risingwave_batch/executor/
managed.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
// 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::pin::pin;

use futures::future::{select, Either};
use futures::stream::StreamExt;
use futures_async_stream::try_stream;
use risingwave_common::array::DataChunk;
use risingwave_common::catalog::Schema;
use tracing::Instrument;

use crate::error::BatchError;
use crate::executor::{BoxedExecutor, Executor};
use crate::task::{ShutdownMsg, ShutdownToken};

/// `ManagedExecutor` build on top of the underlying executor. For now, it does two things:
/// 1. the duration of performance-critical operations will be traced, such as open/next/close.
/// 2. receive shutdown signal
pub struct ManagedExecutor {
    child: BoxedExecutor,
    shutdown_rx: ShutdownToken,
}

impl ManagedExecutor {
    pub fn new(child: BoxedExecutor, shutdown_rx: ShutdownToken) -> Self {
        Self { child, shutdown_rx }
    }
}

impl Executor for ManagedExecutor {
    fn schema(&self) -> &Schema {
        self.child.schema()
    }

    fn identity(&self) -> &str {
        self.child.identity()
    }

    #[try_stream(boxed, ok = DataChunk, error = BatchError)]
    async fn execute(mut self: Box<Self>) {
        let input_desc = self.child.identity().to_string();
        let span = tracing::info_span!("batch_executor", "otel.name" = input_desc);

        let mut child_stream = self.child.execute();

        loop {
            let shutdown = pin!(self.shutdown_rx.cancelled());

            match select(shutdown, child_stream.next().instrument(span.clone())).await {
                Either::Left(_) => break,
                Either::Right((res, _)) => {
                    if let Some(chunk) = res {
                        yield chunk?;
                    } else {
                        return Ok(());
                    }
                }
            }
        }

        match self.shutdown_rx.message() {
            ShutdownMsg::Abort(reason) => {
                return Err(BatchError::aborted(reason));
            }
            ShutdownMsg::Cancel => {
                return Err(BatchError::aborted("cancelled"));
            }
            ShutdownMsg::Init => {}
        }
    }
}