risingwave_batch/task/
broadcast_channel.rsuse std::fmt::{Debug, Formatter};
use std::sync::Arc;
use anyhow::anyhow;
use risingwave_common::array::DataChunk;
use risingwave_pb::batch_plan::exchange_info::BroadcastInfo;
use risingwave_pb::batch_plan::*;
use tokio::sync::mpsc;
use crate::error::BatchError::{Internal, SenderError};
use crate::error::{BatchError, Result as BatchResult, SharedResult};
use crate::task::channel::{ChanReceiver, ChanReceiverImpl, ChanSender, ChanSenderImpl};
use crate::task::data_chunk_in_channel::DataChunkInChannel;
#[derive(Clone)]
pub struct BroadcastSender {
senders: Vec<mpsc::Sender<SharedResult<Option<DataChunkInChannel>>>>,
broadcast_info: BroadcastInfo,
}
impl Debug for BroadcastSender {
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
f.debug_struct("BroadcastSender")
.field("broadcast_info", &self.broadcast_info)
.finish()
}
}
impl ChanSender for BroadcastSender {
async fn send(&mut self, chunk: DataChunk) -> BatchResult<()> {
let broadcast_data_chunk = DataChunkInChannel::new(chunk);
for sender in &self.senders {
sender
.send(Ok(Some(broadcast_data_chunk.clone())))
.await
.map_err(|_| SenderError)?
}
Ok(())
}
async fn close(self, error: Option<Arc<BatchError>>) -> BatchResult<()> {
for sender in self.senders {
sender
.send(error.clone().map(Err).unwrap_or(Ok(None)))
.await
.map_err(|_| SenderError)?
}
Ok(())
}
}
pub struct BroadcastReceiver {
receiver: mpsc::Receiver<SharedResult<Option<DataChunkInChannel>>>,
}
impl ChanReceiver for BroadcastReceiver {
async fn recv(&mut self) -> SharedResult<Option<DataChunkInChannel>> {
match self.receiver.recv().await {
Some(data_chunk) => data_chunk,
None => Err(Arc::new(Internal(anyhow!("broken broadcast_channel")))),
}
}
}
pub fn new_broadcast_channel(
shuffle: &ExchangeInfo,
output_channel_size: usize,
) -> (ChanSenderImpl, Vec<ChanReceiverImpl>) {
let broadcast_info = match shuffle.distribution {
Some(exchange_info::Distribution::BroadcastInfo(ref v)) => *v,
_ => BroadcastInfo::default(),
};
let output_count = broadcast_info.count as usize;
let mut senders = Vec::with_capacity(output_count);
let mut receivers = Vec::with_capacity(output_count);
for _ in 0..output_count {
let (s, r) = mpsc::channel(output_channel_size);
senders.push(s);
receivers.push(r);
}
let channel_sender = ChanSenderImpl::Broadcast(BroadcastSender {
senders,
broadcast_info,
});
let channel_receivers = receivers
.into_iter()
.map(|receiver| ChanReceiverImpl::Broadcast(BroadcastReceiver { receiver }))
.collect::<Vec<_>>();
(channel_sender, channel_receivers)
}