@@ -82,7 +82,7 @@ use tokio::sync::mpsc;
8282use crate :: execution:: memory_pools:: {
8383 create_memory_pool, handle_task_shared_pool_release, parse_memory_pool_config, MemoryPoolConfig ,
8484} ;
85- use crate :: execution:: operators:: ScanExec ;
85+ use crate :: execution:: operators:: { ScanExec , ShuffleScanExec } ;
8686use crate :: execution:: shuffle:: { read_ipc_compressed, CompressionCodec } ;
8787use crate :: execution:: spark_plan:: SparkPlan ;
8888
@@ -151,6 +151,8 @@ struct ExecutionContext {
151151 pub root_op : Option < Arc < SparkPlan > > ,
152152 /// The input sources for the DataFusion plan
153153 pub scans : Vec < ScanExec > ,
154+ /// The shuffle scan input sources for the DataFusion plan
155+ pub shuffle_scans : Vec < ShuffleScanExec > ,
154156 /// The global reference of input sources for the DataFusion plan
155157 pub input_sources : Vec < Arc < GlobalRef > > ,
156158 /// The record batch stream to pull results from
@@ -311,6 +313,7 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan(
311313 partition_count : partition_count as usize ,
312314 root_op : None ,
313315 scans : vec ! [ ] ,
316+ shuffle_scans : vec ! [ ] ,
314317 input_sources,
315318 stream : None ,
316319 batch_receiver : None ,
@@ -491,6 +494,10 @@ fn pull_input_batches(exec_context: &mut ExecutionContext) -> Result<(), CometEr
491494 exec_context. scans . iter_mut ( ) . try_for_each ( |scan| {
492495 scan. get_next_batch ( ) ?;
493496 Ok :: < ( ) , CometError > ( ( ) )
497+ } ) ?;
498+ exec_context. shuffle_scans . iter_mut ( ) . try_for_each ( |scan| {
499+ scan. get_next_batch ( ) ?;
500+ Ok :: < ( ) , CometError > ( ( ) )
494501 } )
495502}
496503
@@ -539,7 +546,7 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan(
539546 let planner =
540547 PhysicalPlanner :: new ( Arc :: clone ( & exec_context. session_ctx ) , partition)
541548 . with_exec_id ( exec_context_id) ;
542- let ( scans, root_op) = planner. create_plan (
549+ let ( scans, shuffle_scans , root_op) = planner. create_plan (
543550 & exec_context. spark_plan ,
544551 & mut exec_context. input_sources . clone ( ) ,
545552 exec_context. partition_count ,
@@ -548,6 +555,7 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan(
548555
549556 exec_context. plan_creation_time += physical_plan_time;
550557 exec_context. scans = scans;
558+ exec_context. shuffle_scans = shuffle_scans;
551559
552560 if exec_context. explain_native {
553561 let formatted_plan_str =
@@ -560,7 +568,7 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan(
560568 // so we should always execute partition 0.
561569 let stream = root_op. native_plan . execute ( 0 , task_ctx) ?;
562570
563- if exec_context. scans . is_empty ( ) {
571+ if exec_context. scans . is_empty ( ) && exec_context . shuffle_scans . is_empty ( ) {
564572 // No JVM data sources — spawn onto tokio so the executor
565573 // thread parks in blocking_recv instead of busy-polling.
566574 //
0 commit comments