risingwave_batch/task/
task_execution.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use std::fmt::{Debug, Formatter};
use std::panic::AssertUnwindSafe;
use std::sync::Arc;

use anyhow::Context;
use futures::StreamExt;
use parking_lot::Mutex;
use risingwave_common::array::DataChunk;
use risingwave_common::util::panic::FutureCatchUnwindExt;
use risingwave_common::util::runtime::BackgroundShutdownRuntime;
use risingwave_common::util::tracing::TracingContext;
use risingwave_expr::expr_context::expr_context_scope;
use risingwave_pb::batch_plan::{PbTaskId, PbTaskOutputId, PlanFragment};
use risingwave_pb::common::BatchQueryEpoch;
use risingwave_pb::plan_common::ExprContext;
use risingwave_pb::task_service::task_info_response::TaskStatus;
use risingwave_pb::task_service::{GetDataResponse, TaskInfoResponse};
use risingwave_pb::PbFieldNotFound;
use thiserror_ext::AsReport;
use tokio::select;
use tokio::task::JoinHandle;
use tracing::Instrument;

use crate::error::BatchError::SenderError;
use crate::error::{BatchError, Result, SharedResult};
use crate::executor::{BoxedExecutor, ExecutorBuilder};
use crate::rpc::service::exchange::ExchangeWriter;
use crate::rpc::service::task_service::TaskInfoResponseResult;
use crate::task::channel::{create_output_channel, ChanReceiverImpl, ChanSenderImpl};
use crate::task::BatchTaskContext;

// Now we will only at most have 2 status for each status channel. Running -> Failed or Finished.
pub const TASK_STATUS_BUFFER_SIZE: usize = 2;

/// Send batch task status (local/distributed) to frontend.
///
///
/// Local mode use `StateReporter::Local`, Distributed mode use `StateReporter::Distributed` to send
/// status (Failed/Finished) update. `StateReporter::Mock` is only used in test and do not takes any
/// effect. Local sender only report Failed update, Distributed sender will also report
/// Finished/Pending/Starting/Aborted etc.
#[derive(Clone)]
pub enum StateReporter {
    Distributed(tokio::sync::mpsc::Sender<TaskInfoResponseResult>),
    Mock(),
}

impl StateReporter {
    pub async fn send(&mut self, val: TaskInfoResponse) -> Result<()> {
        match self {
            Self::Distributed(s) => s.send(Ok(val)).await.map_err(|_| SenderError),
            Self::Mock() => Ok(()),
        }
    }

    pub fn new_with_dist_sender(s: tokio::sync::mpsc::Sender<TaskInfoResponseResult>) -> Self {
        Self::Distributed(s)
    }

    pub fn new_with_test() -> Self {
        Self::Mock()
    }
}

#[derive(PartialEq, Eq, Hash, Clone, Debug, Default)]
pub struct TaskId {
    pub task_id: u64,
    pub stage_id: u32,
    pub query_id: String,
}

#[derive(PartialEq, Eq, Hash, Clone, Default)]
pub struct TaskOutputId {
    pub task_id: TaskId,
    pub output_id: u64,
}

/// More compact formatter compared to derived `fmt::Debug`.
impl Debug for TaskOutputId {
    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
        f.write_fmt(format_args!(
            "TaskOutputId {{ query_id: \"{}\", stage_id: {}, task_id: {}, output_id: {} }}",
            self.task_id.query_id, self.task_id.stage_id, self.task_id.task_id, self.output_id
        ))
    }
}

impl From<&PbTaskId> for TaskId {
    fn from(prost: &PbTaskId) -> Self {
        TaskId {
            task_id: prost.task_id,
            stage_id: prost.stage_id,
            query_id: prost.query_id.clone(),
        }
    }
}

impl TaskId {
    pub fn to_prost(&self) -> PbTaskId {
        PbTaskId {
            task_id: self.task_id,
            stage_id: self.stage_id,
            query_id: self.query_id.clone(),
        }
    }
}

impl TryFrom<&PbTaskOutputId> for TaskOutputId {
    type Error = PbFieldNotFound;

    fn try_from(prost: &PbTaskOutputId) -> std::result::Result<Self, PbFieldNotFound> {
        Ok(TaskOutputId {
            task_id: TaskId::from(prost.get_task_id()?),
            output_id: prost.get_output_id(),
        })
    }
}

impl TaskOutputId {
    pub fn to_prost(&self) -> PbTaskOutputId {
        PbTaskOutputId {
            task_id: Some(self.task_id.to_prost()),
            output_id: self.output_id,
        }
    }
}

pub struct TaskOutput {
    receiver: ChanReceiverImpl,
    output_id: TaskOutputId,
    failure: Arc<Mutex<Option<Arc<BatchError>>>>,
}

impl std::fmt::Debug for TaskOutput {
    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("TaskOutput")
            .field("output_id", &self.output_id)
            .field("failure", &self.failure)
            .finish_non_exhaustive()
    }
}

impl TaskOutput {
    /// Write the data in serialized format to `ExchangeWriter`.
    /// Return whether the data stream is finished.
    async fn take_data_inner(
        &mut self,
        writer: &mut impl ExchangeWriter,
        at_most_num: Option<usize>,
    ) -> Result<bool> {
        let mut cnt: usize = 0;
        let limited = at_most_num.is_some();
        let at_most_num = at_most_num.unwrap_or(usize::MAX);
        loop {
            if limited && cnt >= at_most_num {
                return Ok(false);
            }
            match self.receiver.recv().await {
                // Received some data
                Ok(Some(chunk)) => {
                    trace!(
                        "Task output id: {:?}, data len: {:?}",
                        self.output_id,
                        chunk.cardinality()
                    );
                    let pb = chunk.to_protobuf().await;
                    let resp = GetDataResponse {
                        record_batch: Some(pb),
                    };
                    writer.write(Ok(resp)).await?;
                }
                // Reached EOF
                Ok(None) => {
                    break;
                }
                // Error happened
                Err(e) => {
                    writer.write(Err(tonic::Status::from(&*e))).await?;
                    break;
                }
            }
            cnt += 1;
        }
        Ok(true)
    }

    /// Take at most num data and write the data in serialized format to `ExchangeWriter`.
    /// Return whether the data stream is finished.
    pub async fn take_data_with_num(
        &mut self,
        writer: &mut impl ExchangeWriter,
        num: usize,
    ) -> Result<bool> {
        self.take_data_inner(writer, Some(num)).await
    }

    /// Take all data and write the data in serialized format to `ExchangeWriter`.
    pub async fn take_data(&mut self, writer: &mut impl ExchangeWriter) -> Result<()> {
        let finish = self.take_data_inner(writer, None).await?;
        assert!(finish);
        Ok(())
    }

    /// Directly takes data without serialization.
    pub async fn direct_take_data(&mut self) -> SharedResult<Option<DataChunk>> {
        Ok(self.receiver.recv().await?.map(|c| c.into_data_chunk()))
    }

    pub fn id(&self) -> &TaskOutputId {
        &self.output_id
    }
}

#[derive(Clone, Debug)]
pub enum ShutdownMsg {
    /// Used in init, it never occur in receiver later.
    Init,
    Abort(String),
    Cancel,
}

/// A token which can be used to signal a shutdown request.
pub struct ShutdownSender(tokio::sync::watch::Sender<ShutdownMsg>);

impl ShutdownSender {
    /// Send a cancel message. Return true if the message is sent successfully.
    pub fn cancel(&self) -> bool {
        self.0.send(ShutdownMsg::Cancel).is_ok()
    }

    /// Send an abort message. Return true if the message is sent successfully.
    pub fn abort(&self, msg: impl Into<String>) -> bool {
        self.0.send(ShutdownMsg::Abort(msg.into())).is_ok()
    }
}

/// A token which can be used to receive a shutdown signal.
#[derive(Clone)]
pub struct ShutdownToken(tokio::sync::watch::Receiver<ShutdownMsg>);

impl ShutdownToken {
    /// Create an empty token.
    pub fn empty() -> Self {
        Self::new().1
    }

    /// Create a new token.
    pub fn new() -> (ShutdownSender, Self) {
        let (tx, rx) = tokio::sync::watch::channel(ShutdownMsg::Init);
        (ShutdownSender(tx), ShutdownToken(rx))
    }

    /// Return error if the shutdown token has been triggered.
    pub fn check(&self) -> Result<()> {
        match &*self.0.borrow() {
            ShutdownMsg::Init => Ok(()),
            msg => bail!("Receive shutdown msg: {msg:?}"),
        }
    }

    /// Wait until cancellation is requested.
    ///
    /// # Cancel safety
    /// This method is cancel safe.
    pub async fn cancelled(&mut self) {
        if matches!(*self.0.borrow(), ShutdownMsg::Init) {
            if let Err(_err) = self.0.changed().await {
                std::future::pending::<()>().await;
            }
        }
    }

    /// Return true if the shutdown token has been triggered.
    pub fn is_cancelled(&self) -> bool {
        !matches!(*self.0.borrow(), ShutdownMsg::Init)
    }

    /// Return the current shutdown message.
    pub fn message(&self) -> ShutdownMsg {
        self.0.borrow().clone()
    }
}

/// `BatchTaskExecution` represents a single task execution.
pub struct BatchTaskExecution<C> {
    /// Task id.
    task_id: TaskId,

    /// Inner plan to execute.
    plan: PlanFragment,

    /// Task state.
    state: Mutex<TaskStatus>,

    /// Receivers data of the task.
    receivers: Mutex<Vec<Option<ChanReceiverImpl>>>,

    /// Sender for sending chunks between different executors.
    sender: ChanSenderImpl,

    /// Context for task execution
    context: C,

    /// The execution failure.
    failure: Arc<Mutex<Option<Arc<BatchError>>>>,

    epoch: BatchQueryEpoch,

    /// Runtime for the batch tasks.
    runtime: Arc<BackgroundShutdownRuntime>,

    shutdown_tx: ShutdownSender,
    shutdown_rx: ShutdownToken,
    heartbeat_join_handle: Mutex<Option<JoinHandle<()>>>,
}

impl<C: BatchTaskContext> BatchTaskExecution<C> {
    pub fn new(
        prost_tid: &PbTaskId,
        plan: PlanFragment,
        context: C,
        epoch: BatchQueryEpoch,
        runtime: Arc<BackgroundShutdownRuntime>,
    ) -> Result<Self> {
        let task_id = TaskId::from(prost_tid);

        let (sender, receivers) = create_output_channel(
            plan.get_exchange_info()?,
            context.get_config().developer.output_channel_size,
        )?;

        let mut rts = Vec::new();
        rts.extend(receivers.into_iter().map(Some));

        let (shutdown_tx, shutdown_rx) = ShutdownToken::new();
        Ok(Self {
            task_id,
            plan,
            state: Mutex::new(TaskStatus::Pending),
            receivers: Mutex::new(rts),
            failure: Arc::new(Mutex::new(None)),
            epoch,
            context,
            runtime,
            sender,
            shutdown_tx,
            shutdown_rx,
            heartbeat_join_handle: Mutex::new(None),
        })
    }

    pub fn get_task_id(&self) -> &TaskId {
        &self.task_id
    }

    /// `async_execute` executes the task in background, it spawns a tokio coroutine and returns
    /// immediately. The result produced by the task will be sent to one or more channels, according
    /// to a particular shuffling strategy. For example, in hash shuffling, the result will be
    /// hash partitioned across multiple channels.
    /// To obtain the result, one must pick one of the channels to consume via [`TaskOutputId`]. As
    /// such, parallel consumers are able to consume the result independently.
    pub async fn async_execute(
        self: Arc<Self>,
        state_tx: Option<StateReporter>,
        tracing_context: TracingContext,
        expr_context: ExprContext,
    ) -> Result<()> {
        let mut state_tx = state_tx;
        trace!(
            "Prepare executing plan [{:?}]: {}",
            self.task_id,
            serde_json::to_string_pretty(self.plan.get_root()?).unwrap()
        );

        let exec = expr_context_scope(
            expr_context.clone(),
            ExecutorBuilder::new(
                self.plan.root.as_ref().unwrap(),
                &self.task_id,
                self.context.clone(),
                self.epoch,
                self.shutdown_rx.clone(),
            )
            .build(),
        )
        .await?;

        let sender = self.sender.clone();
        let _failure = self.failure.clone();
        let task_id = self.task_id.clone();

        // After we init the output receivers, it's must safe to schedule next stage -- able to send
        // TaskStatus::Running here.
        // Init the state receivers. Swap out later.
        self.change_state_notify(TaskStatus::Running, state_tx.as_mut(), None)
            .await?;

        // Clone `self` to make compiler happy because of the move block.
        let t_1 = self.clone();
        let this = self.clone();
        async fn notify_panic<C: BatchTaskContext>(
            this: &BatchTaskExecution<C>,
            state_tx: Option<&mut StateReporter>,
        ) {
            let err_str = "execution panic".into();
            if let Err(e) = this
                .change_state_notify(TaskStatus::Failed, state_tx, Some(err_str))
                .await
            {
                warn!(
                    error = %e.as_report(),
                    "The status receiver in FE has closed so the status push is failed",
                );
            }
        }
        // Spawn task for real execution.
        let fut = async move {
            trace!("Executing plan [{:?}]", task_id);
            let sender = sender;
            let mut state_tx_1 = state_tx.clone();

            let task = |task_id: TaskId| async move {
                let span = tracing_context.attach(tracing::info_span!(
                    "batch_execute",
                    task_id = task_id.task_id,
                    stage_id = task_id.stage_id,
                    query_id = task_id.query_id,
                ));

                // We should only pass a reference of sender to execution because we should only
                // close it after task error has been set.
                expr_context_scope(
                    expr_context,
                    t_1.run(exec, sender, state_tx_1.as_mut()).instrument(span),
                )
                .await;
            };

            if let Err(error) = AssertUnwindSafe(task(task_id.clone()))
                .rw_catch_unwind()
                .await
            {
                error!("Batch task {:?} panic: {:?}", task_id, error);
                notify_panic(&this, state_tx.as_mut()).await;
            }
        };

        self.runtime.spawn(fut);

        Ok(())
    }

    /// Change state and notify frontend for task status via streaming GRPC.
    pub async fn change_state_notify(
        &self,
        task_status: TaskStatus,
        state_tx: Option<&mut StateReporter>,
        err_str: Option<String>,
    ) -> Result<()> {
        self.change_state(task_status);
        // Notify frontend the task status.
        if let Some(reporter) = state_tx {
            reporter
                .send(TaskInfoResponse {
                    task_id: Some(self.task_id.to_prost()),
                    task_status: task_status.into(),
                    error_message: err_str.unwrap_or("".to_string()),
                })
                .await
        } else {
            Ok(())
        }
    }

    pub fn change_state(&self, task_status: TaskStatus) {
        *self.state.lock() = task_status;
        tracing::debug!(
            "Task {:?} state changed to {:?}",
            &self.task_id,
            task_status
        );
    }

    async fn run(
        &self,
        root: BoxedExecutor,
        mut sender: ChanSenderImpl,
        state_tx: Option<&mut StateReporter>,
    ) {
        self.context
            .batch_metrics()
            .as_ref()
            .inspect(|m| m.batch_manager_metrics().task_num.inc());
        let mut data_chunk_stream = root.execute();
        let mut state;
        let mut error = None;

        let mut shutdown_rx = self.shutdown_rx.clone();
        loop {
            select! {
                biased;
                // `shutdown_rx` can't be removed here to avoid `sender.send(data_chunk)` blocked whole execution.
                _ = shutdown_rx.cancelled() => {
                    match self.shutdown_rx.message() {
                        ShutdownMsg::Abort(e) => {
                            error = Some(BatchError::Aborted(e));
                            state = TaskStatus::Aborted;
                            break;
                        }
                        ShutdownMsg::Cancel => {
                            state = TaskStatus::Cancelled;
                            break;
                        }
                        ShutdownMsg::Init => {
                            unreachable!("Init message should not be received here!")
                        }
                    }
                }
                data_chunk = data_chunk_stream.next()=> {
                    match data_chunk {
                        Some(Ok(data_chunk)) => {
                            if let Err(e) = sender.send(data_chunk).await {
                                match e {
                                    BatchError::SenderError => {
                                        // This is possible since when we have limit executor in parent
                                        // stage, it may early stop receiving data from downstream, which
                                        // leads to close of channel.
                                        warn!("Task receiver closed!");
                                        state = TaskStatus::Finished;
                                        break;
                                    }
                                    x => {
                                        error!("Failed to send data!");
                                        error = Some(x);
                                        state = TaskStatus::Failed;
                                        break;
                                    }
                                }
                            }
                        }
                        Some(Err(e)) => match self.shutdown_rx.message() {
                            ShutdownMsg::Init => {
                                // There is no message received from shutdown channel, which means it caused
                                // task failed.
                                error!(error = %e.as_report(), "Batch task failed");
                                error = Some(e);
                                state = TaskStatus::Failed;
                                break;
                            }
                            ShutdownMsg::Abort(_) => {
                                error = Some(e);
                                state = TaskStatus::Aborted;
                                break;
                            }
                            ShutdownMsg::Cancel => {
                                state = TaskStatus::Cancelled;
                                break;
                            }
                        },
                        None => {
                            debug!("Batch task {:?} finished successfully.", self.task_id);
                            state = TaskStatus::Finished;
                            break;
                        }
                    }
                }
            }
        }

        let error = error.map(Arc::new);
        self.failure.lock().clone_from(&error);
        let err_str = error.as_ref().map(|e| e.to_report_string());
        if let Err(e) = sender.close(error).await {
            match e {
                SenderError => {
                    // This is possible since when we have limit executor in parent
                    // stage, it may early stop receiving data from downstream, which
                    // leads to close of channel.
                    warn!("Task receiver closed when sending None!");
                }
                _x => {
                    error!("Failed to close task output channel: {:?}", self.task_id);
                    state = TaskStatus::Failed;
                }
            }
        }

        if let Err(e) = self.change_state_notify(state, state_tx, err_str).await {
            warn!(
                error = %e.as_report(),
                "The status receiver in FE has closed so the status push is failed",
            );
        }

        self.context
            .batch_metrics()
            .as_ref()
            .inspect(|m| m.batch_manager_metrics().task_num.dec());
    }

    pub fn abort(&self, err_msg: String) {
        // No need to set state to be Aborted here cuz it will be set by shutdown receiver.
        // Stop task execution.
        if self.shutdown_tx.abort(err_msg) {
            info!("Abort task {:?} done", self.task_id);
        } else {
            debug!("The task has already died before this request.")
        }
    }

    pub fn cancel(&self) {
        if !self.shutdown_tx.cancel() {
            debug!("The task has already died before this request.");
        }
    }

    pub fn get_task_output(&self, output_id: &PbTaskOutputId) -> Result<TaskOutput> {
        let task_id = TaskId::from(output_id.get_task_id()?);
        let receiver = self.receivers.lock()[output_id.get_output_id() as usize]
            .take()
            .with_context(|| {
                format!(
                    "Task{:?}'s output{} has already been taken.",
                    task_id,
                    output_id.get_output_id(),
                )
            })?;
        let task_output = TaskOutput {
            receiver,
            output_id: output_id.try_into()?,
            failure: self.failure.clone(),
        };
        Ok(task_output)
    }

    pub fn check_if_running(&self) -> Result<()> {
        if *self.state.lock() != TaskStatus::Running {
            bail!("task {:?} is not running", self.get_task_id());
        }
        Ok(())
    }

    pub fn check_if_aborted(&self) -> Result<bool> {
        match *self.state.lock() {
            TaskStatus::Aborted => Ok(true),
            TaskStatus::Finished => bail!("task {:?} has been finished", self.get_task_id()),
            _ => Ok(false),
        }
    }

    /// Check the task status: whether has ended.
    pub fn is_end(&self) -> bool {
        let guard = self.state.lock();
        !(*guard == TaskStatus::Running || *guard == TaskStatus::Pending)
    }
}

impl<C> BatchTaskExecution<C> {
    pub(crate) fn set_heartbeat_join_handle(&self, join_handle: JoinHandle<()>) {
        *self.heartbeat_join_handle.lock() = Some(join_handle);
    }

    pub(crate) fn heartbeat_join_handle(&self) -> Option<JoinHandle<()>> {
        self.heartbeat_join_handle.lock().take()
    }
}

#[cfg(test)]
mod tests {
    use super::*;

    #[test]
    fn test_task_output_id_debug() {
        let task_id = TaskId {
            task_id: 1,
            stage_id: 2,
            query_id: "abc".to_string(),
        };
        let task_output_id = TaskOutputId {
            task_id,
            output_id: 3,
        };
        assert_eq!(
            format!("{:?}", task_output_id),
            "TaskOutputId { query_id: \"abc\", stage_id: 2, task_id: 1, output_id: 3 }"
        );
    }
}