risingwave_batch/
exchange_source.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
// 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;
use std::future::Future;

use futures_async_stream::try_stream;
use risingwave_common::array::DataChunk;

use crate::error::{BatchError, Result};
use crate::execution::grpc_exchange::GrpcExchangeSource;
use crate::execution::local_exchange::LocalExchangeSource;
use crate::executor::test_utils::FakeExchangeSource;
use crate::task::TaskId;

/// Each `ExchangeSource` maps to one task, it takes the execution result from task chunk by chunk.
pub trait ExchangeSource: Send + Debug {
    fn take_data(&mut self) -> impl Future<Output = Result<Option<DataChunk>>> + '_;

    /// Get upstream task id.
    fn get_task_id(&self) -> TaskId;
}

#[derive(Debug)]
pub enum ExchangeSourceImpl {
    Grpc(GrpcExchangeSource),
    Local(LocalExchangeSource),
    Fake(FakeExchangeSource),
}

impl ExchangeSourceImpl {
    pub(crate) async fn take_data(&mut self) -> Result<Option<DataChunk>> {
        match self {
            ExchangeSourceImpl::Grpc(grpc) => grpc.take_data().await,
            ExchangeSourceImpl::Local(local) => local.take_data().await,
            ExchangeSourceImpl::Fake(fake) => fake.take_data().await,
        }
    }

    #[expect(dead_code)]
    pub(crate) fn get_task_id(&self) -> TaskId {
        match self {
            ExchangeSourceImpl::Grpc(grpc) => grpc.get_task_id(),
            ExchangeSourceImpl::Local(local) => local.get_task_id(),
            ExchangeSourceImpl::Fake(fake) => fake.get_task_id(),
        }
    }

    #[try_stream(boxed, ok = DataChunk, error = BatchError)]
    pub(crate) async fn take_data_stream(self) {
        let mut source = self;
        loop {
            match source.take_data().await {
                Ok(Some(chunk)) => yield chunk,
                Ok(None) => break,
                Err(e) => return Err(e),
            }
        }
    }
}