1use std::collections::{BTreeMap, VecDeque};
16use std::pin::Pin;
17use std::task::{Context, Poll};
18
19use anyhow::Context as _;
20use futures::future::try_join_all;
21use futures::stream::{FusedStream, FuturesUnordered, StreamFuture};
22use prometheus::Histogram;
23use risingwave_common::array::StreamChunkBuilder;
24use risingwave_common::config::MetricLevel;
25use risingwave_common::metrics::LabelGuardedMetric;
26use tokio::sync::mpsc;
27use tokio::time::Instant;
28
29use super::exchange::input::BoxedInput;
30use super::watermark::*;
31use super::*;
32use crate::executor::exchange::input::{
33 assert_equal_dispatcher_barrier, new_input, process_dispatcher_msg,
34};
35use crate::executor::prelude::*;
36use crate::task::LocalBarrierManager;
37
38pub(crate) enum MergeExecutorUpstream {
39 Singleton(BoxedInput),
40 Merge(SelectReceivers),
41}
42
43pub(crate) struct MergeExecutorInput {
44 upstream: MergeExecutorUpstream,
45 actor_context: ActorContextRef,
46 upstream_fragment_id: UpstreamFragmentId,
47 local_barrier_manager: LocalBarrierManager,
48 executor_stats: Arc<StreamingMetrics>,
49 pub(crate) info: ExecutorInfo,
50 chunk_size: usize,
51}
52
53impl MergeExecutorInput {
54 pub(crate) fn new(
55 upstream: MergeExecutorUpstream,
56 actor_context: ActorContextRef,
57 upstream_fragment_id: UpstreamFragmentId,
58 local_barrier_manager: LocalBarrierManager,
59 executor_stats: Arc<StreamingMetrics>,
60 info: ExecutorInfo,
61 chunk_size: usize,
62 ) -> Self {
63 Self {
64 upstream,
65 actor_context,
66 upstream_fragment_id,
67 local_barrier_manager,
68 executor_stats,
69 info,
70 chunk_size,
71 }
72 }
73
74 pub(crate) fn into_executor(self, barrier_rx: mpsc::UnboundedReceiver<Barrier>) -> Executor {
75 let fragment_id = self.actor_context.fragment_id;
76 let executor = match self.upstream {
77 MergeExecutorUpstream::Singleton(input) => ReceiverExecutor::new(
78 self.actor_context,
79 fragment_id,
80 self.upstream_fragment_id,
81 input,
82 self.local_barrier_manager,
83 self.executor_stats,
84 barrier_rx,
85 )
86 .boxed(),
87 MergeExecutorUpstream::Merge(inputs) => MergeExecutor::new(
88 self.actor_context,
89 fragment_id,
90 self.upstream_fragment_id,
91 inputs,
92 self.local_barrier_manager,
93 self.executor_stats,
94 barrier_rx,
95 self.chunk_size,
96 self.info.schema.clone(),
97 )
98 .boxed(),
99 };
100 (self.info, executor).into()
101 }
102}
103
104impl Stream for MergeExecutorInput {
105 type Item = DispatcherMessageStreamItem;
106
107 fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
108 match &mut self.get_mut().upstream {
109 MergeExecutorUpstream::Singleton(input) => input.poll_next_unpin(cx),
110 MergeExecutorUpstream::Merge(inputs) => inputs.poll_next_unpin(cx),
111 }
112 }
113}
114
115pub struct MergeExecutor {
118 actor_context: ActorContextRef,
120
121 upstreams: SelectReceivers,
123
124 fragment_id: FragmentId,
126
127 upstream_fragment_id: FragmentId,
129
130 local_barrier_manager: LocalBarrierManager,
131
132 metrics: Arc<StreamingMetrics>,
134
135 barrier_rx: mpsc::UnboundedReceiver<Barrier>,
136
137 chunk_size: usize,
139
140 schema: Schema,
142}
143
144impl MergeExecutor {
145 #[allow(clippy::too_many_arguments)]
146 pub fn new(
147 ctx: ActorContextRef,
148 fragment_id: FragmentId,
149 upstream_fragment_id: FragmentId,
150 upstreams: SelectReceivers,
151 local_barrier_manager: LocalBarrierManager,
152 metrics: Arc<StreamingMetrics>,
153 barrier_rx: mpsc::UnboundedReceiver<Barrier>,
154 chunk_size: usize,
155 schema: Schema,
156 ) -> Self {
157 Self {
158 actor_context: ctx,
159 upstreams,
160 fragment_id,
161 upstream_fragment_id,
162 local_barrier_manager,
163 metrics,
164 barrier_rx,
165 chunk_size,
166 schema,
167 }
168 }
169
170 #[cfg(test)]
171 pub fn for_test(
172 actor_id: ActorId,
173 inputs: Vec<super::exchange::permit::Receiver>,
174 local_barrier_manager: crate::task::LocalBarrierManager,
175 schema: Schema,
176 ) -> Self {
177 use super::exchange::input::LocalInput;
178 use crate::executor::exchange::input::Input;
179
180 let barrier_rx = local_barrier_manager.subscribe_barrier(actor_id);
181
182 let metrics = StreamingMetrics::unused();
183 let actor_ctx = ActorContext::for_test(actor_id);
184 let upstream = Self::new_select_receiver(
185 inputs
186 .into_iter()
187 .enumerate()
188 .map(|(idx, input)| LocalInput::new(input, idx as ActorId).boxed_input())
189 .collect(),
190 &metrics,
191 &actor_ctx,
192 );
193
194 Self::new(
195 actor_ctx,
196 514,
197 1919,
198 upstream,
199 local_barrier_manager,
200 metrics.into(),
201 barrier_rx,
202 100,
203 schema,
204 )
205 }
206
207 pub(crate) fn new_select_receiver(
208 upstreams: Vec<BoxedInput>,
209 metrics: &StreamingMetrics,
210 actor_context: &ActorContext,
211 ) -> SelectReceivers {
212 let merge_barrier_align_duration = if metrics.level >= MetricLevel::Debug {
213 Some(
214 metrics
215 .merge_barrier_align_duration
216 .with_guarded_label_values(&[
217 &actor_context.id.to_string(),
218 &actor_context.fragment_id.to_string(),
219 ]),
220 )
221 } else {
222 None
223 };
224
225 SelectReceivers::new(
227 actor_context.id,
228 upstreams,
229 merge_barrier_align_duration.clone(),
230 )
231 }
232
233 #[try_stream(ok = Message, error = StreamExecutorError)]
234 async fn execute_inner(mut self: Box<Self>) {
235 let select_all = self.upstreams;
236 let select_all = BufferChunks::new(select_all, self.chunk_size, self.schema);
237 let actor_id = self.actor_context.id;
238
239 let mut metrics = self.metrics.new_actor_input_metrics(
240 actor_id,
241 self.fragment_id,
242 self.upstream_fragment_id,
243 );
244
245 let mut start_time = Instant::now();
247 pin_mut!(select_all);
248 while let Some(msg) = select_all.next().await {
249 metrics
250 .actor_input_buffer_blocking_duration_ns
251 .inc_by(start_time.elapsed().as_nanos() as u64);
252 let msg: DispatcherMessage = msg?;
253 let mut msg: Message = process_dispatcher_msg(msg, &mut self.barrier_rx).await?;
254
255 match &mut msg {
256 Message::Watermark(_) => {
257 }
259 Message::Chunk(chunk) => {
260 metrics.actor_in_record_cnt.inc_by(chunk.cardinality() as _);
261 }
262 Message::Barrier(barrier) => {
263 tracing::debug!(
264 target: "events::stream::barrier::path",
265 actor_id = actor_id,
266 "receiver receives barrier from path: {:?}",
267 barrier.passed_actors
268 );
269 barrier.passed_actors.push(actor_id);
270
271 if let Some(Mutation::Update(UpdateMutation { dispatchers, .. })) =
272 barrier.mutation.as_deref()
273 && select_all
274 .upstream_actor_ids()
275 .iter()
276 .any(|actor_id| dispatchers.contains_key(actor_id))
277 {
278 select_all
280 .buffered_watermarks
281 .values_mut()
282 .for_each(|buffers| buffers.clear());
283 }
284
285 if let Some(update) =
286 barrier.as_update_merge(self.actor_context.id, self.upstream_fragment_id)
287 {
288 let new_upstream_fragment_id = update
289 .new_upstream_fragment_id
290 .unwrap_or(self.upstream_fragment_id);
291 let removed_upstream_actor_id: HashSet<_> =
292 if update.new_upstream_fragment_id.is_some() {
293 select_all.upstream_actor_ids().iter().copied().collect()
294 } else {
295 update.removed_upstream_actor_id.iter().copied().collect()
296 };
297
298 select_all
300 .buffered_watermarks
301 .values_mut()
302 .for_each(|buffers| buffers.clear());
303
304 if !update.added_upstream_actors.is_empty() {
305 let new_upstreams: Vec<_> = try_join_all(
307 update.added_upstream_actors.iter().map(|upstream_actor| {
308 new_input(
309 &self.local_barrier_manager,
310 self.metrics.clone(),
311 self.actor_context.id,
312 self.fragment_id,
313 upstream_actor,
314 new_upstream_fragment_id,
315 )
316 }),
317 )
318 .await
319 .context("failed to create upstream receivers")?;
320
321 let mut select_new = SelectReceivers::new(
324 self.actor_context.id,
325 new_upstreams,
326 select_all.merge_barrier_align_duration(),
327 );
328 let new_barrier = expect_first_barrier(&mut select_new).await?;
329 assert_equal_dispatcher_barrier(barrier, &new_barrier);
330
331 select_all.add_upstreams_from(select_new);
333
334 select_all
336 .buffered_watermarks
337 .values_mut()
338 .for_each(|buffers| {
339 buffers.add_buffers(
340 update
341 .added_upstream_actors
342 .iter()
343 .map(|actor| actor.actor_id),
344 )
345 });
346 }
347
348 if !removed_upstream_actor_id.is_empty() {
349 select_all.remove_upstreams(&removed_upstream_actor_id);
351
352 for buffers in select_all.buffered_watermarks.values_mut() {
353 buffers.remove_buffer(removed_upstream_actor_id.clone());
356 }
357 }
358
359 self.upstream_fragment_id = new_upstream_fragment_id;
360 metrics = self.metrics.new_actor_input_metrics(
361 actor_id,
362 self.fragment_id,
363 self.upstream_fragment_id,
364 );
365
366 select_all.update_actor_ids();
367 }
368
369 if barrier.is_stop(actor_id) {
370 yield msg;
371 break;
372 }
373 }
374 }
375
376 yield msg;
377 start_time = Instant::now();
378 }
379 }
380}
381
382impl Execute for MergeExecutor {
383 fn execute(self: Box<Self>) -> BoxedMessageStream {
384 self.execute_inner().boxed()
385 }
386}
387
388pub struct SelectReceivers {
390 barrier: Option<DispatcherBarrier>,
392 blocked: Vec<BoxedInput>,
394 active: FuturesUnordered<StreamFuture<BoxedInput>>,
396 upstream_actor_ids: Vec<ActorId>,
398
399 actor_id: u32,
401 buffered_watermarks: BTreeMap<usize, BufferedWatermarks<ActorId>>,
403 merge_barrier_align_duration: Option<LabelGuardedMetric<Histogram>>,
405}
406
407impl Stream for SelectReceivers {
408 type Item = DispatcherMessageStreamItem;
409
410 fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
411 if self.active.is_terminated() {
412 assert!(self.blocked.is_empty());
414 return Poll::Ready(None);
415 }
416
417 let mut start = None;
418 loop {
419 match futures::ready!(self.active.poll_next_unpin(cx)) {
420 Some((Some(Err(e)), _)) => {
422 return Poll::Ready(Some(Err(e)));
423 }
424 Some((Some(Ok(message)), remaining)) => {
426 let actor_id = remaining.actor_id();
427 match message {
428 DispatcherMessage::Chunk(chunk) => {
429 self.active.push(remaining.into_future());
431 return Poll::Ready(Some(Ok(DispatcherMessage::Chunk(chunk))));
432 }
433 DispatcherMessage::Watermark(watermark) => {
434 self.active.push(remaining.into_future());
436 if let Some(watermark) = self.handle_watermark(actor_id, watermark) {
437 return Poll::Ready(Some(Ok(DispatcherMessage::Watermark(
438 watermark,
439 ))));
440 }
441 }
442 DispatcherMessage::Barrier(barrier) => {
443 if self.blocked.is_empty()
445 && self.merge_barrier_align_duration.is_some()
446 {
447 start = Some(Instant::now());
448 }
449 self.blocked.push(remaining);
450 if let Some(current_barrier) = self.barrier.as_ref() {
451 if current_barrier.epoch != barrier.epoch {
452 return Poll::Ready(Some(Err(
453 StreamExecutorError::align_barrier(
454 current_barrier.clone().map_mutation(|_| None),
455 barrier.map_mutation(|_| None),
456 ),
457 )));
458 }
459 } else {
460 self.barrier = Some(barrier);
461 }
462 }
463 }
464 }
465 Some((None, _)) => unreachable!(),
473 None => {
475 if let Some(start) = start
476 && let Some(merge_barrier_align_duration) =
477 &self.merge_barrier_align_duration
478 {
479 merge_barrier_align_duration.observe(start.elapsed().as_secs_f64())
481 }
482 break;
483 }
484 }
485 }
486
487 assert!(self.active.is_terminated());
488 let barrier = self.barrier.take().unwrap();
489
490 let upstreams = std::mem::take(&mut self.blocked);
491 self.extend_active(upstreams);
492 assert!(!self.active.is_terminated());
493
494 Poll::Ready(Some(Ok(DispatcherMessage::Barrier(barrier))))
495 }
496}
497
498impl SelectReceivers {
499 fn new(
500 actor_id: u32,
501 upstreams: Vec<BoxedInput>,
502 merge_barrier_align_duration: Option<LabelGuardedMetric<Histogram>>,
503 ) -> Self {
504 assert!(!upstreams.is_empty());
505 let upstream_actor_ids = upstreams.iter().map(|input| input.actor_id()).collect();
506 let mut this = Self {
507 blocked: Vec::with_capacity(upstreams.len()),
508 active: Default::default(),
509 actor_id,
510 barrier: None,
511 upstream_actor_ids,
512 buffered_watermarks: Default::default(),
513 merge_barrier_align_duration,
514 };
515 this.extend_active(upstreams);
516 this
517 }
518
519 fn extend_active(&mut self, upstreams: impl IntoIterator<Item = BoxedInput>) {
522 assert!(self.blocked.is_empty() && self.barrier.is_none());
523
524 self.active
525 .extend(upstreams.into_iter().map(|s| s.into_future()));
526 }
527
528 fn upstream_actor_ids(&self) -> &[ActorId] {
529 &self.upstream_actor_ids
530 }
531
532 fn update_actor_ids(&mut self) {
533 self.upstream_actor_ids = self
534 .blocked
535 .iter()
536 .map(|input| input.actor_id())
537 .chain(
538 self.active
539 .iter()
540 .map(|input| input.get_ref().unwrap().actor_id()),
541 )
542 .collect();
543 }
544
545 fn handle_watermark(&mut self, actor_id: ActorId, watermark: Watermark) -> Option<Watermark> {
547 let col_idx = watermark.col_idx;
548 let watermarks = self
550 .buffered_watermarks
551 .entry(col_idx)
552 .or_insert_with(|| BufferedWatermarks::with_ids(self.upstream_actor_ids.clone()));
553 watermarks.handle_watermark(actor_id, watermark)
554 }
555
556 fn add_upstreams_from(&mut self, other: Self) {
559 assert!(self.blocked.is_empty() && self.barrier.is_none());
560 assert!(other.blocked.is_empty() && other.barrier.is_none());
561 assert_eq!(self.actor_id, other.actor_id);
562
563 self.active.extend(other.active);
564 }
565
566 fn remove_upstreams(&mut self, upstream_actor_ids: &HashSet<ActorId>) {
569 assert!(self.blocked.is_empty() && self.barrier.is_none());
570
571 let new_upstreams = std::mem::take(&mut self.active)
572 .into_iter()
573 .map(|s| s.into_inner().unwrap())
574 .filter(|u| !upstream_actor_ids.contains(&u.actor_id()));
575 self.extend_active(new_upstreams);
576 }
577
578 fn merge_barrier_align_duration(&self) -> Option<LabelGuardedMetric<Histogram>> {
579 self.merge_barrier_align_duration.clone()
580 }
581}
582
583struct BufferChunks<S: Stream> {
587 inner: S,
588 chunk_builder: StreamChunkBuilder,
589
590 pending_items: VecDeque<S::Item>,
592}
593
594impl<S: Stream> BufferChunks<S> {
595 pub(super) fn new(inner: S, chunk_size: usize, schema: Schema) -> Self {
596 assert!(chunk_size > 0);
597 let chunk_builder = StreamChunkBuilder::new(chunk_size, schema.data_types());
598 Self {
599 inner,
600 chunk_builder,
601 pending_items: VecDeque::new(),
602 }
603 }
604}
605
606impl<S: Stream> std::ops::Deref for BufferChunks<S> {
607 type Target = S;
608
609 fn deref(&self) -> &Self::Target {
610 &self.inner
611 }
612}
613
614impl<S: Stream> std::ops::DerefMut for BufferChunks<S> {
615 fn deref_mut(&mut self) -> &mut Self::Target {
616 &mut self.inner
617 }
618}
619
620impl<S: Stream> Stream for BufferChunks<S>
621where
622 S: Stream<Item = DispatcherMessageStreamItem> + Unpin,
623{
624 type Item = S::Item;
625
626 fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
627 loop {
628 if let Some(item) = self.pending_items.pop_front() {
629 return Poll::Ready(Some(item));
630 }
631
632 match self.inner.poll_next_unpin(cx) {
633 Poll::Pending => {
634 return if let Some(chunk_out) = self.chunk_builder.take() {
635 Poll::Ready(Some(Ok(MessageInner::Chunk(chunk_out))))
636 } else {
637 Poll::Pending
638 };
639 }
640
641 Poll::Ready(Some(result)) => {
642 if let Ok(MessageInner::Chunk(chunk)) = result {
643 for row in chunk.records() {
644 if let Some(chunk_out) = self.chunk_builder.append_record(row) {
645 self.pending_items
646 .push_back(Ok(MessageInner::Chunk(chunk_out)));
647 }
648 }
649 } else {
650 return if let Some(chunk_out) = self.chunk_builder.take() {
651 self.pending_items.push_back(result);
652 Poll::Ready(Some(Ok(MessageInner::Chunk(chunk_out))))
653 } else {
654 Poll::Ready(Some(result))
655 };
656 }
657 }
658
659 Poll::Ready(None) => {
660 unreachable!("SelectReceivers should never return None");
662 }
663 }
664 }
665 }
666}
667
668#[cfg(test)]
669mod tests {
670 use std::sync::atomic::{AtomicBool, Ordering};
671 use std::time::Duration;
672
673 use assert_matches::assert_matches;
674 use futures::FutureExt;
675 use futures::future::try_join_all;
676 use risingwave_common::array::Op;
677 use risingwave_common::util::epoch::test_epoch;
678 use risingwave_pb::task_service::exchange_service_server::{
679 ExchangeService, ExchangeServiceServer,
680 };
681 use risingwave_pb::task_service::{
682 GetDataRequest, GetDataResponse, GetStreamRequest, GetStreamResponse, PbPermits,
683 };
684 use tokio::time::sleep;
685 use tokio_stream::wrappers::ReceiverStream;
686 use tonic::{Request, Response, Status, Streaming};
687
688 use super::*;
689 use crate::executor::exchange::input::{Input, LocalInput, RemoteInput};
690 use crate::executor::exchange::permit::channel_for_test;
691 use crate::executor::{BarrierInner as Barrier, MessageInner as Message};
692 use crate::task::NewOutputRequest;
693 use crate::task::barrier_test_utils::LocalBarrierTestEnv;
694 use crate::task::test_utils::helper_make_local_actor;
695
696 fn build_test_chunk(size: u64) -> StreamChunk {
697 let ops = vec![Op::Insert; size as usize];
698 StreamChunk::new(ops, vec![])
699 }
700
701 #[tokio::test]
702 async fn test_buffer_chunks() {
703 let test_env = LocalBarrierTestEnv::for_test().await;
704
705 let (tx, rx) = channel_for_test();
706 let input = LocalInput::new(rx, 1).boxed_input();
707 let mut buffer = BufferChunks::new(input, 100, Schema::new(vec![]));
708
709 tx.send(Message::Chunk(build_test_chunk(10)).into())
711 .await
712 .unwrap();
713 assert_matches!(buffer.next().await.unwrap().unwrap(), Message::Chunk(chunk) => {
714 assert_eq!(chunk.ops().len() as u64, 10);
715 });
716
717 tx.send(Message::Chunk(build_test_chunk(10)).into())
719 .await
720 .unwrap();
721 tx.send(Message::Chunk(build_test_chunk(10)).into())
722 .await
723 .unwrap();
724 assert_matches!(buffer.next().await.unwrap().unwrap(), Message::Chunk(chunk) => {
725 assert_eq!(chunk.ops().len() as u64, 20);
726 });
727
728 tx.send(
730 Message::Watermark(Watermark {
731 col_idx: 0,
732 data_type: DataType::Int64,
733 val: ScalarImpl::Int64(233),
734 })
735 .into(),
736 )
737 .await
738 .unwrap();
739 assert_matches!(buffer.next().await.unwrap().unwrap(), Message::Watermark(watermark) => {
740 assert_eq!(watermark.val, ScalarImpl::Int64(233));
741 });
742
743 tx.send(Message::Chunk(build_test_chunk(10)).into())
745 .await
746 .unwrap();
747 tx.send(Message::Chunk(build_test_chunk(10)).into())
748 .await
749 .unwrap();
750 tx.send(
751 Message::Watermark(Watermark {
752 col_idx: 0,
753 data_type: DataType::Int64,
754 val: ScalarImpl::Int64(233),
755 })
756 .into(),
757 )
758 .await
759 .unwrap();
760 assert_matches!(buffer.next().await.unwrap().unwrap(), Message::Chunk(chunk) => {
761 assert_eq!(chunk.ops().len() as u64, 20);
762 });
763 assert_matches!(buffer.next().await.unwrap().unwrap(), Message::Watermark(watermark) => {
764 assert_eq!(watermark.val, ScalarImpl::Int64(233));
765 });
766
767 let barrier = Barrier::new_test_barrier(test_epoch(1));
769 test_env.inject_barrier(&barrier, [2]);
770 tx.send(Message::Barrier(barrier.clone().into_dispatcher()).into())
771 .await
772 .unwrap();
773 assert_matches!(buffer.next().await.unwrap().unwrap(), Message::Barrier(Barrier { epoch: barrier_epoch, mutation: _, .. }) => {
774 assert_eq!(barrier_epoch.curr, test_epoch(1));
775 });
776
777 tx.send(Message::Chunk(build_test_chunk(10)).into())
779 .await
780 .unwrap();
781 tx.send(Message::Chunk(build_test_chunk(10)).into())
782 .await
783 .unwrap();
784 let barrier = Barrier::new_test_barrier(test_epoch(2));
785 test_env.inject_barrier(&barrier, [2]);
786 tx.send(Message::Barrier(barrier.clone().into_dispatcher()).into())
787 .await
788 .unwrap();
789 assert_matches!(buffer.next().await.unwrap().unwrap(), Message::Chunk(chunk) => {
790 assert_eq!(chunk.ops().len() as u64, 20);
791 });
792 assert_matches!(buffer.next().await.unwrap().unwrap(), Message::Barrier(Barrier { epoch: barrier_epoch, mutation: _, .. }) => {
793 assert_eq!(barrier_epoch.curr, test_epoch(2));
794 });
795 }
796
797 #[tokio::test]
798 async fn test_merger() {
799 const CHANNEL_NUMBER: usize = 10;
800 let mut txs = Vec::with_capacity(CHANNEL_NUMBER);
801 let mut rxs = Vec::with_capacity(CHANNEL_NUMBER);
802 for _i in 0..CHANNEL_NUMBER {
803 let (tx, rx) = channel_for_test();
804 txs.push(tx);
805 rxs.push(rx);
806 }
807 let barrier_test_env = LocalBarrierTestEnv::for_test().await;
808 let actor_id = 233;
809 let mut handles = Vec::with_capacity(CHANNEL_NUMBER);
810
811 let epochs = (10..1000u64)
812 .step_by(10)
813 .map(|idx| (idx, test_epoch(idx)))
814 .collect_vec();
815 let mut prev_epoch = 0;
816 let prev_epoch = &mut prev_epoch;
817 let barriers: HashMap<_, _> = epochs
818 .iter()
819 .map(|(_, epoch)| {
820 let barrier = Barrier::with_prev_epoch_for_test(*epoch, *prev_epoch);
821 *prev_epoch = *epoch;
822 barrier_test_env.inject_barrier(&barrier, [actor_id]);
823 (*epoch, barrier)
824 })
825 .collect();
826 let b2 = Barrier::with_prev_epoch_for_test(test_epoch(1000), *prev_epoch)
827 .with_mutation(Mutation::Stop(HashSet::default()));
828 barrier_test_env.inject_barrier(&b2, [actor_id]);
829 barrier_test_env.flush_all_events().await;
830
831 for (tx_id, tx) in txs.into_iter().enumerate() {
832 let epochs = epochs.clone();
833 let barriers = barriers.clone();
834 let b2 = b2.clone();
835 let handle = tokio::spawn(async move {
836 for (idx, epoch) in epochs {
837 if idx % 20 == 0 {
838 tx.send(Message::Chunk(build_test_chunk(10)).into())
839 .await
840 .unwrap();
841 } else {
842 tx.send(
843 Message::Watermark(Watermark {
844 col_idx: (idx as usize / 20 + tx_id) % CHANNEL_NUMBER,
845 data_type: DataType::Int64,
846 val: ScalarImpl::Int64(idx as i64),
847 })
848 .into(),
849 )
850 .await
851 .unwrap();
852 }
853 tx.send(Message::Barrier(barriers[&epoch].clone().into_dispatcher()).into())
854 .await
855 .unwrap();
856 sleep(Duration::from_millis(1)).await;
857 }
858 tx.send(Message::Barrier(b2.clone().into_dispatcher()).into())
859 .await
860 .unwrap();
861 });
862 handles.push(handle);
863 }
864
865 let merger = MergeExecutor::for_test(
866 actor_id,
867 rxs,
868 barrier_test_env.local_barrier_manager.clone(),
869 Schema::new(vec![]),
870 );
871 let mut merger = merger.boxed().execute();
872 for (idx, epoch) in epochs {
873 if idx % 20 == 0 {
874 let mut count = 0usize;
876 while count < 100 {
877 assert_matches!(merger.next().await.unwrap().unwrap(), Message::Chunk(chunk) => {
878 count += chunk.ops().len();
879 });
880 }
881 assert_eq!(count, 100);
882 } else if idx as usize / 20 >= CHANNEL_NUMBER - 1 {
883 for _ in 0..CHANNEL_NUMBER {
885 assert_matches!(merger.next().await.unwrap().unwrap(), Message::Watermark(watermark) => {
886 assert_eq!(watermark.val, ScalarImpl::Int64((idx - 20 * (CHANNEL_NUMBER as u64 - 1)) as i64));
887 });
888 }
889 }
890 assert_matches!(merger.next().await.unwrap().unwrap(), Message::Barrier(Barrier{epoch:barrier_epoch,mutation:_,..}) => {
892 assert_eq!(barrier_epoch.curr, epoch);
893 });
894 }
895 assert_matches!(
896 merger.next().await.unwrap().unwrap(),
897 Message::Barrier(Barrier {
898 mutation,
899 ..
900 }) if mutation.as_deref().unwrap().is_stop()
901 );
902
903 for handle in handles {
904 handle.await.unwrap();
905 }
906 }
907
908 #[tokio::test]
909 async fn test_configuration_change() {
910 let actor_id = 233;
911 let (untouched, old, new) = (234, 235, 238); let barrier_test_env = LocalBarrierTestEnv::for_test().await;
913 let metrics = Arc::new(StreamingMetrics::unused());
914
915 let (upstream_fragment_id, fragment_id) = (10, 18);
920
921 let inputs: Vec<_> =
922 try_join_all([untouched, old].into_iter().map(async |upstream_actor_id| {
923 new_input(
924 &barrier_test_env.local_barrier_manager,
925 metrics.clone(),
926 actor_id,
927 fragment_id,
928 &helper_make_local_actor(upstream_actor_id),
929 upstream_fragment_id,
930 )
931 .await
932 }))
933 .await
934 .unwrap();
935
936 let merge_updates = maplit::hashmap! {
937 (actor_id, upstream_fragment_id) => MergeUpdate {
938 actor_id,
939 upstream_fragment_id,
940 new_upstream_fragment_id: None,
941 added_upstream_actors: vec![helper_make_local_actor(new)],
942 removed_upstream_actor_id: vec![old],
943 }
944 };
945
946 let b1 = Barrier::new_test_barrier(test_epoch(1)).with_mutation(Mutation::Update(
947 UpdateMutation {
948 dispatchers: Default::default(),
949 merges: merge_updates,
950 vnode_bitmaps: Default::default(),
951 dropped_actors: Default::default(),
952 actor_splits: Default::default(),
953 actor_new_dispatchers: Default::default(),
954 },
955 ));
956 barrier_test_env.inject_barrier(&b1, [actor_id]);
957 barrier_test_env.flush_all_events().await;
958
959 let barrier_rx = barrier_test_env
960 .local_barrier_manager
961 .subscribe_barrier(actor_id);
962 let actor_ctx = ActorContext::for_test(actor_id);
963 let upstream = MergeExecutor::new_select_receiver(inputs, &metrics, &actor_ctx);
964
965 let mut merge = MergeExecutor::new(
966 actor_ctx,
967 fragment_id,
968 upstream_fragment_id,
969 upstream,
970 barrier_test_env.local_barrier_manager.clone(),
971 metrics.clone(),
972 barrier_rx,
973 100,
974 Schema::new(vec![]),
975 )
976 .boxed()
977 .execute();
978
979 let mut txs = HashMap::new();
980 macro_rules! send {
981 ($actors:expr, $msg:expr) => {
982 for actor in $actors {
983 txs.get(&actor).unwrap().send($msg).await.unwrap();
984 }
985 };
986 }
987
988 macro_rules! assert_recv_pending {
989 () => {
990 assert!(
991 merge
992 .next()
993 .now_or_never()
994 .flatten()
995 .transpose()
996 .unwrap()
997 .is_none()
998 );
999 };
1000 }
1001 macro_rules! recv {
1002 () => {
1003 merge.next().await.transpose().unwrap()
1004 };
1005 }
1006
1007 macro_rules! collect_upstream_tx {
1008 ($actors:expr) => {
1009 for upstream_id in $actors {
1010 let mut output_requests = barrier_test_env
1011 .take_pending_new_output_requests(upstream_id)
1012 .await;
1013 assert_eq!(output_requests.len(), 1);
1014 let (downstream_actor_id, request) = output_requests.pop().unwrap();
1015 assert_eq!(actor_id, downstream_actor_id);
1016 let NewOutputRequest::Local(tx) = request else {
1017 unreachable!()
1018 };
1019 txs.insert(upstream_id, tx);
1020 }
1021 };
1022 }
1023
1024 assert_recv_pending!();
1025 barrier_test_env.flush_all_events().await;
1026
1027 collect_upstream_tx!([untouched, old]);
1029
1030 send!([untouched, old], Message::Chunk(build_test_chunk(1)).into());
1032 assert_eq!(2, recv!().unwrap().as_chunk().unwrap().cardinality()); assert_recv_pending!();
1034
1035 send!(
1036 [untouched, old],
1037 Message::Barrier(b1.clone().into_dispatcher()).into()
1038 );
1039 assert_recv_pending!(); collect_upstream_tx!([new]);
1042
1043 send!([new], Message::Barrier(b1.clone().into_dispatcher()).into());
1044 recv!().unwrap().as_barrier().unwrap(); send!([untouched, new], Message::Chunk(build_test_chunk(1)).into());
1048 assert_eq!(2, recv!().unwrap().as_chunk().unwrap().cardinality()); assert_recv_pending!();
1050 }
1051
1052 struct FakeExchangeService {
1053 rpc_called: Arc<AtomicBool>,
1054 }
1055
1056 fn exchange_client_test_barrier() -> crate::executor::Barrier {
1057 Barrier::new_test_barrier(test_epoch(1))
1058 }
1059
1060 #[async_trait::async_trait]
1061 impl ExchangeService for FakeExchangeService {
1062 type GetDataStream = ReceiverStream<std::result::Result<GetDataResponse, Status>>;
1063 type GetStreamStream = ReceiverStream<std::result::Result<GetStreamResponse, Status>>;
1064
1065 async fn get_data(
1066 &self,
1067 _: Request<GetDataRequest>,
1068 ) -> std::result::Result<Response<Self::GetDataStream>, Status> {
1069 unimplemented!()
1070 }
1071
1072 async fn get_stream(
1073 &self,
1074 _request: Request<Streaming<GetStreamRequest>>,
1075 ) -> std::result::Result<Response<Self::GetStreamStream>, Status> {
1076 let (tx, rx) = tokio::sync::mpsc::channel(10);
1077 self.rpc_called.store(true, Ordering::SeqCst);
1078 let stream_chunk = StreamChunk::default().to_protobuf();
1080 tx.send(Ok(GetStreamResponse {
1081 message: Some(PbStreamMessageBatch {
1082 stream_message_batch: Some(
1083 risingwave_pb::stream_plan::stream_message_batch::StreamMessageBatch::StreamChunk(
1084 stream_chunk,
1085 ),
1086 ),
1087 }),
1088 permits: Some(PbPermits::default()),
1089 }))
1090 .await
1091 .unwrap();
1092 let barrier = exchange_client_test_barrier();
1094 tx.send(Ok(GetStreamResponse {
1095 message: Some(PbStreamMessageBatch {
1096 stream_message_batch: Some(
1097 risingwave_pb::stream_plan::stream_message_batch::StreamMessageBatch::BarrierBatch(
1098 BarrierBatch {
1099 barriers: vec![barrier.to_protobuf()],
1100 },
1101 ),
1102 ),
1103 }),
1104 permits: Some(PbPermits::default()),
1105 }))
1106 .await
1107 .unwrap();
1108 Ok(Response::new(ReceiverStream::new(rx)))
1109 }
1110 }
1111
1112 #[tokio::test]
1113 async fn test_stream_exchange_client() {
1114 let rpc_called = Arc::new(AtomicBool::new(false));
1115 let server_run = Arc::new(AtomicBool::new(false));
1116 let addr = "127.0.0.1:12348".parse().unwrap();
1117
1118 let (shutdown_send, shutdown_recv) = tokio::sync::oneshot::channel();
1120 let exchange_svc = ExchangeServiceServer::new(FakeExchangeService {
1121 rpc_called: rpc_called.clone(),
1122 });
1123 let cp_server_run = server_run.clone();
1124 let join_handle = tokio::spawn(async move {
1125 cp_server_run.store(true, Ordering::SeqCst);
1126 tonic::transport::Server::builder()
1127 .add_service(exchange_svc)
1128 .serve_with_shutdown(addr, async move {
1129 shutdown_recv.await.unwrap();
1130 })
1131 .await
1132 .unwrap();
1133 });
1134
1135 sleep(Duration::from_secs(1)).await;
1136 assert!(server_run.load(Ordering::SeqCst));
1137
1138 let test_env = LocalBarrierTestEnv::for_test().await;
1139
1140 let remote_input = {
1141 RemoteInput::new(
1142 &test_env.local_barrier_manager,
1143 addr.into(),
1144 (0, 0),
1145 (0, 0),
1146 Arc::new(StreamingMetrics::unused()),
1147 )
1148 .await
1149 .unwrap()
1150 };
1151
1152 test_env.inject_barrier(&exchange_client_test_barrier(), [remote_input.actor_id()]);
1153
1154 pin_mut!(remote_input);
1155
1156 assert_matches!(remote_input.next().await.unwrap().unwrap(), Message::Chunk(chunk) => {
1157 let (ops, columns, visibility) = chunk.into_inner();
1158 assert!(ops.is_empty());
1159 assert!(columns.is_empty());
1160 assert!(visibility.is_empty());
1161 });
1162 assert_matches!(remote_input.next().await.unwrap().unwrap(), Message::Barrier(Barrier { epoch: barrier_epoch, mutation: _, .. }) => {
1163 assert_eq!(barrier_epoch.curr, test_epoch(1));
1164 });
1165 assert!(rpc_called.load(Ordering::SeqCst));
1166
1167 shutdown_send.send(()).unwrap();
1168 join_handle.await.unwrap();
1169 }
1170}