risingwave_batch/execution/
grpc_exchange.rsuse 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;
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 {
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())
}
}