risingwave_frontend/scheduler/
plan_fragmenter.rs

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