risingwave_frontend/stream_fragmenter/graph/
fragment_graph.rs

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