risingwave_frontend/optimizer/plan_node/
logical_delete.rsuse risingwave_common::catalog::TableVersionId;
use super::utils::impl_distill_by_unit;
use super::{
gen_filter_and_pushdown, generic, BatchDelete, ColPrunable, ExprRewritable, Logical,
LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream,
};
use crate::catalog::TableId;
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 LogicalDelete {
pub base: PlanBase<Logical>,
core: generic::Delete<PlanRef>,
}
impl From<generic::Delete<PlanRef>> for LogicalDelete {
fn from(core: generic::Delete<PlanRef>) -> Self {
let base = PlanBase::new_logical_with_core(&core);
Self { base, core }
}
}
impl LogicalDelete {
#[must_use]
pub fn table_id(&self) -> TableId {
self.core.table_id
}
pub fn has_returning(&self) -> bool {
self.core.returning
}
pub fn table_version_id(&self) -> TableVersionId {
self.core.table_version_id
}
}
impl PlanTreeNodeUnary for LogicalDelete {
fn input(&self) -> PlanRef {
self.core.input.clone()
}
fn clone_with_input(&self, input: PlanRef) -> Self {
let mut core = self.core.clone();
core.input = input;
core.into()
}
}
impl_plan_tree_node_for_unary! { LogicalDelete }
impl_distill_by_unit!(LogicalDelete, core, "LogicalDelete");
impl ColPrunable for LogicalDelete {
fn prune_col(&self, required_cols: &[usize], ctx: &mut ColumnPruningContext) -> PlanRef {
let pruned_input = {
let input = &self.core.input;
let required_cols: Vec<_> = (0..input.schema().len()).collect();
input.prune_col(&required_cols, ctx)
};
LogicalProject::with_out_col_idx(
self.clone_with_input(pruned_input).into(),
required_cols.iter().copied(),
)
.into()
}
}
impl ExprRewritable for LogicalDelete {}
impl ExprVisitable for LogicalDelete {}
impl PredicatePushdown for LogicalDelete {
fn predicate_pushdown(
&self,
predicate: Condition,
ctx: &mut PredicatePushdownContext,
) -> PlanRef {
gen_filter_and_pushdown(self, predicate, Condition::true_cond(), ctx)
}
}
impl ToBatch for LogicalDelete {
fn to_batch(&self) -> Result<PlanRef> {
let new_input = self.input().to_batch()?;
let mut core = self.core.clone();
core.input = new_input;
Ok(BatchDelete::new(core).into())
}
}
impl ToStream for LogicalDelete {
fn to_stream(&self, _ctx: &mut ToStreamContext) -> Result<PlanRef> {
unreachable!("delete should always be converted to batch plan");
}
fn logical_rewrite_for_stream(
&self,
_ctx: &mut RewriteStreamContext,
) -> Result<(PlanRef, ColIndexMapping)> {
unreachable!("delete should always be converted to batch plan");
}
}