risingwave_frontend/stream_fragmenter/graph/
fragment_graph.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
// 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::HashMap;
use std::rc::Rc;

use risingwave_pb::stream_plan::stream_fragment_graph::{
    StreamFragment as StreamFragmentProto, StreamFragmentEdge as StreamFragmentEdgeProto,
};
use risingwave_pb::stream_plan::{
    DispatchStrategy, FragmentTypeFlag, StreamFragmentGraph as StreamFragmentGraphProto, StreamNode,
};
use thiserror_ext::AsReport;

pub type LocalFragmentId = u32;

/// [`StreamFragment`] represent a fragment node in fragment DAG.
#[derive(Clone, Debug)]
pub struct StreamFragment {
    /// the allocated fragment id.
    pub fragment_id: LocalFragmentId,

    /// root stream node in this fragment.
    pub node: Option<Box<StreamNode>>,

    /// Bitwise-OR of type Flags of this fragment.
    pub fragment_type_mask: u32,

    /// Mark whether this fragment requires exactly one actor.
    pub requires_singleton: bool,

    /// Number of table ids (stateful states) for this fragment.
    pub table_ids_cnt: u32,

    /// Mark the upstream table ids of this fragment.
    pub upstream_table_ids: Vec<u32>,
}

/// An edge between the nodes in the fragment graph.
#[derive(Debug, Clone)]
pub struct StreamFragmentEdge {
    /// Dispatch strategy for the fragment.
    pub dispatch_strategy: DispatchStrategy,

    /// A unique identifier of this edge. Generally it should be exchange node's operator id. When
    /// rewriting fragments into delta joins or when inserting 1-to-1 exchange, there will be
    /// virtual links generated.
    pub link_id: u64,
}

impl StreamFragment {
    pub fn new(fragment_id: LocalFragmentId) -> Self {
        Self {
            fragment_id,
            fragment_type_mask: FragmentTypeFlag::FragmentUnspecified as u32,
            requires_singleton: false,
            node: None,
            table_ids_cnt: 0,
            upstream_table_ids: vec![],
        }
    }

    pub fn to_protobuf(&self) -> StreamFragmentProto {
        StreamFragmentProto {
            fragment_id: self.fragment_id,
            node: self.node.clone().map(|n| *n),
            fragment_type_mask: self.fragment_type_mask,
            requires_singleton: self.requires_singleton,
            table_ids_cnt: self.table_ids_cnt,
            upstream_table_ids: self.upstream_table_ids.clone(),
        }
    }
}

/// [`StreamFragmentGraph`] stores a fragment graph (DAG).
#[derive(Default)]
pub struct StreamFragmentGraph {
    /// stores all the fragments in the graph.
    fragments: HashMap<LocalFragmentId, Rc<StreamFragment>>,

    /// stores edges between fragments: (upstream, downstream) => edge.
    edges: HashMap<(LocalFragmentId, LocalFragmentId), StreamFragmentEdgeProto>,
}

impl StreamFragmentGraph {
    pub fn to_protobuf(&self) -> StreamFragmentGraphProto {
        StreamFragmentGraphProto {
            fragments: self
                .fragments
                .iter()
                .map(|(k, v)| (*k, v.to_protobuf()))
                .collect(),
            edges: self.edges.values().cloned().collect(),

            // Following fields will be filled later in `build_graph` based on session context.
            ctx: None,
            dependent_table_ids: vec![],
            table_ids_cnt: 0,
            parallelism: None,
            max_parallelism: 0,
        }
    }

    /// Adds a fragment to the graph.
    pub fn add_fragment(&mut self, stream_fragment: Rc<StreamFragment>) {
        let id = stream_fragment.fragment_id;
        let ret = self.fragments.insert(id, stream_fragment);
        assert!(ret.is_none(), "fragment already exists: {:?}", id);
    }

    pub fn get_fragment(&self, fragment_id: &LocalFragmentId) -> Option<&Rc<StreamFragment>> {
        self.fragments.get(fragment_id)
    }

    /// Links upstream to downstream in the graph.
    pub fn add_edge(
        &mut self,
        upstream_id: LocalFragmentId,
        downstream_id: LocalFragmentId,
        edge: StreamFragmentEdge,
    ) {
        self.try_add_edge(upstream_id, downstream_id, edge).unwrap();
    }

    /// Try to link upstream to downstream in the graph.
    ///
    /// If the edge between upstream and downstream already exists, return an error.
    pub fn try_add_edge(
        &mut self,
        upstream_id: LocalFragmentId,
        downstream_id: LocalFragmentId,
        edge: StreamFragmentEdge,
    ) -> Result<(), String> {
        let edge = StreamFragmentEdgeProto {
            upstream_id,
            downstream_id,
            dispatch_strategy: Some(edge.dispatch_strategy),
            link_id: edge.link_id,
        };

        self.edges
            .try_insert((upstream_id, downstream_id), edge)
            .map(|_| ())
            .map_err(|e| {
                format!(
                    "edge between {} and {} already exists: {}",
                    upstream_id,
                    downstream_id,
                    e.to_report_string()
                )
            })
    }
}