risingwave_batch/task/
hash_shuffle_channel.rs1use std::fmt::{Debug, Formatter};
16use std::sync::Arc;
17
18use anyhow::anyhow;
19use risingwave_common::array::DataChunk;
20use risingwave_common::bitmap::Bitmap;
21use risingwave_common::util::hash_util::Crc32FastBuilder;
22use risingwave_pb::batch_plan::exchange_info::HashInfo;
23use risingwave_pb::batch_plan::*;
24use tokio::sync::mpsc;
25
26use crate::error::BatchError::{Internal, SenderError};
27use crate::error::{BatchError, Result as BatchResult, SharedResult};
28use crate::task::channel::{ChanReceiver, ChanReceiverImpl, ChanSender, ChanSenderImpl};
29use crate::task::data_chunk_in_channel::DataChunkInChannel;
30#[derive(Clone)]
31pub struct HashShuffleSender {
32 senders: Vec<mpsc::Sender<SharedResult<Option<DataChunkInChannel>>>>,
33 hash_info: HashInfo,
34}
35
36impl Debug for HashShuffleSender {
37 fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
38 f.debug_struct("HashShuffleSender")
39 .field("hash_info", &self.hash_info)
40 .finish()
41 }
42}
43
44pub struct HashShuffleReceiver {
45 receiver: mpsc::Receiver<SharedResult<Option<DataChunkInChannel>>>,
46}
47
48fn generate_hash_values(chunk: &DataChunk, hash_info: &HashInfo) -> BatchResult<Vec<usize>> {
49 let output_count = hash_info.output_count as usize;
50
51 let hasher_builder = Crc32FastBuilder;
52
53 let hash_values = chunk
54 .get_hash_values(
55 &hash_info
56 .key
57 .iter()
58 .map(|idx| *idx as usize)
59 .collect::<Vec<_>>(),
60 hasher_builder,
61 )
62 .iter_mut()
63 .map(|hash_value| hash_value.value() as usize % output_count)
64 .collect::<Vec<_>>();
65 Ok(hash_values)
66}
67
68fn generate_new_data_chunks(
70 chunk: &DataChunk,
71 hash_info: &exchange_info::HashInfo,
72 hash_values: &[usize],
73) -> Vec<DataChunk> {
74 let output_count = hash_info.output_count as usize;
75 let mut vis_maps = vec![vec![]; output_count];
76 hash_values.iter().for_each(|hash| {
77 for (sink_id, vis_map) in vis_maps.iter_mut().enumerate() {
78 if *hash == sink_id {
79 vis_map.push(true);
80 } else {
81 vis_map.push(false);
82 }
83 }
84 });
85 let mut res = Vec::with_capacity(output_count);
86 for (sink_id, vis_map_vec) in vis_maps.into_iter().enumerate() {
87 let vis_map = Bitmap::from_bool_slice(&vis_map_vec) & chunk.visibility();
88 let new_data_chunk = chunk.with_visibility(vis_map);
89 trace!(
90 "send to sink:{}, cardinality:{}",
91 sink_id,
92 new_data_chunk.cardinality()
93 );
94 res.push(new_data_chunk);
95 }
96 res
97}
98
99impl ChanSender for HashShuffleSender {
100 async fn send(&mut self, chunk: DataChunk) -> BatchResult<()> {
101 self.send_chunk(chunk).await
102 }
103
104 async fn close(self, error: Option<Arc<BatchError>>) -> BatchResult<()> {
105 self.send_done(error).await
106 }
107}
108
109impl HashShuffleSender {
110 async fn send_chunk(&mut self, chunk: DataChunk) -> BatchResult<()> {
111 let hash_values = generate_hash_values(&chunk, &self.hash_info)?;
112 let new_data_chunks = generate_new_data_chunks(&chunk, &self.hash_info, &hash_values);
113
114 for (sink_id, new_data_chunk) in new_data_chunks.into_iter().enumerate() {
115 trace!(
116 "send to sink:{}, cardinality:{}",
117 sink_id,
118 new_data_chunk.cardinality()
119 );
120 if new_data_chunk.cardinality() > 0 {
123 self.senders[sink_id]
124 .send(Ok(Some(DataChunkInChannel::new(new_data_chunk))))
125 .await
126 .map_err(|_| SenderError)?
127 }
128 }
129 Ok(())
130 }
131
132 async fn send_done(self, error: Option<Arc<BatchError>>) -> BatchResult<()> {
133 for sender in self.senders {
134 sender
135 .send(error.clone().map(Err).unwrap_or(Ok(None)))
136 .await
137 .map_err(|_| SenderError)?
138 }
139
140 Ok(())
141 }
142}
143
144impl ChanReceiver for HashShuffleReceiver {
145 async fn recv(&mut self) -> SharedResult<Option<DataChunkInChannel>> {
146 match self.receiver.recv().await {
147 Some(data_chunk) => data_chunk,
148 None => Err(Arc::new(Internal(anyhow!("broken hash_shuffle_channel")))),
150 }
151 }
152}
153
154pub fn new_hash_shuffle_channel(
155 shuffle: &ExchangeInfo,
156 output_channel_size: usize,
157) -> (ChanSenderImpl, Vec<ChanReceiverImpl>) {
158 let hash_info = match shuffle.distribution {
159 Some(exchange_info::Distribution::HashInfo(ref v)) => v.clone(),
160 _ => exchange_info::HashInfo::default(),
161 };
162
163 let output_count = hash_info.output_count as usize;
164 let mut senders = Vec::with_capacity(output_count);
165 let mut receivers = Vec::with_capacity(output_count);
166 for _ in 0..output_count {
167 let (s, r) = mpsc::channel(output_channel_size);
168 senders.push(s);
169 receivers.push(r);
170 }
171 let channel_sender = ChanSenderImpl::HashShuffle(HashShuffleSender { senders, hash_info });
172 let channel_receivers = receivers
173 .into_iter()
174 .map(|receiver| ChanReceiverImpl::HashShuffle(HashShuffleReceiver { receiver }))
175 .collect::<Vec<_>>();
176 (channel_sender, channel_receivers)
177}