risingwave_frontend/scheduler/
plan_fragmenter.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::cmp::min;
16use std::collections::{HashMap, HashSet};
17use std::fmt::{Debug, Display, Formatter};
18use std::num::NonZeroU64;
19
20use anyhow::anyhow;
21use async_recursion::async_recursion;
22use enum_as_inner::EnumAsInner;
23use futures::TryStreamExt;
24use itertools::Itertools;
25use petgraph::{Directed, Graph};
26use pgwire::pg_server::SessionId;
27use risingwave_batch::error::BatchError;
28use risingwave_batch::worker_manager::worker_node_manager::WorkerNodeSelector;
29use risingwave_common::bitmap::{Bitmap, BitmapBuilder};
30use risingwave_common::catalog::Schema;
31use risingwave_common::hash::table_distribution::TableDistribution;
32use risingwave_common::hash::{WorkerSlotId, WorkerSlotMapping};
33use risingwave_common::util::scan_range::ScanRange;
34use risingwave_connector::source::filesystem::opendal_source::opendal_enumerator::OpendalEnumerator;
35use risingwave_connector::source::filesystem::opendal_source::{
36    BatchPosixFsEnumerator, OpendalAzblob, OpendalGcs, OpendalS3,
37};
38use risingwave_connector::source::iceberg::{
39    IcebergFileScanTask, IcebergSplit, IcebergSplitEnumerator,
40};
41use risingwave_connector::source::kafka::KafkaSplitEnumerator;
42use risingwave_connector::source::prelude::DatagenSplitEnumerator;
43use risingwave_connector::source::reader::reader::build_opendal_fs_list_for_batch;
44use risingwave_connector::source::{
45    ConnectorProperties, SourceEnumeratorContext, SplitEnumerator, SplitImpl,
46};
47use risingwave_pb::batch_plan::plan_node::NodeBody;
48use risingwave_pb::batch_plan::{ExchangeInfo, ScanRange as ScanRangeProto};
49use risingwave_pb::plan_common::Field as PbField;
50use serde::ser::SerializeStruct;
51use serde::{Serialize, Serializer};
52use uuid::Uuid;
53
54use super::SchedulerError;
55use crate::TableCatalog;
56use crate::catalog::TableId;
57use crate::catalog::catalog_service::CatalogReader;
58use crate::optimizer::plan_node::generic::{GenericPlanRef, PhysicalPlanRef};
59use crate::optimizer::plan_node::{
60    BatchIcebergScan, BatchKafkaScan, BatchPlanNodeType, BatchPlanRef as PlanRef, BatchSource,
61    PlanNodeId,
62};
63use crate::optimizer::property::Distribution;
64use crate::scheduler::SchedulerResult;
65
66#[derive(Clone, Debug, Hash, Eq, PartialEq)]
67pub struct QueryId {
68    pub id: String,
69}
70
71impl std::fmt::Display for QueryId {
72    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
73        write!(f, "QueryId:{}", self.id)
74    }
75}
76
77#[derive(Copy, Clone, Hash, PartialEq, Eq, Ord, PartialOrd)]
78pub struct StageId(u32);
79
80impl Display for StageId {
81    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
82        write!(f, "{}", self.0)
83    }
84}
85
86impl Debug for StageId {
87    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
88        write!(f, "{:?}", self.0)
89    }
90}
91
92impl From<StageId> for u32 {
93    fn from(value: StageId) -> Self {
94        value.0
95    }
96}
97
98impl From<u32> for StageId {
99    fn from(value: u32) -> Self {
100        StageId(value)
101    }
102}
103
104impl Serialize for StageId {
105    fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
106    where
107        S: Serializer,
108    {
109        self.0.serialize(serializer)
110    }
111}
112
113impl StageId {
114    pub fn inc(&mut self) {
115        self.0 += 1;
116    }
117}
118
119// Root stage always has only one task.
120pub const ROOT_TASK_ID: u64 = 0;
121// Root task has only one output.
122pub const ROOT_TASK_OUTPUT_ID: u64 = 0;
123pub type TaskId = u64;
124
125/// Generated by [`BatchPlanFragmenter`] and used in query execution graph.
126#[derive(Debug)]
127#[cfg_attr(test, derive(Clone))]
128pub struct ExecutionPlanNode {
129    pub plan_node_id: PlanNodeId,
130    pub plan_node_type: BatchPlanNodeType,
131    pub node: NodeBody,
132    pub schema: Vec<PbField>,
133
134    pub children: Vec<ExecutionPlanNode>,
135
136    /// The stage id of the source of `BatchExchange`.
137    /// Used to find `ExchangeSource` from scheduler when creating `PlanNode`.
138    ///
139    /// `None` when this node is not `BatchExchange`.
140    pub source_stage_id: Option<StageId>,
141}
142
143impl Serialize for ExecutionPlanNode {
144    fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
145    where
146        S: serde::Serializer,
147    {
148        let mut state = serializer.serialize_struct("QueryStage", 5)?;
149        state.serialize_field("plan_node_id", &self.plan_node_id)?;
150        state.serialize_field("plan_node_type", &self.plan_node_type)?;
151        state.serialize_field("schema", &self.schema)?;
152        state.serialize_field("children", &self.children)?;
153        state.serialize_field("source_stage_id", &self.source_stage_id)?;
154        state.end()
155    }
156}
157
158impl TryFrom<PlanRef> for ExecutionPlanNode {
159    type Error = SchedulerError;
160
161    fn try_from(plan_node: PlanRef) -> Result<Self, Self::Error> {
162        Ok(Self {
163            plan_node_id: plan_node.plan_base().id(),
164            plan_node_type: plan_node.node_type(),
165            node: plan_node.try_to_batch_prost_body()?,
166            children: vec![],
167            schema: plan_node.schema().to_prost(),
168            source_stage_id: None,
169        })
170    }
171}
172
173impl ExecutionPlanNode {
174    pub fn node_type(&self) -> BatchPlanNodeType {
175        self.plan_node_type
176    }
177}
178
179/// `BatchPlanFragmenter` splits a query plan into fragments.
180pub struct BatchPlanFragmenter {
181    query_id: QueryId,
182    next_stage_id: StageId,
183    worker_node_manager: WorkerNodeSelector,
184    catalog_reader: CatalogReader,
185
186    batch_parallelism: usize,
187
188    stage_graph_builder: Option<StageGraphBuilder>,
189    stage_graph: Option<StageGraph>,
190}
191
192impl Default for QueryId {
193    fn default() -> Self {
194        Self {
195            id: Uuid::new_v4().to_string(),
196        }
197    }
198}
199
200impl BatchPlanFragmenter {
201    pub fn new(
202        worker_node_manager: WorkerNodeSelector,
203        catalog_reader: CatalogReader,
204        batch_parallelism: Option<NonZeroU64>,
205        batch_node: PlanRef,
206    ) -> SchedulerResult<Self> {
207        // if batch_parallelism is None, it means no limit, we will use the available nodes count as
208        // parallelism.
209        // if batch_parallelism is Some(num), we will use the min(num, the available
210        // nodes count) as parallelism.
211        let batch_parallelism = if let Some(num) = batch_parallelism {
212            // can be 0 if no available serving worker
213            min(
214                num.get() as usize,
215                worker_node_manager.schedule_unit_count(),
216            )
217        } else {
218            // can be 0 if no available serving worker
219            worker_node_manager.schedule_unit_count()
220        };
221
222        let mut plan_fragmenter = Self {
223            query_id: Default::default(),
224            next_stage_id: 0.into(),
225            worker_node_manager,
226            catalog_reader,
227            batch_parallelism,
228            stage_graph_builder: Some(StageGraphBuilder::new(batch_parallelism)),
229            stage_graph: None,
230        };
231        plan_fragmenter.split_into_stage(batch_node)?;
232        Ok(plan_fragmenter)
233    }
234
235    /// Split the plan node into each stages, based on exchange node.
236    fn split_into_stage(&mut self, batch_node: PlanRef) -> SchedulerResult<()> {
237        let root_stage_id = self.new_stage(
238            batch_node,
239            Some(Distribution::Single.to_prost(
240                1,
241                &self.catalog_reader,
242                &self.worker_node_manager,
243            )?),
244        )?;
245        self.stage_graph = Some(
246            self.stage_graph_builder
247                .take()
248                .unwrap()
249                .build(root_stage_id),
250        );
251        Ok(())
252    }
253}
254
255/// The fragmented query generated by [`BatchPlanFragmenter`].
256#[derive(Debug)]
257#[cfg_attr(test, derive(Clone))]
258pub struct Query {
259    /// Query id should always be unique.
260    pub query_id: QueryId,
261    pub stage_graph: StageGraph,
262}
263
264impl Query {
265    pub fn leaf_stages(&self) -> Vec<StageId> {
266        let mut ret_leaf_stages = Vec::new();
267        for stage_id in self.stage_graph.stages.keys() {
268            if self
269                .stage_graph
270                .get_child_stages_unchecked(stage_id)
271                .is_empty()
272            {
273                ret_leaf_stages.push(*stage_id);
274            }
275        }
276        ret_leaf_stages
277    }
278
279    pub fn get_parents(&self, stage_id: &StageId) -> &HashSet<StageId> {
280        self.stage_graph.parent_edges.get(stage_id).unwrap()
281    }
282
283    pub fn root_stage_id(&self) -> StageId {
284        self.stage_graph.root_stage_id
285    }
286
287    pub fn query_id(&self) -> &QueryId {
288        &self.query_id
289    }
290
291    pub fn stages_with_table_scan(&self) -> HashSet<StageId> {
292        self.stage_graph
293            .stages
294            .iter()
295            .filter_map(|(stage_id, stage_query)| {
296                if stage_query.has_table_scan() {
297                    Some(*stage_id)
298                } else {
299                    None
300                }
301            })
302            .collect()
303    }
304
305    pub fn has_lookup_join_stage(&self) -> bool {
306        self.stage_graph
307            .stages
308            .iter()
309            .any(|(_stage_id, stage_query)| stage_query.has_lookup_join())
310    }
311
312    pub fn stage(&self, stage_id: StageId) -> &QueryStage {
313        &self.stage_graph.stages[&stage_id]
314    }
315}
316
317#[derive(Debug, Clone)]
318pub enum SourceFetchParameters {
319    KafkaTimebound {
320        lower: Option<i64>,
321        upper: Option<i64>,
322    },
323    Empty,
324}
325
326#[derive(Debug, Clone)]
327pub enum UnpartitionedData {
328    Iceberg(IcebergFileScanTask),
329}
330
331#[derive(Debug, Clone)]
332pub struct SourceFetchInfo {
333    pub schema: Schema,
334    /// These are user-configured connector properties.
335    /// e.g. host, username, etc...
336    pub connector: ConnectorProperties,
337    /// These parameters are internally derived by the plan node.
338    /// e.g. predicate pushdown for iceberg, timebound for kafka.
339    pub fetch_parameters: SourceFetchParameters,
340}
341
342#[derive(Clone)]
343pub enum SourceScanInfo {
344    /// Split Info
345    Incomplete(SourceFetchInfo),
346    Unpartitioned(UnpartitionedData),
347    Complete(Vec<SplitImpl>),
348}
349
350impl SourceScanInfo {
351    pub fn new(fetch_info: SourceFetchInfo) -> Self {
352        Self::Incomplete(fetch_info)
353    }
354
355    pub async fn complete(self, batch_parallelism: usize) -> SchedulerResult<Self> {
356        match self {
357            SourceScanInfo::Incomplete(fetch_info) => fetch_info.complete(batch_parallelism).await,
358            SourceScanInfo::Unpartitioned(data) => data.complete(batch_parallelism),
359            SourceScanInfo::Complete(_) => {
360                unreachable!("Never call complete when SourceScanInfo is already complete")
361            }
362        }
363    }
364
365    pub fn split_info(&self) -> SchedulerResult<&Vec<SplitImpl>> {
366        match self {
367            Self::Incomplete(_) => Err(SchedulerError::Internal(anyhow!(
368                "Should not get split info from incomplete source scan info"
369            ))),
370            Self::Unpartitioned(_) => Err(SchedulerError::Internal(anyhow!(
371                "Should not get split info from unpartitioned source scan info"
372            ))),
373            Self::Complete(split_info) => Ok(split_info),
374        }
375    }
376}
377
378impl UnpartitionedData {
379    fn complete(self, batch_parallelism: usize) -> SchedulerResult<SourceScanInfo> {
380        macro_rules! split_iceberg_tasks {
381            ($tasks:expr, $variant:ident) => {
382                IcebergSplitEnumerator::split_n_vecs($tasks, batch_parallelism)
383                    .into_iter()
384                    .enumerate()
385                    .map(|(id, tasks)| {
386                        SplitImpl::Iceberg(IcebergSplit {
387                            split_id: id.try_into().unwrap(),
388                            task: IcebergFileScanTask::$variant(tasks),
389                        })
390                    })
391                    .collect()
392            };
393        }
394
395        let splits = match self {
396            UnpartitionedData::Iceberg(task) => match task {
397                IcebergFileScanTask::Data(tasks) => split_iceberg_tasks!(tasks, Data),
398                IcebergFileScanTask::EqualityDelete(tasks) => {
399                    split_iceberg_tasks!(tasks, EqualityDelete)
400                }
401                IcebergFileScanTask::PositionDelete(tasks) => {
402                    split_iceberg_tasks!(tasks, PositionDelete)
403                }
404            },
405        };
406        Ok(SourceScanInfo::Complete(splits))
407    }
408}
409
410impl SourceFetchInfo {
411    async fn complete(self, _batch_parallelism: usize) -> SchedulerResult<SourceScanInfo> {
412        match (self.connector, self.fetch_parameters) {
413            (
414                ConnectorProperties::Kafka(prop),
415                SourceFetchParameters::KafkaTimebound { lower, upper },
416            ) => {
417                let mut kafka_enumerator =
418                    KafkaSplitEnumerator::new(*prop, SourceEnumeratorContext::dummy().into())
419                        .await?;
420                let split_info = kafka_enumerator
421                    .list_splits_batch(lower, upper)
422                    .await?
423                    .into_iter()
424                    .map(SplitImpl::Kafka)
425                    .collect_vec();
426
427                Ok(SourceScanInfo::Complete(split_info))
428            }
429            (ConnectorProperties::Datagen(prop), SourceFetchParameters::Empty) => {
430                let mut datagen_enumerator =
431                    DatagenSplitEnumerator::new(*prop, SourceEnumeratorContext::dummy().into())
432                        .await?;
433                let split_info = datagen_enumerator.list_splits().await?;
434                let res = split_info.into_iter().map(SplitImpl::Datagen).collect_vec();
435
436                Ok(SourceScanInfo::Complete(res))
437            }
438            (ConnectorProperties::OpendalS3(prop), SourceFetchParameters::Empty) => {
439                let lister: OpendalEnumerator<OpendalS3> = OpendalEnumerator::new_s3_source(
440                    &prop.s3_properties,
441                    prop.assume_role,
442                    prop.fs_common.compression_format,
443                )?;
444                let stream = build_opendal_fs_list_for_batch(lister);
445
446                let batch_res: Vec<_> = stream.try_collect().await?;
447                let res = batch_res
448                    .into_iter()
449                    .map(SplitImpl::OpendalS3)
450                    .collect_vec();
451
452                Ok(SourceScanInfo::Complete(res))
453            }
454            (ConnectorProperties::Gcs(prop), SourceFetchParameters::Empty) => {
455                let lister: OpendalEnumerator<OpendalGcs> =
456                    OpendalEnumerator::new_gcs_source(*prop)?;
457                let stream = build_opendal_fs_list_for_batch(lister);
458                let batch_res: Vec<_> = stream.try_collect().await?;
459                let res = batch_res.into_iter().map(SplitImpl::Gcs).collect_vec();
460
461                Ok(SourceScanInfo::Complete(res))
462            }
463            (ConnectorProperties::Azblob(prop), SourceFetchParameters::Empty) => {
464                let lister: OpendalEnumerator<OpendalAzblob> =
465                    OpendalEnumerator::new_azblob_source(*prop)?;
466                let stream = build_opendal_fs_list_for_batch(lister);
467                let batch_res: Vec<_> = stream.try_collect().await?;
468                let res = batch_res.into_iter().map(SplitImpl::Azblob).collect_vec();
469
470                Ok(SourceScanInfo::Complete(res))
471            }
472            (ConnectorProperties::BatchPosixFs(prop), SourceFetchParameters::Empty) => {
473                use risingwave_connector::source::SplitEnumerator;
474                let mut enumerator = BatchPosixFsEnumerator::new(
475                    *prop,
476                    risingwave_connector::source::SourceEnumeratorContext::dummy().into(),
477                )
478                .await?;
479                let splits = enumerator.list_splits().await?;
480                let res = splits
481                    .into_iter()
482                    .map(SplitImpl::BatchPosixFs)
483                    .collect_vec();
484
485                Ok(SourceScanInfo::Complete(res))
486            }
487            (connector, _) => Err(SchedulerError::Internal(anyhow!(
488                "Unsupported to query directly from this {} source, \
489                 please create a table or streaming job from it",
490                connector.kind()
491            ))),
492        }
493    }
494}
495
496#[derive(Clone, Debug)]
497pub struct TableScanInfo {
498    /// The name of the table to scan.
499    name: String,
500
501    /// Indicates the table partitions to be read by scan tasks. Unnecessary partitions are already
502    /// pruned.
503    ///
504    /// For singleton table, this field is still `Some` and only contains a single partition with
505    /// full vnode bitmap, since we need to know where to schedule the singleton scan task.
506    ///
507    /// `None` iff the table is a system table.
508    partitions: Option<HashMap<WorkerSlotId, TablePartitionInfo>>,
509}
510
511impl TableScanInfo {
512    /// For normal tables, `partitions` should always be `Some`.
513    pub fn new(name: String, partitions: HashMap<WorkerSlotId, TablePartitionInfo>) -> Self {
514        Self {
515            name,
516            partitions: Some(partitions),
517        }
518    }
519
520    /// For system table, there's no partition info.
521    pub fn system_table(name: String) -> Self {
522        Self {
523            name,
524            partitions: None,
525        }
526    }
527
528    pub fn name(&self) -> &str {
529        self.name.as_ref()
530    }
531
532    pub fn partitions(&self) -> Option<&HashMap<WorkerSlotId, TablePartitionInfo>> {
533        self.partitions.as_ref()
534    }
535}
536
537#[derive(Clone, Debug)]
538pub struct TablePartitionInfo {
539    pub vnode_bitmap: Bitmap,
540    pub scan_ranges: Vec<ScanRangeProto>,
541}
542
543#[derive(Clone, Debug, EnumAsInner)]
544pub enum PartitionInfo {
545    Table(TablePartitionInfo),
546    Source(Vec<SplitImpl>),
547    File(Vec<String>),
548}
549
550#[derive(Clone, Debug)]
551pub struct FileScanInfo {
552    pub file_location: Vec<String>,
553}
554
555/// Fragment part of `Query`.
556#[cfg_attr(test, derive(Clone))]
557pub struct QueryStage {
558    pub id: StageId,
559    pub root: ExecutionPlanNode,
560    pub exchange_info: Option<ExchangeInfo>,
561    pub parallelism: Option<u32>,
562    /// Indicates whether this stage contains a table scan node and the table's information if so.
563    pub table_scan_info: Option<TableScanInfo>,
564    pub source_info: Option<SourceScanInfo>,
565    pub file_scan_info: Option<FileScanInfo>,
566    pub has_lookup_join: bool,
567    pub dml_table_id: Option<TableId>,
568    pub session_id: SessionId,
569    pub batch_enable_distributed_dml: bool,
570
571    /// Used to generate exchange information when complete source scan information.
572    children_exchange_distribution: Option<HashMap<StageId, Distribution>>,
573}
574
575impl QueryStage {
576    /// If true, this stage contains table scan executor that creates
577    /// Hummock iterators to read data from table. The iterator is initialized during
578    /// the executor building process on the batch execution engine.
579    pub fn has_table_scan(&self) -> bool {
580        self.table_scan_info.is_some()
581    }
582
583    /// If true, this stage contains lookup join executor.
584    /// We need to delay epoch unpin util the end of the query.
585    pub fn has_lookup_join(&self) -> bool {
586        self.has_lookup_join
587    }
588
589    pub fn with_exchange_info(
590        self,
591        exchange_info: Option<ExchangeInfo>,
592        parallelism: Option<u32>,
593    ) -> Self {
594        if let Some(exchange_info) = exchange_info {
595            Self {
596                id: self.id,
597                root: self.root,
598                exchange_info: Some(exchange_info),
599                parallelism,
600                table_scan_info: self.table_scan_info,
601                source_info: self.source_info,
602                file_scan_info: self.file_scan_info,
603                has_lookup_join: self.has_lookup_join,
604                dml_table_id: self.dml_table_id,
605                session_id: self.session_id,
606                batch_enable_distributed_dml: self.batch_enable_distributed_dml,
607                children_exchange_distribution: self.children_exchange_distribution,
608            }
609        } else {
610            self
611        }
612    }
613
614    pub fn with_exchange_info_and_complete_source_info(
615        self,
616        exchange_info: Option<ExchangeInfo>,
617        source_info: SourceScanInfo,
618        task_parallelism: u32,
619    ) -> Self {
620        assert!(matches!(source_info, SourceScanInfo::Complete(_)));
621        let exchange_info = if let Some(exchange_info) = exchange_info {
622            Some(exchange_info)
623        } else {
624            self.exchange_info
625        };
626        Self {
627            id: self.id,
628            root: self.root,
629            exchange_info,
630            parallelism: Some(task_parallelism),
631            table_scan_info: self.table_scan_info,
632            source_info: Some(source_info),
633            file_scan_info: self.file_scan_info,
634            has_lookup_join: self.has_lookup_join,
635            dml_table_id: self.dml_table_id,
636            session_id: self.session_id,
637            batch_enable_distributed_dml: self.batch_enable_distributed_dml,
638            children_exchange_distribution: None,
639        }
640    }
641}
642
643impl Debug for QueryStage {
644    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
645        f.debug_struct("QueryStage")
646            .field("id", &self.id)
647            .field("parallelism", &self.parallelism)
648            .field("exchange_info", &self.exchange_info)
649            .field("has_table_scan", &self.has_table_scan())
650            .finish()
651    }
652}
653
654impl Serialize for QueryStage {
655    fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
656    where
657        S: serde::Serializer,
658    {
659        let mut state = serializer.serialize_struct("QueryStage", 3)?;
660        state.serialize_field("root", &self.root)?;
661        state.serialize_field("parallelism", &self.parallelism)?;
662        state.serialize_field("exchange_info", &self.exchange_info)?;
663        state.end()
664    }
665}
666
667struct QueryStageBuilder {
668    id: StageId,
669    root: Option<ExecutionPlanNode>,
670    parallelism: Option<u32>,
671    exchange_info: Option<ExchangeInfo>,
672
673    children_stages: Vec<StageId>,
674    /// See also [`QueryStage::table_scan_info`].
675    table_scan_info: Option<TableScanInfo>,
676    source_info: Option<SourceScanInfo>,
677    file_scan_file: Option<FileScanInfo>,
678    has_lookup_join: bool,
679    dml_table_id: Option<TableId>,
680    session_id: SessionId,
681    batch_enable_distributed_dml: bool,
682
683    children_exchange_distribution: HashMap<StageId, Distribution>,
684}
685
686impl QueryStageBuilder {
687    fn new(
688        id: StageId,
689        parallelism: Option<u32>,
690        exchange_info: Option<ExchangeInfo>,
691        table_scan_info: Option<TableScanInfo>,
692        source_info: Option<SourceScanInfo>,
693        file_scan_file: Option<FileScanInfo>,
694        has_lookup_join: bool,
695        dml_table_id: Option<TableId>,
696        session_id: SessionId,
697        batch_enable_distributed_dml: bool,
698    ) -> Self {
699        Self {
700            id,
701            root: None,
702            parallelism,
703            exchange_info,
704            children_stages: vec![],
705            table_scan_info,
706            source_info,
707            file_scan_file,
708            has_lookup_join,
709            dml_table_id,
710            session_id,
711            batch_enable_distributed_dml,
712            children_exchange_distribution: HashMap::new(),
713        }
714    }
715
716    fn finish(self, stage_graph_builder: &mut StageGraphBuilder) -> StageId {
717        let children_exchange_distribution = if self.parallelism.is_none() {
718            Some(self.children_exchange_distribution)
719        } else {
720            None
721        };
722        let stage = QueryStage {
723            id: self.id,
724            root: self.root.unwrap(),
725            exchange_info: self.exchange_info,
726            parallelism: self.parallelism,
727            table_scan_info: self.table_scan_info,
728            source_info: self.source_info,
729            file_scan_info: self.file_scan_file,
730            has_lookup_join: self.has_lookup_join,
731            dml_table_id: self.dml_table_id,
732            session_id: self.session_id,
733            batch_enable_distributed_dml: self.batch_enable_distributed_dml,
734            children_exchange_distribution,
735        };
736
737        let stage_id = stage.id;
738        stage_graph_builder.add_node(stage);
739        for child_stage_id in self.children_stages {
740            stage_graph_builder.link_to_child(self.id, child_stage_id);
741        }
742        stage_id
743    }
744}
745
746/// Maintains how each stage are connected.
747#[derive(Debug, Serialize)]
748#[cfg_attr(test, derive(Clone))]
749pub struct StageGraph {
750    pub root_stage_id: StageId,
751    pub stages: HashMap<StageId, QueryStage>,
752    /// Traverse from top to down. Used in split plan into stages.
753    child_edges: HashMap<StageId, HashSet<StageId>>,
754    /// Traverse from down to top. Used in schedule each stage.
755    parent_edges: HashMap<StageId, HashSet<StageId>>,
756
757    batch_parallelism: usize,
758}
759
760enum StageCompleteInfo {
761    ExchangeInfo((Option<ExchangeInfo>, Option<u32>)),
762    ExchangeWithSourceInfo((Option<ExchangeInfo>, SourceScanInfo, u32)),
763}
764
765impl StageGraph {
766    pub fn get_child_stages_unchecked(&self, stage_id: &StageId) -> &HashSet<StageId> {
767        self.child_edges.get(stage_id).unwrap()
768    }
769
770    pub fn get_child_stages(&self, stage_id: &StageId) -> Option<&HashSet<StageId>> {
771        self.child_edges.get(stage_id)
772    }
773
774    /// Returns stage ids in topology order, s.t. child stage always appears before its parent.
775    pub fn stage_ids_by_topo_order(&self) -> impl Iterator<Item = StageId> {
776        let mut stack = Vec::with_capacity(self.stages.len());
777        stack.push(self.root_stage_id);
778        let mut ret = Vec::with_capacity(self.stages.len());
779        let mut existing = HashSet::with_capacity(self.stages.len());
780
781        while let Some(s) = stack.pop() {
782            if !existing.contains(&s) {
783                ret.push(s);
784                existing.insert(s);
785                stack.extend(&self.child_edges[&s]);
786            }
787        }
788
789        ret.into_iter().rev()
790    }
791
792    async fn complete(
793        self,
794        catalog_reader: &CatalogReader,
795        worker_node_manager: &WorkerNodeSelector,
796    ) -> SchedulerResult<StageGraph> {
797        let mut complete_stages = HashMap::new();
798        self.complete_stage(
799            self.root_stage_id,
800            None,
801            &mut complete_stages,
802            catalog_reader,
803            worker_node_manager,
804        )
805        .await?;
806        let mut stages = self.stages;
807        Ok(StageGraph {
808            root_stage_id: self.root_stage_id,
809            stages: complete_stages
810                .into_iter()
811                .map(|(stage_id, info)| {
812                    let stage = stages.remove(&stage_id).expect("should exist");
813                    let stage = match info {
814                        StageCompleteInfo::ExchangeInfo((exchange_info, parallelism)) => {
815                            stage.with_exchange_info(exchange_info, parallelism)
816                        }
817                        StageCompleteInfo::ExchangeWithSourceInfo((
818                            exchange_info,
819                            source_info,
820                            parallelism,
821                        )) => stage.with_exchange_info_and_complete_source_info(
822                            exchange_info,
823                            source_info,
824                            parallelism,
825                        ),
826                    };
827                    (stage_id, stage)
828                })
829                .collect(),
830            child_edges: self.child_edges,
831            parent_edges: self.parent_edges,
832            batch_parallelism: self.batch_parallelism,
833        })
834    }
835
836    #[async_recursion]
837    async fn complete_stage(
838        &self,
839        stage_id: StageId,
840        exchange_info: Option<ExchangeInfo>,
841        complete_stages: &mut HashMap<StageId, StageCompleteInfo>,
842        catalog_reader: &CatalogReader,
843        worker_node_manager: &WorkerNodeSelector,
844    ) -> SchedulerResult<()> {
845        let stage = &self.stages[&stage_id];
846        let parallelism = if stage.parallelism.is_some() {
847            // If the stage has parallelism, it means it's a complete stage.
848            complete_stages.insert(
849                stage.id,
850                StageCompleteInfo::ExchangeInfo((exchange_info, stage.parallelism)),
851            );
852            None
853        } else if matches!(
854            stage.source_info,
855            Some(SourceScanInfo::Incomplete(_)) | Some(SourceScanInfo::Unpartitioned(_))
856        ) {
857            let complete_source_info = stage
858                .source_info
859                .as_ref()
860                .unwrap()
861                .clone()
862                .complete(self.batch_parallelism)
863                .await?;
864
865            // For batch reading file source, the number of files involved is typically large.
866            // In order to avoid generating a task for each file, the parallelism of tasks is limited here.
867            // The minimum `task_parallelism` is 1. Additionally, `task_parallelism`
868            // must be greater than the number of files to read. Therefore, we first take the
869            // minimum of the number of files and (self.batch_parallelism / 2). If the number of
870            // files is 0, we set task_parallelism to 1.
871
872            let task_parallelism = match &stage.source_info {
873                Some(SourceScanInfo::Incomplete(source_fetch_info)) => {
874                    match source_fetch_info.connector {
875                        ConnectorProperties::Gcs(_)
876                        | ConnectorProperties::OpendalS3(_)
877                        | ConnectorProperties::Azblob(_) => (min(
878                            complete_source_info.split_info().unwrap().len() as u32,
879                            (self.batch_parallelism / 2) as u32,
880                        ))
881                        .max(1),
882                        _ => complete_source_info.split_info().unwrap().len() as u32,
883                    }
884                }
885                _ => complete_source_info.split_info().unwrap().len() as u32,
886            };
887            // For file source batch read, all the files  to be read are divide into several parts to prevent the task from taking up too many resources.
888            // todo(wcy-fdu): Currently it will be divided into half of batch_parallelism groups, and this will be changed to configurable later.
889            let complete_stage_info = StageCompleteInfo::ExchangeWithSourceInfo((
890                exchange_info,
891                complete_source_info,
892                task_parallelism,
893            ));
894            complete_stages.insert(stage.id, complete_stage_info);
895            Some(task_parallelism)
896        } else {
897            assert!(stage.file_scan_info.is_some());
898            let parallelism = min(
899                self.batch_parallelism / 2,
900                stage.file_scan_info.as_ref().unwrap().file_location.len(),
901            );
902            complete_stages.insert(
903                stage.id,
904                StageCompleteInfo::ExchangeInfo((exchange_info, Some(parallelism as u32))),
905            );
906            None
907        };
908
909        for child_stage_id in self
910            .child_edges
911            .get(&stage.id)
912            .map(|edges| edges.iter())
913            .into_iter()
914            .flatten()
915        {
916            let exchange_info = if let Some(parallelism) = parallelism {
917                let exchange_distribution = stage
918                    .children_exchange_distribution
919                    .as_ref()
920                    .unwrap()
921                    .get(child_stage_id)
922                    .expect("Exchange distribution is not consistent with the stage graph");
923                Some(exchange_distribution.to_prost(
924                    parallelism,
925                    catalog_reader,
926                    worker_node_manager,
927                )?)
928            } else {
929                None
930            };
931            self.complete_stage(
932                *child_stage_id,
933                exchange_info,
934                complete_stages,
935                catalog_reader,
936                worker_node_manager,
937            )
938            .await?;
939        }
940
941        Ok(())
942    }
943
944    /// Converts the `StageGraph` into a `petgraph::graph::Graph<String, String>`.
945    pub fn to_petgraph(&self) -> Graph<String, String, Directed> {
946        let mut graph = Graph::<String, String, Directed>::new();
947
948        let mut node_indices = HashMap::new();
949
950        // Add all stages as nodes
951        for (&stage_id, stage_ref) in self.stages.iter().sorted_by_key(|(id, _)| **id) {
952            let node_label = format!("Stage {}: {:?}", stage_id, stage_ref);
953            let node_index = graph.add_node(node_label);
954            node_indices.insert(stage_id, node_index);
955        }
956
957        // Add edges between stages based on child_edges
958        for (&parent_id, children) in &self.child_edges {
959            if let Some(&parent_index) = node_indices.get(&parent_id) {
960                for &child_id in children {
961                    if let Some(&child_index) = node_indices.get(&child_id) {
962                        // Add an edge from parent to child
963                        graph.add_edge(parent_index, child_index, "".to_owned());
964                    }
965                }
966            }
967        }
968
969        graph
970    }
971}
972
973struct StageGraphBuilder {
974    stages: HashMap<StageId, QueryStage>,
975    child_edges: HashMap<StageId, HashSet<StageId>>,
976    parent_edges: HashMap<StageId, HashSet<StageId>>,
977    batch_parallelism: usize,
978}
979
980impl StageGraphBuilder {
981    pub fn new(batch_parallelism: usize) -> Self {
982        Self {
983            stages: HashMap::new(),
984            child_edges: HashMap::new(),
985            parent_edges: HashMap::new(),
986            batch_parallelism,
987        }
988    }
989
990    pub fn build(self, root_stage_id: StageId) -> StageGraph {
991        StageGraph {
992            root_stage_id,
993            stages: self.stages,
994            child_edges: self.child_edges,
995            parent_edges: self.parent_edges,
996            batch_parallelism: self.batch_parallelism,
997        }
998    }
999
1000    /// Link parent stage and child stage. Maintain the mappings of parent -> child and child ->
1001    /// parent.
1002    pub fn link_to_child(&mut self, parent_id: StageId, child_id: StageId) {
1003        self.child_edges
1004            .get_mut(&parent_id)
1005            .unwrap()
1006            .insert(child_id);
1007        self.parent_edges
1008            .get_mut(&child_id)
1009            .unwrap()
1010            .insert(parent_id);
1011    }
1012
1013    pub fn add_node(&mut self, stage: QueryStage) {
1014        // Insert here so that left/root stages also has linkage.
1015        self.child_edges.insert(stage.id, HashSet::new());
1016        self.parent_edges.insert(stage.id, HashSet::new());
1017        self.stages.insert(stage.id, stage);
1018    }
1019}
1020
1021impl BatchPlanFragmenter {
1022    /// After split, the `stage_graph` in the framenter may has the stage with incomplete source
1023    /// info, we need to fetch the source info to complete the stage in this function.
1024    /// Why separate this two step(`split()` and `generate_complete_query()`)?
1025    /// The step of fetching source info is a async operation so that we can't do it in the split
1026    /// step.
1027    pub async fn generate_complete_query(self) -> SchedulerResult<Query> {
1028        let stage_graph = self.stage_graph.unwrap();
1029        let new_stage_graph = stage_graph
1030            .complete(&self.catalog_reader, &self.worker_node_manager)
1031            .await?;
1032        Ok(Query {
1033            query_id: self.query_id,
1034            stage_graph: new_stage_graph,
1035        })
1036    }
1037
1038    fn new_stage(
1039        &mut self,
1040        root: PlanRef,
1041        exchange_info: Option<ExchangeInfo>,
1042    ) -> SchedulerResult<StageId> {
1043        let next_stage_id = self.next_stage_id;
1044        self.next_stage_id.inc();
1045
1046        let mut table_scan_info = None;
1047        let mut source_info = None;
1048        let mut file_scan_info = None;
1049
1050        // For current implementation, we can guarantee that each stage has only one table
1051        // scan(except System table) or one source.
1052        if let Some(info) = self.collect_stage_table_scan(root.clone())? {
1053            table_scan_info = Some(info);
1054        } else if let Some(info) = Self::collect_stage_source(root.clone())? {
1055            source_info = Some(info);
1056        } else if let Some(info) = Self::collect_stage_file_scan(root.clone())? {
1057            file_scan_info = Some(info);
1058        }
1059
1060        let mut has_lookup_join = false;
1061        let parallelism = match root.distribution() {
1062            Distribution::Single => {
1063                if let Some(info) = &mut table_scan_info {
1064                    if let Some(partitions) = &mut info.partitions {
1065                        if partitions.len() != 1 {
1066                            // This is rare case, but it's possible on the internal state of the
1067                            // Source operator.
1068                            tracing::warn!(
1069                                "The stage has single distribution, but contains a scan of table `{}` with {} partitions. A single random worker will be assigned",
1070                                info.name,
1071                                partitions.len()
1072                            );
1073
1074                            *partitions = partitions
1075                                .drain()
1076                                .take(1)
1077                                .update(|(_, info)| {
1078                                    info.vnode_bitmap = Bitmap::ones(info.vnode_bitmap.len());
1079                                })
1080                                .collect();
1081                        }
1082                    } else {
1083                        // System table
1084                    }
1085                } else if source_info.is_some() {
1086                    return Err(SchedulerError::Internal(anyhow!(
1087                        "The stage has single distribution, but contains a source operator"
1088                    )));
1089                }
1090                1
1091            }
1092            _ => {
1093                if let Some(table_scan_info) = &table_scan_info {
1094                    table_scan_info
1095                        .partitions
1096                        .as_ref()
1097                        .map(|m| m.len())
1098                        .unwrap_or(1)
1099                } else if let Some(lookup_join_parallelism) =
1100                    self.collect_stage_lookup_join_parallelism(root.clone())?
1101                {
1102                    has_lookup_join = true;
1103                    lookup_join_parallelism
1104                } else if source_info.is_some() {
1105                    0
1106                } else if file_scan_info.is_some() {
1107                    1
1108                } else {
1109                    self.batch_parallelism
1110                }
1111            }
1112        };
1113        if source_info.is_none() && file_scan_info.is_none() && parallelism == 0 {
1114            return Err(BatchError::EmptyWorkerNodes.into());
1115        }
1116        let parallelism = if parallelism == 0 {
1117            None
1118        } else {
1119            Some(parallelism as u32)
1120        };
1121        let dml_table_id = Self::collect_dml_table_id(&root);
1122        let mut builder = QueryStageBuilder::new(
1123            next_stage_id,
1124            parallelism,
1125            exchange_info,
1126            table_scan_info,
1127            source_info,
1128            file_scan_info,
1129            has_lookup_join,
1130            dml_table_id,
1131            root.ctx().session_ctx().session_id(),
1132            root.ctx()
1133                .session_ctx()
1134                .config()
1135                .batch_enable_distributed_dml(),
1136        );
1137
1138        self.visit_node(root, &mut builder, None)?;
1139
1140        Ok(builder.finish(self.stage_graph_builder.as_mut().unwrap()))
1141    }
1142
1143    fn visit_node(
1144        &mut self,
1145        node: PlanRef,
1146        builder: &mut QueryStageBuilder,
1147        parent_exec_node: Option<&mut ExecutionPlanNode>,
1148    ) -> SchedulerResult<()> {
1149        match node.node_type() {
1150            BatchPlanNodeType::BatchExchange => {
1151                self.visit_exchange(node, builder, parent_exec_node)?;
1152            }
1153            _ => {
1154                let mut execution_plan_node = ExecutionPlanNode::try_from(node.clone())?;
1155
1156                for child in node.inputs() {
1157                    self.visit_node(child, builder, Some(&mut execution_plan_node))?;
1158                }
1159
1160                if let Some(parent) = parent_exec_node {
1161                    parent.children.push(execution_plan_node);
1162                } else {
1163                    builder.root = Some(execution_plan_node);
1164                }
1165            }
1166        }
1167        Ok(())
1168    }
1169
1170    fn visit_exchange(
1171        &mut self,
1172        node: PlanRef,
1173        builder: &mut QueryStageBuilder,
1174        parent_exec_node: Option<&mut ExecutionPlanNode>,
1175    ) -> SchedulerResult<()> {
1176        let mut execution_plan_node = ExecutionPlanNode::try_from(node.clone())?;
1177        let child_exchange_info = if let Some(parallelism) = builder.parallelism {
1178            Some(node.distribution().to_prost(
1179                parallelism,
1180                &self.catalog_reader,
1181                &self.worker_node_manager,
1182            )?)
1183        } else {
1184            None
1185        };
1186        let child_stage_id = self.new_stage(node.inputs()[0].clone(), child_exchange_info)?;
1187        execution_plan_node.source_stage_id = Some(child_stage_id);
1188        if builder.parallelism.is_none() {
1189            builder
1190                .children_exchange_distribution
1191                .insert(child_stage_id, node.distribution().clone());
1192        }
1193
1194        if let Some(parent) = parent_exec_node {
1195            parent.children.push(execution_plan_node);
1196        } else {
1197            builder.root = Some(execution_plan_node);
1198        }
1199
1200        builder.children_stages.push(child_stage_id);
1201        Ok(())
1202    }
1203
1204    /// Check whether this stage contains a source node.
1205    /// If so, use  `SplitEnumeratorImpl` to get the split info from exteneral source.
1206    ///
1207    /// For current implementation, we can guarantee that each stage has only one source.
1208    fn collect_stage_source(node: PlanRef) -> SchedulerResult<Option<SourceScanInfo>> {
1209        if node.node_type() == BatchPlanNodeType::BatchExchange {
1210            // Do not visit next stage.
1211            return Ok(None);
1212        }
1213
1214        if let Some(batch_kafka_node) = node.as_batch_kafka_scan() {
1215            let batch_kafka_scan: &BatchKafkaScan = batch_kafka_node;
1216            let source_catalog = batch_kafka_scan.source_catalog();
1217            if let Some(source_catalog) = source_catalog {
1218                let property =
1219                    ConnectorProperties::extract(source_catalog.with_properties.clone(), false)?;
1220                let timestamp_bound = batch_kafka_scan.kafka_timestamp_range_value();
1221                return Ok(Some(SourceScanInfo::new(SourceFetchInfo {
1222                    schema: batch_kafka_scan.base.schema().clone(),
1223                    connector: property,
1224                    fetch_parameters: SourceFetchParameters::KafkaTimebound {
1225                        lower: timestamp_bound.0,
1226                        upper: timestamp_bound.1,
1227                    },
1228                })));
1229            }
1230        } else if let Some(batch_iceberg_scan) = node.as_batch_iceberg_scan() {
1231            let batch_iceberg_scan: &BatchIcebergScan = batch_iceberg_scan;
1232            let task = batch_iceberg_scan.task.clone();
1233            return Ok(Some(SourceScanInfo::Unpartitioned(
1234                UnpartitionedData::Iceberg(task),
1235            )));
1236        } else if let Some(source_node) = node.as_batch_source() {
1237            // TODO: use specific batch operator instead of batch source.
1238            let source_node: &BatchSource = source_node;
1239            let source_catalog = source_node.source_catalog();
1240            if let Some(source_catalog) = source_catalog {
1241                let property =
1242                    ConnectorProperties::extract(source_catalog.with_properties.clone(), false)?;
1243                return Ok(Some(SourceScanInfo::new(SourceFetchInfo {
1244                    schema: source_node.base.schema().clone(),
1245                    connector: property,
1246                    fetch_parameters: SourceFetchParameters::Empty,
1247                })));
1248            }
1249        }
1250
1251        node.inputs()
1252            .into_iter()
1253            .find_map(|n| Self::collect_stage_source(n).transpose())
1254            .transpose()
1255    }
1256
1257    fn collect_stage_file_scan(node: PlanRef) -> SchedulerResult<Option<FileScanInfo>> {
1258        if node.node_type() == BatchPlanNodeType::BatchExchange {
1259            // Do not visit next stage.
1260            return Ok(None);
1261        }
1262
1263        if let Some(batch_file_scan) = node.as_batch_file_scan() {
1264            return Ok(Some(FileScanInfo {
1265                file_location: batch_file_scan.core.file_location(),
1266            }));
1267        }
1268
1269        node.inputs()
1270            .into_iter()
1271            .find_map(|n| Self::collect_stage_file_scan(n).transpose())
1272            .transpose()
1273    }
1274
1275    /// Check whether this stage contains a table scan node and the table's information if so.
1276    ///
1277    /// If there are multiple scan nodes in this stage, they must have the same distribution, but
1278    /// maybe different vnodes partition. We just use the same partition for all the scan nodes.
1279    fn collect_stage_table_scan(&self, node: PlanRef) -> SchedulerResult<Option<TableScanInfo>> {
1280        let build_table_scan_info = |name, table_catalog: &TableCatalog, scan_range| {
1281            let vnode_mapping = self
1282                .worker_node_manager
1283                .fragment_mapping(table_catalog.fragment_id)?;
1284            let partitions = derive_partitions(scan_range, table_catalog, &vnode_mapping)?;
1285            let info = TableScanInfo::new(name, partitions);
1286            Ok(Some(info))
1287        };
1288        if node.node_type() == BatchPlanNodeType::BatchExchange {
1289            // Do not visit next stage.
1290            return Ok(None);
1291        }
1292        if let Some(scan_node) = node.as_batch_sys_seq_scan() {
1293            let name = scan_node.core().table.name.clone();
1294            Ok(Some(TableScanInfo::system_table(name)))
1295        } else if let Some(scan_node) = node.as_batch_log_seq_scan() {
1296            build_table_scan_info(
1297                scan_node.core().table_name.clone(),
1298                &scan_node.core().table,
1299                &[],
1300            )
1301        } else if let Some(scan_node) = node.as_batch_seq_scan() {
1302            build_table_scan_info(
1303                scan_node.core().table_name().to_owned(),
1304                &scan_node.core().table_catalog,
1305                scan_node.scan_ranges(),
1306            )
1307        } else {
1308            node.inputs()
1309                .into_iter()
1310                .find_map(|n| self.collect_stage_table_scan(n).transpose())
1311                .transpose()
1312        }
1313    }
1314
1315    /// Returns the dml table id if any.
1316    fn collect_dml_table_id(node: &PlanRef) -> Option<TableId> {
1317        if node.node_type() == BatchPlanNodeType::BatchExchange {
1318            return None;
1319        }
1320        if let Some(insert) = node.as_batch_insert() {
1321            Some(insert.core.table_id)
1322        } else if let Some(update) = node.as_batch_update() {
1323            Some(update.core.table_id)
1324        } else if let Some(delete) = node.as_batch_delete() {
1325            Some(delete.core.table_id)
1326        } else {
1327            node.inputs()
1328                .into_iter()
1329                .find_map(|n| Self::collect_dml_table_id(&n))
1330        }
1331    }
1332
1333    fn collect_stage_lookup_join_parallelism(
1334        &self,
1335        node: PlanRef,
1336    ) -> SchedulerResult<Option<usize>> {
1337        if node.node_type() == BatchPlanNodeType::BatchExchange {
1338            // Do not visit next stage.
1339            return Ok(None);
1340        }
1341        if let Some(lookup_join) = node.as_batch_lookup_join() {
1342            let table_catalog = lookup_join.right_table();
1343            let vnode_mapping = self
1344                .worker_node_manager
1345                .fragment_mapping(table_catalog.fragment_id)?;
1346            let parallelism = vnode_mapping.iter().sorted().dedup().count();
1347            Ok(Some(parallelism))
1348        } else {
1349            node.inputs()
1350                .into_iter()
1351                .find_map(|n| self.collect_stage_lookup_join_parallelism(n).transpose())
1352                .transpose()
1353        }
1354    }
1355}
1356
1357/// Try to derive the partition to read from the scan range.
1358/// It can be derived if the value of the distribution key is already known.
1359fn derive_partitions(
1360    scan_ranges: &[ScanRange],
1361    table_catalog: &TableCatalog,
1362    vnode_mapping: &WorkerSlotMapping,
1363) -> SchedulerResult<HashMap<WorkerSlotId, TablePartitionInfo>> {
1364    let vnode_mapping = if table_catalog.vnode_count.value() != vnode_mapping.len() {
1365        // The vnode count mismatch occurs only in special cases where a hash-distributed fragment
1366        // contains singleton internal tables. e.g., the state table of `Source` executors.
1367        // In this case, we reduce the vnode mapping to a single vnode as only `SINGLETON_VNODE` is used.
1368        assert_eq!(
1369            table_catalog.vnode_count.value(),
1370            1,
1371            "fragment vnode count {} does not match table vnode count {}",
1372            vnode_mapping.len(),
1373            table_catalog.vnode_count.value(),
1374        );
1375        &WorkerSlotMapping::new_single(vnode_mapping.iter().next().unwrap())
1376    } else {
1377        vnode_mapping
1378    };
1379    let vnode_count = vnode_mapping.len();
1380
1381    let mut partitions: HashMap<WorkerSlotId, (BitmapBuilder, Vec<_>)> = HashMap::new();
1382
1383    if scan_ranges.is_empty() {
1384        return Ok(vnode_mapping
1385            .to_bitmaps()
1386            .into_iter()
1387            .map(|(k, vnode_bitmap)| {
1388                (
1389                    k,
1390                    TablePartitionInfo {
1391                        vnode_bitmap,
1392                        scan_ranges: vec![],
1393                    },
1394                )
1395            })
1396            .collect());
1397    }
1398
1399    let table_distribution = TableDistribution::new_from_storage_table_desc(
1400        Some(Bitmap::ones(vnode_count).into()),
1401        &table_catalog.table_desc().try_to_protobuf()?,
1402    );
1403
1404    for scan_range in scan_ranges {
1405        let vnode = scan_range.try_compute_vnode(&table_distribution);
1406        match vnode {
1407            None => {
1408                // put this scan_range to all partitions
1409                vnode_mapping.to_bitmaps().into_iter().for_each(
1410                    |(worker_slot_id, vnode_bitmap)| {
1411                        let (bitmap, scan_ranges) = partitions
1412                            .entry(worker_slot_id)
1413                            .or_insert_with(|| (BitmapBuilder::zeroed(vnode_count), vec![]));
1414                        vnode_bitmap
1415                            .iter()
1416                            .enumerate()
1417                            .for_each(|(vnode, b)| bitmap.set(vnode, b));
1418                        scan_ranges.push(scan_range.to_protobuf());
1419                    },
1420                );
1421            }
1422            // scan a single partition
1423            Some(vnode) => {
1424                let worker_slot_id = vnode_mapping[vnode];
1425                let (bitmap, scan_ranges) = partitions
1426                    .entry(worker_slot_id)
1427                    .or_insert_with(|| (BitmapBuilder::zeroed(vnode_count), vec![]));
1428                bitmap.set(vnode.to_index(), true);
1429                scan_ranges.push(scan_range.to_protobuf());
1430            }
1431        }
1432    }
1433
1434    Ok(partitions
1435        .into_iter()
1436        .map(|(k, (bitmap, scan_ranges))| {
1437            (
1438                k,
1439                TablePartitionInfo {
1440                    vnode_bitmap: bitmap.finish(),
1441                    scan_ranges,
1442                },
1443            )
1444        })
1445        .collect())
1446}
1447
1448#[cfg(test)]
1449mod tests {
1450    use std::collections::{HashMap, HashSet};
1451
1452    use risingwave_pb::batch_plan::plan_node::NodeBody;
1453
1454    use crate::optimizer::plan_node::BatchPlanNodeType;
1455    use crate::scheduler::plan_fragmenter::StageId;
1456
1457    #[tokio::test]
1458    async fn test_fragmenter() {
1459        let query = crate::scheduler::distributed::tests::create_query().await;
1460
1461        assert_eq!(query.stage_graph.root_stage_id, 0.into());
1462        assert_eq!(query.stage_graph.stages.len(), 4);
1463
1464        // Check the mappings of child edges.
1465        assert_eq!(
1466            query.stage_graph.child_edges[&0.into()],
1467            HashSet::from_iter([1.into()])
1468        );
1469        assert_eq!(
1470            query.stage_graph.child_edges[&1.into()],
1471            HashSet::from_iter([2.into(), 3.into()])
1472        );
1473        assert_eq!(query.stage_graph.child_edges[&2.into()], HashSet::new());
1474        assert_eq!(query.stage_graph.child_edges[&3.into()], HashSet::new());
1475
1476        // Check the mappings of parent edges.
1477        assert_eq!(query.stage_graph.parent_edges[&0.into()], HashSet::new());
1478        assert_eq!(
1479            query.stage_graph.parent_edges[&1.into()],
1480            HashSet::from_iter([0.into()])
1481        );
1482        assert_eq!(
1483            query.stage_graph.parent_edges[&2.into()],
1484            HashSet::from_iter([1.into()])
1485        );
1486        assert_eq!(
1487            query.stage_graph.parent_edges[&3.into()],
1488            HashSet::from_iter([1.into()])
1489        );
1490
1491        // Verify topology order
1492        {
1493            let stage_id_to_pos: HashMap<StageId, usize> = query
1494                .stage_graph
1495                .stage_ids_by_topo_order()
1496                .enumerate()
1497                .map(|(pos, stage_id)| (stage_id, pos))
1498                .collect();
1499
1500            for stage_id in query.stage_graph.stages.keys() {
1501                let stage_pos = stage_id_to_pos[stage_id];
1502                for child_stage_id in &query.stage_graph.child_edges[stage_id] {
1503                    let child_pos = stage_id_to_pos[child_stage_id];
1504                    assert!(stage_pos > child_pos);
1505                }
1506            }
1507        }
1508
1509        // Check plan node in each stages.
1510        let root_exchange = query.stage_graph.stages.get(&0.into()).unwrap();
1511        assert_eq!(
1512            root_exchange.root.node_type(),
1513            BatchPlanNodeType::BatchExchange
1514        );
1515        assert_eq!(root_exchange.root.source_stage_id, Some(1.into()));
1516        assert!(matches!(root_exchange.root.node, NodeBody::Exchange(_)));
1517        assert_eq!(root_exchange.parallelism, Some(1));
1518        assert!(!root_exchange.has_table_scan());
1519
1520        let join_node = query.stage_graph.stages.get(&1.into()).unwrap();
1521        assert_eq!(join_node.root.node_type(), BatchPlanNodeType::BatchHashJoin);
1522        assert_eq!(join_node.parallelism, Some(24));
1523
1524        assert!(matches!(join_node.root.node, NodeBody::HashJoin(_)));
1525        assert_eq!(join_node.root.source_stage_id, None);
1526        assert_eq!(2, join_node.root.children.len());
1527
1528        assert!(matches!(
1529            join_node.root.children[0].node,
1530            NodeBody::Exchange(_)
1531        ));
1532        assert_eq!(join_node.root.children[0].source_stage_id, Some(2.into()));
1533        assert_eq!(0, join_node.root.children[0].children.len());
1534
1535        assert!(matches!(
1536            join_node.root.children[1].node,
1537            NodeBody::Exchange(_)
1538        ));
1539        assert_eq!(join_node.root.children[1].source_stage_id, Some(3.into()));
1540        assert_eq!(0, join_node.root.children[1].children.len());
1541        assert!(!join_node.has_table_scan());
1542
1543        let scan_node1 = query.stage_graph.stages.get(&2.into()).unwrap();
1544        assert_eq!(scan_node1.root.node_type(), BatchPlanNodeType::BatchSeqScan);
1545        assert_eq!(scan_node1.root.source_stage_id, None);
1546        assert_eq!(0, scan_node1.root.children.len());
1547        assert!(scan_node1.has_table_scan());
1548
1549        let scan_node2 = query.stage_graph.stages.get(&3.into()).unwrap();
1550        assert_eq!(scan_node2.root.node_type(), BatchPlanNodeType::BatchFilter);
1551        assert_eq!(scan_node2.root.source_stage_id, None);
1552        assert_eq!(1, scan_node2.root.children.len());
1553        assert!(scan_node2.has_table_scan());
1554    }
1555}