1use std::collections::{BTreeMap, HashMap, HashSet};
16use std::num::NonZeroUsize;
17use std::ops::{Deref, DerefMut};
18use std::sync::LazyLock;
19use std::sync::atomic::AtomicU32;
20
21use anyhow::{Context, anyhow};
22use enum_as_inner::EnumAsInner;
23use itertools::Itertools;
24use risingwave_common::bail;
25use risingwave_common::catalog::{
26 CDC_SOURCE_COLUMN_NUM, ColumnCatalog, Field, FragmentTypeFlag, FragmentTypeMask, TableId,
27 generate_internal_table_name_with_type,
28};
29use risingwave_common::hash::VnodeCount;
30use risingwave_common::id::JobId;
31use risingwave_common::util::iter_util::ZipEqFast;
32use risingwave_common::util::stream_graph_visitor::{
33 self, visit_stream_node_cont, visit_stream_node_cont_mut,
34};
35use risingwave_connector::sink::catalog::SinkType;
36use risingwave_meta_model::WorkerId;
37use risingwave_pb::catalog::{PbSink, PbTable, Table};
38use risingwave_pb::ddl_service::TableJobType;
39use risingwave_pb::plan_common::{PbColumnCatalog, PbColumnDesc};
40use risingwave_pb::stream_plan::dispatch_output_mapping::TypePair;
41use risingwave_pb::stream_plan::stream_fragment_graph::{
42 Parallelism, StreamFragment, StreamFragmentEdge as StreamFragmentEdgeProto,
43};
44use risingwave_pb::stream_plan::stream_node::{NodeBody, PbNodeBody};
45use risingwave_pb::stream_plan::{
46 BackfillOrder, DispatchOutputMapping, DispatchStrategy, DispatcherType, PbStreamNode,
47 PbStreamScanType, StreamFragmentGraph as StreamFragmentGraphProto, StreamNode, StreamScanNode,
48 StreamScanType,
49};
50
51use crate::barrier::{SharedFragmentInfo, SnapshotBackfillInfo};
52use crate::controller::id::IdGeneratorManager;
53use crate::manager::{MetaSrvEnv, StreamingJob, StreamingJobType};
54use crate::model::{ActorId, Fragment, FragmentId, StreamActor};
55use crate::stream::stream_graph::id::{
56 GlobalActorIdGen, GlobalFragmentId, GlobalFragmentIdGen, GlobalTableIdGen,
57};
58use crate::stream::stream_graph::schedule::Distribution;
59use crate::{MetaError, MetaResult};
60
61#[derive(Debug, Clone)]
64pub(super) struct BuildingFragment {
65 inner: StreamFragment,
67
68 job_id: Option<JobId>,
70
71 upstream_job_columns: HashMap<JobId, Vec<PbColumnDesc>>,
76}
77
78impl BuildingFragment {
79 fn new(
82 id: GlobalFragmentId,
83 fragment: StreamFragment,
84 job: &StreamingJob,
85 table_id_gen: GlobalTableIdGen,
86 ) -> Self {
87 let mut fragment = StreamFragment {
88 fragment_id: id.as_global_id(),
89 ..fragment
90 };
91
92 Self::fill_internal_tables(&mut fragment, job, table_id_gen);
94
95 let job_id = Self::fill_job(&mut fragment, job).then(|| job.id());
96 let upstream_job_columns =
97 Self::extract_upstream_columns_except_cross_db_backfill(&fragment);
98
99 Self {
100 inner: fragment,
101 job_id,
102 upstream_job_columns,
103 }
104 }
105
106 fn extract_internal_tables(&self) -> Vec<Table> {
108 let mut fragment = self.inner.clone();
109 let mut tables = Vec::new();
110 stream_graph_visitor::visit_internal_tables(&mut fragment, |table, _| {
111 tables.push(table.clone());
112 });
113 tables
114 }
115
116 fn fill_internal_tables(
118 fragment: &mut StreamFragment,
119 job: &StreamingJob,
120 table_id_gen: GlobalTableIdGen,
121 ) {
122 let fragment_id = fragment.fragment_id;
123 stream_graph_visitor::visit_internal_tables(fragment, |table, table_type_name| {
124 table.id = table_id_gen
125 .to_global_id(table.id.as_raw_id())
126 .as_global_id();
127 table.schema_id = job.schema_id();
128 table.database_id = job.database_id();
129 table.name = generate_internal_table_name_with_type(
130 &job.name(),
131 fragment_id,
132 table.id,
133 table_type_name,
134 );
135 table.fragment_id = fragment_id;
136 table.owner = job.owner();
137 table.job_id = Some(job.id());
138 });
139 }
140
141 fn fill_job(fragment: &mut StreamFragment, job: &StreamingJob) -> bool {
143 let job_id = job.id();
144 let fragment_id = fragment.fragment_id;
145 let mut has_job = false;
146
147 stream_graph_visitor::visit_fragment_mut(fragment, |node_body| match node_body {
148 NodeBody::Materialize(materialize_node) => {
149 materialize_node.table_id = job_id.as_mv_table_id();
150
151 let table = materialize_node.table.insert(job.table().unwrap().clone());
153 table.fragment_id = fragment_id; if cfg!(not(debug_assertions)) {
156 table.definition = job.name();
157 }
158
159 has_job = true;
160 }
161 NodeBody::Sink(sink_node) => {
162 sink_node.sink_desc.as_mut().unwrap().id = job_id.as_sink_id();
163
164 has_job = true;
165 }
166 NodeBody::Dml(dml_node) => {
167 dml_node.table_id = job_id.as_mv_table_id();
168 dml_node.table_version_id = job.table_version_id().unwrap();
169 }
170 NodeBody::StreamFsFetch(fs_fetch_node) => {
171 if let StreamingJob::Table(table_source, _, _) = job
172 && let Some(node_inner) = fs_fetch_node.node_inner.as_mut()
173 && let Some(source) = table_source
174 {
175 node_inner.source_id = source.id;
176 if let Some(id) = source.optional_associated_table_id {
177 node_inner.associated_table_id = Some(id.into());
178 }
179 }
180 }
181 NodeBody::Source(source_node) => {
182 match job {
183 StreamingJob::Table(source, _table, _table_job_type) => {
186 if let Some(source_inner) = source_node.source_inner.as_mut()
187 && let Some(source) = source
188 {
189 debug_assert_ne!(source.id, job_id.as_raw_id());
190 source_inner.source_id = source.id;
191 if let Some(id) = source.optional_associated_table_id {
192 source_inner.associated_table_id = Some(id.into());
193 }
194 }
195 }
196 StreamingJob::Source(source) => {
197 has_job = true;
198 if let Some(source_inner) = source_node.source_inner.as_mut() {
199 debug_assert_eq!(source.id, job_id.as_raw_id());
200 source_inner.source_id = source.id;
201 if let Some(id) = source.optional_associated_table_id {
202 source_inner.associated_table_id = Some(id.into());
203 }
204 }
205 }
206 _ => {}
208 }
209 }
210 NodeBody::StreamCdcScan(node) => {
211 if let Some(table_desc) = node.cdc_table_desc.as_mut() {
212 table_desc.table_id = job_id.as_mv_table_id();
213 }
214 }
215 NodeBody::VectorIndexWrite(node) => {
216 let table = node.table.as_mut().unwrap();
217 table.id = job_id.as_mv_table_id();
218 table.database_id = job.database_id();
219 table.schema_id = job.schema_id();
220 table.fragment_id = fragment_id;
221 #[cfg(not(debug_assertions))]
222 {
223 table.definition = job.name();
224 }
225
226 has_job = true;
227 }
228 _ => {}
229 });
230
231 has_job
232 }
233
234 fn extract_upstream_columns_except_cross_db_backfill(
236 fragment: &StreamFragment,
237 ) -> HashMap<JobId, Vec<PbColumnDesc>> {
238 let mut table_columns = HashMap::new();
239
240 stream_graph_visitor::visit_fragment(fragment, |node_body| {
241 let (table_id, column_ids) = match node_body {
242 NodeBody::StreamScan(stream_scan) => {
243 if stream_scan.get_stream_scan_type().unwrap()
244 == StreamScanType::CrossDbSnapshotBackfill
245 {
246 return;
247 }
248 (
249 stream_scan.table_id.as_job_id(),
250 stream_scan.upstream_columns(),
251 )
252 }
253 NodeBody::CdcFilter(cdc_filter) => (
254 cdc_filter.upstream_source_id.as_share_source_job_id(),
255 vec![],
256 ),
257 NodeBody::SourceBackfill(backfill) => (
258 backfill.upstream_source_id.as_share_source_job_id(),
259 backfill.column_descs(),
261 ),
262 _ => return,
263 };
264 table_columns
265 .try_insert(table_id, column_ids)
266 .expect("currently there should be no two same upstream tables in a fragment");
267 });
268
269 table_columns
270 }
271
272 pub fn has_shuffled_backfill(&self) -> bool {
273 let stream_node = match self.inner.node.as_ref() {
274 Some(node) => node,
275 _ => return false,
276 };
277 let mut has_shuffled_backfill = false;
278 let has_shuffled_backfill_mut_ref = &mut has_shuffled_backfill;
279 visit_stream_node_cont(stream_node, |node| {
280 let is_shuffled_backfill = if let Some(node) = &node.node_body
281 && let Some(node) = node.as_stream_scan()
282 {
283 node.stream_scan_type == StreamScanType::ArrangementBackfill as i32
284 || node.stream_scan_type == StreamScanType::SnapshotBackfill as i32
285 } else {
286 false
287 };
288 if is_shuffled_backfill {
289 *has_shuffled_backfill_mut_ref = true;
290 false
291 } else {
292 true
293 }
294 });
295 has_shuffled_backfill
296 }
297}
298
299impl Deref for BuildingFragment {
300 type Target = StreamFragment;
301
302 fn deref(&self) -> &Self::Target {
303 &self.inner
304 }
305}
306
307impl DerefMut for BuildingFragment {
308 fn deref_mut(&mut self) -> &mut Self::Target {
309 &mut self.inner
310 }
311}
312
313#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, EnumAsInner)]
316pub(super) enum EdgeId {
317 Internal {
319 link_id: u64,
322 },
323
324 UpstreamExternal {
327 upstream_job_id: JobId,
329 downstream_fragment_id: GlobalFragmentId,
331 },
332
333 DownstreamExternal(DownstreamExternalEdgeId),
336}
337
338#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)]
339pub(super) struct DownstreamExternalEdgeId {
340 pub(super) original_upstream_fragment_id: GlobalFragmentId,
342 pub(super) downstream_fragment_id: GlobalFragmentId,
344}
345
346#[derive(Debug, Clone)]
350pub(super) struct StreamFragmentEdge {
351 pub id: EdgeId,
353
354 pub dispatch_strategy: DispatchStrategy,
356}
357
358impl StreamFragmentEdge {
359 fn from_protobuf(edge: &StreamFragmentEdgeProto) -> Self {
360 Self {
361 id: EdgeId::Internal {
364 link_id: edge.link_id,
365 },
366 dispatch_strategy: edge.get_dispatch_strategy().unwrap().clone(),
367 }
368 }
369}
370
371fn clone_fragment(
372 fragment: &Fragment,
373 id_generator_manager: &IdGeneratorManager,
374 actor_id_counter: &AtomicU32,
375) -> Fragment {
376 let fragment_id = GlobalFragmentIdGen::new(id_generator_manager, 1)
377 .to_global_id(0)
378 .as_global_id();
379 let actor_id_gen = GlobalActorIdGen::new(actor_id_counter, fragment.actors.len() as _);
380 Fragment {
381 fragment_id,
382 fragment_type_mask: fragment.fragment_type_mask,
383 distribution_type: fragment.distribution_type,
384 actors: fragment
385 .actors
386 .iter()
387 .enumerate()
388 .map(|(i, actor)| StreamActor {
389 actor_id: actor_id_gen.to_global_id(i as _).as_global_id(),
390 fragment_id,
391 vnode_bitmap: actor.vnode_bitmap.clone(),
392 mview_definition: actor.mview_definition.clone(),
393 expr_context: actor.expr_context.clone(),
394 })
395 .collect(),
396 state_table_ids: fragment.state_table_ids.clone(),
397 maybe_vnode_count: fragment.maybe_vnode_count,
398 nodes: fragment.nodes.clone(),
399 }
400}
401
402pub fn check_sink_fragments_support_refresh_schema(
403 fragments: &BTreeMap<FragmentId, Fragment>,
404) -> MetaResult<()> {
405 if fragments.len() != 1 {
406 return Err(anyhow!(
407 "sink with auto schema change should have only 1 fragment, but got {:?}",
408 fragments.len()
409 )
410 .into());
411 }
412 let (_, fragment) = fragments.first_key_value().expect("non-empty");
413 let sink_node = &fragment.nodes;
414 let PbNodeBody::Sink(_) = sink_node.node_body.as_ref().unwrap() else {
415 return Err(anyhow!("expect PbNodeBody::Sink but got: {:?}", sink_node.node_body).into());
416 };
417 let [stream_scan_node] = sink_node.input.as_slice() else {
418 panic!("Sink has more than 1 input: {:?}", sink_node.input);
419 };
420 let PbNodeBody::StreamScan(scan) = stream_scan_node.node_body.as_ref().unwrap() else {
421 return Err(anyhow!(
422 "expect PbNodeBody::StreamScan but got: {:?}",
423 stream_scan_node.node_body
424 )
425 .into());
426 };
427 let stream_scan_type = PbStreamScanType::try_from(scan.stream_scan_type).unwrap();
428 if stream_scan_type != PbStreamScanType::ArrangementBackfill {
429 return Err(anyhow!(
430 "unsupported stream_scan_type for auto refresh schema: {:?}",
431 stream_scan_type
432 )
433 .into());
434 }
435 let [merge_node, _batch_plan_node] = stream_scan_node.input.as_slice() else {
436 panic!(
437 "the number of StreamScan inputs is not 2: {:?}",
438 stream_scan_node.input
439 );
440 };
441 let NodeBody::Merge(_) = merge_node.node_body.as_ref().unwrap() else {
442 return Err(anyhow!(
443 "expect PbNodeBody::Merge but got: {:?}",
444 merge_node.node_body
445 )
446 .into());
447 };
448 Ok(())
449}
450
451pub fn rewrite_refresh_schema_sink_fragment(
452 original_sink_fragment: &Fragment,
453 sink: &PbSink,
454 newly_added_columns: &[ColumnCatalog],
455 upstream_table: &PbTable,
456 upstream_table_fragment_id: FragmentId,
457 id_generator_manager: &IdGeneratorManager,
458 actor_id_counter: &AtomicU32,
459) -> MetaResult<(Fragment, Vec<PbColumnCatalog>, Option<PbTable>)> {
460 let mut new_sink_columns = sink.columns.clone();
461 fn extend_sink_columns(
462 sink_columns: &mut Vec<PbColumnCatalog>,
463 new_columns: &[ColumnCatalog],
464 get_column_name: impl Fn(&String) -> String,
465 ) {
466 let next_column_id = sink_columns
467 .iter()
468 .map(|col| col.column_desc.as_ref().unwrap().column_id + 1)
469 .max()
470 .unwrap_or(1);
471 sink_columns.extend(new_columns.iter().enumerate().map(|(i, col)| {
472 let mut col = col.to_protobuf();
473 let column_desc = col.column_desc.as_mut().unwrap();
474 column_desc.column_id = next_column_id + (i as i32);
475 column_desc.name = get_column_name(&column_desc.name);
476 col
477 }));
478 }
479 extend_sink_columns(&mut new_sink_columns, newly_added_columns, |name| {
480 name.clone()
481 });
482
483 let mut new_sink_fragment = clone_fragment(
484 original_sink_fragment,
485 id_generator_manager,
486 actor_id_counter,
487 );
488 let sink_node = &mut new_sink_fragment.nodes;
489 let PbNodeBody::Sink(sink_node_body) = sink_node.node_body.as_mut().unwrap() else {
490 return Err(anyhow!("expect PbNodeBody::Sink but got: {:?}", sink_node.node_body).into());
491 };
492 let [stream_scan_node] = sink_node.input.as_mut_slice() else {
493 panic!("Sink has more than 1 input: {:?}", sink_node.input);
494 };
495 let PbNodeBody::StreamScan(scan) = stream_scan_node.node_body.as_mut().unwrap() else {
496 return Err(anyhow!(
497 "expect PbNodeBody::StreamScan but got: {:?}",
498 stream_scan_node.node_body
499 )
500 .into());
501 };
502 let [merge_node, _batch_plan_node] = stream_scan_node.input.as_mut_slice() else {
503 panic!(
504 "the number of StreamScan inputs is not 2: {:?}",
505 stream_scan_node.input
506 );
507 };
508 let NodeBody::Merge(merge) = merge_node.node_body.as_mut().unwrap() else {
509 return Err(anyhow!(
510 "expect PbNodeBody::Merge but got: {:?}",
511 merge_node.node_body
512 )
513 .into());
514 };
515 sink_node.identity = {
518 let sink_type = SinkType::from_proto(sink.sink_type());
519 let sink_type_str = sink_type.type_str();
520 let column_names = new_sink_columns
521 .iter()
522 .map(|col| {
523 ColumnCatalog::from(col.clone())
524 .name_with_hidden()
525 .to_string()
526 })
527 .join(", ");
528 let downstream_pk = if !sink_type.is_append_only() {
529 let downstream_pk = sink
530 .downstream_pk
531 .iter()
532 .map(|i| &sink.columns[*i as usize].column_desc.as_ref().unwrap().name)
533 .collect_vec();
534 format!(", downstream_pk: {downstream_pk:?}")
535 } else {
536 "".to_owned()
537 };
538 format!("StreamSink {{ type: {sink_type_str}, columns: [{column_names}]{downstream_pk} }}")
539 };
540 sink_node
541 .fields
542 .extend(newly_added_columns.iter().map(|col| {
543 Field::new(
544 format!("{}.{}", upstream_table.name, col.column_desc.name),
545 col.data_type().clone(),
546 )
547 .to_prost()
548 }));
549
550 let new_log_store_table = if let Some(log_store_table) = &mut sink_node_body.table {
551 extend_sink_columns(&mut log_store_table.columns, newly_added_columns, |name| {
552 format!("{}_{}", upstream_table.name, name)
553 });
554 Some(log_store_table.clone())
555 } else {
556 None
557 };
558 sink_node_body.sink_desc.as_mut().unwrap().column_catalogs = new_sink_columns.clone();
559
560 stream_scan_node
562 .fields
563 .extend(newly_added_columns.iter().map(|col| {
564 Field::new(
565 format!("{}.{}", upstream_table.name, col.column_desc.name),
566 col.data_type().clone(),
567 )
568 .to_prost()
569 }));
570 stream_scan_node.identity = {
572 let columns = stream_scan_node
573 .fields
574 .iter()
575 .map(|col| &col.name)
576 .join(", ");
577 format!("StreamTableScan {{ table: t, columns: [{columns}] }}")
578 };
579
580 let stream_scan_type = PbStreamScanType::try_from(scan.stream_scan_type).unwrap();
581 if stream_scan_type != PbStreamScanType::ArrangementBackfill {
582 return Err(anyhow!(
583 "unsupported stream_scan_type for auto refresh schema: {:?}",
584 stream_scan_type
585 )
586 .into());
587 }
588 scan.arrangement_table = Some(upstream_table.clone());
589 scan.output_indices.extend(
590 (0..newly_added_columns.len()).map(|i| (i + scan.upstream_column_ids.len()) as u32),
591 );
592 scan.upstream_column_ids.extend(
593 newly_added_columns
594 .iter()
595 .map(|col| col.column_id().get_id()),
596 );
597 let table_desc = scan.table_desc.as_mut().unwrap();
598 table_desc
599 .value_indices
600 .extend((0..newly_added_columns.len()).map(|i| (i + table_desc.columns.len()) as u32));
601 table_desc.columns.extend(
602 newly_added_columns
603 .iter()
604 .map(|col| col.column_desc.to_protobuf()),
605 );
606
607 merge_node.fields = scan
609 .upstream_column_ids
610 .iter()
611 .map(|&column_id| {
612 let col = upstream_table
613 .columns
614 .iter()
615 .find(|c| c.column_desc.as_ref().unwrap().column_id == column_id)
616 .unwrap();
617 let col_desc = col.column_desc.as_ref().unwrap();
618 Field::new(
619 col_desc.name.clone(),
620 col_desc.column_type.as_ref().unwrap().into(),
621 )
622 .to_prost()
623 })
624 .collect();
625 merge.upstream_fragment_id = upstream_table_fragment_id;
626 Ok((new_sink_fragment, new_sink_columns, new_log_store_table))
627}
628
629pub type FragmentBackfillOrder = HashMap<FragmentId, Vec<FragmentId>>;
634
635#[derive(Default, Debug)]
642pub struct StreamFragmentGraph {
643 pub(super) fragments: HashMap<GlobalFragmentId, BuildingFragment>,
645
646 pub(super) downstreams:
648 HashMap<GlobalFragmentId, HashMap<GlobalFragmentId, StreamFragmentEdge>>,
649
650 pub(super) upstreams: HashMap<GlobalFragmentId, HashMap<GlobalFragmentId, StreamFragmentEdge>>,
652
653 dependent_table_ids: HashSet<TableId>,
655
656 specified_parallelism: Option<NonZeroUsize>,
659
660 max_parallelism: usize,
670
671 backfill_order: BackfillOrder,
673}
674
675impl StreamFragmentGraph {
676 pub fn new(
679 env: &MetaSrvEnv,
680 proto: StreamFragmentGraphProto,
681 job: &StreamingJob,
682 ) -> MetaResult<Self> {
683 let fragment_id_gen =
684 GlobalFragmentIdGen::new(env.id_gen_manager(), proto.fragments.len() as u64);
685 let table_id_gen = GlobalTableIdGen::new(env.id_gen_manager(), proto.table_ids_cnt as u64);
689
690 let fragments: HashMap<_, _> = proto
692 .fragments
693 .into_iter()
694 .map(|(id, fragment)| {
695 let id = fragment_id_gen.to_global_id(id.as_raw_id());
696 let fragment = BuildingFragment::new(id, fragment, job, table_id_gen);
697 (id, fragment)
698 })
699 .collect();
700
701 assert_eq!(
702 fragments
703 .values()
704 .map(|f| f.extract_internal_tables().len() as u32)
705 .sum::<u32>(),
706 proto.table_ids_cnt
707 );
708
709 let mut downstreams = HashMap::new();
711 let mut upstreams = HashMap::new();
712
713 for edge in proto.edges {
714 let upstream_id = fragment_id_gen.to_global_id(edge.upstream_id.as_raw_id());
715 let downstream_id = fragment_id_gen.to_global_id(edge.downstream_id.as_raw_id());
716 let edge = StreamFragmentEdge::from_protobuf(&edge);
717
718 upstreams
719 .entry(downstream_id)
720 .or_insert_with(HashMap::new)
721 .try_insert(upstream_id, edge.clone())
722 .unwrap();
723 downstreams
724 .entry(upstream_id)
725 .or_insert_with(HashMap::new)
726 .try_insert(downstream_id, edge)
727 .unwrap();
728 }
729
730 let dependent_table_ids = proto.dependent_table_ids.iter().copied().collect();
733
734 let specified_parallelism = if let Some(Parallelism { parallelism }) = proto.parallelism {
735 Some(NonZeroUsize::new(parallelism as usize).context("parallelism should not be 0")?)
736 } else {
737 None
738 };
739
740 let max_parallelism = proto.max_parallelism as usize;
741 let backfill_order = proto.backfill_order.unwrap_or(BackfillOrder {
742 order: Default::default(),
743 });
744
745 Ok(Self {
746 fragments,
747 downstreams,
748 upstreams,
749 dependent_table_ids,
750 specified_parallelism,
751 max_parallelism,
752 backfill_order,
753 })
754 }
755
756 pub fn incomplete_internal_tables(&self) -> BTreeMap<TableId, Table> {
762 let mut tables = BTreeMap::new();
763 for fragment in self.fragments.values() {
764 for table in fragment.extract_internal_tables() {
765 let table_id = table.id;
766 tables
767 .try_insert(table_id, table)
768 .unwrap_or_else(|_| panic!("duplicated table id `{}`", table_id));
769 }
770 }
771 tables
772 }
773
774 pub fn refill_internal_table_ids(&mut self, table_id_map: HashMap<TableId, TableId>) {
777 for fragment in self.fragments.values_mut() {
778 stream_graph_visitor::visit_internal_tables(
779 &mut fragment.inner,
780 |table, _table_type_name| {
781 let target = table_id_map.get(&table.id).cloned().unwrap();
782 table.id = target;
783 },
784 );
785 }
786 }
787
788 pub fn fit_internal_tables_trivial(
791 &mut self,
792 mut old_internal_tables: Vec<Table>,
793 ) -> MetaResult<()> {
794 let mut new_internal_table_ids = Vec::new();
795 for fragment in self.fragments.values() {
796 for table in &fragment.extract_internal_tables() {
797 new_internal_table_ids.push(table.id);
798 }
799 }
800
801 if new_internal_table_ids.len() != old_internal_tables.len() {
802 bail!(
803 "Different number of internal tables. New: {}, Old: {}",
804 new_internal_table_ids.len(),
805 old_internal_tables.len()
806 );
807 }
808 old_internal_tables.sort_by(|a, b| a.id.cmp(&b.id));
809 new_internal_table_ids.sort();
810
811 let internal_table_id_map = new_internal_table_ids
812 .into_iter()
813 .zip_eq_fast(old_internal_tables.into_iter())
814 .collect::<HashMap<_, _>>();
815
816 for fragment in self.fragments.values_mut() {
819 stream_graph_visitor::visit_internal_tables(
820 &mut fragment.inner,
821 |table, _table_type_name| {
822 let target = internal_table_id_map.get(&table.id).cloned().unwrap();
824 *table = target;
825 },
826 );
827 }
828
829 Ok(())
830 }
831
832 pub fn fit_internal_table_ids_with_mapping(&mut self, mut matches: HashMap<TableId, Table>) {
834 for fragment in self.fragments.values_mut() {
835 stream_graph_visitor::visit_internal_tables(
836 &mut fragment.inner,
837 |table, _table_type_name| {
838 let target = matches.remove(&table.id).unwrap_or_else(|| {
839 panic!("no matching table for table {}({})", table.id, table.name)
840 });
841 table.id = target.id;
842 table.maybe_vnode_count = target.maybe_vnode_count;
843 },
844 );
845 }
846 }
847
848 pub fn table_fragment_id(&self) -> FragmentId {
850 self.fragments
851 .values()
852 .filter(|b| b.job_id.is_some())
853 .map(|b| b.fragment_id)
854 .exactly_one()
855 .expect("require exactly 1 materialize/sink/cdc source node when creating the streaming job")
856 }
857
858 pub fn dml_fragment_id(&self) -> Option<FragmentId> {
860 self.fragments
861 .values()
862 .filter(|b| {
863 FragmentTypeMask::from(b.fragment_type_mask).contains(FragmentTypeFlag::Dml)
864 })
865 .map(|b| b.fragment_id)
866 .at_most_one()
867 .expect("require at most 1 dml node when creating the streaming job")
868 }
869
870 pub fn dependent_table_ids(&self) -> &HashSet<TableId> {
872 &self.dependent_table_ids
873 }
874
875 pub fn specified_parallelism(&self) -> Option<NonZeroUsize> {
877 self.specified_parallelism
878 }
879
880 pub fn max_parallelism(&self) -> usize {
882 self.max_parallelism
883 }
884
885 fn get_downstreams(
887 &self,
888 fragment_id: GlobalFragmentId,
889 ) -> &HashMap<GlobalFragmentId, StreamFragmentEdge> {
890 self.downstreams.get(&fragment_id).unwrap_or(&EMPTY_HASHMAP)
891 }
892
893 fn get_upstreams(
895 &self,
896 fragment_id: GlobalFragmentId,
897 ) -> &HashMap<GlobalFragmentId, StreamFragmentEdge> {
898 self.upstreams.get(&fragment_id).unwrap_or(&EMPTY_HASHMAP)
899 }
900
901 pub fn collect_snapshot_backfill_info(
902 &self,
903 ) -> MetaResult<(Option<SnapshotBackfillInfo>, SnapshotBackfillInfo)> {
904 Self::collect_snapshot_backfill_info_impl(self.fragments.values().map(|fragment| {
905 (
906 fragment.node.as_ref().unwrap(),
907 fragment.fragment_type_mask.into(),
908 )
909 }))
910 }
911
912 pub fn collect_snapshot_backfill_info_impl(
914 fragments: impl IntoIterator<Item = (&PbStreamNode, FragmentTypeMask)>,
915 ) -> MetaResult<(Option<SnapshotBackfillInfo>, SnapshotBackfillInfo)> {
916 let mut prev_stream_scan: Option<(Option<SnapshotBackfillInfo>, StreamScanNode)> = None;
917 let mut cross_db_info = SnapshotBackfillInfo {
918 upstream_mv_table_id_to_backfill_epoch: Default::default(),
919 };
920 let mut result = Ok(());
921 for (node, fragment_type_mask) in fragments {
922 visit_stream_node_cont(node, |node| {
923 if let Some(NodeBody::StreamScan(stream_scan)) = node.node_body.as_ref() {
924 let stream_scan_type = StreamScanType::try_from(stream_scan.stream_scan_type)
925 .expect("invalid stream_scan_type");
926 let is_snapshot_backfill = match stream_scan_type {
927 StreamScanType::SnapshotBackfill => {
928 assert!(
929 fragment_type_mask
930 .contains(FragmentTypeFlag::SnapshotBackfillStreamScan)
931 );
932 true
933 }
934 StreamScanType::CrossDbSnapshotBackfill => {
935 assert!(
936 fragment_type_mask
937 .contains(FragmentTypeFlag::CrossDbSnapshotBackfillStreamScan)
938 );
939 cross_db_info
940 .upstream_mv_table_id_to_backfill_epoch
941 .insert(stream_scan.table_id, stream_scan.snapshot_backfill_epoch);
942
943 return true;
944 }
945 _ => false,
946 };
947
948 match &mut prev_stream_scan {
949 Some((prev_snapshot_backfill_info, prev_stream_scan)) => {
950 match (prev_snapshot_backfill_info, is_snapshot_backfill) {
951 (Some(prev_snapshot_backfill_info), true) => {
952 prev_snapshot_backfill_info
953 .upstream_mv_table_id_to_backfill_epoch
954 .insert(
955 stream_scan.table_id,
956 stream_scan.snapshot_backfill_epoch,
957 );
958 true
959 }
960 (None, false) => true,
961 (_, _) => {
962 result = Err(anyhow!("must be either all snapshot_backfill or no snapshot_backfill. Curr: {stream_scan:?} Prev: {prev_stream_scan:?}").into());
963 false
964 }
965 }
966 }
967 None => {
968 prev_stream_scan = Some((
969 if is_snapshot_backfill {
970 Some(SnapshotBackfillInfo {
971 upstream_mv_table_id_to_backfill_epoch: HashMap::from_iter(
972 [(
973 stream_scan.table_id,
974 stream_scan.snapshot_backfill_epoch,
975 )],
976 ),
977 })
978 } else {
979 None
980 },
981 *stream_scan.clone(),
982 ));
983 true
984 }
985 }
986 } else {
987 true
988 }
989 })
990 }
991 result.map(|_| {
992 (
993 prev_stream_scan
994 .map(|(snapshot_backfill_info, _)| snapshot_backfill_info)
995 .unwrap_or(None),
996 cross_db_info,
997 )
998 })
999 }
1000
1001 pub fn collect_backfill_mapping(&self) -> HashMap<u32, Vec<FragmentId>> {
1003 let mut mapping = HashMap::new();
1004 for (fragment_id, fragment) in &self.fragments {
1005 let fragment_id = fragment_id.as_global_id();
1006 let fragment_mask = fragment.fragment_type_mask;
1007 let candidates = [FragmentTypeFlag::StreamScan, FragmentTypeFlag::SourceScan];
1008 let has_some_scan = candidates
1009 .into_iter()
1010 .any(|flag| (fragment_mask & flag as u32) > 0);
1011 if has_some_scan {
1012 visit_stream_node_cont(fragment.node.as_ref().unwrap(), |node| {
1013 match node.node_body.as_ref() {
1014 Some(NodeBody::StreamScan(stream_scan)) => {
1015 let table_id = stream_scan.table_id;
1016 let fragments: &mut Vec<_> =
1017 mapping.entry(table_id.as_raw_id()).or_default();
1018 fragments.push(fragment_id);
1019 false
1021 }
1022 Some(NodeBody::SourceBackfill(source_backfill)) => {
1023 let source_id = source_backfill.upstream_source_id;
1024 let fragments: &mut Vec<_> =
1025 mapping.entry(source_id.as_raw_id()).or_default();
1026 fragments.push(fragment_id);
1027 false
1029 }
1030 _ => true,
1031 }
1032 })
1033 }
1034 }
1035 mapping
1036 }
1037
1038 pub fn create_fragment_backfill_ordering(&self) -> FragmentBackfillOrder {
1042 let mapping = self.collect_backfill_mapping();
1043 let mut fragment_ordering: HashMap<FragmentId, Vec<FragmentId>> = HashMap::new();
1044
1045 for (rel_id, downstream_rel_ids) in &self.backfill_order.order {
1047 let fragment_ids = mapping.get(rel_id).unwrap();
1048 for fragment_id in fragment_ids {
1049 let downstream_fragment_ids = downstream_rel_ids
1050 .data
1051 .iter()
1052 .flat_map(|downstream_rel_id| mapping.get(downstream_rel_id).unwrap().iter())
1053 .copied()
1054 .collect();
1055 fragment_ordering.insert(*fragment_id, downstream_fragment_ids);
1056 }
1057 }
1058
1059 if fragment_ordering.is_empty() {
1062 for value in mapping.values() {
1063 for &fragment_id in value {
1064 fragment_ordering.entry(fragment_id).or_default();
1065 }
1066 }
1067 }
1068
1069 let locality_provider_dependencies = self.find_locality_provider_dependencies();
1071
1072 let backfill_fragments: HashSet<FragmentId> = mapping.values().flatten().copied().collect();
1073
1074 let all_locality_provider_fragments: HashSet<FragmentId> =
1077 locality_provider_dependencies.keys().copied().collect();
1078 let downstream_locality_provider_fragments: HashSet<FragmentId> =
1079 locality_provider_dependencies
1080 .values()
1081 .flatten()
1082 .copied()
1083 .collect();
1084 let locality_provider_root_fragments: Vec<FragmentId> = all_locality_provider_fragments
1085 .difference(&downstream_locality_provider_fragments)
1086 .copied()
1087 .collect();
1088
1089 for &backfill_fragment_id in &backfill_fragments {
1092 fragment_ordering
1093 .entry(backfill_fragment_id)
1094 .or_default()
1095 .extend(locality_provider_root_fragments.iter().copied());
1096 }
1097
1098 for (fragment_id, downstream_fragments) in locality_provider_dependencies {
1100 fragment_ordering
1101 .entry(fragment_id)
1102 .or_default()
1103 .extend(downstream_fragments);
1104 }
1105
1106 fragment_ordering
1107 }
1108
1109 pub fn find_locality_provider_fragment_state_table_mapping(
1110 &self,
1111 ) -> HashMap<FragmentId, Vec<TableId>> {
1112 let mut mapping: HashMap<FragmentId, Vec<TableId>> = HashMap::new();
1113
1114 for (fragment_id, fragment) in &self.fragments {
1115 let fragment_id = fragment_id.as_global_id();
1116
1117 if let Some(node) = fragment.node.as_ref() {
1119 let mut state_table_ids = Vec::new();
1120
1121 visit_stream_node_cont(node, |stream_node| {
1122 if let Some(NodeBody::LocalityProvider(locality_provider)) =
1123 stream_node.node_body.as_ref()
1124 {
1125 let state_table_id = locality_provider
1127 .state_table
1128 .as_ref()
1129 .expect("must have state table")
1130 .id;
1131 state_table_ids.push(state_table_id);
1132 false } else {
1134 true }
1136 });
1137
1138 if !state_table_ids.is_empty() {
1139 mapping.insert(fragment_id, state_table_ids);
1140 }
1141 }
1142 }
1143
1144 mapping
1145 }
1146
1147 pub fn find_locality_provider_dependencies(&self) -> HashMap<FragmentId, Vec<FragmentId>> {
1155 let mut locality_provider_fragments = HashSet::new();
1156 let mut dependencies: HashMap<FragmentId, Vec<FragmentId>> = HashMap::new();
1157
1158 for (fragment_id, fragment) in &self.fragments {
1160 let fragment_id = fragment_id.as_global_id();
1161 let has_locality_provider = self.fragment_has_locality_provider(fragment);
1162
1163 if has_locality_provider {
1164 locality_provider_fragments.insert(fragment_id);
1165 dependencies.entry(fragment_id).or_default();
1166 }
1167 }
1168
1169 for &provider_fragment_id in &locality_provider_fragments {
1173 let provider_fragment_global_id = GlobalFragmentId::new(provider_fragment_id);
1174
1175 let mut visited = HashSet::new();
1177 let mut downstream_locality_providers = Vec::new();
1178
1179 self.collect_downstream_locality_providers(
1180 provider_fragment_global_id,
1181 &locality_provider_fragments,
1182 &mut visited,
1183 &mut downstream_locality_providers,
1184 );
1185
1186 dependencies
1188 .entry(provider_fragment_id)
1189 .or_default()
1190 .extend(downstream_locality_providers);
1191 }
1192
1193 dependencies
1194 }
1195
1196 fn fragment_has_locality_provider(&self, fragment: &BuildingFragment) -> bool {
1197 let mut has_locality_provider = false;
1198
1199 if let Some(node) = fragment.node.as_ref() {
1200 visit_stream_node_cont(node, |stream_node| {
1201 if let Some(NodeBody::LocalityProvider(_)) = stream_node.node_body.as_ref() {
1202 has_locality_provider = true;
1203 false } else {
1205 true }
1207 });
1208 }
1209
1210 has_locality_provider
1211 }
1212
1213 fn collect_downstream_locality_providers(
1215 &self,
1216 current_fragment_id: GlobalFragmentId,
1217 locality_provider_fragments: &HashSet<FragmentId>,
1218 visited: &mut HashSet<GlobalFragmentId>,
1219 downstream_providers: &mut Vec<FragmentId>,
1220 ) {
1221 if visited.contains(¤t_fragment_id) {
1222 return;
1223 }
1224 visited.insert(current_fragment_id);
1225
1226 for &downstream_id in self.get_downstreams(current_fragment_id).keys() {
1228 let downstream_fragment_id = downstream_id.as_global_id();
1229
1230 if locality_provider_fragments.contains(&downstream_fragment_id) {
1232 downstream_providers.push(downstream_fragment_id);
1233 }
1234
1235 self.collect_downstream_locality_providers(
1237 downstream_id,
1238 locality_provider_fragments,
1239 visited,
1240 downstream_providers,
1241 );
1242 }
1243 }
1244}
1245
1246pub fn fill_snapshot_backfill_epoch(
1249 node: &mut StreamNode,
1250 snapshot_backfill_info: Option<&SnapshotBackfillInfo>,
1251 cross_db_snapshot_backfill_info: &SnapshotBackfillInfo,
1252) -> MetaResult<bool> {
1253 let mut result = Ok(());
1254 let mut applied = false;
1255 visit_stream_node_cont_mut(node, |node| {
1256 if let Some(NodeBody::StreamScan(stream_scan)) = node.node_body.as_mut()
1257 && (stream_scan.stream_scan_type == StreamScanType::SnapshotBackfill as i32
1258 || stream_scan.stream_scan_type == StreamScanType::CrossDbSnapshotBackfill as i32)
1259 {
1260 result = try {
1261 let table_id = stream_scan.table_id;
1262 let snapshot_epoch = cross_db_snapshot_backfill_info
1263 .upstream_mv_table_id_to_backfill_epoch
1264 .get(&table_id)
1265 .or_else(|| {
1266 snapshot_backfill_info.and_then(|snapshot_backfill_info| {
1267 snapshot_backfill_info
1268 .upstream_mv_table_id_to_backfill_epoch
1269 .get(&table_id)
1270 })
1271 })
1272 .ok_or_else(|| anyhow!("upstream table id not covered: {}", table_id))?
1273 .ok_or_else(|| anyhow!("upstream table id not set: {}", table_id))?;
1274 if let Some(prev_snapshot_epoch) =
1275 stream_scan.snapshot_backfill_epoch.replace(snapshot_epoch)
1276 {
1277 Err(anyhow!(
1278 "snapshot backfill epoch set again: {} {} {}",
1279 table_id,
1280 prev_snapshot_epoch,
1281 snapshot_epoch
1282 ))?;
1283 }
1284 applied = true;
1285 };
1286 result.is_ok()
1287 } else {
1288 true
1289 }
1290 });
1291 result.map(|_| applied)
1292}
1293
1294static EMPTY_HASHMAP: LazyLock<HashMap<GlobalFragmentId, StreamFragmentEdge>> =
1295 LazyLock::new(HashMap::new);
1296
1297#[derive(Debug, Clone, EnumAsInner)]
1300pub(super) enum EitherFragment {
1301 Building(BuildingFragment),
1303
1304 Existing(SharedFragmentInfo),
1306}
1307
1308#[derive(Debug)]
1317pub struct CompleteStreamFragmentGraph {
1318 building_graph: StreamFragmentGraph,
1320
1321 existing_fragments: HashMap<GlobalFragmentId, SharedFragmentInfo>,
1323
1324 existing_actor_location: HashMap<ActorId, WorkerId>,
1326
1327 extra_downstreams: HashMap<GlobalFragmentId, HashMap<GlobalFragmentId, StreamFragmentEdge>>,
1329
1330 extra_upstreams: HashMap<GlobalFragmentId, HashMap<GlobalFragmentId, StreamFragmentEdge>>,
1332}
1333
1334pub struct FragmentGraphUpstreamContext {
1335 pub upstream_root_fragments: HashMap<JobId, (SharedFragmentInfo, PbStreamNode)>,
1338 pub upstream_actor_location: HashMap<ActorId, WorkerId>,
1339}
1340
1341pub struct FragmentGraphDownstreamContext {
1342 pub original_root_fragment_id: FragmentId,
1343 pub downstream_fragments: Vec<(DispatcherType, SharedFragmentInfo, PbStreamNode)>,
1344 pub downstream_actor_location: HashMap<ActorId, WorkerId>,
1345}
1346
1347impl CompleteStreamFragmentGraph {
1348 #[cfg(test)]
1351 pub fn for_test(graph: StreamFragmentGraph) -> Self {
1352 Self {
1353 building_graph: graph,
1354 existing_fragments: Default::default(),
1355 existing_actor_location: Default::default(),
1356 extra_downstreams: Default::default(),
1357 extra_upstreams: Default::default(),
1358 }
1359 }
1360
1361 pub fn with_upstreams(
1365 graph: StreamFragmentGraph,
1366 upstream_context: FragmentGraphUpstreamContext,
1367 job_type: StreamingJobType,
1368 ) -> MetaResult<Self> {
1369 Self::build_helper(graph, Some(upstream_context), None, job_type)
1370 }
1371
1372 pub fn with_downstreams(
1375 graph: StreamFragmentGraph,
1376 downstream_context: FragmentGraphDownstreamContext,
1377 job_type: StreamingJobType,
1378 ) -> MetaResult<Self> {
1379 Self::build_helper(graph, None, Some(downstream_context), job_type)
1380 }
1381
1382 pub fn with_upstreams_and_downstreams(
1384 graph: StreamFragmentGraph,
1385 upstream_context: FragmentGraphUpstreamContext,
1386 downstream_context: FragmentGraphDownstreamContext,
1387 job_type: StreamingJobType,
1388 ) -> MetaResult<Self> {
1389 Self::build_helper(
1390 graph,
1391 Some(upstream_context),
1392 Some(downstream_context),
1393 job_type,
1394 )
1395 }
1396
1397 fn build_helper(
1399 mut graph: StreamFragmentGraph,
1400 upstream_ctx: Option<FragmentGraphUpstreamContext>,
1401 downstream_ctx: Option<FragmentGraphDownstreamContext>,
1402 job_type: StreamingJobType,
1403 ) -> MetaResult<Self> {
1404 let mut extra_downstreams = HashMap::new();
1405 let mut extra_upstreams = HashMap::new();
1406 let mut existing_fragments = HashMap::new();
1407
1408 let mut existing_actor_location = HashMap::new();
1409
1410 if let Some(FragmentGraphUpstreamContext {
1411 upstream_root_fragments,
1412 upstream_actor_location,
1413 }) = upstream_ctx
1414 {
1415 for (&id, fragment) in &mut graph.fragments {
1416 let uses_shuffled_backfill = fragment.has_shuffled_backfill();
1417
1418 for (&upstream_job_id, required_columns) in &fragment.upstream_job_columns {
1419 let (upstream_fragment, nodes) = upstream_root_fragments
1420 .get(&upstream_job_id)
1421 .context("upstream fragment not found")?;
1422 let upstream_root_fragment_id =
1423 GlobalFragmentId::new(upstream_fragment.fragment_id);
1424
1425 let edge = match job_type {
1426 StreamingJobType::Table(TableJobType::SharedCdcSource) => {
1427 assert_ne!(
1430 (fragment.fragment_type_mask & FragmentTypeFlag::CdcFilter as u32),
1431 0
1432 );
1433
1434 tracing::debug!(
1435 ?upstream_root_fragment_id,
1436 ?required_columns,
1437 identity = ?fragment.inner.get_node().unwrap().get_identity(),
1438 current_frag_id=?id,
1439 "CdcFilter with upstream source fragment"
1440 );
1441
1442 StreamFragmentEdge {
1443 id: EdgeId::UpstreamExternal {
1444 upstream_job_id,
1445 downstream_fragment_id: id,
1446 },
1447 dispatch_strategy: DispatchStrategy {
1450 r#type: DispatcherType::NoShuffle as _,
1451 dist_key_indices: vec![], output_mapping: DispatchOutputMapping::identical(
1453 CDC_SOURCE_COLUMN_NUM as _,
1454 )
1455 .into(),
1456 },
1457 }
1458 }
1459
1460 StreamingJobType::MaterializedView
1462 | StreamingJobType::Sink
1463 | StreamingJobType::Index => {
1464 if upstream_fragment
1467 .fragment_type_mask
1468 .contains(FragmentTypeFlag::Mview)
1469 {
1470 let (dist_key_indices, output_mapping) = {
1472 let mview_node =
1473 nodes.get_node_body().unwrap().as_materialize().unwrap();
1474 let all_columns = mview_node.column_descs();
1475 let dist_key_indices = mview_node.dist_key_indices();
1476 let output_mapping = gen_output_mapping(
1477 required_columns,
1478 &all_columns,
1479 )
1480 .context(
1481 "BUG: column not found in the upstream materialized view",
1482 )?;
1483 (dist_key_indices, output_mapping)
1484 };
1485 let dispatch_strategy = mv_on_mv_dispatch_strategy(
1486 uses_shuffled_backfill,
1487 dist_key_indices,
1488 output_mapping,
1489 );
1490
1491 StreamFragmentEdge {
1492 id: EdgeId::UpstreamExternal {
1493 upstream_job_id,
1494 downstream_fragment_id: id,
1495 },
1496 dispatch_strategy,
1497 }
1498 }
1499 else if upstream_fragment
1502 .fragment_type_mask
1503 .contains(FragmentTypeFlag::Source)
1504 {
1505 let output_mapping = {
1506 let source_node =
1507 nodes.get_node_body().unwrap().as_source().unwrap();
1508
1509 let all_columns = source_node.column_descs().unwrap();
1510 gen_output_mapping(required_columns, &all_columns).context(
1511 "BUG: column not found in the upstream source node",
1512 )?
1513 };
1514
1515 StreamFragmentEdge {
1516 id: EdgeId::UpstreamExternal {
1517 upstream_job_id,
1518 downstream_fragment_id: id,
1519 },
1520 dispatch_strategy: DispatchStrategy {
1523 r#type: DispatcherType::NoShuffle as _,
1524 dist_key_indices: vec![], output_mapping: Some(output_mapping),
1526 },
1527 }
1528 } else {
1529 bail!(
1530 "the upstream fragment should be a MView or Source, got fragment type: {:b}",
1531 upstream_fragment.fragment_type_mask
1532 )
1533 }
1534 }
1535 StreamingJobType::Source | StreamingJobType::Table(_) => {
1536 bail!(
1537 "the streaming job shouldn't have an upstream fragment, job_type: {:?}",
1538 job_type
1539 )
1540 }
1541 };
1542
1543 extra_downstreams
1545 .entry(upstream_root_fragment_id)
1546 .or_insert_with(HashMap::new)
1547 .try_insert(id, edge.clone())
1548 .unwrap();
1549 extra_upstreams
1550 .entry(id)
1551 .or_insert_with(HashMap::new)
1552 .try_insert(upstream_root_fragment_id, edge)
1553 .unwrap();
1554 }
1555 }
1556
1557 existing_fragments.extend(
1558 upstream_root_fragments
1559 .into_values()
1560 .map(|(f, _)| (GlobalFragmentId::new(f.fragment_id), f)),
1561 );
1562
1563 existing_actor_location.extend(upstream_actor_location);
1564 }
1565
1566 if let Some(FragmentGraphDownstreamContext {
1567 original_root_fragment_id,
1568 downstream_fragments,
1569 downstream_actor_location,
1570 }) = downstream_ctx
1571 {
1572 let original_table_fragment_id = GlobalFragmentId::new(original_root_fragment_id);
1573 let table_fragment_id = GlobalFragmentId::new(graph.table_fragment_id());
1574
1575 for (dispatcher_type, fragment, nodes) in &downstream_fragments {
1578 let id = GlobalFragmentId::new(fragment.fragment_id);
1579
1580 let output_columns = {
1582 let mut res = None;
1583
1584 stream_graph_visitor::visit_stream_node_body(nodes, |node_body| {
1585 let columns = match node_body {
1586 NodeBody::StreamScan(stream_scan) => stream_scan.upstream_columns(),
1587 NodeBody::SourceBackfill(source_backfill) => {
1588 source_backfill.column_descs()
1590 }
1591 _ => return,
1592 };
1593 res = Some(columns);
1594 });
1595
1596 res.context("failed to locate downstream scan")?
1597 };
1598
1599 let table_fragment = graph.fragments.get(&table_fragment_id).unwrap();
1600 let nodes = table_fragment.node.as_ref().unwrap();
1601
1602 let (dist_key_indices, output_mapping) = match job_type {
1603 StreamingJobType::Table(_) | StreamingJobType::MaterializedView => {
1604 let mview_node = nodes.get_node_body().unwrap().as_materialize().unwrap();
1605 let all_columns = mview_node.column_descs();
1606 let dist_key_indices = mview_node.dist_key_indices();
1607 let output_mapping = gen_output_mapping(&output_columns, &all_columns)
1608 .ok_or_else(|| {
1609 MetaError::invalid_parameter(
1610 "unable to drop the column due to \
1611 being referenced by downstream materialized views or sinks",
1612 )
1613 })?;
1614 (dist_key_indices, output_mapping)
1615 }
1616
1617 StreamingJobType::Source => {
1618 let source_node = nodes.get_node_body().unwrap().as_source().unwrap();
1619 let all_columns = source_node.column_descs().unwrap();
1620 let output_mapping = gen_output_mapping(&output_columns, &all_columns)
1621 .ok_or_else(|| {
1622 MetaError::invalid_parameter(
1623 "unable to drop the column due to \
1624 being referenced by downstream materialized views or sinks",
1625 )
1626 })?;
1627 assert_eq!(*dispatcher_type, DispatcherType::NoShuffle);
1628 (
1629 vec![], output_mapping,
1631 )
1632 }
1633
1634 _ => bail!("unsupported job type for replacement: {job_type:?}"),
1635 };
1636
1637 let edge = StreamFragmentEdge {
1638 id: EdgeId::DownstreamExternal(DownstreamExternalEdgeId {
1639 original_upstream_fragment_id: original_table_fragment_id,
1640 downstream_fragment_id: id,
1641 }),
1642 dispatch_strategy: DispatchStrategy {
1643 r#type: *dispatcher_type as i32,
1644 output_mapping: Some(output_mapping),
1645 dist_key_indices,
1646 },
1647 };
1648
1649 extra_downstreams
1650 .entry(table_fragment_id)
1651 .or_insert_with(HashMap::new)
1652 .try_insert(id, edge.clone())
1653 .unwrap();
1654 extra_upstreams
1655 .entry(id)
1656 .or_insert_with(HashMap::new)
1657 .try_insert(table_fragment_id, edge)
1658 .unwrap();
1659 }
1660
1661 existing_fragments.extend(
1662 downstream_fragments
1663 .into_iter()
1664 .map(|(_, f, _)| (GlobalFragmentId::new(f.fragment_id), f)),
1665 );
1666
1667 existing_actor_location.extend(downstream_actor_location);
1668 }
1669
1670 Ok(Self {
1671 building_graph: graph,
1672 existing_fragments,
1673 existing_actor_location,
1674 extra_downstreams,
1675 extra_upstreams,
1676 })
1677 }
1678}
1679
1680fn gen_output_mapping(
1682 required_columns: &[PbColumnDesc],
1683 upstream_columns: &[PbColumnDesc],
1684) -> Option<DispatchOutputMapping> {
1685 let len = required_columns.len();
1686 let mut indices = vec![0; len];
1687 let mut types = None;
1688
1689 for (i, r) in required_columns.iter().enumerate() {
1690 let (ui, u) = upstream_columns
1691 .iter()
1692 .find_position(|&u| u.column_id == r.column_id)?;
1693 indices[i] = ui as u32;
1694
1695 if u.column_type != r.column_type {
1698 types.get_or_insert_with(|| vec![TypePair::default(); len])[i] = TypePair {
1699 upstream: u.column_type.clone(),
1700 downstream: r.column_type.clone(),
1701 };
1702 }
1703 }
1704
1705 let types = types.unwrap_or(Vec::new());
1707
1708 Some(DispatchOutputMapping { indices, types })
1709}
1710
1711fn mv_on_mv_dispatch_strategy(
1712 uses_shuffled_backfill: bool,
1713 dist_key_indices: Vec<u32>,
1714 output_mapping: DispatchOutputMapping,
1715) -> DispatchStrategy {
1716 if uses_shuffled_backfill {
1717 if !dist_key_indices.is_empty() {
1718 DispatchStrategy {
1719 r#type: DispatcherType::Hash as _,
1720 dist_key_indices,
1721 output_mapping: Some(output_mapping),
1722 }
1723 } else {
1724 DispatchStrategy {
1725 r#type: DispatcherType::Simple as _,
1726 dist_key_indices: vec![], output_mapping: Some(output_mapping),
1728 }
1729 }
1730 } else {
1731 DispatchStrategy {
1732 r#type: DispatcherType::NoShuffle as _,
1733 dist_key_indices: vec![], output_mapping: Some(output_mapping),
1735 }
1736 }
1737}
1738
1739impl CompleteStreamFragmentGraph {
1740 pub(super) fn all_fragment_ids(&self) -> impl Iterator<Item = GlobalFragmentId> + '_ {
1743 self.building_graph
1744 .fragments
1745 .keys()
1746 .chain(self.existing_fragments.keys())
1747 .copied()
1748 }
1749
1750 pub(super) fn all_edges(
1752 &self,
1753 ) -> impl Iterator<Item = (GlobalFragmentId, GlobalFragmentId, &StreamFragmentEdge)> + '_ {
1754 self.building_graph
1755 .downstreams
1756 .iter()
1757 .chain(self.extra_downstreams.iter())
1758 .flat_map(|(&from, tos)| tos.iter().map(move |(&to, edge)| (from, to, edge)))
1759 }
1760
1761 pub(super) fn existing_distribution(&self) -> HashMap<GlobalFragmentId, Distribution> {
1763 self.existing_fragments
1764 .iter()
1765 .map(|(&id, f)| {
1766 (
1767 id,
1768 Distribution::from_fragment(f, &self.existing_actor_location),
1769 )
1770 })
1771 .collect()
1772 }
1773
1774 pub(super) fn topo_order(&self) -> MetaResult<Vec<GlobalFragmentId>> {
1781 let mut topo = Vec::new();
1782 let mut downstream_cnts = HashMap::new();
1783
1784 for fragment_id in self.all_fragment_ids() {
1786 let downstream_cnt = self.get_downstreams(fragment_id).count();
1788 if downstream_cnt == 0 {
1789 topo.push(fragment_id);
1790 } else {
1791 downstream_cnts.insert(fragment_id, downstream_cnt);
1792 }
1793 }
1794
1795 let mut i = 0;
1796 while let Some(&fragment_id) = topo.get(i) {
1797 i += 1;
1798 for (upstream_job_id, _) in self.get_upstreams(fragment_id) {
1800 let downstream_cnt = downstream_cnts.get_mut(&upstream_job_id).unwrap();
1801 *downstream_cnt -= 1;
1802 if *downstream_cnt == 0 {
1803 downstream_cnts.remove(&upstream_job_id);
1804 topo.push(upstream_job_id);
1805 }
1806 }
1807 }
1808
1809 if !downstream_cnts.is_empty() {
1810 bail!("graph is not a DAG");
1812 }
1813
1814 Ok(topo)
1815 }
1816
1817 pub(super) fn seal_fragment(
1820 &self,
1821 id: GlobalFragmentId,
1822 actors: Vec<StreamActor>,
1823 distribution: Distribution,
1824 stream_node: StreamNode,
1825 ) -> Fragment {
1826 let building_fragment = self.get_fragment(id).into_building().unwrap();
1827 let internal_tables = building_fragment.extract_internal_tables();
1828 let BuildingFragment {
1829 inner,
1830 job_id,
1831 upstream_job_columns: _,
1832 } = building_fragment;
1833
1834 let distribution_type = distribution.to_distribution_type();
1835 let vnode_count = distribution.vnode_count();
1836
1837 let materialized_fragment_id =
1838 if FragmentTypeMask::from(inner.fragment_type_mask).contains(FragmentTypeFlag::Mview) {
1839 job_id.map(JobId::as_mv_table_id)
1840 } else {
1841 None
1842 };
1843
1844 let vector_index_fragment_id =
1845 if inner.fragment_type_mask & FragmentTypeFlag::VectorIndexWrite as u32 != 0 {
1846 job_id.map(JobId::as_mv_table_id)
1847 } else {
1848 None
1849 };
1850
1851 let state_table_ids = internal_tables
1852 .iter()
1853 .map(|t| t.id)
1854 .chain(materialized_fragment_id)
1855 .chain(vector_index_fragment_id)
1856 .collect();
1857
1858 Fragment {
1859 fragment_id: inner.fragment_id,
1860 fragment_type_mask: inner.fragment_type_mask.into(),
1861 distribution_type,
1862 actors,
1863 state_table_ids,
1864 maybe_vnode_count: VnodeCount::set(vnode_count).to_protobuf(),
1865 nodes: stream_node,
1866 }
1867 }
1868
1869 pub(super) fn get_fragment(&self, fragment_id: GlobalFragmentId) -> EitherFragment {
1872 if let Some(fragment) = self.existing_fragments.get(&fragment_id) {
1873 EitherFragment::Existing(fragment.clone())
1874 } else {
1875 EitherFragment::Building(
1876 self.building_graph
1877 .fragments
1878 .get(&fragment_id)
1879 .unwrap()
1880 .clone(),
1881 )
1882 }
1883 }
1884
1885 pub(super) fn get_downstreams(
1888 &self,
1889 fragment_id: GlobalFragmentId,
1890 ) -> impl Iterator<Item = (GlobalFragmentId, &StreamFragmentEdge)> {
1891 self.building_graph
1892 .get_downstreams(fragment_id)
1893 .iter()
1894 .chain(
1895 self.extra_downstreams
1896 .get(&fragment_id)
1897 .into_iter()
1898 .flatten(),
1899 )
1900 .map(|(&id, edge)| (id, edge))
1901 }
1902
1903 pub(super) fn get_upstreams(
1906 &self,
1907 fragment_id: GlobalFragmentId,
1908 ) -> impl Iterator<Item = (GlobalFragmentId, &StreamFragmentEdge)> {
1909 self.building_graph
1910 .get_upstreams(fragment_id)
1911 .iter()
1912 .chain(self.extra_upstreams.get(&fragment_id).into_iter().flatten())
1913 .map(|(&id, edge)| (id, edge))
1914 }
1915
1916 pub(super) fn building_fragments(&self) -> &HashMap<GlobalFragmentId, BuildingFragment> {
1918 &self.building_graph.fragments
1919 }
1920
1921 pub(super) fn building_fragments_mut(
1923 &mut self,
1924 ) -> &mut HashMap<GlobalFragmentId, BuildingFragment> {
1925 &mut self.building_graph.fragments
1926 }
1927
1928 pub(super) fn max_parallelism(&self) -> usize {
1930 self.building_graph.max_parallelism()
1931 }
1932}