risingwave_frontend/optimizer/plan_node/
logical_union.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use std::cmp::max;
use std::collections::BTreeMap;

use itertools::Itertools;
use risingwave_common::catalog::Schema;
use risingwave_common::types::{DataType, Scalar};

use super::utils::impl_distill_by_unit;
use super::{
    ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream,
};
use crate::error::Result;
use crate::expr::{ExprImpl, InputRef, Literal};
use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
use crate::optimizer::plan_node::generic::GenericPlanRef;
use crate::optimizer::plan_node::stream_union::StreamUnion;
use crate::optimizer::plan_node::{
    generic, BatchHashAgg, BatchUnion, ColumnPruningContext, LogicalProject, PlanTreeNode,
    PredicatePushdownContext, RewriteStreamContext, ToStreamContext,
};
use crate::optimizer::property::RequiredDist;
use crate::utils::{ColIndexMapping, Condition};
use crate::Explain;

/// `LogicalUnion` returns the union of the rows of its inputs.
/// If `all` is false, it needs to eliminate duplicates.
#[derive(Debug, Clone, PartialEq, Eq, Hash)]
pub struct LogicalUnion {
    pub base: PlanBase<Logical>,
    core: generic::Union<PlanRef>,
}

impl LogicalUnion {
    pub fn new(all: bool, inputs: Vec<PlanRef>) -> Self {
        assert!(Schema::all_type_eq(inputs.iter().map(|x| x.schema())));
        Self::new_with_source_col(all, inputs, None)
    }

    /// It is used by streaming processing. We need to use `source_col` to identify the record came
    /// from which source input.
    pub fn new_with_source_col(all: bool, inputs: Vec<PlanRef>, source_col: Option<usize>) -> Self {
        let core = generic::Union {
            all,
            inputs,
            source_col,
        };
        let base = PlanBase::new_logical_with_core(&core);
        LogicalUnion { base, core }
    }

    pub fn create(all: bool, inputs: Vec<PlanRef>) -> PlanRef {
        LogicalUnion::new(all, inputs).into()
    }

    pub fn all(&self) -> bool {
        self.core.all
    }

    pub fn source_col(&self) -> Option<usize> {
        self.core.source_col
    }
}

impl PlanTreeNode for LogicalUnion {
    fn inputs(&self) -> smallvec::SmallVec<[crate::optimizer::PlanRef; 2]> {
        self.core.inputs.clone().into_iter().collect()
    }

    fn clone_with_inputs(&self, inputs: &[crate::optimizer::PlanRef]) -> PlanRef {
        Self::new_with_source_col(self.all(), inputs.to_vec(), self.core.source_col).into()
    }
}

impl_distill_by_unit!(LogicalUnion, core, "LogicalUnion");

impl ColPrunable for LogicalUnion {
    fn prune_col(&self, required_cols: &[usize], ctx: &mut ColumnPruningContext) -> PlanRef {
        let new_inputs = self
            .inputs()
            .iter()
            .map(|input| input.prune_col(required_cols, ctx))
            .collect_vec();
        self.clone_with_inputs(&new_inputs)
    }
}

impl ExprRewritable for LogicalUnion {}

impl ExprVisitable for LogicalUnion {}

impl PredicatePushdown for LogicalUnion {
    fn predicate_pushdown(
        &self,
        predicate: Condition,
        ctx: &mut PredicatePushdownContext,
    ) -> PlanRef {
        let new_inputs = self
            .inputs()
            .iter()
            .map(|input| input.predicate_pushdown(predicate.clone(), ctx))
            .collect_vec();
        self.clone_with_inputs(&new_inputs)
    }
}

impl ToBatch for LogicalUnion {
    fn to_batch(&self) -> Result<PlanRef> {
        let new_inputs = self
            .inputs()
            .iter()
            .map(|input| input.to_batch())
            .try_collect()?;
        let new_logical = generic::Union {
            all: true,
            inputs: new_inputs,
            source_col: None,
        };
        // We still need to handle !all even if we already have `UnionToDistinctRule`, because it
        // can be generated by index selection which is an optimization during the `to_batch`.
        // Convert union to union all + agg
        if !self.all() {
            let batch_union = BatchUnion::new(new_logical).into();
            Ok(BatchHashAgg::new(
                generic::Agg::new(vec![], (0..self.base.schema().len()).collect(), batch_union)
                    .with_enable_two_phase(false),
            )
            .into())
        } else {
            Ok(BatchUnion::new(new_logical).into())
        }
    }
}

impl ToStream for LogicalUnion {
    fn to_stream(&self, ctx: &mut ToStreamContext) -> Result<PlanRef> {
        // TODO: use round robin distribution instead of using hash distribution of all inputs.
        let dist = RequiredDist::hash_shard(self.base.stream_key().unwrap_or_else(|| {
            panic!(
                "should always have a stream key in the stream plan but not, sub plan: {}",
                PlanRef::from(self.clone()).explain_to_string()
            )
        }));
        let new_inputs: Result<Vec<_>> = self
            .inputs()
            .iter()
            .map(|input| input.to_stream_with_dist_required(&dist, ctx))
            .collect();
        let new_logical = generic::Union {
            all: true,
            inputs: new_inputs?,
            ..self.core
        };
        assert!(
            self.all(),
            "After UnionToDistinctRule, union should become union all"
        );
        Ok(StreamUnion::new(new_logical).into())
    }

    fn logical_rewrite_for_stream(
        &self,
        ctx: &mut RewriteStreamContext,
    ) -> Result<(PlanRef, ColIndexMapping)> {
        let original_schema = self.base.schema().clone();
        let original_schema_len = original_schema.len();
        let mut rewrites = vec![];
        for input in &self.core.inputs {
            rewrites.push(input.logical_rewrite_for_stream(ctx)?);
        }

        let original_schema_contain_all_input_stream_keys =
            rewrites.iter().all(|(new_input, col_index_mapping)| {
                let original_schema_new_pos = (0..original_schema_len)
                    .map(|x| col_index_mapping.map(x))
                    .collect_vec();
                new_input
                    .expect_stream_key()
                    .iter()
                    .all(|x| original_schema_new_pos.contains(x))
            });

        if original_schema_contain_all_input_stream_keys {
            // Add one more column at the end of the original schema to identify the record came
            // from which input. [original_schema + source_col]
            let new_inputs = rewrites
                .into_iter()
                .enumerate()
                .map(|(i, (new_input, col_index_mapping))| {
                    // original_schema
                    let mut exprs = (0..original_schema_len)
                        .map(|x| {
                            ExprImpl::InputRef(
                                InputRef::new(
                                    col_index_mapping.map(x),
                                    original_schema.fields[x].data_type.clone(),
                                )
                                .into(),
                            )
                        })
                        .collect_vec();
                    // source_col
                    exprs.push(ExprImpl::Literal(
                        Literal::new(Some((i as i32).to_scalar_value()), DataType::Int32).into(),
                    ));
                    LogicalProject::create(new_input, exprs)
                })
                .collect_vec();
            let new_union = LogicalUnion::new_with_source_col(
                self.all(),
                new_inputs,
                Some(original_schema_len),
            );
            // We have already used project to map rewrite input to the origin schema, so we can use
            // identity with the new schema len.
            let out_col_change =
                ColIndexMapping::identity_or_none(original_schema_len, new_union.schema().len());
            Ok((new_union.into(), out_col_change))
        } else {
            // In order to ensure all inputs have the same schema for new union, we construct new
            // schema like that: [original_schema + merged_stream_key + source_col]
            // where merged_stream_key is merged by the types of each input stream key.
            // If all inputs have the same stream key column types, we have a small merged_stream_key. Otherwise, we will have a large merged_stream_key.

            let (merged_stream_key_types, types_offset) = {
                let mut max_types_counter = BTreeMap::default();
                for (new_input, _) in &rewrites {
                    let mut types_counter = BTreeMap::default();
                    for x in new_input.expect_stream_key() {
                        types_counter
                            .entry(new_input.schema().fields[*x].data_type())
                            .and_modify(|x| *x += 1)
                            .or_insert(1);
                    }
                    for (key, val) in types_counter {
                        max_types_counter
                            .entry(key)
                            .and_modify(|x| *x = max(*x, val))
                            .or_insert(val);
                    }
                }

                let mut merged_stream_key_types = vec![];
                let mut types_offset = BTreeMap::default();
                let mut offset = 0;
                for (key, val) in max_types_counter {
                    let _ = types_offset.insert(key.clone(), offset);
                    offset += val;
                    merged_stream_key_types.extend(std::iter::repeat(key.clone()).take(val));
                }

                (merged_stream_key_types, types_offset)
            };

            let input_stream_key_nulls = merged_stream_key_types
                .iter()
                .map(|t| ExprImpl::Literal(Literal::new(None, t.clone()).into()))
                .collect_vec();

            let new_inputs = rewrites
                .into_iter()
                .enumerate()
                .map(|(i, (new_input, col_index_mapping))| {
                    // original_schema
                    let mut exprs = (0..original_schema_len)
                        .map(|x| {
                            ExprImpl::InputRef(
                                InputRef::new(
                                    col_index_mapping.map(x),
                                    original_schema.fields[x].data_type.clone(),
                                )
                                .into(),
                            )
                        })
                        .collect_vec();
                    // merged_stream_key
                    let mut input_stream_keys = input_stream_key_nulls.clone();
                    let mut types_counter = BTreeMap::default();
                    for stream_key_idx in new_input.expect_stream_key() {
                        let data_type =
                            new_input.schema().fields[*stream_key_idx].data_type.clone();
                        let count = *types_counter
                            .entry(data_type.clone())
                            .and_modify(|x| *x += 1)
                            .or_insert(1);
                        let type_start_offset = *types_offset.get(&data_type).unwrap();

                        input_stream_keys[type_start_offset + count - 1] =
                            ExprImpl::InputRef(InputRef::new(*stream_key_idx, data_type).into());
                    }
                    exprs.extend(input_stream_keys);
                    // source_col
                    exprs.push(ExprImpl::Literal(
                        Literal::new(Some((i as i32).to_scalar_value()), DataType::Int32).into(),
                    ));
                    LogicalProject::create(new_input, exprs)
                })
                .collect_vec();

            let new_union = LogicalUnion::new_with_source_col(
                self.all(),
                new_inputs,
                Some(original_schema_len + merged_stream_key_types.len()),
            );
            // We have already used project to map rewrite input to the origin schema, so we can use
            // identity with the new schema len.
            let out_col_change =
                ColIndexMapping::identity_or_none(original_schema_len, new_union.schema().len());
            Ok((new_union.into(), out_col_change))
        }
    }
}

#[cfg(test)]
mod tests {

    use risingwave_common::catalog::Field;

    use super::*;
    use crate::optimizer::optimizer_context::OptimizerContext;
    use crate::optimizer::plan_node::{LogicalValues, PlanTreeNodeUnary};

    #[tokio::test]
    async fn test_prune_union() {
        let ty = DataType::Int32;
        let ctx = OptimizerContext::mock().await;
        let fields: Vec<Field> = vec![
            Field::with_name(ty.clone(), "v1"),
            Field::with_name(ty.clone(), "v2"),
            Field::with_name(ty.clone(), "v3"),
        ];
        let values1 = LogicalValues::new(vec![], Schema { fields }, ctx);

        let values2 = values1.clone();

        let union: PlanRef = LogicalUnion::new(false, vec![values1.into(), values2.into()]).into();

        // Perform the prune
        let required_cols = vec![1, 2];
        let plan = union.prune_col(
            &required_cols,
            &mut ColumnPruningContext::new(union.clone()),
        );

        // Check the result
        let union = plan.as_logical_union().unwrap();
        assert_eq!(union.base.schema().len(), 2);
    }

    #[tokio::test]
    async fn test_union_to_batch() {
        let ty = DataType::Int32;
        let ctx = OptimizerContext::mock().await;
        let fields: Vec<Field> = vec![
            Field::with_name(ty.clone(), "v1"),
            Field::with_name(ty.clone(), "v2"),
            Field::with_name(ty.clone(), "v3"),
        ];
        let values1 = LogicalValues::new(vec![], Schema { fields }, ctx);

        let values2 = values1.clone();

        let union = LogicalUnion::new(false, vec![values1.into(), values2.into()]);

        let plan = union.to_batch().unwrap();
        let agg: &BatchHashAgg = plan.as_batch_hash_agg().unwrap();
        let agg_input = agg.input();
        let union = agg_input.as_batch_union().unwrap();

        assert_eq!(union.inputs().len(), 2);
    }
}