risingwave_frontend/optimizer/plan_node/
stream_exchange.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
// 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 pretty_xmlish::{Pretty, XmlNode};
use risingwave_pb::stream_plan::stream_node::NodeBody;
use risingwave_pb::stream_plan::{DispatchStrategy, DispatcherType, ExchangeNode};

use super::stream::prelude::*;
use super::utils::{childless_record, plan_node_name, Distill};
use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode};
use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
use crate::optimizer::property::{
    Distribution, DistributionDisplay, MonotonicityMap, RequiredDist,
};
use crate::stream_fragmenter::BuildFragmentGraphState;

/// `StreamExchange` imposes a particular distribution on its input
/// without changing its content.
#[derive(Debug, Clone, PartialEq, Eq, Hash)]
pub struct StreamExchange {
    pub base: PlanBase<Stream>,
    input: PlanRef,
    no_shuffle: bool,
}

impl StreamExchange {
    pub fn new(input: PlanRef, dist: Distribution) -> Self {
        let columns_monotonicity = if input.distribution().satisfies(&RequiredDist::single()) {
            // If the input is a singleton, the monotonicity will be preserved during shuffle
            // since we use ordered channel/buffer when exchanging data.
            input.columns_monotonicity().clone()
        } else {
            MonotonicityMap::new()
        };
        assert!(!input.schema().is_empty());
        let base = PlanBase::new_stream(
            input.ctx(),
            input.schema().clone(),
            input.stream_key().map(|v| v.to_vec()),
            input.functional_dependency().clone(),
            dist,
            input.append_only(), // append-only property won't change
            input.emit_on_window_close(),
            input.watermark_columns().clone(),
            columns_monotonicity,
        );
        StreamExchange {
            base,
            input,
            no_shuffle: false,
        }
    }

    pub fn new_no_shuffle(input: PlanRef) -> Self {
        let ctx = input.ctx();
        let base = PlanBase::new_stream(
            ctx,
            input.schema().clone(),
            input.stream_key().map(|v| v.to_vec()),
            input.functional_dependency().clone(),
            input.distribution().clone(),
            input.append_only(), // append-only property won't change
            input.emit_on_window_close(),
            input.watermark_columns().clone(),
            input.columns_monotonicity().clone(),
        );
        StreamExchange {
            base,
            input,
            no_shuffle: true,
        }
    }

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

impl Distill for StreamExchange {
    fn distill<'a>(&self) -> XmlNode<'a> {
        let distribution_display = DistributionDisplay {
            distribution: self.base.distribution(),
            input_schema: self.input.schema(),
        };
        childless_record(
            plan_node_name!(
                "StreamExchange",
                { "no_shuffle", self.no_shuffle },
            ),
            vec![("dist", Pretty::display(&distribution_display))],
        )
    }
}

impl PlanTreeNodeUnary for StreamExchange {
    fn input(&self) -> PlanRef {
        self.input.clone()
    }

    fn clone_with_input(&self, input: PlanRef) -> Self {
        if self.no_shuffle {
            Self::new_no_shuffle(input)
        } else {
            Self::new(input, self.distribution().clone())
        }
    }
}
impl_plan_tree_node_for_unary! {StreamExchange}

impl StreamNode for StreamExchange {
    fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> NodeBody {
        NodeBody::Exchange(ExchangeNode {
            strategy: if self.no_shuffle {
                Some(DispatchStrategy {
                    r#type: DispatcherType::NoShuffle as i32,
                    dist_key_indices: vec![],
                    output_indices: (0..self.schema().len() as u32).collect(),
                })
            } else {
                Some(DispatchStrategy {
                    r#type: match &self.base.distribution() {
                        Distribution::HashShard(_) => DispatcherType::Hash,
                        Distribution::Single => DispatcherType::Simple,
                        Distribution::Broadcast => DispatcherType::Broadcast,
                        _ => panic!("Do not allow Any or AnyShard in serialization process"),
                    } as i32,
                    dist_key_indices: match &self.base.distribution() {
                        Distribution::HashShard(keys) => {
                            keys.iter().map(|num| *num as u32).collect()
                        }
                        _ => vec![],
                    },
                    output_indices: (0..self.schema().len() as u32).collect(),
                })
            },
        })
    }
}

impl ExprRewritable for StreamExchange {}

impl ExprVisitable for StreamExchange {}