@@ -170,7 +170,8 @@ impl ExecutionPlan for ShuffleReaderExec {
170170 context : Arc < TaskContext > ,
171171 ) -> Result < SendableRecordBatchStream > {
172172 let task_id = context. task_id ( ) . unwrap_or_else ( || partition. to_string ( ) ) ;
173- info ! ( "ShuffleReaderExec::execute({task_id}) partition={partition}, num_locations={}, stage_id={}" ,
173+ info ! (
174+ "ShuffleReaderExec::execute({task_id}) partition={partition}, num_locations={}, stage_id={}" ,
174175 self . partition. get( partition) . map( |p| p. len( ) ) . unwrap_or( 0 ) ,
175176 self . stage_id,
176177 ) ;
@@ -496,7 +497,10 @@ fn send_fetch_partitions(
496497 ) ;
497498 info ! (
498499 "send_fetch_partitions: {} total locations (memory={}, local={}, object_store={}, remote={})" ,
499- locations. memory. len( ) + locations. local. len( ) + locations. object_store. len( ) + locations. remote. len( ) ,
500+ locations. memory. len( )
501+ + locations. local. len( )
502+ + locations. object_store. len( )
503+ + locations. remote. len( ) ,
500504 locations. memory. len( ) ,
501505 locations. local. len( ) ,
502506 locations. object_store. len( ) ,
@@ -522,8 +526,18 @@ fn send_fetch_partitions(
522526 let customize_endpoint_c = customize_endpoint. clone ( ) ;
523527 let metrics_callback_c = metrics_callback. clone ( ) ;
524528 let local_locations = locations. local ;
529+ let local_count = local_locations. len ( ) ;
525530 spawned_tasks. push ( SpawnedTask :: spawn ( async move {
526- for p in local_locations {
531+ for ( i, p) in local_locations. into_iter ( ) . enumerate ( ) {
532+ info ! (
533+ "fetch_local[{}/{}]: reading {}/{}/{} from {}" ,
534+ i + 1 ,
535+ local_count,
536+ p. partition_id. job_id,
537+ p. partition_id. stage_id,
538+ p. partition_id. partition_id,
539+ p. path
540+ ) ;
527541 let start_time = std:: time:: Instant :: now ( ) ;
528542 let r = PartitionReaderEnum :: Local
529543 . fetch_partition (
@@ -534,6 +548,17 @@ fn send_fetch_partitions(
534548 use_tls,
535549 )
536550 . await ;
551+ let ok = r. is_ok ( ) ;
552+ info ! (
553+ "fetch_local[{}/{}]: {}/{}/{} completed in {:.3}s, ok={}" ,
554+ i + 1 ,
555+ local_count,
556+ p. partition_id. job_id,
557+ p. partition_id. stage_id,
558+ p. partition_id. partition_id,
559+ start_time. elapsed( ) . as_secs_f64( ) ,
560+ ok
561+ ) ;
537562
538563 // Record local read metrics if callback is set and read succeeded
539564 if r. is_ok ( )
@@ -712,8 +737,7 @@ async fn fetch_partition_remote(
712737 let port = metadata. port ;
713738 info ! (
714739 "fetch_partition_remote: fetching {}/{}/{} from {}:{}" ,
715- partition_id. job_id, partition_id. stage_id, partition_id. partition_id,
716- host, port
740+ partition_id. job_id, partition_id. stage_id, partition_id. partition_id, host, port
717741 ) ;
718742 let mut ballista_client = BallistaClient :: try_new (
719743 host,
0 commit comments