risingwave_batch/task/
fifo_channel.rsuse std::fmt::{Debug, Formatter};
use std::sync::Arc;
use anyhow::anyhow;
use risingwave_common::array::DataChunk;
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 FifoSender {
sender: mpsc::Sender<SharedResult<Option<DataChunkInChannel>>>,
}
impl Debug for FifoSender {
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
f.debug_struct("FifoSender").finish()
}
}
pub struct FifoReceiver {
receiver: mpsc::Receiver<SharedResult<Option<DataChunkInChannel>>>,
}
impl ChanSender for FifoSender {
async fn send(&mut self, chunk: DataChunk) -> BatchResult<()> {
let data = DataChunkInChannel::new(chunk);
self.sender
.send(Ok(Some(data)))
.await
.map_err(|_| SenderError)
}
async fn close(self, error: Option<Arc<BatchError>>) -> BatchResult<()> {
let result = error.map(Err).unwrap_or(Ok(None));
self.sender.send(result).await.map_err(|_| SenderError)
}
}
impl ChanReceiver for FifoReceiver {
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 fifo_channel")))),
}
}
}
pub fn new_fifo_channel(output_channel_size: usize) -> (ChanSenderImpl, Vec<ChanReceiverImpl>) {
let (s, r) = mpsc::channel(output_channel_size);
(
ChanSenderImpl::Fifo(FifoSender { sender: s }),
vec![ChanReceiverImpl::Fifo(FifoReceiver { receiver: r })],
)
}
mod tests {
#[tokio::test]
async fn test_recv_not_fail_on_closed_channel() {
use crate::task::fifo_channel::new_fifo_channel;
let (sender, mut receivers) = new_fifo_channel(64);
assert_eq!(receivers.len(), 1);
drop(sender);
let receiver = receivers.get_mut(0).unwrap();
assert!(receiver.recv().await.is_err());
}
}