risingwave_frontend/optimizer/plan_node/generic/
expand.rs

1// Copyright 2022 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 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;
25use crate::utils::ColIndexMappingRewriteExt;
26
27/// [`Expand`] expand one row multiple times according to `column_subsets` and also keep
28/// original columns of input. It can be used to implement distinct aggregation and group set.
29///
30/// This is the schema of `Expand`:
31/// | expanded columns(i.e. some columns are set to null) | original columns of input | flag |.
32///
33/// Aggregates use expanded columns as their arguments and original columns for their filter. `flag`
34/// is used to distinguish between different `subset`s in `column_subsets`.
35#[derive(Debug, Clone, PartialEq, Eq, Hash)]
36pub struct Expand<PlanRef> {
37    // `column_subsets` has many `subset`s which specifies the columns that need to be
38    // reserved and other columns will be filled with NULL.
39    pub column_subsets: Vec<Vec<usize>>,
40    pub input: PlanRef,
41}
42
43impl<PlanRef: GenericPlanRef> Expand<PlanRef> {
44    pub fn output_len(&self) -> usize {
45        self.input.schema().len() * 2 + 1
46    }
47
48    pub fn clone_with_input<OtherPlanRef>(&self, input: OtherPlanRef) -> Expand<OtherPlanRef> {
49        Expand {
50            column_subsets: self.column_subsets.clone(),
51            input,
52        }
53    }
54}
55
56impl<PlanRef: GenericPlanRef> GenericPlanNode for Expand<PlanRef> {
57    fn schema(&self) -> Schema {
58        let mut fields = self
59            .input
60            .schema()
61            .fields()
62            .iter()
63            .map(|f| Field::with_name(f.data_type(), format!("{}_expanded", f.name)))
64            .collect::<Vec<_>>();
65        fields.extend(self.input.schema().fields().iter().cloned());
66        fields.push(Field::with_name(DataType::Int64, "flag"));
67        Schema::new(fields)
68    }
69
70    fn stream_key(&self) -> Option<Vec<usize>> {
71        let input_schema_len = self.input.schema().len();
72        let mut pk_indices = self
73            .input
74            .stream_key()?
75            .iter()
76            .map(|&pk| pk + input_schema_len)
77            .collect_vec();
78        // The last column should be the flag.
79        pk_indices.push(input_schema_len * 2);
80        Some(pk_indices)
81    }
82
83    fn ctx(&self) -> OptimizerContextRef {
84        self.input.ctx()
85    }
86
87    fn functional_dependency(&self) -> FunctionalDependencySet {
88        // Expanded columns may be nulled by different `column_subsets`, so input FDs are only
89        // guaranteed on the original input columns preserved in the second half of the schema.
90        self.i2o_col_mapping()
91            .rewrite_functional_dependency_set(self.input.functional_dependency().clone())
92    }
93}
94
95impl<PlanRef: GenericPlanRef> DistillUnit for Expand<PlanRef> {
96    fn distill_with_name<'a>(&self, name: impl Into<Str<'a>>) -> XmlNode<'a> {
97        childless_record(name, vec![("column_subsets", self.column_subsets_pretty())])
98    }
99}
100
101impl<PlanRef: GenericPlanRef> Expand<PlanRef> {
102    fn column_subsets_pretty<'a>(&self) -> Pretty<'a> {
103        Pretty::Array(
104            self.column_subsets
105                .iter()
106                .map(|subset| {
107                    subset
108                        .iter()
109                        .map(|&i| FieldDisplay(self.input.schema().fields.get(i).unwrap()))
110                        .map(|d| Pretty::display(&d))
111                        .collect()
112                })
113                .map(Pretty::Array)
114                .collect(),
115        )
116    }
117
118    pub fn i2o_col_mapping(&self) -> ColIndexMapping {
119        let input_len = self.input.schema().len();
120        let map = (0..input_len)
121            .map(|source| Some(source + input_len))
122            .collect_vec();
123        ColIndexMapping::new(map, self.output_len())
124    }
125
126    pub fn o2i_col_mapping(&self) -> ColIndexMapping {
127        self.i2o_col_mapping()
128            .inverse()
129            .expect("must be invertible")
130    }
131
132    pub fn decompose(self) -> (PlanRef, Vec<Vec<usize>>) {
133        (self.input, self.column_subsets)
134    }
135}