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;
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    BatchPosixFsEnumerator, 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::TableCatalog;
58use crate::catalog::TableId;
59use crate::catalog::catalog_service::CatalogReader;
60use crate::optimizer::plan_node::generic::{GenericPlanRef, PhysicalPlanRef};
61use crate::optimizer::plan_node::utils::to_iceberg_time_travel_as_of;
62use crate::optimizer::plan_node::{
63    BatchIcebergScan, BatchKafkaScan, BatchPlanNodeType, BatchPlanRef as PlanRef, BatchSource,
64    PlanNodeId,
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: BatchPlanNodeType,
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) -> BatchPlanNodeType {
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            (ConnectorProperties::BatchPosixFs(prop), SourceFetchParameters::Empty) => {
390                use risingwave_connector::source::SplitEnumerator;
391                let mut enumerator = BatchPosixFsEnumerator::new(
392                    *prop,
393                    risingwave_connector::source::SourceEnumeratorContext::dummy().into(),
394                )
395                .await?;
396                let splits = enumerator.list_splits().await?;
397                let res = splits
398                    .into_iter()
399                    .map(SplitImpl::BatchPosixFs)
400                    .collect_vec();
401
402                Ok(SourceScanInfo::Complete(res))
403            }
404            (
405                ConnectorProperties::Iceberg(prop),
406                SourceFetchParameters::IcebergSpecificInfo(iceberg_specific_info),
407            ) => {
408                let iceberg_enumerator =
409                    IcebergSplitEnumerator::new(*prop, SourceEnumeratorContext::dummy().into())
410                        .await?;
411
412                let time_travel_info = to_iceberg_time_travel_as_of(&fetch_info.as_of, &timezone)?;
413
414                let split_info = iceberg_enumerator
415                    .list_splits_batch(
416                        fetch_info.schema,
417                        time_travel_info,
418                        batch_parallelism,
419                        iceberg_specific_info.iceberg_scan_type,
420                        iceberg_specific_info.predicate,
421                    )
422                    .await?
423                    .into_iter()
424                    .map(SplitImpl::Iceberg)
425                    .collect_vec();
426
427                Ok(SourceScanInfo::Complete(split_info))
428            }
429            (connector, _) => Err(SchedulerError::Internal(anyhow!(
430                "Unsupported to query directly from this {} source, \
431                 please create a table or streaming job from it",
432                connector.kind()
433            ))),
434        }
435    }
436
437    pub fn split_info(&self) -> SchedulerResult<&Vec<SplitImpl>> {
438        match self {
439            Self::Incomplete(_) => Err(SchedulerError::Internal(anyhow!(
440                "Should not get split info from incomplete source scan info"
441            ))),
442            Self::Complete(split_info) => Ok(split_info),
443        }
444    }
445}
446
447#[derive(Clone, Debug)]
448pub struct TableScanInfo {
449    /// The name of the table to scan.
450    name: String,
451
452    /// Indicates the table partitions to be read by scan tasks. Unnecessary partitions are already
453    /// pruned.
454    ///
455    /// For singleton table, this field is still `Some` and only contains a single partition with
456    /// full vnode bitmap, since we need to know where to schedule the singleton scan task.
457    ///
458    /// `None` iff the table is a system table.
459    partitions: Option<HashMap<WorkerSlotId, TablePartitionInfo>>,
460}
461
462impl TableScanInfo {
463    /// For normal tables, `partitions` should always be `Some`.
464    pub fn new(name: String, partitions: HashMap<WorkerSlotId, TablePartitionInfo>) -> Self {
465        Self {
466            name,
467            partitions: Some(partitions),
468        }
469    }
470
471    /// For system table, there's no partition info.
472    pub fn system_table(name: String) -> Self {
473        Self {
474            name,
475            partitions: None,
476        }
477    }
478
479    pub fn name(&self) -> &str {
480        self.name.as_ref()
481    }
482
483    pub fn partitions(&self) -> Option<&HashMap<WorkerSlotId, TablePartitionInfo>> {
484        self.partitions.as_ref()
485    }
486}
487
488#[derive(Clone, Debug)]
489pub struct TablePartitionInfo {
490    pub vnode_bitmap: Bitmap,
491    pub scan_ranges: Vec<ScanRangeProto>,
492}
493
494#[derive(Clone, Debug, EnumAsInner)]
495pub enum PartitionInfo {
496    Table(TablePartitionInfo),
497    Source(Vec<SplitImpl>),
498    File(Vec<String>),
499}
500
501#[derive(Clone, Debug)]
502pub struct FileScanInfo {
503    pub file_location: Vec<String>,
504}
505
506/// Fragment part of `Query`.
507#[derive(Clone)]
508pub struct QueryStage {
509    pub query_id: QueryId,
510    pub id: StageId,
511    pub root: Arc<ExecutionPlanNode>,
512    pub exchange_info: Option<ExchangeInfo>,
513    pub parallelism: Option<u32>,
514    /// Indicates whether this stage contains a table scan node and the table's information if so.
515    pub table_scan_info: Option<TableScanInfo>,
516    pub source_info: Option<SourceScanInfo>,
517    pub file_scan_info: Option<FileScanInfo>,
518    pub has_lookup_join: bool,
519    pub dml_table_id: Option<TableId>,
520    pub session_id: SessionId,
521    pub batch_enable_distributed_dml: bool,
522
523    /// Used to generate exchange information when complete source scan information.
524    children_exchange_distribution: Option<HashMap<StageId, Distribution>>,
525}
526
527impl QueryStage {
528    /// If true, this stage contains table scan executor that creates
529    /// Hummock iterators to read data from table. The iterator is initialized during
530    /// the executor building process on the batch execution engine.
531    pub fn has_table_scan(&self) -> bool {
532        self.table_scan_info.is_some()
533    }
534
535    /// If true, this stage contains lookup join executor.
536    /// We need to delay epoch unpin util the end of the query.
537    pub fn has_lookup_join(&self) -> bool {
538        self.has_lookup_join
539    }
540
541    pub fn clone_with_exchange_info(
542        &self,
543        exchange_info: Option<ExchangeInfo>,
544        parallelism: Option<u32>,
545    ) -> Self {
546        if let Some(exchange_info) = exchange_info {
547            return Self {
548                query_id: self.query_id.clone(),
549                id: self.id,
550                root: self.root.clone(),
551                exchange_info: Some(exchange_info),
552                parallelism,
553                table_scan_info: self.table_scan_info.clone(),
554                source_info: self.source_info.clone(),
555                file_scan_info: self.file_scan_info.clone(),
556                has_lookup_join: self.has_lookup_join,
557                dml_table_id: self.dml_table_id,
558                session_id: self.session_id,
559                batch_enable_distributed_dml: self.batch_enable_distributed_dml,
560                children_exchange_distribution: self.children_exchange_distribution.clone(),
561            };
562        }
563        self.clone()
564    }
565
566    pub fn clone_with_exchange_info_and_complete_source_info(
567        &self,
568        exchange_info: Option<ExchangeInfo>,
569        source_info: SourceScanInfo,
570        task_parallelism: u32,
571    ) -> Self {
572        assert!(matches!(source_info, SourceScanInfo::Complete(_)));
573        let exchange_info = if let Some(exchange_info) = exchange_info {
574            Some(exchange_info)
575        } else {
576            self.exchange_info.clone()
577        };
578        Self {
579            query_id: self.query_id.clone(),
580            id: self.id,
581            root: self.root.clone(),
582            exchange_info,
583            parallelism: Some(task_parallelism),
584            table_scan_info: self.table_scan_info.clone(),
585            source_info: Some(source_info),
586            file_scan_info: self.file_scan_info.clone(),
587            has_lookup_join: self.has_lookup_join,
588            dml_table_id: self.dml_table_id,
589            session_id: self.session_id,
590            batch_enable_distributed_dml: self.batch_enable_distributed_dml,
591            children_exchange_distribution: None,
592        }
593    }
594}
595
596impl Debug for QueryStage {
597    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
598        f.debug_struct("QueryStage")
599            .field("id", &self.id)
600            .field("parallelism", &self.parallelism)
601            .field("exchange_info", &self.exchange_info)
602            .field("has_table_scan", &self.has_table_scan())
603            .finish()
604    }
605}
606
607impl Serialize for QueryStage {
608    fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
609    where
610        S: serde::Serializer,
611    {
612        let mut state = serializer.serialize_struct("QueryStage", 3)?;
613        state.serialize_field("root", &self.root)?;
614        state.serialize_field("parallelism", &self.parallelism)?;
615        state.serialize_field("exchange_info", &self.exchange_info)?;
616        state.end()
617    }
618}
619
620pub type QueryStageRef = Arc<QueryStage>;
621
622struct QueryStageBuilder {
623    query_id: QueryId,
624    id: StageId,
625    root: Option<Arc<ExecutionPlanNode>>,
626    parallelism: Option<u32>,
627    exchange_info: Option<ExchangeInfo>,
628
629    children_stages: Vec<QueryStageRef>,
630    /// See also [`QueryStage::table_scan_info`].
631    table_scan_info: Option<TableScanInfo>,
632    source_info: Option<SourceScanInfo>,
633    file_scan_file: Option<FileScanInfo>,
634    has_lookup_join: bool,
635    dml_table_id: Option<TableId>,
636    session_id: SessionId,
637    batch_enable_distributed_dml: bool,
638
639    children_exchange_distribution: HashMap<StageId, Distribution>,
640}
641
642impl QueryStageBuilder {
643    #[allow(clippy::too_many_arguments)]
644    fn new(
645        id: StageId,
646        query_id: QueryId,
647        parallelism: Option<u32>,
648        exchange_info: Option<ExchangeInfo>,
649        table_scan_info: Option<TableScanInfo>,
650        source_info: Option<SourceScanInfo>,
651        file_scan_file: Option<FileScanInfo>,
652        has_lookup_join: bool,
653        dml_table_id: Option<TableId>,
654        session_id: SessionId,
655        batch_enable_distributed_dml: bool,
656    ) -> Self {
657        Self {
658            query_id,
659            id,
660            root: None,
661            parallelism,
662            exchange_info,
663            children_stages: vec![],
664            table_scan_info,
665            source_info,
666            file_scan_file,
667            has_lookup_join,
668            dml_table_id,
669            session_id,
670            batch_enable_distributed_dml,
671            children_exchange_distribution: HashMap::new(),
672        }
673    }
674
675    fn finish(self, stage_graph_builder: &mut StageGraphBuilder) -> QueryStageRef {
676        let children_exchange_distribution = if self.parallelism.is_none() {
677            Some(self.children_exchange_distribution)
678        } else {
679            None
680        };
681        let stage = Arc::new(QueryStage {
682            query_id: self.query_id,
683            id: self.id,
684            root: self.root.unwrap(),
685            exchange_info: self.exchange_info,
686            parallelism: self.parallelism,
687            table_scan_info: self.table_scan_info,
688            source_info: self.source_info,
689            file_scan_info: self.file_scan_file,
690            has_lookup_join: self.has_lookup_join,
691            dml_table_id: self.dml_table_id,
692            session_id: self.session_id,
693            batch_enable_distributed_dml: self.batch_enable_distributed_dml,
694            children_exchange_distribution,
695        });
696
697        stage_graph_builder.add_node(stage.clone());
698        for child_stage in self.children_stages {
699            stage_graph_builder.link_to_child(self.id, child_stage.id);
700        }
701        stage
702    }
703}
704
705/// Maintains how each stage are connected.
706#[derive(Debug, Serialize)]
707#[cfg_attr(test, derive(Clone))]
708pub struct StageGraph {
709    pub root_stage_id: StageId,
710    pub stages: HashMap<StageId, QueryStageRef>,
711    /// Traverse from top to down. Used in split plan into stages.
712    child_edges: HashMap<StageId, HashSet<StageId>>,
713    /// Traverse from down to top. Used in schedule each stage.
714    parent_edges: HashMap<StageId, HashSet<StageId>>,
715
716    batch_parallelism: usize,
717}
718
719impl StageGraph {
720    pub fn get_child_stages_unchecked(&self, stage_id: &StageId) -> &HashSet<StageId> {
721        self.child_edges.get(stage_id).unwrap()
722    }
723
724    pub fn get_child_stages(&self, stage_id: &StageId) -> Option<&HashSet<StageId>> {
725        self.child_edges.get(stage_id)
726    }
727
728    /// Returns stage ids in topology order, s.t. child stage always appears before its parent.
729    pub fn stage_ids_by_topo_order(&self) -> impl Iterator<Item = StageId> {
730        let mut stack = Vec::with_capacity(self.stages.len());
731        stack.push(self.root_stage_id);
732        let mut ret = Vec::with_capacity(self.stages.len());
733        let mut existing = HashSet::with_capacity(self.stages.len());
734
735        while let Some(s) = stack.pop() {
736            if !existing.contains(&s) {
737                ret.push(s);
738                existing.insert(s);
739                stack.extend(&self.child_edges[&s]);
740            }
741        }
742
743        ret.into_iter().rev()
744    }
745
746    async fn complete(
747        self,
748        catalog_reader: &CatalogReader,
749        worker_node_manager: &WorkerNodeSelector,
750        timezone: String,
751    ) -> SchedulerResult<StageGraph> {
752        let mut complete_stages = HashMap::new();
753        self.complete_stage(
754            self.stages.get(&self.root_stage_id).unwrap().clone(),
755            None,
756            &mut complete_stages,
757            catalog_reader,
758            worker_node_manager,
759            timezone,
760        )
761        .await?;
762        Ok(StageGraph {
763            root_stage_id: self.root_stage_id,
764            stages: complete_stages,
765            child_edges: self.child_edges,
766            parent_edges: self.parent_edges,
767            batch_parallelism: self.batch_parallelism,
768        })
769    }
770
771    #[async_recursion]
772    async fn complete_stage(
773        &self,
774        stage: QueryStageRef,
775        exchange_info: Option<ExchangeInfo>,
776        complete_stages: &mut HashMap<StageId, QueryStageRef>,
777        catalog_reader: &CatalogReader,
778        worker_node_manager: &WorkerNodeSelector,
779        timezone: String,
780    ) -> SchedulerResult<()> {
781        let parallelism = if stage.parallelism.is_some() {
782            // If the stage has parallelism, it means it's a complete stage.
783            complete_stages.insert(
784                stage.id,
785                Arc::new(stage.clone_with_exchange_info(exchange_info, stage.parallelism)),
786            );
787            None
788        } else if matches!(stage.source_info, Some(SourceScanInfo::Incomplete(_))) {
789            let complete_source_info = stage
790                .source_info
791                .as_ref()
792                .unwrap()
793                .clone()
794                .complete(self.batch_parallelism, timezone.to_owned())
795                .await?;
796
797            // For batch reading file source, the number of files involved is typically large.
798            // In order to avoid generating a task for each file, the parallelism of tasks is limited here.
799            // The minimum `task_parallelism` is 1. Additionally, `task_parallelism`
800            // must be greater than the number of files to read. Therefore, we first take the
801            // minimum of the number of files and (self.batch_parallelism / 2). If the number of
802            // files is 0, we set task_parallelism to 1.
803
804            let task_parallelism = match &stage.source_info {
805                Some(SourceScanInfo::Incomplete(source_fetch_info)) => {
806                    match source_fetch_info.connector {
807                        ConnectorProperties::Gcs(_)
808                        | ConnectorProperties::OpendalS3(_)
809                        | ConnectorProperties::Azblob(_) => (min(
810                            complete_source_info.split_info().unwrap().len() as u32,
811                            (self.batch_parallelism / 2) as u32,
812                        ))
813                        .max(1),
814                        _ => complete_source_info.split_info().unwrap().len() as u32,
815                    }
816                }
817                _ => unreachable!(),
818            };
819            // 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.
820            // todo(wcy-fdu): Currently it will be divided into half of batch_parallelism groups, and this will be changed to configurable later.
821            let complete_stage = Arc::new(stage.clone_with_exchange_info_and_complete_source_info(
822                exchange_info,
823                complete_source_info,
824                task_parallelism,
825            ));
826            let parallelism = complete_stage.parallelism;
827            complete_stages.insert(stage.id, complete_stage);
828            parallelism
829        } else {
830            assert!(stage.file_scan_info.is_some());
831            let parallelism = min(
832                self.batch_parallelism / 2,
833                stage.file_scan_info.as_ref().unwrap().file_location.len(),
834            );
835            complete_stages.insert(
836                stage.id,
837                Arc::new(stage.clone_with_exchange_info(exchange_info, Some(parallelism as u32))),
838            );
839            None
840        };
841
842        for child_stage_id in self.child_edges.get(&stage.id).unwrap_or(&HashSet::new()) {
843            let exchange_info = if let Some(parallelism) = parallelism {
844                let exchange_distribution = stage
845                    .children_exchange_distribution
846                    .as_ref()
847                    .unwrap()
848                    .get(child_stage_id)
849                    .expect("Exchange distribution is not consistent with the stage graph");
850                Some(exchange_distribution.to_prost(
851                    parallelism,
852                    catalog_reader,
853                    worker_node_manager,
854                )?)
855            } else {
856                None
857            };
858            self.complete_stage(
859                self.stages.get(child_stage_id).unwrap().clone(),
860                exchange_info,
861                complete_stages,
862                catalog_reader,
863                worker_node_manager,
864                timezone.to_owned(),
865            )
866            .await?;
867        }
868
869        Ok(())
870    }
871
872    /// Converts the `StageGraph` into a `petgraph::graph::Graph<String, String>`.
873    pub fn to_petgraph(&self) -> Graph<String, String, Directed> {
874        let mut graph = Graph::<String, String, Directed>::new();
875
876        let mut node_indices = HashMap::new();
877
878        // Add all stages as nodes
879        for (&stage_id, stage_ref) in self.stages.iter().sorted_by_key(|(id, _)| **id) {
880            let node_label = format!("Stage {}: {:?}", stage_id, stage_ref);
881            let node_index = graph.add_node(node_label);
882            node_indices.insert(stage_id, node_index);
883        }
884
885        // Add edges between stages based on child_edges
886        for (&parent_id, children) in &self.child_edges {
887            if let Some(&parent_index) = node_indices.get(&parent_id) {
888                for &child_id in children {
889                    if let Some(&child_index) = node_indices.get(&child_id) {
890                        // Add an edge from parent to child
891                        graph.add_edge(parent_index, child_index, "".to_owned());
892                    }
893                }
894            }
895        }
896
897        graph
898    }
899}
900
901struct StageGraphBuilder {
902    stages: HashMap<StageId, QueryStageRef>,
903    child_edges: HashMap<StageId, HashSet<StageId>>,
904    parent_edges: HashMap<StageId, HashSet<StageId>>,
905    batch_parallelism: usize,
906}
907
908impl StageGraphBuilder {
909    pub fn new(batch_parallelism: usize) -> Self {
910        Self {
911            stages: HashMap::new(),
912            child_edges: HashMap::new(),
913            parent_edges: HashMap::new(),
914            batch_parallelism,
915        }
916    }
917
918    pub fn build(self, root_stage_id: StageId) -> StageGraph {
919        StageGraph {
920            root_stage_id,
921            stages: self.stages,
922            child_edges: self.child_edges,
923            parent_edges: self.parent_edges,
924            batch_parallelism: self.batch_parallelism,
925        }
926    }
927
928    /// Link parent stage and child stage. Maintain the mappings of parent -> child and child ->
929    /// parent.
930    pub fn link_to_child(&mut self, parent_id: StageId, child_id: StageId) {
931        self.child_edges
932            .get_mut(&parent_id)
933            .unwrap()
934            .insert(child_id);
935        self.parent_edges
936            .get_mut(&child_id)
937            .unwrap()
938            .insert(parent_id);
939    }
940
941    pub fn add_node(&mut self, stage: QueryStageRef) {
942        // Insert here so that left/root stages also has linkage.
943        self.child_edges.insert(stage.id, HashSet::new());
944        self.parent_edges.insert(stage.id, HashSet::new());
945        self.stages.insert(stage.id, stage);
946    }
947}
948
949impl BatchPlanFragmenter {
950    /// After split, the `stage_graph` in the framenter may has the stage with incomplete source
951    /// info, we need to fetch the source info to complete the stage in this function.
952    /// Why separate this two step(`split()` and `generate_complete_query()`)?
953    /// The step of fetching source info is a async operation so that we can't do it in the split
954    /// step.
955    pub async fn generate_complete_query(self) -> SchedulerResult<Query> {
956        let stage_graph = self.stage_graph.unwrap();
957        let new_stage_graph = stage_graph
958            .complete(
959                &self.catalog_reader,
960                &self.worker_node_manager,
961                self.timezone.to_owned(),
962            )
963            .await?;
964        Ok(Query {
965            query_id: self.query_id,
966            stage_graph: new_stage_graph,
967        })
968    }
969
970    fn new_stage(
971        &mut self,
972        root: PlanRef,
973        exchange_info: Option<ExchangeInfo>,
974    ) -> SchedulerResult<QueryStageRef> {
975        let next_stage_id = self.next_stage_id;
976        self.next_stage_id += 1;
977
978        let mut table_scan_info = self.collect_stage_table_scan(root.clone())?;
979        // For current implementation, we can guarantee that each stage has only one table
980        // scan(except System table) or one source.
981        let source_info = if table_scan_info.is_none() {
982            Self::collect_stage_source(root.clone())?
983        } else {
984            None
985        };
986
987        let file_scan_info = if table_scan_info.is_none() && source_info.is_none() {
988            Self::collect_stage_file_scan(root.clone())?
989        } else {
990            None
991        };
992
993        let mut has_lookup_join = false;
994        let parallelism = match root.distribution() {
995            Distribution::Single => {
996                if let Some(info) = &mut table_scan_info {
997                    if let Some(partitions) = &mut info.partitions {
998                        if partitions.len() != 1 {
999                            // This is rare case, but it's possible on the internal state of the
1000                            // Source operator.
1001                            tracing::warn!(
1002                                "The stage has single distribution, but contains a scan of table `{}` with {} partitions. A single random worker will be assigned",
1003                                info.name,
1004                                partitions.len()
1005                            );
1006
1007                            *partitions = partitions
1008                                .drain()
1009                                .take(1)
1010                                .update(|(_, info)| {
1011                                    info.vnode_bitmap = Bitmap::ones(info.vnode_bitmap.len());
1012                                })
1013                                .collect();
1014                        }
1015                    } else {
1016                        // System table
1017                    }
1018                } else if source_info.is_some() {
1019                    return Err(SchedulerError::Internal(anyhow!(
1020                        "The stage has single distribution, but contains a source operator"
1021                    )));
1022                }
1023                1
1024            }
1025            _ => {
1026                if let Some(table_scan_info) = &table_scan_info {
1027                    table_scan_info
1028                        .partitions
1029                        .as_ref()
1030                        .map(|m| m.len())
1031                        .unwrap_or(1)
1032                } else if let Some(lookup_join_parallelism) =
1033                    self.collect_stage_lookup_join_parallelism(root.clone())?
1034                {
1035                    has_lookup_join = true;
1036                    lookup_join_parallelism
1037                } else if source_info.is_some() {
1038                    0
1039                } else if file_scan_info.is_some() {
1040                    1
1041                } else {
1042                    self.batch_parallelism
1043                }
1044            }
1045        };
1046        if source_info.is_none() && file_scan_info.is_none() && parallelism == 0 {
1047            return Err(BatchError::EmptyWorkerNodes.into());
1048        }
1049        let parallelism = if parallelism == 0 {
1050            None
1051        } else {
1052            Some(parallelism as u32)
1053        };
1054        let dml_table_id = Self::collect_dml_table_id(&root);
1055        let mut builder = QueryStageBuilder::new(
1056            next_stage_id,
1057            self.query_id.clone(),
1058            parallelism,
1059            exchange_info,
1060            table_scan_info,
1061            source_info,
1062            file_scan_info,
1063            has_lookup_join,
1064            dml_table_id,
1065            root.ctx().session_ctx().session_id(),
1066            root.ctx()
1067                .session_ctx()
1068                .config()
1069                .batch_enable_distributed_dml(),
1070        );
1071
1072        self.visit_node(root, &mut builder, None)?;
1073
1074        Ok(builder.finish(self.stage_graph_builder.as_mut().unwrap()))
1075    }
1076
1077    fn visit_node(
1078        &mut self,
1079        node: PlanRef,
1080        builder: &mut QueryStageBuilder,
1081        parent_exec_node: Option<&mut ExecutionPlanNode>,
1082    ) -> SchedulerResult<()> {
1083        match node.node_type() {
1084            BatchPlanNodeType::BatchExchange => {
1085                self.visit_exchange(node.clone(), builder, parent_exec_node)?;
1086            }
1087            _ => {
1088                let mut execution_plan_node = ExecutionPlanNode::try_from(node.clone())?;
1089
1090                for child in node.inputs() {
1091                    self.visit_node(child, builder, Some(&mut execution_plan_node))?;
1092                }
1093
1094                if let Some(parent) = parent_exec_node {
1095                    parent.children.push(Arc::new(execution_plan_node));
1096                } else {
1097                    builder.root = Some(Arc::new(execution_plan_node));
1098                }
1099            }
1100        }
1101        Ok(())
1102    }
1103
1104    fn visit_exchange(
1105        &mut self,
1106        node: PlanRef,
1107        builder: &mut QueryStageBuilder,
1108        parent_exec_node: Option<&mut ExecutionPlanNode>,
1109    ) -> SchedulerResult<()> {
1110        let mut execution_plan_node = ExecutionPlanNode::try_from(node.clone())?;
1111        let child_exchange_info = if let Some(parallelism) = builder.parallelism {
1112            Some(node.distribution().to_prost(
1113                parallelism,
1114                &self.catalog_reader,
1115                &self.worker_node_manager,
1116            )?)
1117        } else {
1118            None
1119        };
1120        let child_stage = self.new_stage(node.inputs()[0].clone(), child_exchange_info)?;
1121        execution_plan_node.source_stage_id = Some(child_stage.id);
1122        if builder.parallelism.is_none() {
1123            builder
1124                .children_exchange_distribution
1125                .insert(child_stage.id, node.distribution().clone());
1126        }
1127
1128        if let Some(parent) = parent_exec_node {
1129            parent.children.push(Arc::new(execution_plan_node));
1130        } else {
1131            builder.root = Some(Arc::new(execution_plan_node));
1132        }
1133
1134        builder.children_stages.push(child_stage);
1135        Ok(())
1136    }
1137
1138    /// Check whether this stage contains a source node.
1139    /// If so, use  `SplitEnumeratorImpl` to get the split info from exteneral source.
1140    ///
1141    /// For current implementation, we can guarantee that each stage has only one source.
1142    fn collect_stage_source(node: PlanRef) -> SchedulerResult<Option<SourceScanInfo>> {
1143        if node.node_type() == BatchPlanNodeType::BatchExchange {
1144            // Do not visit next stage.
1145            return Ok(None);
1146        }
1147
1148        if let Some(batch_kafka_node) = node.as_batch_kafka_scan() {
1149            let batch_kafka_scan: &BatchKafkaScan = batch_kafka_node;
1150            let source_catalog = batch_kafka_scan.source_catalog();
1151            if let Some(source_catalog) = source_catalog {
1152                let property =
1153                    ConnectorProperties::extract(source_catalog.with_properties.clone(), false)?;
1154                let timestamp_bound = batch_kafka_scan.kafka_timestamp_range_value();
1155                return Ok(Some(SourceScanInfo::new(SourceFetchInfo {
1156                    schema: batch_kafka_scan.base.schema().clone(),
1157                    connector: property,
1158                    fetch_parameters: SourceFetchParameters::KafkaTimebound {
1159                        lower: timestamp_bound.0,
1160                        upper: timestamp_bound.1,
1161                    },
1162                    as_of: None,
1163                })));
1164            }
1165        } else if let Some(batch_iceberg_scan) = node.as_batch_iceberg_scan() {
1166            let batch_iceberg_scan: &BatchIcebergScan = batch_iceberg_scan;
1167            let source_catalog = batch_iceberg_scan.source_catalog();
1168            if let Some(source_catalog) = source_catalog {
1169                let property =
1170                    ConnectorProperties::extract(source_catalog.with_properties.clone(), false)?;
1171                let as_of = batch_iceberg_scan.as_of();
1172                return Ok(Some(SourceScanInfo::new(SourceFetchInfo {
1173                    schema: batch_iceberg_scan.base.schema().clone(),
1174                    connector: property,
1175                    fetch_parameters: SourceFetchParameters::IcebergSpecificInfo(
1176                        IcebergSpecificInfo {
1177                            predicate: batch_iceberg_scan.predicate.clone(),
1178                            iceberg_scan_type: batch_iceberg_scan.iceberg_scan_type(),
1179                        },
1180                    ),
1181                    as_of,
1182                })));
1183            }
1184        } else if let Some(source_node) = node.as_batch_source() {
1185            // TODO: use specific batch operator instead of batch source.
1186            let source_node: &BatchSource = source_node;
1187            let source_catalog = source_node.source_catalog();
1188            if let Some(source_catalog) = source_catalog {
1189                let property =
1190                    ConnectorProperties::extract(source_catalog.with_properties.clone(), false)?;
1191                let as_of = source_node.as_of();
1192                return Ok(Some(SourceScanInfo::new(SourceFetchInfo {
1193                    schema: source_node.base.schema().clone(),
1194                    connector: property,
1195                    fetch_parameters: SourceFetchParameters::Empty,
1196                    as_of,
1197                })));
1198            }
1199        }
1200
1201        node.inputs()
1202            .into_iter()
1203            .find_map(|n| Self::collect_stage_source(n).transpose())
1204            .transpose()
1205    }
1206
1207    fn collect_stage_file_scan(node: PlanRef) -> SchedulerResult<Option<FileScanInfo>> {
1208        if node.node_type() == BatchPlanNodeType::BatchExchange {
1209            // Do not visit next stage.
1210            return Ok(None);
1211        }
1212
1213        if let Some(batch_file_scan) = node.as_batch_file_scan() {
1214            return Ok(Some(FileScanInfo {
1215                file_location: batch_file_scan.core.file_location().clone(),
1216            }));
1217        }
1218
1219        node.inputs()
1220            .into_iter()
1221            .find_map(|n| Self::collect_stage_file_scan(n).transpose())
1222            .transpose()
1223    }
1224
1225    /// Check whether this stage contains a table scan node and the table's information if so.
1226    ///
1227    /// If there are multiple scan nodes in this stage, they must have the same distribution, but
1228    /// maybe different vnodes partition. We just use the same partition for all the scan nodes.
1229    fn collect_stage_table_scan(&self, node: PlanRef) -> SchedulerResult<Option<TableScanInfo>> {
1230        let build_table_scan_info = |name, table_catalog: &TableCatalog, scan_range| {
1231            let vnode_mapping = self
1232                .worker_node_manager
1233                .fragment_mapping(table_catalog.fragment_id)?;
1234            let partitions = derive_partitions(scan_range, table_catalog, &vnode_mapping)?;
1235            let info = TableScanInfo::new(name, partitions);
1236            Ok(Some(info))
1237        };
1238        if node.node_type() == BatchPlanNodeType::BatchExchange {
1239            // Do not visit next stage.
1240            return Ok(None);
1241        }
1242        if let Some(scan_node) = node.as_batch_sys_seq_scan() {
1243            let name = scan_node.core().table.name.clone();
1244            Ok(Some(TableScanInfo::system_table(name)))
1245        } else if let Some(scan_node) = node.as_batch_log_seq_scan() {
1246            build_table_scan_info(
1247                scan_node.core().table_name.to_owned(),
1248                &scan_node.core().table,
1249                &[],
1250            )
1251        } else if let Some(scan_node) = node.as_batch_seq_scan() {
1252            build_table_scan_info(
1253                scan_node.core().table_name().to_owned(),
1254                &scan_node.core().table_catalog,
1255                scan_node.scan_ranges(),
1256            )
1257        } else {
1258            node.inputs()
1259                .into_iter()
1260                .find_map(|n| self.collect_stage_table_scan(n).transpose())
1261                .transpose()
1262        }
1263    }
1264
1265    /// Returns the dml table id if any.
1266    fn collect_dml_table_id(node: &PlanRef) -> Option<TableId> {
1267        if node.node_type() == BatchPlanNodeType::BatchExchange {
1268            return None;
1269        }
1270        if let Some(insert) = node.as_batch_insert() {
1271            Some(insert.core.table_id)
1272        } else if let Some(update) = node.as_batch_update() {
1273            Some(update.core.table_id)
1274        } else if let Some(delete) = node.as_batch_delete() {
1275            Some(delete.core.table_id)
1276        } else {
1277            node.inputs()
1278                .into_iter()
1279                .find_map(|n| Self::collect_dml_table_id(&n))
1280        }
1281    }
1282
1283    fn collect_stage_lookup_join_parallelism(
1284        &self,
1285        node: PlanRef,
1286    ) -> SchedulerResult<Option<usize>> {
1287        if node.node_type() == BatchPlanNodeType::BatchExchange {
1288            // Do not visit next stage.
1289            return Ok(None);
1290        }
1291        if let Some(lookup_join) = node.as_batch_lookup_join() {
1292            let table_catalog = lookup_join.right_table();
1293            let vnode_mapping = self
1294                .worker_node_manager
1295                .fragment_mapping(table_catalog.fragment_id)?;
1296            let parallelism = vnode_mapping.iter().sorted().dedup().count();
1297            Ok(Some(parallelism))
1298        } else {
1299            node.inputs()
1300                .into_iter()
1301                .find_map(|n| self.collect_stage_lookup_join_parallelism(n).transpose())
1302                .transpose()
1303        }
1304    }
1305}
1306
1307/// Try to derive the partition to read from the scan range.
1308/// It can be derived if the value of the distribution key is already known.
1309fn derive_partitions(
1310    scan_ranges: &[ScanRange],
1311    table_catalog: &TableCatalog,
1312    vnode_mapping: &WorkerSlotMapping,
1313) -> SchedulerResult<HashMap<WorkerSlotId, TablePartitionInfo>> {
1314    let vnode_mapping = if table_catalog.vnode_count.value() != vnode_mapping.len() {
1315        // The vnode count mismatch occurs only in special cases where a hash-distributed fragment
1316        // contains singleton internal tables. e.g., the state table of `Source` executors.
1317        // In this case, we reduce the vnode mapping to a single vnode as only `SINGLETON_VNODE` is used.
1318        assert_eq!(
1319            table_catalog.vnode_count.value(),
1320            1,
1321            "fragment vnode count {} does not match table vnode count {}",
1322            vnode_mapping.len(),
1323            table_catalog.vnode_count.value(),
1324        );
1325        &WorkerSlotMapping::new_single(vnode_mapping.iter().next().unwrap())
1326    } else {
1327        vnode_mapping
1328    };
1329    let vnode_count = vnode_mapping.len();
1330
1331    let mut partitions: HashMap<WorkerSlotId, (BitmapBuilder, Vec<_>)> = HashMap::new();
1332
1333    if scan_ranges.is_empty() {
1334        return Ok(vnode_mapping
1335            .to_bitmaps()
1336            .into_iter()
1337            .map(|(k, vnode_bitmap)| {
1338                (
1339                    k,
1340                    TablePartitionInfo {
1341                        vnode_bitmap,
1342                        scan_ranges: vec![],
1343                    },
1344                )
1345            })
1346            .collect());
1347    }
1348
1349    let table_distribution = TableDistribution::new_from_storage_table_desc(
1350        Some(Bitmap::ones(vnode_count).into()),
1351        &table_catalog.table_desc().try_to_protobuf()?,
1352    );
1353
1354    for scan_range in scan_ranges {
1355        let vnode = scan_range.try_compute_vnode(&table_distribution);
1356        match vnode {
1357            None => {
1358                // put this scan_range to all partitions
1359                vnode_mapping.to_bitmaps().into_iter().for_each(
1360                    |(worker_slot_id, vnode_bitmap)| {
1361                        let (bitmap, scan_ranges) = partitions
1362                            .entry(worker_slot_id)
1363                            .or_insert_with(|| (BitmapBuilder::zeroed(vnode_count), vec![]));
1364                        vnode_bitmap
1365                            .iter()
1366                            .enumerate()
1367                            .for_each(|(vnode, b)| bitmap.set(vnode, b));
1368                        scan_ranges.push(scan_range.to_protobuf());
1369                    },
1370                );
1371            }
1372            // scan a single partition
1373            Some(vnode) => {
1374                let worker_slot_id = vnode_mapping[vnode];
1375                let (bitmap, scan_ranges) = partitions
1376                    .entry(worker_slot_id)
1377                    .or_insert_with(|| (BitmapBuilder::zeroed(vnode_count), vec![]));
1378                bitmap.set(vnode.to_index(), true);
1379                scan_ranges.push(scan_range.to_protobuf());
1380            }
1381        }
1382    }
1383
1384    Ok(partitions
1385        .into_iter()
1386        .map(|(k, (bitmap, scan_ranges))| {
1387            (
1388                k,
1389                TablePartitionInfo {
1390                    vnode_bitmap: bitmap.finish(),
1391                    scan_ranges,
1392                },
1393            )
1394        })
1395        .collect())
1396}
1397
1398#[cfg(test)]
1399mod tests {
1400    use std::collections::{HashMap, HashSet};
1401
1402    use risingwave_pb::batch_plan::plan_node::NodeBody;
1403
1404    use crate::optimizer::plan_node::BatchPlanNodeType;
1405    use crate::scheduler::plan_fragmenter::StageId;
1406
1407    #[tokio::test]
1408    async fn test_fragmenter() {
1409        let query = crate::scheduler::distributed::tests::create_query().await;
1410
1411        assert_eq!(query.stage_graph.root_stage_id, 0);
1412        assert_eq!(query.stage_graph.stages.len(), 4);
1413
1414        // Check the mappings of child edges.
1415        assert_eq!(query.stage_graph.child_edges[&0], [1].into());
1416        assert_eq!(query.stage_graph.child_edges[&1], [2, 3].into());
1417        assert_eq!(query.stage_graph.child_edges[&2], HashSet::new());
1418        assert_eq!(query.stage_graph.child_edges[&3], HashSet::new());
1419
1420        // Check the mappings of parent edges.
1421        assert_eq!(query.stage_graph.parent_edges[&0], HashSet::new());
1422        assert_eq!(query.stage_graph.parent_edges[&1], [0].into());
1423        assert_eq!(query.stage_graph.parent_edges[&2], [1].into());
1424        assert_eq!(query.stage_graph.parent_edges[&3], [1].into());
1425
1426        // Verify topology order
1427        {
1428            let stage_id_to_pos: HashMap<StageId, usize> = query
1429                .stage_graph
1430                .stage_ids_by_topo_order()
1431                .enumerate()
1432                .map(|(pos, stage_id)| (stage_id, pos))
1433                .collect();
1434
1435            for stage_id in query.stage_graph.stages.keys() {
1436                let stage_pos = stage_id_to_pos[stage_id];
1437                for child_stage_id in &query.stage_graph.child_edges[stage_id] {
1438                    let child_pos = stage_id_to_pos[child_stage_id];
1439                    assert!(stage_pos > child_pos);
1440                }
1441            }
1442        }
1443
1444        // Check plan node in each stages.
1445        let root_exchange = query.stage_graph.stages.get(&0).unwrap();
1446        assert_eq!(
1447            root_exchange.root.node_type(),
1448            BatchPlanNodeType::BatchExchange
1449        );
1450        assert_eq!(root_exchange.root.source_stage_id, Some(1));
1451        assert!(matches!(root_exchange.root.node, NodeBody::Exchange(_)));
1452        assert_eq!(root_exchange.parallelism, Some(1));
1453        assert!(!root_exchange.has_table_scan());
1454
1455        let join_node = query.stage_graph.stages.get(&1).unwrap();
1456        assert_eq!(join_node.root.node_type(), BatchPlanNodeType::BatchHashJoin);
1457        assert_eq!(join_node.parallelism, Some(24));
1458
1459        assert!(matches!(join_node.root.node, NodeBody::HashJoin(_)));
1460        assert_eq!(join_node.root.source_stage_id, None);
1461        assert_eq!(2, join_node.root.children.len());
1462
1463        assert!(matches!(
1464            join_node.root.children[0].node,
1465            NodeBody::Exchange(_)
1466        ));
1467        assert_eq!(join_node.root.children[0].source_stage_id, Some(2));
1468        assert_eq!(0, join_node.root.children[0].children.len());
1469
1470        assert!(matches!(
1471            join_node.root.children[1].node,
1472            NodeBody::Exchange(_)
1473        ));
1474        assert_eq!(join_node.root.children[1].source_stage_id, Some(3));
1475        assert_eq!(0, join_node.root.children[1].children.len());
1476        assert!(!join_node.has_table_scan());
1477
1478        let scan_node1 = query.stage_graph.stages.get(&2).unwrap();
1479        assert_eq!(scan_node1.root.node_type(), BatchPlanNodeType::BatchSeqScan);
1480        assert_eq!(scan_node1.root.source_stage_id, None);
1481        assert_eq!(0, scan_node1.root.children.len());
1482        assert!(scan_node1.has_table_scan());
1483
1484        let scan_node2 = query.stage_graph.stages.get(&3).unwrap();
1485        assert_eq!(scan_node2.root.node_type(), BatchPlanNodeType::BatchFilter);
1486        assert_eq!(scan_node2.root.source_stage_id, None);
1487        assert_eq!(1, scan_node2.root.children.len());
1488        assert!(scan_node2.has_table_scan());
1489    }
1490}