Skip to main content

risingwave_frontend/stream_fragmenter/graph/
fragment_graph.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 std::collections::HashMap;
16use std::rc::Rc;
17
18use risingwave_common::catalog::FragmentTypeMask;
19use risingwave_pb::id::FragmentId;
20use risingwave_pb::stream_plan::stream_fragment_graph::{
21    StreamFragment as StreamFragmentProto, StreamFragmentEdge as StreamFragmentEdgeProto,
22};
23use risingwave_pb::stream_plan::{DispatchStrategy, StreamNode};
24
25pub type LocalFragmentId = FragmentId;
26
27/// [`StreamFragment`] represent a fragment node in fragment DAG.
28#[derive(Clone, Debug)]
29pub struct StreamFragment {
30    /// the allocated fragment id.
31    pub fragment_id: LocalFragmentId,
32
33    /// root stream node in this fragment.
34    pub node: Option<Box<StreamNode>>,
35
36    /// Bitwise-OR of type Flags of this fragment.
37    pub fragment_type_mask: FragmentTypeMask,
38
39    /// Mark whether this fragment requires exactly one actor.
40    pub requires_singleton: bool,
41}
42
43/// An edge between the nodes in the fragment graph.
44#[derive(Debug, Clone)]
45pub struct StreamFragmentEdge {
46    /// Dispatch strategy for the fragment.
47    pub dispatch_strategy: DispatchStrategy,
48
49    /// A unique identifier of this edge. Generally it should be exchange node's operator id. When
50    /// rewriting fragments into delta joins or when inserting 1-to-1 exchange, there will be
51    /// virtual links generated.
52    pub link_id: u64,
53}
54
55impl StreamFragment {
56    pub fn new(fragment_id: LocalFragmentId) -> Self {
57        Self {
58            fragment_id,
59            fragment_type_mask: FragmentTypeMask::empty(),
60            requires_singleton: false,
61            node: None,
62        }
63    }
64
65    pub fn to_protobuf(&self) -> StreamFragmentProto {
66        StreamFragmentProto {
67            fragment_id: self.fragment_id,
68            node: self.node.clone().map(|n| *n),
69            fragment_type_mask: self.fragment_type_mask.into(),
70            requires_singleton: self.requires_singleton,
71        }
72    }
73}
74
75/// [`StreamFragmentGraph`] stores a fragment graph (DAG).
76#[derive(Default)]
77pub struct StreamFragmentGraph {
78    /// stores all the fragments in the graph.
79    pub(in crate::stream_fragmenter) fragments: HashMap<LocalFragmentId, Rc<StreamFragment>>,
80
81    /// stores edges between fragments: (upstream, downstream) => edge.
82    pub(in crate::stream_fragmenter) edges:
83        HashMap<(LocalFragmentId, LocalFragmentId), StreamFragmentEdgeProto>,
84}
85
86impl StreamFragmentGraph {
87    /// Adds a fragment to the graph.
88    pub fn add_fragment(&mut self, stream_fragment: Rc<StreamFragment>) {
89        let id = stream_fragment.fragment_id;
90        let ret = self.fragments.insert(id, stream_fragment);
91        assert!(ret.is_none(), "fragment already exists: {:?}", id);
92    }
93
94    pub fn get_fragment(&self, fragment_id: &LocalFragmentId) -> Option<&Rc<StreamFragment>> {
95        self.fragments.get(fragment_id)
96    }
97
98    /// Links upstream to downstream in the graph.
99    pub fn add_edge(
100        &mut self,
101        upstream_id: LocalFragmentId,
102        downstream_id: LocalFragmentId,
103        edge: StreamFragmentEdge,
104    ) {
105        self.try_add_edge(upstream_id, downstream_id, edge).unwrap();
106    }
107
108    /// Try to link upstream to downstream in the graph.
109    ///
110    /// If the edge between upstream and downstream already exists, return an error.
111    pub fn try_add_edge(
112        &mut self,
113        upstream_id: LocalFragmentId,
114        downstream_id: LocalFragmentId,
115        edge: StreamFragmentEdge,
116    ) -> Result<(), String> {
117        let edge = StreamFragmentEdgeProto {
118            upstream_id,
119            downstream_id,
120            dispatch_strategy: Some(edge.dispatch_strategy),
121            link_id: edge.link_id,
122        };
123
124        self.edges
125            .try_insert((upstream_id, downstream_id), edge)
126            .map(|_| ())
127            .map_err(|e| {
128                format!(
129                    "edge between {upstream_id} and {downstream_id} already exists (existing: {:?})",
130                    e.entry.get(),
131                )
132            })
133    }
134}