@@ -32,13 +32,12 @@ use task_stats_alloc::{TaskLocalBytesAllocated, BYTES_ALLOCATED};
32
32
use tokio:: runtime:: Runtime ;
33
33
use tokio:: sync:: oneshot:: { Receiver , Sender } ;
34
34
use tokio_metrics:: TaskMonitor ;
35
- use tonic:: Status ;
36
35
37
36
use crate :: error:: BatchError :: { Aborted , SenderError } ;
38
37
use crate :: error:: { to_rw_error, BatchError , Result as BatchResult } ;
39
38
use crate :: executor:: { BoxedExecutor , ExecutorBuilder } ;
40
39
use crate :: rpc:: service:: exchange:: ExchangeWriter ;
41
- use crate :: rpc:: service:: task_service:: { GetDataResponseResult , TaskInfoResponseResult } ;
40
+ use crate :: rpc:: service:: task_service:: TaskInfoResponseResult ;
42
41
use crate :: task:: channel:: { create_output_channel, ChanReceiverImpl , ChanSenderImpl } ;
43
42
use crate :: task:: BatchTaskContext ;
44
43
@@ -96,33 +95,18 @@ where
96
95
/// effect. Local sender only report Failed update, Distributed sender will also report
97
96
/// Finished/Pending/Starting/Aborted etc.
98
97
pub enum StateReporter {
99
- Local ( tokio:: sync:: mpsc:: Sender < GetDataResponseResult > ) ,
100
98
Distributed ( tokio:: sync:: mpsc:: Sender < TaskInfoResponseResult > ) ,
101
99
Mock ( ) ,
102
100
}
103
101
104
102
impl StateReporter {
105
103
pub async fn send ( & mut self , val : TaskInfoResponse ) -> BatchResult < ( ) > {
106
104
match self {
107
- Self :: Local ( s) => {
108
- // A hack here to convert task failure message to data error
109
- match val. task_status ( ) {
110
- TaskStatus :: Failed => s
111
- . send ( Err ( Status :: internal ( val. error_message ) ) )
112
- . await
113
- . map_err ( |_| SenderError ) ,
114
- _ => Ok ( ( ) ) ,
115
- }
116
- }
117
105
Self :: Distributed ( s) => s. send ( Ok ( val) ) . await . map_err ( |_| SenderError ) ,
118
106
Self :: Mock ( ) => Ok ( ( ) ) ,
119
107
}
120
108
}
121
109
122
- pub fn new_with_local_sender ( s : tokio:: sync:: mpsc:: Sender < GetDataResponseResult > ) -> Self {
123
- Self :: Local ( s)
124
- }
125
-
126
110
pub fn new_with_dist_sender ( s : tokio:: sync:: mpsc:: Sender < TaskInfoResponseResult > ) -> Self {
127
111
Self :: Distributed ( s)
128
112
}
@@ -355,7 +339,7 @@ impl<C: BatchTaskContext> BatchTaskExecution<C> {
355
339
/// hash partitioned across multiple channels.
356
340
/// To obtain the result, one must pick one of the channels to consume via [`TaskOutputId`]. As
357
341
/// such, parallel consumers are able to consume the result independently.
358
- pub async fn async_execute ( self : Arc < Self > , state_tx : StateReporter ) -> Result < ( ) > {
342
+ pub async fn async_execute ( self : Arc < Self > , state_tx : Option < StateReporter > ) -> Result < ( ) > {
359
343
let mut state_tx = state_tx;
360
344
trace ! (
361
345
"Prepare executing plan [{:?}]: {}" ,
@@ -382,7 +366,7 @@ impl<C: BatchTaskContext> BatchTaskExecution<C> {
382
366
// After we init the output receivers, it's must safe to schedule next stage -- able to send
383
367
// TaskStatus::Running here.
384
368
// Init the state receivers. Swap out later.
385
- self . change_state_notify ( TaskStatus :: Running , & mut state_tx, None )
369
+ self . change_state_notify ( TaskStatus :: Running , state_tx. as_mut ( ) , None )
386
370
. await ?;
387
371
388
372
// Clone `self` to make compiler happy because of the move block.
@@ -398,7 +382,7 @@ impl<C: BatchTaskContext> BatchTaskExecution<C> {
398
382
let task = |task_id : TaskId | async move {
399
383
// We should only pass a reference of sender to execution because we should only
400
384
// close it after task error has been set.
401
- t_1. run ( exec, sender, shutdown_rx, & mut state_tx)
385
+ t_1. run ( exec, sender, shutdown_rx, state_tx. as_mut ( ) )
402
386
. in_span ( {
403
387
let mut span = Span :: enter_with_local_parent ( "batch_execute" ) ;
404
388
span. add_property ( || ( "task_id" , task_id. task_id . to_string ( ) ) ) ;
@@ -470,18 +454,22 @@ impl<C: BatchTaskContext> BatchTaskExecution<C> {
470
454
pub async fn change_state_notify (
471
455
& self ,
472
456
task_status : TaskStatus ,
473
- state_tx : & mut StateReporter ,
457
+ state_tx : Option < & mut StateReporter > ,
474
458
err_str : Option < String > ,
475
459
) -> BatchResult < ( ) > {
476
460
self . change_state ( task_status) ;
477
461
// Notify frontend the task status.
478
- state_tx
479
- . send ( TaskInfoResponse {
480
- task_id : Some ( self . task_id . to_prost ( ) ) ,
481
- task_status : task_status. into ( ) ,
482
- error_message : err_str. unwrap_or ( "" . to_string ( ) ) ,
483
- } )
484
- . await
462
+ if let Some ( reporter) = state_tx {
463
+ reporter
464
+ . send ( TaskInfoResponse {
465
+ task_id : Some ( self . task_id . to_prost ( ) ) ,
466
+ task_status : task_status. into ( ) ,
467
+ error_message : err_str. unwrap_or ( "" . to_string ( ) ) ,
468
+ } )
469
+ . await
470
+ } else {
471
+ Ok ( ( ) )
472
+ }
485
473
}
486
474
487
475
pub fn change_state ( & self , task_status : TaskStatus ) {
@@ -493,7 +481,7 @@ impl<C: BatchTaskContext> BatchTaskExecution<C> {
493
481
root : BoxedExecutor ,
494
482
mut sender : ChanSenderImpl ,
495
483
mut shutdown_rx : Receiver < String > ,
496
- state_tx : & mut StateReporter ,
484
+ state_tx : Option < & mut StateReporter > ,
497
485
) {
498
486
let mut data_chunk_stream = root. execute ( ) ;
499
487
let mut state;
0 commit comments