@@ -30,11 +30,11 @@ use std::{any::Any, vec};
30
30
use super :: metrics:: { ExecutionPlanMetricsSet , MetricsSet } ;
31
31
use super :: {
32
32
DisplayAs , ExecutionPlanProperties , MaybeBatch , RecordBatchStream ,
33
- RepartitionExecBase , RepartitionMetrics , SendableRecordBatchStream ,
33
+ RepartitionExecBase , SendableRecordBatchStream ,
34
34
} ;
35
35
use crate :: common:: SharedMemoryReservation ;
36
36
use crate :: execution_plan:: CardinalityEffect ;
37
- use crate :: metrics:: BaselineMetrics ;
37
+ use crate :: metrics:: { self , BaselineMetrics , MetricBuilder } ;
38
38
use crate :: repartition:: distributor_channels:: {
39
39
DistributionReceiver , DistributionSender ,
40
40
} ;
@@ -398,22 +398,38 @@ impl OnDemandRepartitionExec {
398
398
partition : usize ,
399
399
buffer_tx : tokio:: sync:: mpsc:: Sender < RecordBatch > ,
400
400
context : Arc < TaskContext > ,
401
+ fetch_time : metrics:: Time ,
402
+ send_buffer_time : metrics:: Time ,
401
403
) -> Result < ( ) > {
404
+ let timer = fetch_time. timer ( ) ;
402
405
let mut stream = input. execute ( partition, context) . map_err ( |e| {
403
406
internal_datafusion_err ! (
404
407
"Error executing input partition {} for on demand repartitioning: {}" ,
405
408
partition,
406
409
e
407
410
)
408
411
} ) ?;
409
- while let Some ( batch) = stream. next ( ) . await {
410
- buffer_tx. send ( batch?) . await . map_err ( |e| {
411
- internal_datafusion_err ! (
412
- "Error sending batch to buffer channel for partition {}: {}" ,
413
- partition,
414
- e
415
- )
416
- } ) ?;
412
+ timer. done ( ) ;
413
+
414
+ loop {
415
+ let timer = fetch_time. timer ( ) ;
416
+ let batch = stream. next ( ) . await ;
417
+ timer. done ( ) ;
418
+
419
+ // send the batch to the buffer channel
420
+ if let Some ( batch) = batch {
421
+ let timer = send_buffer_time. timer ( ) ;
422
+ buffer_tx. send ( batch?) . await . map_err ( |e| {
423
+ internal_datafusion_err ! (
424
+ "Error sending batch to buffer channel for partition {}: {}" ,
425
+ partition,
426
+ e
427
+ )
428
+ } ) ?;
429
+ timer. done ( ) ;
430
+ } else {
431
+ break ;
432
+ }
417
433
}
418
434
419
435
Ok ( ( ) )
@@ -432,7 +448,7 @@ impl OnDemandRepartitionExec {
432
448
> ,
433
449
partitioning : Partitioning ,
434
450
output_partition_rx : Receiver < usize > ,
435
- metrics : RepartitionMetrics ,
451
+ metrics : OnDemandRepartitionMetrics ,
436
452
context : Arc < TaskContext > ,
437
453
) -> Result < ( ) > {
438
454
// execute the child operator in a separate task
@@ -442,6 +458,8 @@ impl OnDemandRepartitionExec {
442
458
partition,
443
459
buffer_tx,
444
460
Arc :: clone ( & context) ,
461
+ metrics. fetch_time ,
462
+ metrics. send_buffer_time ,
445
463
) ) ;
446
464
447
465
// While there are still outputs to send to, keep pulling inputs
@@ -501,6 +519,50 @@ impl OnDemandRepartitionExec {
501
519
}
502
520
}
503
521
522
+ #[ derive( Debug , Clone ) ]
523
+ pub ( crate ) struct OnDemandRepartitionMetrics {
524
+ /// Time in nanos to execute child operator and fetch batches
525
+ fetch_time : metrics:: Time ,
526
+ /// Time in nanos for sending resulting batches to buffer channels.
527
+ send_buffer_time : metrics:: Time ,
528
+ /// Time in nanos for sending resulting batches to channels.
529
+ ///
530
+ /// One metric per output partition.
531
+ send_time : Vec < metrics:: Time > ,
532
+ }
533
+
534
+ impl OnDemandRepartitionMetrics {
535
+ pub fn new (
536
+ input_partition : usize ,
537
+ num_output_partitions : usize ,
538
+ metrics : & ExecutionPlanMetricsSet ,
539
+ ) -> Self {
540
+ // Time in nanos to execute child operator and fetch batches
541
+ let fetch_time =
542
+ MetricBuilder :: new ( metrics) . subset_time ( "fetch_time" , input_partition) ;
543
+
544
+ // Time in nanos for sending resulting batches to channels
545
+ let send_time = ( 0 ..num_output_partitions)
546
+ . map ( |output_partition| {
547
+ let label =
548
+ metrics:: Label :: new ( "outputPartition" , output_partition. to_string ( ) ) ;
549
+ MetricBuilder :: new ( metrics)
550
+ . with_label ( label)
551
+ . subset_time ( "send_time" , input_partition)
552
+ } )
553
+ . collect ( ) ;
554
+
555
+ // Time in nanos for sending resulting batches to buffer channels
556
+ let send_buffer_time =
557
+ MetricBuilder :: new ( metrics) . subset_time ( "send_buffer_time" , input_partition) ;
558
+ Self {
559
+ fetch_time,
560
+ send_time,
561
+ send_buffer_time,
562
+ }
563
+ }
564
+ }
565
+
504
566
/// This struct converts a receiver to a stream.
505
567
/// Receiver receives data on an SPSC channel.
506
568
struct OnDemandPerPartitionStream {
0 commit comments