risingwave_frontend/optimizer/plan_node/generic/
expand.rs1use itertools::Itertools;
16use pretty_xmlish::{Pretty, Str, XmlNode};
17use risingwave_common::catalog::{Field, FieldDisplay, Schema};
18use risingwave_common::types::DataType;
19use risingwave_common::util::column_index_mapping::ColIndexMapping;
20
21use super::{DistillUnit, GenericPlanNode, GenericPlanRef};
22use crate::optimizer::optimizer_context::OptimizerContextRef;
23use crate::optimizer::plan_node::utils::childless_record;
24use crate::optimizer::property::FunctionalDependencySet;
25
26#[derive(Debug, Clone, PartialEq, Eq, Hash)]
35pub struct Expand<PlanRef> {
36 pub column_subsets: Vec<Vec<usize>>,
39 pub input: PlanRef,
40}
41
42impl<PlanRef: GenericPlanRef> Expand<PlanRef> {
43 pub fn output_len(&self) -> usize {
44 self.input.schema().len() * 2 + 1
45 }
46
47 fn flag_index(&self) -> usize {
48 self.output_len() - 1
49 }
50
51 pub fn clone_with_input<OtherPlanRef>(&self, input: OtherPlanRef) -> Expand<OtherPlanRef> {
52 Expand {
53 column_subsets: self.column_subsets.clone(),
54 input,
55 }
56 }
57}
58
59impl<PlanRef: GenericPlanRef> GenericPlanNode for Expand<PlanRef> {
60 fn schema(&self) -> Schema {
61 let mut fields = self
62 .input
63 .schema()
64 .fields()
65 .iter()
66 .map(|f| Field::with_name(f.data_type(), format!("{}_expanded", f.name)))
67 .collect::<Vec<_>>();
68 fields.extend(self.input.schema().fields().iter().cloned());
69 fields.push(Field::with_name(DataType::Int64, "flag"));
70 Schema::new(fields)
71 }
72
73 fn stream_key(&self) -> Option<Vec<usize>> {
74 let input_schema_len = self.input.schema().len();
75 let mut pk_indices = self
76 .input
77 .stream_key()?
78 .iter()
79 .map(|&pk| pk + input_schema_len)
80 .collect_vec();
81 pk_indices.push(input_schema_len * 2);
83 Some(pk_indices)
84 }
85
86 fn ctx(&self) -> OptimizerContextRef {
87 self.input.ctx()
88 }
89
90 fn functional_dependency(&self) -> FunctionalDependencySet {
91 let input_fd = self
92 .input
93 .functional_dependency()
94 .clone()
95 .into_dependencies();
96 let output_len = self.output_len();
97 let flag_index = self.flag_index();
98
99 self.input
100 .functional_dependency()
101 .as_dependencies()
102 .iter()
103 .map(|_input_fd| {})
104 .collect_vec();
105
106 let mut current_fd = FunctionalDependencySet::new(output_len);
107 for mut fd in input_fd {
108 fd.grow(output_len);
109 fd.set_from(flag_index, true);
110 current_fd.add_functional_dependency(fd);
111 }
112 current_fd
113 }
114}
115
116impl<PlanRef: GenericPlanRef> DistillUnit for Expand<PlanRef> {
117 fn distill_with_name<'a>(&self, name: impl Into<Str<'a>>) -> XmlNode<'a> {
118 childless_record(name, vec![("column_subsets", self.column_subsets_pretty())])
119 }
120}
121
122impl<PlanRef: GenericPlanRef> Expand<PlanRef> {
123 fn column_subsets_pretty<'a>(&self) -> Pretty<'a> {
124 Pretty::Array(
125 self.column_subsets
126 .iter()
127 .map(|subset| {
128 subset
129 .iter()
130 .map(|&i| FieldDisplay(self.input.schema().fields.get(i).unwrap()))
131 .map(|d| Pretty::display(&d))
132 .collect()
133 })
134 .map(Pretty::Array)
135 .collect(),
136 )
137 }
138
139 pub fn i2o_col_mapping(&self) -> ColIndexMapping {
140 let input_len = self.input.schema().len();
141 let map = (0..input_len)
142 .map(|source| Some(source + input_len))
143 .collect_vec();
144 ColIndexMapping::new(map, self.output_len())
145 }
146
147 pub fn o2i_col_mapping(&self) -> ColIndexMapping {
148 self.i2o_col_mapping()
149 .inverse()
150 .expect("must be invertible")
151 }
152
153 pub fn decompose(self) -> (PlanRef, Vec<Vec<usize>>) {
154 (self.input, self.column_subsets)
155 }
156}