risingwave_frontend/optimizer/plan_node/generic/
delete.rsuse std::hash::Hash;
use educe::Educe;
use pretty_xmlish::{Pretty, Str, XmlNode};
use risingwave_common::catalog::{Field, Schema, TableVersionId};
use risingwave_common::types::DataType;
use super::{DistillUnit, GenericPlanNode, GenericPlanRef};
use crate::catalog::TableId;
use crate::optimizer::plan_node::utils::childless_record;
use crate::optimizer::property::FunctionalDependencySet;
use crate::OptimizerContextRef;
#[derive(Debug, Clone, Educe)]
#[educe(PartialEq, Eq, Hash)]
pub struct Delete<PlanRef: Eq + Hash> {
#[educe(PartialEq(ignore))]
#[educe(Hash(ignore))]
pub table_name: String, pub table_id: TableId,
pub table_version_id: TableVersionId,
pub input: PlanRef,
pub returning: bool,
}
impl<PlanRef: GenericPlanRef> Delete<PlanRef> {
pub fn output_len(&self) -> usize {
if self.returning {
self.input.schema().len()
} else {
1
}
}
}
impl<PlanRef: GenericPlanRef> GenericPlanNode for Delete<PlanRef> {
fn functional_dependency(&self) -> FunctionalDependencySet {
FunctionalDependencySet::new(self.output_len())
}
fn schema(&self) -> Schema {
if self.returning {
self.input.schema().clone()
} else {
Schema::new(vec![Field::unnamed(DataType::Int64)])
}
}
fn stream_key(&self) -> Option<Vec<usize>> {
if self.returning {
Some(self.input.stream_key()?.to_vec())
} else {
Some(vec![])
}
}
fn ctx(&self) -> OptimizerContextRef {
self.input.ctx()
}
}
impl<PlanRef: Eq + Hash> Delete<PlanRef> {
pub fn new(
input: PlanRef,
table_name: String,
table_id: TableId,
table_version_id: TableVersionId,
returning: bool,
) -> Self {
Self {
table_name,
table_id,
table_version_id,
input,
returning,
}
}
}
impl<PlanRef: Eq + Hash> DistillUnit for Delete<PlanRef> {
fn distill_with_name<'a>(&self, name: impl Into<Str<'a>>) -> XmlNode<'a> {
let mut vec = Vec::with_capacity(if self.returning { 2 } else { 1 });
vec.push(("table", Pretty::from(self.table_name.clone())));
if self.returning {
vec.push(("returning", Pretty::display(&true)));
}
childless_record(name, vec)
}
}