Skip to content

Commit 70f46f1

Browse files
refactor: Remove state reporter for local mode (risingwavelabs#8477)
1 parent eb24cda commit 70f46f1

File tree

3 files changed

+19
-32
lines changed

3 files changed

+19
-32
lines changed

src/batch/src/rpc/service/task_service.rs

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -130,8 +130,7 @@ impl TaskService for BatchServiceImpl {
130130
let task = BatchTaskExecution::new(&task_id, plan, context, epoch, self.mgr.runtime())?;
131131
let task = Arc::new(task);
132132
let (tx, rx) = tokio::sync::mpsc::channel(LOCAL_EXECUTE_BUFFER_SIZE);
133-
let state_reporter = StateReporter::new_with_local_sender(tx.clone());
134-
if let Err(e) = task.clone().async_execute(state_reporter).await {
133+
if let Err(e) = task.clone().async_execute(None).await {
135134
error!(
136135
"failed to build executors and trigger execution of Task {:?}: {}",
137136
task_id, e

src/batch/src/task/task_execution.rs

Lines changed: 17 additions & 29 deletions
Original file line numberDiff line numberDiff line change
@@ -32,13 +32,12 @@ use task_stats_alloc::{TaskLocalBytesAllocated, BYTES_ALLOCATED};
3232
use tokio::runtime::Runtime;
3333
use tokio::sync::oneshot::{Receiver, Sender};
3434
use tokio_metrics::TaskMonitor;
35-
use tonic::Status;
3635

3736
use crate::error::BatchError::{Aborted, SenderError};
3837
use crate::error::{to_rw_error, BatchError, Result as BatchResult};
3938
use crate::executor::{BoxedExecutor, ExecutorBuilder};
4039
use crate::rpc::service::exchange::ExchangeWriter;
41-
use crate::rpc::service::task_service::{GetDataResponseResult, TaskInfoResponseResult};
40+
use crate::rpc::service::task_service::TaskInfoResponseResult;
4241
use crate::task::channel::{create_output_channel, ChanReceiverImpl, ChanSenderImpl};
4342
use crate::task::BatchTaskContext;
4443

@@ -96,33 +95,18 @@ where
9695
/// effect. Local sender only report Failed update, Distributed sender will also report
9796
/// Finished/Pending/Starting/Aborted etc.
9897
pub enum StateReporter {
99-
Local(tokio::sync::mpsc::Sender<GetDataResponseResult>),
10098
Distributed(tokio::sync::mpsc::Sender<TaskInfoResponseResult>),
10199
Mock(),
102100
}
103101

104102
impl StateReporter {
105103
pub async fn send(&mut self, val: TaskInfoResponse) -> BatchResult<()> {
106104
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-
}
117105
Self::Distributed(s) => s.send(Ok(val)).await.map_err(|_| SenderError),
118106
Self::Mock() => Ok(()),
119107
}
120108
}
121109

122-
pub fn new_with_local_sender(s: tokio::sync::mpsc::Sender<GetDataResponseResult>) -> Self {
123-
Self::Local(s)
124-
}
125-
126110
pub fn new_with_dist_sender(s: tokio::sync::mpsc::Sender<TaskInfoResponseResult>) -> Self {
127111
Self::Distributed(s)
128112
}
@@ -355,7 +339,7 @@ impl<C: BatchTaskContext> BatchTaskExecution<C> {
355339
/// hash partitioned across multiple channels.
356340
/// To obtain the result, one must pick one of the channels to consume via [`TaskOutputId`]. As
357341
/// 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<()> {
359343
let mut state_tx = state_tx;
360344
trace!(
361345
"Prepare executing plan [{:?}]: {}",
@@ -382,7 +366,7 @@ impl<C: BatchTaskContext> BatchTaskExecution<C> {
382366
// After we init the output receivers, it's must safe to schedule next stage -- able to send
383367
// TaskStatus::Running here.
384368
// 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)
386370
.await?;
387371

388372
// Clone `self` to make compiler happy because of the move block.
@@ -398,7 +382,7 @@ impl<C: BatchTaskContext> BatchTaskExecution<C> {
398382
let task = |task_id: TaskId| async move {
399383
// We should only pass a reference of sender to execution because we should only
400384
// 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())
402386
.in_span({
403387
let mut span = Span::enter_with_local_parent("batch_execute");
404388
span.add_property(|| ("task_id", task_id.task_id.to_string()));
@@ -470,18 +454,22 @@ impl<C: BatchTaskContext> BatchTaskExecution<C> {
470454
pub async fn change_state_notify(
471455
&self,
472456
task_status: TaskStatus,
473-
state_tx: &mut StateReporter,
457+
state_tx: Option<&mut StateReporter>,
474458
err_str: Option<String>,
475459
) -> BatchResult<()> {
476460
self.change_state(task_status);
477461
// 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+
}
485473
}
486474

487475
pub fn change_state(&self, task_status: TaskStatus) {
@@ -493,7 +481,7 @@ impl<C: BatchTaskContext> BatchTaskExecution<C> {
493481
root: BoxedExecutor,
494482
mut sender: ChanSenderImpl,
495483
mut shutdown_rx: Receiver<String>,
496-
state_tx: &mut StateReporter,
484+
state_tx: Option<&mut StateReporter>,
497485
) {
498486
let mut data_chunk_stream = root.execute();
499487
let mut state;

src/batch/src/task/task_manager.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -108,7 +108,7 @@ impl BatchManager {
108108
))
109109
.into())
110110
};
111-
task.async_execute(state_reporter).await?;
111+
task.async_execute(Some(state_reporter)).await?;
112112
ret
113113
}
114114

0 commit comments

Comments
 (0)