risingwave_frontend/optimizer/plan_node/
logical_max_one_row.rsuse pretty_xmlish::XmlNode;
use super::generic::DistillUnit;
use super::utils::Distill;
use super::{
gen_filter_and_pushdown, generic, BatchMaxOneRow, ColPrunable, ExprRewritable, Logical,
PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream,
};
use crate::error::Result;
use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
use crate::optimizer::plan_node::{
ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext,
};
use crate::utils::{ColIndexMapping, Condition};
#[derive(Debug, Clone, PartialEq, Eq, Hash)]
pub struct LogicalMaxOneRow {
pub base: PlanBase<Logical>,
pub(super) core: generic::MaxOneRow<PlanRef>,
}
impl LogicalMaxOneRow {
fn new(core: generic::MaxOneRow<PlanRef>) -> Self {
let base = PlanBase::new_logical_with_core(&core);
LogicalMaxOneRow { base, core }
}
pub fn create(input: PlanRef) -> PlanRef {
let core = generic::MaxOneRow { input };
Self::new(core).into()
}
}
impl PlanTreeNodeUnary for LogicalMaxOneRow {
fn input(&self) -> PlanRef {
self.core.input.clone()
}
fn clone_with_input(&self, input: PlanRef) -> Self {
let core = generic::MaxOneRow { input };
Self::new(core)
}
#[must_use]
fn rewrite_with_input(
&self,
input: PlanRef,
input_col_change: ColIndexMapping,
) -> (Self, ColIndexMapping) {
(self.clone_with_input(input), input_col_change)
}
}
impl_plan_tree_node_for_unary! {LogicalMaxOneRow}
impl Distill for LogicalMaxOneRow {
fn distill<'a>(&self) -> XmlNode<'a> {
self.core.distill_with_name("LogicalMaxOneRow")
}
}
impl ColPrunable for LogicalMaxOneRow {
fn prune_col(&self, required_cols: &[usize], ctx: &mut ColumnPruningContext) -> PlanRef {
let new_input = self.input().prune_col(required_cols, ctx);
self.clone_with_input(new_input).into()
}
}
impl ExprRewritable for LogicalMaxOneRow {}
impl ExprVisitable for LogicalMaxOneRow {}
impl PredicatePushdown for LogicalMaxOneRow {
fn predicate_pushdown(
&self,
predicate: Condition,
ctx: &mut PredicatePushdownContext,
) -> PlanRef {
gen_filter_and_pushdown(self, predicate, Condition::true_cond(), ctx)
}
}
impl ToBatch for LogicalMaxOneRow {
fn to_batch(&self) -> Result<PlanRef> {
let input = self.input().to_batch()?;
let core = generic::MaxOneRow { input };
Ok(BatchMaxOneRow::new(core).into())
}
}
impl ToStream for LogicalMaxOneRow {
fn to_stream(&self, _ctx: &mut ToStreamContext) -> Result<PlanRef> {
unreachable!("should already bail out after subquery unnesting")
}
fn logical_rewrite_for_stream(
&self,
ctx: &mut RewriteStreamContext,
) -> Result<(PlanRef, ColIndexMapping)> {
let (input, input_col_change) = self.input().logical_rewrite_for_stream(ctx)?;
let (this, out_col_change) = self.rewrite_with_input(input, input_col_change);
Ok((this.into(), out_col_change))
}
}