risingwave_frontend/optimizer/plan_node/
derive.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
// 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::collections::HashSet;

use fixedbitset::FixedBitSet;
use itertools::Itertools;
use risingwave_common::catalog::{ColumnCatalog, ColumnDesc, Schema, USER_COLUMN_ID_OFFSET};
use risingwave_common::util::sort_util::{ColumnOrder, OrderType};

use super::PlanRef;
use crate::error::{ErrorCode, Result};
use crate::optimizer::plan_node::generic::PhysicalPlanRef;
use crate::optimizer::property::Order;

pub(crate) fn derive_columns(
    input_schema: &Schema,
    out_names: Vec<String>,
    user_cols: &FixedBitSet,
) -> Result<Vec<ColumnCatalog>> {
    // Validate and deduplicate column names.
    let mut col_names = HashSet::new();
    for name in &out_names {
        if !col_names.insert(name.clone()) {
            Err(ErrorCode::InvalidInputSyntax(format!(
                "column \"{}\" specified more than once",
                name
            )))?;
        }
    }

    let mut out_name_iter = out_names.into_iter();
    let columns = input_schema
        .fields()
        .iter()
        .enumerate()
        .map(|(i, field)| {
            let mut c = ColumnCatalog {
                column_desc: ColumnDesc::from_field_with_column_id(
                    field,
                    i as i32 + USER_COLUMN_ID_OFFSET,
                ),
                is_hidden: !user_cols.contains(i),
            };
            c.column_desc.name = if !c.is_hidden {
                out_name_iter.next().unwrap()
            } else {
                let mut name = field.name.clone();
                let mut count = 0;

                while !col_names.insert(name.clone()) {
                    count += 1;
                    name = format!("{}#{}", field.name, count);
                }

                name
            };
            c
        })
        .collect_vec();

    // We should use up all of the `out_name`s
    assert_eq!(out_name_iter.next(), None);

    Ok(columns)
}

/// Derive the pk and the stream key for tables and sinks.
pub(crate) fn derive_pk(
    input: PlanRef,
    user_order_by: Order,
    columns: &[ColumnCatalog],
) -> (Vec<ColumnOrder>, Vec<usize>) {
    // Note(congyi): avoid pk duplication
    let stream_key = input
        .expect_stream_key()
        .iter()
        .copied()
        .unique()
        .collect_vec();
    let schema = input.schema();

    // Assert the uniqueness of column names and IDs, including hidden columns.
    if let Some(name) = columns.iter().map(|c| c.name()).duplicates().next() {
        panic!("duplicated column name \"{name}\"");
    }
    if let Some(id) = columns.iter().map(|c| c.column_id()).duplicates().next() {
        panic!("duplicated column ID {id}");
    }
    // Assert that the schema of given `columns` is correct.
    assert_eq!(
        columns.iter().map(|c| c.data_type().clone()).collect_vec(),
        input.schema().data_types()
    );

    let mut in_order = FixedBitSet::with_capacity(schema.len());
    let mut pk = vec![];

    let func_dep = input.functional_dependency();
    let user_order_by =
        func_dep.minimize_order_key(user_order_by, input.distribution().dist_column_indices());

    for order in &user_order_by.column_orders {
        let idx = order.column_index;
        pk.push(order.clone());
        in_order.insert(idx);
    }

    for &idx in &stream_key {
        if in_order.contains(idx) {
            continue;
        }
        pk.push(ColumnOrder::new(idx, OrderType::ascending()));
        in_order.insert(idx);
    }

    (pk, stream_key)
}