risingwave_frontend/optimizer/plan_node/generic/
project_set.rs1use pretty_xmlish::{Pretty, Str, XmlNode};
16use risingwave_common::catalog::{Field, Schema};
17use risingwave_common::types::DataType;
18
19use super::{DistillUnit, GenericPlanNode, GenericPlanRef};
20use crate::expr::{Expr, ExprDisplay, ExprImpl, ExprRewriter, ExprVisitor};
21use crate::optimizer::optimizer_context::OptimizerContextRef;
22use crate::optimizer::plan_node::BatchPlanRef;
23use crate::optimizer::plan_node::batch::BatchPlanNodeMetadata;
24use crate::optimizer::plan_node::utils::childless_record;
25use crate::optimizer::property::{FunctionalDependencySet, Order};
26use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt};
27
28#[derive(Debug, Clone, PartialEq, Eq, Hash)]
37pub struct ProjectSet<PlanRef> {
38 pub select_list: Vec<ExprImpl>,
39 pub input: PlanRef,
40}
41
42impl<PlanRef> ProjectSet<PlanRef> {
43 pub(crate) fn clone_with_input<OtherPlanRef>(
44 &self,
45 input: OtherPlanRef,
46 ) -> ProjectSet<OtherPlanRef> {
47 ProjectSet {
48 select_list: self.select_list.clone(),
49 input,
50 }
51 }
52
53 pub(crate) fn rewrite_exprs(&mut self, r: &mut dyn ExprRewriter) {
54 self.select_list = self
55 .select_list
56 .iter()
57 .map(|e| r.rewrite_expr(e.clone()))
58 .collect();
59 }
60
61 pub(crate) fn visit_exprs(&self, v: &mut dyn ExprVisitor) {
62 self.select_list.iter().for_each(|e| v.visit_expr(e));
63 }
64
65 pub fn decompose(self) -> (Vec<ExprImpl>, PlanRef) {
66 (self.select_list, self.input)
67 }
68}
69
70impl<PlanRef> DistillUnit for ProjectSet<PlanRef> {
71 fn distill_with_name<'a>(&self, name: impl Into<Str<'a>>) -> XmlNode<'a> {
72 let fields = vec![("select_list", Pretty::debug(&self.select_list))];
73 childless_record(name, fields)
74 }
75}
76
77impl<PlanRef: GenericPlanRef> GenericPlanNode for ProjectSet<PlanRef> {
78 fn schema(&self) -> Schema {
79 let input_schema = self.input.schema();
80 let o2i = self.o2i_col_mapping();
81 let mut fields = vec![Field::with_name(DataType::Int64, "projected_row_id")];
82 fields.extend(self.select_list.iter().enumerate().map(|(idx, expr)| {
83 let idx = idx + 1;
84 let name = match o2i.try_map(idx) {
86 Some(input_idx) => input_schema.fields()[input_idx].name.clone(),
87 None => format!("{:?}", ExprDisplay { expr, input_schema }),
88 };
89 Field::with_name(expr.return_type(), name)
90 }));
91
92 Schema { fields }
93 }
94
95 fn stream_key(&self) -> Option<Vec<usize>> {
96 let i2o = self.i2o_col_mapping();
97 let mut pk = self
98 .input
99 .stream_key()?
100 .iter()
101 .map(|pk_col| i2o.try_map(*pk_col))
102 .collect::<Option<Vec<_>>>()
103 .unwrap_or_default();
104 pk.push(0);
106 Some(pk)
107 }
108
109 fn ctx(&self) -> OptimizerContextRef {
110 self.input.ctx()
111 }
112
113 fn functional_dependency(&self) -> FunctionalDependencySet {
114 let i2o = self.i2o_col_mapping();
115 i2o.rewrite_functional_dependency_set(self.input.functional_dependency().clone())
116 }
117}
118
119impl<PlanRef: GenericPlanRef> ProjectSet<PlanRef> {
120 pub fn o2i_col_mapping(&self) -> ColIndexMapping {
122 let input_len = self.input.schema().len();
123 let mut map = vec![None; 1 + self.select_list.len()];
124 for (i, item) in self.select_list.iter().enumerate() {
125 if let ExprImpl::InputRef(input) = item {
126 map[1 + i] = Some(input.index())
127 }
128 }
129 ColIndexMapping::new(map, input_len)
130 }
131
132 pub fn i2o_col_mapping(&self) -> ColIndexMapping {
135 let input_len = self.input.schema().len();
136 let mut map = vec![None; input_len];
137 for (i, item) in self.select_list.iter().enumerate() {
138 if let ExprImpl::InputRef(input) = item {
139 map[input.index()] = Some(1 + i)
140 }
141 }
142 ColIndexMapping::new(map, 1 + self.select_list.len())
143 }
144}
145
146impl ProjectSet<BatchPlanRef> {
147 pub fn get_out_column_index_order(&self) -> Order {
149 self.i2o_col_mapping()
150 .rewrite_provided_order(self.input.order())
151 }
152}