risingwave_frontend/optimizer/plan_node/generic/
except.rsuse pretty_xmlish::{Pretty, Str, XmlNode};
use risingwave_common::catalog::Schema;
use super::{DistillUnit, GenericPlanNode, GenericPlanRef};
use crate::optimizer::optimizer_context::OptimizerContextRef;
use crate::optimizer::plan_node::utils::childless_record;
use crate::optimizer::property::FunctionalDependencySet;
#[derive(Debug, Clone, PartialEq, Eq, Hash)]
pub struct Except<PlanRef> {
pub all: bool,
pub inputs: Vec<PlanRef>,
}
impl<PlanRef: GenericPlanRef> GenericPlanNode for Except<PlanRef> {
fn schema(&self) -> Schema {
self.inputs[0].schema().clone()
}
fn stream_key(&self) -> Option<Vec<usize>> {
Some(self.inputs[0].stream_key()?.to_vec())
}
fn ctx(&self) -> OptimizerContextRef {
self.inputs[0].ctx()
}
fn functional_dependency(&self) -> FunctionalDependencySet {
FunctionalDependencySet::new(self.inputs[0].schema().len())
}
}
impl<PlanRef> DistillUnit for Except<PlanRef> {
fn distill_with_name<'a>(&self, name: impl Into<Str<'a>>) -> XmlNode<'a> {
childless_record(name, vec![("all", Pretty::debug(&self.all))])
}
}