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
52impl<PlanRef: GenericPlanRef> GenericPlanNode for Expand<PlanRef> {
53 fn schema(&self) -> Schema {
54 let mut fields = self
55 .input
56 .schema()
57 .fields()
58 .iter()
59 .map(|f| Field::with_name(f.data_type(), format!("{}_expanded", f.name)))
60 .collect::<Vec<_>>();
61 fields.extend(self.input.schema().fields().iter().cloned());
62 fields.push(Field::with_name(DataType::Int64, "flag"));
63 Schema::new(fields)
64 }
65
66 fn stream_key(&self) -> Option<Vec<usize>> {
67 let input_schema_len = self.input.schema().len();
68 let mut pk_indices = self
69 .input
70 .stream_key()?
71 .iter()
72 .map(|&pk| pk + input_schema_len)
73 .collect_vec();
74 pk_indices.push(input_schema_len * 2);
76 Some(pk_indices)
77 }
78
79 fn ctx(&self) -> OptimizerContextRef {
80 self.input.ctx()
81 }
82
83 fn functional_dependency(&self) -> FunctionalDependencySet {
84 let input_fd = self
85 .input
86 .functional_dependency()
87 .clone()
88 .into_dependencies();
89 let output_len = self.output_len();
90 let flag_index = self.flag_index();
91
92 self.input
93 .functional_dependency()
94 .as_dependencies()
95 .iter()
96 .map(|_input_fd| {})
97 .collect_vec();
98
99 let mut current_fd = FunctionalDependencySet::new(output_len);
100 for mut fd in input_fd {
101 fd.grow(output_len);
102 fd.set_from(flag_index, true);
103 current_fd.add_functional_dependency(fd);
104 }
105 current_fd
106 }
107}
108
109impl<PlanRef: GenericPlanRef> DistillUnit for Expand<PlanRef> {
110 fn distill_with_name<'a>(&self, name: impl Into<Str<'a>>) -> XmlNode<'a> {
111 childless_record(name, vec![("column_subsets", self.column_subsets_pretty())])
112 }
113}
114
115impl<PlanRef: GenericPlanRef> Expand<PlanRef> {
116 fn column_subsets_pretty<'a>(&self) -> Pretty<'a> {
117 Pretty::Array(
118 self.column_subsets
119 .iter()
120 .map(|subset| {
121 subset
122 .iter()
123 .map(|&i| FieldDisplay(self.input.schema().fields.get(i).unwrap()))
124 .map(|d| Pretty::display(&d))
125 .collect()
126 })
127 .map(Pretty::Array)
128 .collect(),
129 )
130 }
131
132 pub fn i2o_col_mapping(&self) -> ColIndexMapping {
133 let input_len = self.input.schema().len();
134 let map = (0..input_len)
135 .map(|source| Some(source + input_len))
136 .collect_vec();
137 ColIndexMapping::new(map, self.output_len())
138 }
139
140 pub fn o2i_col_mapping(&self) -> ColIndexMapping {
141 self.i2o_col_mapping()
142 .inverse()
143 .expect("must be invertible")
144 }
145
146 pub fn decompose(self) -> (PlanRef, Vec<Vec<usize>>) {
147 (self.input, self.column_subsets)
148 }
149}