risingwave_frontend/optimizer/plan_node/generic/
delete.rs

1// Copyright 2025 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15use std::hash::Hash;
16
17use educe::Educe;
18use pretty_xmlish::{Pretty, Str, XmlNode};
19use risingwave_common::catalog::{Field, Schema, TableVersionId};
20use risingwave_common::types::DataType;
21
22use super::{DistillUnit, GenericPlanNode, GenericPlanRef};
23use crate::OptimizerContextRef;
24use crate::catalog::TableId;
25use crate::optimizer::plan_node::utils::childless_record;
26use crate::optimizer::property::FunctionalDependencySet;
27
28#[derive(Debug, Clone, Educe)]
29#[educe(PartialEq, Eq, Hash)]
30pub struct Delete<PlanRef: Eq + Hash> {
31    #[educe(PartialEq(ignore))]
32    #[educe(Hash(ignore))]
33    pub table_name: String, // explain-only
34    pub table_id: TableId,
35    pub table_version_id: TableVersionId,
36    pub input: PlanRef,
37    pub returning: bool,
38}
39
40impl<PlanRef: GenericPlanRef> Delete<PlanRef> {
41    pub fn output_len(&self) -> usize {
42        if self.returning {
43            self.input.schema().len()
44        } else {
45            1
46        }
47    }
48}
49
50impl<PlanRef: GenericPlanRef> GenericPlanNode for Delete<PlanRef> {
51    fn functional_dependency(&self) -> FunctionalDependencySet {
52        FunctionalDependencySet::new(self.output_len())
53    }
54
55    fn schema(&self) -> Schema {
56        if self.returning {
57            self.input.schema().clone()
58        } else {
59            Schema::new(vec![Field::unnamed(DataType::Int64)])
60        }
61    }
62
63    fn stream_key(&self) -> Option<Vec<usize>> {
64        if self.returning {
65            Some(self.input.stream_key()?.to_vec())
66        } else {
67            Some(vec![])
68        }
69    }
70
71    fn ctx(&self) -> OptimizerContextRef {
72        self.input.ctx()
73    }
74}
75
76impl<PlanRef: Eq + Hash> Delete<PlanRef> {
77    pub fn new(
78        input: PlanRef,
79        table_name: String,
80        table_id: TableId,
81        table_version_id: TableVersionId,
82        returning: bool,
83    ) -> Self {
84        Self {
85            table_name,
86            table_id,
87            table_version_id,
88            input,
89            returning,
90        }
91    }
92}
93
94impl<PlanRef: Eq + Hash> DistillUnit for Delete<PlanRef> {
95    fn distill_with_name<'a>(&self, name: impl Into<Str<'a>>) -> XmlNode<'a> {
96        let mut vec = Vec::with_capacity(if self.returning { 2 } else { 1 });
97        vec.push(("table", Pretty::from(self.table_name.clone())));
98        if self.returning {
99            vec.push(("returning", Pretty::display(&true)));
100        }
101        childless_record(name, vec)
102    }
103}