risingwave_batch/execution/
grpc_exchange.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
// 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 futures::StreamExt;
use risingwave_common::array::DataChunk;
use risingwave_expr::expr_context::capture_expr_context;
use risingwave_pb::batch_plan::exchange_source::LocalExecutePlan::{self, Plan};
use risingwave_pb::batch_plan::TaskOutputId;
use risingwave_pb::task_service::{ExecuteRequest, GetDataResponse};
use risingwave_rpc_client::error::RpcError;
use risingwave_rpc_client::ComputeClient;
use tonic::Streaming;

use crate::error::Result;
use crate::exchange_source::ExchangeSource;
use crate::task::TaskId;

/// Use grpc client as the source.
pub struct GrpcExchangeSource {
    stream: Streaming<GetDataResponse>,

    task_output_id: TaskOutputId,
}

impl GrpcExchangeSource {
    pub async fn create(
        client: ComputeClient,
        task_output_id: TaskOutputId,
        local_execute_plan: Option<LocalExecutePlan>,
    ) -> Result<Self> {
        let task_id = task_output_id.get_task_id()?.clone();
        let stream = match local_execute_plan {
            // When in the local execution mode, `GrpcExchangeSource` would send out
            // `ExecuteRequest` and get the data chunks back in a single RPC.
            Some(local_execute_plan) => {
                let plan = try_match_expand!(local_execute_plan, Plan)?;
                let execute_request = ExecuteRequest {
                    task_id: Some(task_id),
                    plan: plan.plan,
                    epoch: plan.epoch,
                    tracing_context: plan.tracing_context,
                    expr_context: Some(capture_expr_context()?),
                };
                client.execute(execute_request).await?
            }
            None => client.get_data(task_output_id.clone()).await?,
        };
        let source = Self {
            stream,
            task_output_id,
        };
        Ok(source)
    }
}

impl Debug for GrpcExchangeSource {
    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("GrpcExchangeSource")
            .field("task_output_id", &self.task_output_id)
            .finish()
    }
}

impl ExchangeSource for GrpcExchangeSource {
    async fn take_data(&mut self) -> Result<Option<DataChunk>> {
        let res = match self.stream.next().await {
            None => {
                return Ok(None);
            }
            Some(r) => r,
        };
        let task_data = res.map_err(RpcError::from_batch_status)?;
        let data = DataChunk::from_protobuf(task_data.get_record_batch()?)?.compact();
        trace!(
            "Receiver taskOutput = {:?}, data = {:?}",
            self.task_output_id,
            data
        );

        Ok(Some(data))
    }

    fn get_task_id(&self) -> TaskId {
        TaskId::from(self.task_output_id.get_task_id().unwrap())
    }
}