risingwave_stream/executor/
dispatch.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use std::collections::{HashMap, HashSet};
use std::future::Future;
use std::iter::repeat_with;
use std::ops::{Deref, DerefMut};
use std::time::Duration;

use futures::TryStreamExt;
use itertools::Itertools;
use risingwave_common::array::Op;
use risingwave_common::bitmap::BitmapBuilder;
use risingwave_common::hash::{ActorMapping, ExpandedActorMapping, VirtualNode};
use risingwave_common::metrics::LabelGuardedIntCounter;
use risingwave_common::util::iter_util::ZipEqFast;
use risingwave_pb::stream_plan::update_mutation::PbDispatcherUpdate;
use risingwave_pb::stream_plan::PbDispatcher;
use smallvec::{smallvec, SmallVec};
use tokio::time::Instant;
use tracing::{event, Instrument};

use super::exchange::output::{new_output, BoxedOutput};
use super::{
    AddMutation, DispatcherBarrier, DispatcherMessage, TroublemakerExecutor, UpdateMutation,
};
use crate::executor::prelude::*;
use crate::executor::StreamConsumer;
use crate::task::{DispatcherId, SharedContext};

/// [`DispatchExecutor`] consumes messages and send them into downstream actors. Usually,
/// data chunks will be dispatched with some specified policy, while control message
/// such as barriers will be distributed to all receivers.
pub struct DispatchExecutor {
    input: Executor,
    inner: DispatchExecutorInner,
}

struct DispatcherWithMetrics {
    dispatcher: DispatcherImpl,
    actor_output_buffer_blocking_duration_ns: LabelGuardedIntCounter<3>,
}

impl DispatcherWithMetrics {
    pub fn record_output_buffer_blocking_duration(&self, duration: Duration) {
        let ns = duration.as_nanos() as u64;
        self.actor_output_buffer_blocking_duration_ns.inc_by(ns);
    }
}

impl Debug for DispatcherWithMetrics {
    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
        self.dispatcher.fmt(f)
    }
}

impl Deref for DispatcherWithMetrics {
    type Target = DispatcherImpl;

    fn deref(&self) -> &Self::Target {
        &self.dispatcher
    }
}

impl DerefMut for DispatcherWithMetrics {
    fn deref_mut(&mut self) -> &mut Self::Target {
        &mut self.dispatcher
    }
}

struct DispatchExecutorMetrics {
    actor_id_str: String,
    fragment_id_str: String,
    metrics: Arc<StreamingMetrics>,
    actor_out_record_cnt: LabelGuardedIntCounter<2>,
}

impl DispatchExecutorMetrics {
    fn monitor_dispatcher(&self, dispatcher: DispatcherImpl) -> DispatcherWithMetrics {
        DispatcherWithMetrics {
            actor_output_buffer_blocking_duration_ns: self
                .metrics
                .actor_output_buffer_blocking_duration_ns
                .with_guarded_label_values(&[
                    &self.actor_id_str,
                    &self.fragment_id_str,
                    dispatcher.dispatcher_id_str(),
                ]),
            dispatcher,
        }
    }
}

struct DispatchExecutorInner {
    dispatchers: Vec<DispatcherWithMetrics>,
    actor_id: u32,
    context: Arc<SharedContext>,
    metrics: DispatchExecutorMetrics,
}

impl DispatchExecutorInner {
    async fn dispatch(&mut self, msg: Message) -> StreamResult<()> {
        let limit = (self.context.config.developer).exchange_concurrent_dispatchers;

        match msg {
            Message::Watermark(watermark) => {
                futures::stream::iter(self.dispatchers.iter_mut())
                    .map(Ok)
                    .try_for_each_concurrent(limit, |dispatcher| async {
                        let start_time = Instant::now();
                        dispatcher.dispatch_watermark(watermark.clone()).await?;
                        dispatcher.record_output_buffer_blocking_duration(start_time.elapsed());
                        StreamResult::Ok(())
                    })
                    .await?;
            }
            Message::Chunk(chunk) => {
                futures::stream::iter(self.dispatchers.iter_mut())
                    .map(Ok)
                    .try_for_each_concurrent(limit, |dispatcher| async {
                        let start_time = Instant::now();
                        dispatcher.dispatch_data(chunk.clone()).await?;
                        dispatcher.record_output_buffer_blocking_duration(start_time.elapsed());
                        StreamResult::Ok(())
                    })
                    .await?;

                self.metrics
                    .actor_out_record_cnt
                    .inc_by(chunk.cardinality() as _);
            }
            Message::Barrier(barrier) => {
                let mutation = barrier.mutation.clone();
                self.pre_mutate_dispatchers(&mutation)?;

                futures::stream::iter(self.dispatchers.iter_mut())
                    .map(Ok)
                    .try_for_each_concurrent(limit, |dispatcher| async {
                        let start_time = Instant::now();
                        dispatcher
                            .dispatch_barrier(barrier.clone().into_dispatcher())
                            .await?;
                        dispatcher.record_output_buffer_blocking_duration(start_time.elapsed());
                        StreamResult::Ok(())
                    })
                    .await?;

                self.post_mutate_dispatchers(&mutation)?;
            }
        };

        Ok(())
    }

    /// Add new dispatchers to the executor. Will check whether their ids are unique.
    fn add_dispatchers<'a>(
        &mut self,
        new_dispatchers: impl IntoIterator<Item = &'a PbDispatcher>,
    ) -> StreamResult<()> {
        let new_dispatchers: Vec<_> = new_dispatchers
            .into_iter()
            .map(|d| {
                DispatcherImpl::new(&self.context, self.actor_id, d)
                    .map(|dispatcher| self.metrics.monitor_dispatcher(dispatcher))
            })
            .try_collect()?;

        self.dispatchers.extend(new_dispatchers);

        assert!(
            self.dispatchers
                .iter()
                .map(|d| d.dispatcher_id())
                .all_unique(),
            "dispatcher ids must be unique: {:?}",
            self.dispatchers
        );

        Ok(())
    }

    fn find_dispatcher(&mut self, dispatcher_id: DispatcherId) -> &mut DispatcherImpl {
        self.dispatchers
            .iter_mut()
            .find(|d| d.dispatcher_id() == dispatcher_id)
            .unwrap_or_else(|| panic!("dispatcher {}:{} not found", self.actor_id, dispatcher_id))
    }

    /// Update the dispatcher BEFORE we actually dispatch this barrier. We'll only add the new
    /// outputs.
    fn pre_update_dispatcher(&mut self, update: &PbDispatcherUpdate) -> StreamResult<()> {
        let outputs: Vec<_> = update
            .added_downstream_actor_id
            .iter()
            .map(|&id| new_output(&self.context, self.actor_id, id))
            .try_collect()?;

        let dispatcher = self.find_dispatcher(update.dispatcher_id);
        dispatcher.add_outputs(outputs);

        Ok(())
    }

    /// Update the dispatcher AFTER we dispatch this barrier. We'll remove some outputs and finally
    /// update the hash mapping.
    fn post_update_dispatcher(&mut self, update: &PbDispatcherUpdate) -> StreamResult<()> {
        let ids = update.removed_downstream_actor_id.iter().copied().collect();

        let dispatcher = self.find_dispatcher(update.dispatcher_id);
        dispatcher.remove_outputs(&ids);

        // The hash mapping is only used by the hash dispatcher.
        //
        // We specify a single upstream hash mapping for scaling the downstream fragment. However,
        // it's possible that there're multiple upstreams with different exchange types, for
        // example, the `Broadcast` inner side of the dynamic filter. There're too many combinations
        // to handle here, so we just ignore the `hash_mapping` field for any other exchange types.
        if let DispatcherImpl::Hash(dispatcher) = dispatcher {
            dispatcher.hash_mapping =
                ActorMapping::from_protobuf(update.get_hash_mapping()?).to_expanded();
        }

        Ok(())
    }

    /// For `Add` and `Update`, update the dispatchers before we dispatch the barrier.
    fn pre_mutate_dispatchers(&mut self, mutation: &Option<Arc<Mutation>>) -> StreamResult<()> {
        let Some(mutation) = mutation.as_deref() else {
            return Ok(());
        };

        match mutation {
            Mutation::Add(AddMutation { adds, .. }) => {
                if let Some(new_dispatchers) = adds.get(&self.actor_id) {
                    self.add_dispatchers(new_dispatchers)?;
                }
            }
            Mutation::Update(UpdateMutation {
                dispatchers,
                actor_new_dispatchers: actor_dispatchers,
                ..
            }) => {
                if let Some(new_dispatchers) = actor_dispatchers.get(&self.actor_id) {
                    self.add_dispatchers(new_dispatchers)?;
                }

                if let Some(updates) = dispatchers.get(&self.actor_id) {
                    for update in updates {
                        self.pre_update_dispatcher(update)?;
                    }
                }
            }
            Mutation::AddAndUpdate(
                AddMutation { adds, .. },
                UpdateMutation {
                    dispatchers,
                    actor_new_dispatchers: actor_dispatchers,
                    ..
                },
            ) => {
                if let Some(new_dispatchers) = adds.get(&self.actor_id) {
                    self.add_dispatchers(new_dispatchers)?;
                }

                if let Some(new_dispatchers) = actor_dispatchers.get(&self.actor_id) {
                    self.add_dispatchers(new_dispatchers)?;
                }

                if let Some(updates) = dispatchers.get(&self.actor_id) {
                    for update in updates {
                        self.pre_update_dispatcher(update)?;
                    }
                }
            }
            _ => {}
        }

        Ok(())
    }

    /// For `Stop` and `Update`, update the dispatchers after we dispatch the barrier.
    fn post_mutate_dispatchers(&mut self, mutation: &Option<Arc<Mutation>>) -> StreamResult<()> {
        let Some(mutation) = mutation.as_deref() else {
            return Ok(());
        };

        match mutation {
            Mutation::Stop(stops) => {
                // Remove outputs only if this actor itself is not to be stopped.
                if !stops.contains(&self.actor_id) {
                    for dispatcher in &mut self.dispatchers {
                        dispatcher.remove_outputs(stops);
                    }
                }
            }
            Mutation::Update(UpdateMutation {
                dispatchers,
                dropped_actors,
                ..
            })
            | Mutation::AddAndUpdate(
                _,
                UpdateMutation {
                    dispatchers,
                    dropped_actors,
                    ..
                },
            ) => {
                if let Some(updates) = dispatchers.get(&self.actor_id) {
                    for update in updates {
                        self.post_update_dispatcher(update)?;
                    }
                }

                if !dropped_actors.contains(&self.actor_id) {
                    for dispatcher in &mut self.dispatchers {
                        dispatcher.remove_outputs(dropped_actors);
                    }
                }
            }
            _ => {}
        };

        // After stopping the downstream mview, the outputs of some dispatcher might be empty and we
        // should clean up them.
        self.dispatchers.retain(|d| !d.is_empty());

        Ok(())
    }
}

impl DispatchExecutor {
    pub fn new(
        mut input: Executor,
        dispatchers: Vec<DispatcherImpl>,
        actor_id: u32,
        fragment_id: u32,
        context: Arc<SharedContext>,
        metrics: Arc<StreamingMetrics>,
        chunk_size: usize,
    ) -> Self {
        if crate::consistency::insane() {
            // make some trouble before dispatching to avoid generating invalid dist key.
            let mut info = input.info().clone();
            info.identity = format!("{} (embedded trouble)", info.identity);
            let troublemaker = TroublemakerExecutor::new(input, chunk_size);
            input = (info, troublemaker).into();
        }

        let actor_id_str = actor_id.to_string();
        let fragment_id_str = fragment_id.to_string();
        let actor_out_record_cnt = metrics
            .actor_out_record_cnt
            .with_guarded_label_values(&[&actor_id_str, &fragment_id_str]);
        let metrics = DispatchExecutorMetrics {
            actor_id_str,
            fragment_id_str,
            metrics,
            actor_out_record_cnt,
        };
        let dispatchers = dispatchers
            .into_iter()
            .map(|dispatcher| metrics.monitor_dispatcher(dispatcher))
            .collect();
        Self {
            input,
            inner: DispatchExecutorInner {
                dispatchers,
                actor_id,
                context,
                metrics,
            },
        }
    }
}

impl StreamConsumer for DispatchExecutor {
    type BarrierStream = impl Stream<Item = StreamResult<Barrier>> + Send;

    fn execute(mut self: Box<Self>) -> Self::BarrierStream {
        #[try_stream]
        async move {
            let input = self.input.execute();

            #[for_await]
            for msg in input {
                let msg: Message = msg?;
                let (barrier, span, tracing_span) = match msg {
                    Message::Chunk(_) => (
                        None,
                        "dispatch_chunk",
                        tracing::info_span!("dispatch_chunk"),
                    ),
                    Message::Barrier(ref barrier) => (
                        Some(barrier.clone()),
                        "dispatch_barrier",
                        tracing::info_span!("dispatch_barrier"),
                    ),
                    Message::Watermark(_) => (
                        None,
                        "dispatch_watermark",
                        tracing::info_span!("dispatch_watermark"),
                    ),
                };

                self.inner
                    .dispatch(msg)
                    .instrument(tracing_span)
                    .instrument_await(span)
                    .await?;
                if let Some(barrier) = barrier {
                    yield barrier;
                }
            }
        }
    }
}

#[derive(Debug)]
pub enum DispatcherImpl {
    Hash(HashDataDispatcher),
    Broadcast(BroadcastDispatcher),
    Simple(SimpleDispatcher),
    RoundRobin(RoundRobinDataDispatcher),
}

impl DispatcherImpl {
    pub fn new(
        context: &SharedContext,
        actor_id: ActorId,
        dispatcher: &PbDispatcher,
    ) -> StreamResult<Self> {
        let outputs = dispatcher
            .downstream_actor_id
            .iter()
            .map(|&down_id| new_output(context, actor_id, down_id))
            .collect::<StreamResult<Vec<_>>>()?;

        let output_indices = dispatcher
            .output_indices
            .iter()
            .map(|&i| i as usize)
            .collect_vec();

        use risingwave_pb::stream_plan::DispatcherType::*;
        let dispatcher_impl = match dispatcher.get_type()? {
            Hash => {
                assert!(!outputs.is_empty());
                let dist_key_indices = dispatcher
                    .dist_key_indices
                    .iter()
                    .map(|i| *i as usize)
                    .collect();

                let hash_mapping =
                    ActorMapping::from_protobuf(dispatcher.get_hash_mapping()?).to_expanded();

                DispatcherImpl::Hash(HashDataDispatcher::new(
                    outputs,
                    dist_key_indices,
                    output_indices,
                    hash_mapping,
                    dispatcher.dispatcher_id,
                ))
            }
            Broadcast => DispatcherImpl::Broadcast(BroadcastDispatcher::new(
                outputs,
                output_indices,
                dispatcher.dispatcher_id,
            )),
            Simple | NoShuffle => {
                let [output]: [_; 1] = outputs.try_into().unwrap();
                DispatcherImpl::Simple(SimpleDispatcher::new(
                    output,
                    output_indices,
                    dispatcher.dispatcher_id,
                ))
            }
            Unspecified => unreachable!(),
        };

        Ok(dispatcher_impl)
    }
}

macro_rules! impl_dispatcher {
    ($( { $variant_name:ident } ),*) => {
        impl DispatcherImpl {
            pub async fn dispatch_data(&mut self, chunk: StreamChunk) -> StreamResult<()> {
                match self {
                    $( Self::$variant_name(inner) => inner.dispatch_data(chunk).await, )*
                }
            }

            pub async fn dispatch_barrier(&mut self, barrier: DispatcherBarrier) -> StreamResult<()> {
                match self {
                    $( Self::$variant_name(inner) => inner.dispatch_barrier(barrier).await, )*
                }
            }

            pub async fn dispatch_watermark(&mut self, watermark: Watermark) -> StreamResult<()> {
                match self {
                    $( Self::$variant_name(inner) => inner.dispatch_watermark(watermark).await, )*
                }
            }

            pub fn add_outputs(&mut self, outputs: impl IntoIterator<Item = BoxedOutput>) {
                match self {
                    $(Self::$variant_name(inner) => inner.add_outputs(outputs), )*
                }
            }

            pub fn remove_outputs(&mut self, actor_ids: &HashSet<ActorId>) {
                match self {
                    $(Self::$variant_name(inner) => inner.remove_outputs(actor_ids), )*
                }
            }

            pub fn dispatcher_id(&self) -> DispatcherId {
                match self {
                    $(Self::$variant_name(inner) => inner.dispatcher_id(), )*
                }
            }

            pub fn dispatcher_id_str(&self) -> &str {
                match self {
                    $(Self::$variant_name(inner) => inner.dispatcher_id_str(), )*
                }
            }

            pub fn is_empty(&self) -> bool {
                match self {
                    $(Self::$variant_name(inner) => inner.is_empty(), )*
                }
            }
        }
    }
}

macro_rules! for_all_dispatcher_variants {
    ($macro:ident) => {
        $macro! {
            { Hash },
            { Broadcast },
            { Simple },
            { RoundRobin }
        }
    };
}

for_all_dispatcher_variants! { impl_dispatcher }

pub trait DispatchFuture<'a> = Future<Output = StreamResult<()>> + Send;

pub trait Dispatcher: Debug + 'static {
    /// Dispatch a data chunk to downstream actors.
    fn dispatch_data(&mut self, chunk: StreamChunk) -> impl DispatchFuture<'_>;
    /// Dispatch a barrier to downstream actors, generally by broadcasting it.
    fn dispatch_barrier(&mut self, barrier: DispatcherBarrier) -> impl DispatchFuture<'_>;
    /// Dispatch a watermark to downstream actors, generally by broadcasting it.
    fn dispatch_watermark(&mut self, watermark: Watermark) -> impl DispatchFuture<'_>;

    /// Add new outputs to the dispatcher.
    fn add_outputs(&mut self, outputs: impl IntoIterator<Item = BoxedOutput>);
    /// Remove outputs to `actor_ids` from the dispatcher.
    fn remove_outputs(&mut self, actor_ids: &HashSet<ActorId>);

    /// The ID of the dispatcher. A [`DispatchExecutor`] may have multiple dispatchers with
    /// different IDs.
    ///
    /// Note that the dispatcher id is always equal to the downstream fragment id.
    /// See also `proto/stream_plan.proto`.
    fn dispatcher_id(&self) -> DispatcherId;

    /// Dispatcher id in string. See [`Dispatcher::dispatcher_id`].
    fn dispatcher_id_str(&self) -> &str;

    /// Whether the dispatcher has no outputs. If so, it'll be cleaned up from the
    /// [`DispatchExecutor`].
    fn is_empty(&self) -> bool;
}

/// Concurrently broadcast a message to all outputs.
///
/// Note that this does not follow `concurrent_dispatchers` in the config and the concurrency is
/// always unlimited.
async fn broadcast_concurrent(
    outputs: impl IntoIterator<Item = &'_ mut BoxedOutput>,
    message: DispatcherMessage,
) -> StreamResult<()> {
    futures::future::try_join_all(
        outputs
            .into_iter()
            .map(|output| output.send(message.clone())),
    )
    .await?;
    Ok(())
}

#[derive(Debug)]
pub struct RoundRobinDataDispatcher {
    outputs: Vec<BoxedOutput>,
    output_indices: Vec<usize>,
    cur: usize,
    dispatcher_id: DispatcherId,
    dispatcher_id_str: String,
}

impl RoundRobinDataDispatcher {
    pub fn new(
        outputs: Vec<BoxedOutput>,
        output_indices: Vec<usize>,
        dispatcher_id: DispatcherId,
    ) -> Self {
        Self {
            outputs,
            output_indices,
            cur: 0,
            dispatcher_id,
            dispatcher_id_str: dispatcher_id.to_string(),
        }
    }
}

impl Dispatcher for RoundRobinDataDispatcher {
    async fn dispatch_data(&mut self, chunk: StreamChunk) -> StreamResult<()> {
        let chunk = if self.output_indices.len() < chunk.columns().len() {
            chunk
                .project(&self.output_indices)
                .eliminate_adjacent_noop_update()
        } else {
            chunk.project(&self.output_indices)
        };

        self.outputs[self.cur]
            .send(DispatcherMessage::Chunk(chunk))
            .await?;
        self.cur += 1;
        self.cur %= self.outputs.len();
        Ok(())
    }

    async fn dispatch_barrier(&mut self, barrier: DispatcherBarrier) -> StreamResult<()> {
        // always broadcast barrier
        broadcast_concurrent(&mut self.outputs, DispatcherMessage::Barrier(barrier)).await
    }

    async fn dispatch_watermark(&mut self, watermark: Watermark) -> StreamResult<()> {
        if let Some(watermark) = watermark.transform_with_indices(&self.output_indices) {
            // always broadcast watermark
            broadcast_concurrent(&mut self.outputs, DispatcherMessage::Watermark(watermark))
                .await?;
        }
        Ok(())
    }

    fn add_outputs(&mut self, outputs: impl IntoIterator<Item = BoxedOutput>) {
        self.outputs.extend(outputs);
    }

    fn remove_outputs(&mut self, actor_ids: &HashSet<ActorId>) {
        self.outputs
            .extract_if(|output| actor_ids.contains(&output.actor_id()))
            .count();
        self.cur = self.cur.min(self.outputs.len() - 1);
    }

    fn dispatcher_id(&self) -> DispatcherId {
        self.dispatcher_id
    }

    fn dispatcher_id_str(&self) -> &str {
        &self.dispatcher_id_str
    }

    fn is_empty(&self) -> bool {
        self.outputs.is_empty()
    }
}

pub struct HashDataDispatcher {
    outputs: Vec<BoxedOutput>,
    keys: Vec<usize>,
    output_indices: Vec<usize>,
    /// Mapping from virtual node to actor id, used for hash data dispatcher to dispatch tasks to
    /// different downstream actors.
    hash_mapping: ExpandedActorMapping,
    dispatcher_id: DispatcherId,
    dispatcher_id_str: String,
}

impl Debug for HashDataDispatcher {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("HashDataDispatcher")
            .field("outputs", &self.outputs)
            .field("keys", &self.keys)
            .field("dispatcher_id", &self.dispatcher_id)
            .finish_non_exhaustive()
    }
}

impl HashDataDispatcher {
    pub fn new(
        outputs: Vec<BoxedOutput>,
        keys: Vec<usize>,
        output_indices: Vec<usize>,
        hash_mapping: ExpandedActorMapping,
        dispatcher_id: DispatcherId,
    ) -> Self {
        Self {
            outputs,
            keys,
            output_indices,
            hash_mapping,
            dispatcher_id,
            dispatcher_id_str: dispatcher_id.to_string(),
        }
    }
}

impl Dispatcher for HashDataDispatcher {
    fn add_outputs(&mut self, outputs: impl IntoIterator<Item = BoxedOutput>) {
        self.outputs.extend(outputs);
    }

    async fn dispatch_barrier(&mut self, barrier: DispatcherBarrier) -> StreamResult<()> {
        // always broadcast barrier
        broadcast_concurrent(&mut self.outputs, DispatcherMessage::Barrier(barrier)).await
    }

    async fn dispatch_watermark(&mut self, watermark: Watermark) -> StreamResult<()> {
        if let Some(watermark) = watermark.transform_with_indices(&self.output_indices) {
            // always broadcast watermark
            broadcast_concurrent(&mut self.outputs, DispatcherMessage::Watermark(watermark))
                .await?;
        }
        Ok(())
    }

    async fn dispatch_data(&mut self, chunk: StreamChunk) -> StreamResult<()> {
        // A chunk can be shuffled into multiple output chunks that to be sent to downstreams.
        // In these output chunks, the only difference are visibility map, which is calculated
        // by the hash value of each line in the input chunk.
        let num_outputs = self.outputs.len();

        // get hash value of every line by its key
        let vnode_count = self.hash_mapping.len();
        let vnodes = VirtualNode::compute_chunk(chunk.data_chunk(), &self.keys, vnode_count);

        tracing::debug!(target: "events::stream::dispatch::hash", "\n{}\n keys {:?} => {:?}", chunk.to_pretty(), self.keys, vnodes);

        let mut vis_maps = repeat_with(|| BitmapBuilder::with_capacity(chunk.capacity()))
            .take(num_outputs)
            .collect_vec();
        let mut last_vnode_when_update_delete = None;
        let mut new_ops: Vec<Op> = Vec::with_capacity(chunk.capacity());

        // Apply output indices after calculating the vnode.
        let chunk = if self.output_indices.len() < chunk.columns().len() {
            chunk
                .project(&self.output_indices)
                .eliminate_adjacent_noop_update()
        } else {
            chunk.project(&self.output_indices)
        };

        for ((vnode, &op), visible) in vnodes
            .iter()
            .copied()
            .zip_eq_fast(chunk.ops())
            .zip_eq_fast(chunk.visibility().iter())
        {
            // Build visibility map for every output chunk.
            for (output, vis_map) in self.outputs.iter().zip_eq_fast(vis_maps.iter_mut()) {
                vis_map.append(visible && self.hash_mapping[vnode.to_index()] == output.actor_id());
            }

            if !visible {
                assert!(
                    last_vnode_when_update_delete.is_none(),
                    "invisible row between U- and U+, op = {op:?}",
                );
                new_ops.push(op);
                continue;
            }

            // The 'update' message, noted by an `UpdateDelete` and a successive `UpdateInsert`,
            // need to be rewritten to common `Delete` and `Insert` if they were dispatched to
            // different actors.
            if op == Op::UpdateDelete {
                last_vnode_when_update_delete = Some(vnode);
            } else if op == Op::UpdateInsert {
                if vnode
                    != last_vnode_when_update_delete
                        .take()
                        .expect("missing U- before U+")
                {
                    new_ops.push(Op::Delete);
                    new_ops.push(Op::Insert);
                } else {
                    new_ops.push(Op::UpdateDelete);
                    new_ops.push(Op::UpdateInsert);
                }
            } else {
                new_ops.push(op);
            }
        }
        assert!(
            last_vnode_when_update_delete.is_none(),
            "missing U+ after U-"
        );

        let ops = new_ops;

        // individually output StreamChunk integrated with vis_map
        futures::future::try_join_all(
            vis_maps
                .into_iter()
                .zip_eq_fast(self.outputs.iter_mut())
                .map(|(vis_map, output)| async {
                    let vis_map = vis_map.finish();
                    // columns is not changed in this function
                    let new_stream_chunk =
                        StreamChunk::with_visibility(ops.clone(), chunk.columns().into(), vis_map);
                    if new_stream_chunk.cardinality() > 0 {
                        event!(
                            tracing::Level::TRACE,
                            msg = "chunk",
                            downstream = output.actor_id(),
                            "send = \n{:#?}",
                            new_stream_chunk
                        );
                        output
                            .send(DispatcherMessage::Chunk(new_stream_chunk))
                            .await?;
                    }
                    StreamResult::Ok(())
                }),
        )
        .await?;

        Ok(())
    }

    fn remove_outputs(&mut self, actor_ids: &HashSet<ActorId>) {
        self.outputs
            .extract_if(|output| actor_ids.contains(&output.actor_id()))
            .count();
    }

    fn dispatcher_id(&self) -> DispatcherId {
        self.dispatcher_id
    }

    fn dispatcher_id_str(&self) -> &str {
        &self.dispatcher_id_str
    }

    fn is_empty(&self) -> bool {
        self.outputs.is_empty()
    }
}

/// `BroadcastDispatcher` dispatches message to all outputs.
#[derive(Debug)]
pub struct BroadcastDispatcher {
    outputs: HashMap<ActorId, BoxedOutput>,
    output_indices: Vec<usize>,
    dispatcher_id: DispatcherId,
    dispatcher_id_str: String,
}

impl BroadcastDispatcher {
    pub fn new(
        outputs: impl IntoIterator<Item = BoxedOutput>,
        output_indices: Vec<usize>,
        dispatcher_id: DispatcherId,
    ) -> Self {
        Self {
            outputs: Self::into_pairs(outputs).collect(),
            output_indices,
            dispatcher_id,
            dispatcher_id_str: dispatcher_id.to_string(),
        }
    }

    fn into_pairs(
        outputs: impl IntoIterator<Item = BoxedOutput>,
    ) -> impl Iterator<Item = (ActorId, BoxedOutput)> {
        outputs
            .into_iter()
            .map(|output| (output.actor_id(), output))
    }
}

impl Dispatcher for BroadcastDispatcher {
    async fn dispatch_data(&mut self, chunk: StreamChunk) -> StreamResult<()> {
        let chunk = if self.output_indices.len() < chunk.columns().len() {
            chunk
                .project(&self.output_indices)
                .eliminate_adjacent_noop_update()
        } else {
            chunk.project(&self.output_indices)
        };
        broadcast_concurrent(self.outputs.values_mut(), DispatcherMessage::Chunk(chunk)).await
    }

    async fn dispatch_barrier(&mut self, barrier: DispatcherBarrier) -> StreamResult<()> {
        // always broadcast barrier
        broadcast_concurrent(
            self.outputs.values_mut(),
            DispatcherMessage::Barrier(barrier),
        )
        .await
    }

    async fn dispatch_watermark(&mut self, watermark: Watermark) -> StreamResult<()> {
        if let Some(watermark) = watermark.transform_with_indices(&self.output_indices) {
            // always broadcast watermark
            broadcast_concurrent(
                self.outputs.values_mut(),
                DispatcherMessage::Watermark(watermark),
            )
            .await?;
        }
        Ok(())
    }

    fn add_outputs(&mut self, outputs: impl IntoIterator<Item = BoxedOutput>) {
        self.outputs.extend(Self::into_pairs(outputs));
    }

    fn remove_outputs(&mut self, actor_ids: &HashSet<ActorId>) {
        self.outputs
            .extract_if(|actor_id, _| actor_ids.contains(actor_id))
            .count();
    }

    fn dispatcher_id(&self) -> DispatcherId {
        self.dispatcher_id
    }

    fn dispatcher_id_str(&self) -> &str {
        &self.dispatcher_id_str
    }

    fn is_empty(&self) -> bool {
        self.outputs.is_empty()
    }
}

/// `SimpleDispatcher` dispatches message to a single output.
#[derive(Debug)]
pub struct SimpleDispatcher {
    /// In most cases, there is exactly one output. However, in some cases of configuration change,
    /// the field needs to be temporarily set to 0 or 2 outputs.
    ///
    /// - When dropping a materialized view, the output will be removed and this field becomes
    ///   empty. The [`DispatchExecutor`] will immediately clean-up this empty dispatcher before
    ///   finishing processing the current mutation.
    /// - When migrating a singleton fragment, the new output will be temporarily added in `pre`
    ///   stage and this field becomes multiple, which is for broadcasting this configuration
    ///   change barrier to both old and new downstream actors. In `post` stage, the old output
    ///   will be removed and this field becomes single again.
    ///
    /// Therefore, when dispatching data, we assert that there's exactly one output by
    /// `Self::output`.
    output: SmallVec<[BoxedOutput; 2]>,
    output_indices: Vec<usize>,
    dispatcher_id: DispatcherId,
    dispatcher_id_str: String,
}

impl SimpleDispatcher {
    pub fn new(
        output: BoxedOutput,
        output_indices: Vec<usize>,
        dispatcher_id: DispatcherId,
    ) -> Self {
        Self {
            output: smallvec![output],
            output_indices,
            dispatcher_id,
            dispatcher_id_str: dispatcher_id.to_string(),
        }
    }
}

impl Dispatcher for SimpleDispatcher {
    fn add_outputs(&mut self, outputs: impl IntoIterator<Item = BoxedOutput>) {
        self.output.extend(outputs);
        assert!(self.output.len() <= 2);
    }

    async fn dispatch_barrier(&mut self, barrier: DispatcherBarrier) -> StreamResult<()> {
        // Only barrier is allowed to be dispatched to multiple outputs during migration.
        for output in &mut self.output {
            output
                .send(DispatcherMessage::Barrier(barrier.clone()))
                .await?;
        }
        Ok(())
    }

    async fn dispatch_data(&mut self, chunk: StreamChunk) -> StreamResult<()> {
        let output = self
            .output
            .iter_mut()
            .exactly_one()
            .expect("expect exactly one output");

        let chunk = if self.output_indices.len() < chunk.columns().len() {
            chunk
                .project(&self.output_indices)
                .eliminate_adjacent_noop_update()
        } else {
            chunk.project(&self.output_indices)
        };
        output.send(DispatcherMessage::Chunk(chunk)).await
    }

    async fn dispatch_watermark(&mut self, watermark: Watermark) -> StreamResult<()> {
        let output = self
            .output
            .iter_mut()
            .exactly_one()
            .expect("expect exactly one output");

        if let Some(watermark) = watermark.transform_with_indices(&self.output_indices) {
            output.send(DispatcherMessage::Watermark(watermark)).await?;
        }
        Ok(())
    }

    fn remove_outputs(&mut self, actor_ids: &HashSet<ActorId>) {
        self.output
            .retain(|output| !actor_ids.contains(&output.actor_id()));
    }

    fn dispatcher_id(&self) -> DispatcherId {
        self.dispatcher_id
    }

    fn dispatcher_id_str(&self) -> &str {
        &self.dispatcher_id_str
    }

    fn is_empty(&self) -> bool {
        self.output.is_empty()
    }
}

#[cfg(test)]
mod tests {
    use std::hash::{BuildHasher, Hasher};
    use std::sync::Mutex;

    use async_trait::async_trait;
    use futures::pin_mut;
    use risingwave_common::array::stream_chunk::StreamChunkTestExt;
    use risingwave_common::array::{Array, ArrayBuilder, I32ArrayBuilder};
    use risingwave_common::config;
    use risingwave_common::util::epoch::test_epoch;
    use risingwave_common::util::hash_util::Crc32FastBuilder;
    use risingwave_pb::stream_plan::DispatcherType;

    use super::*;
    use crate::executor::exchange::output::Output;
    use crate::executor::exchange::permit::channel_for_test;
    use crate::executor::receiver::ReceiverExecutor;
    use crate::executor::{BarrierInner as Barrier, MessageInner as Message};
    use crate::task::barrier_test_utils::LocalBarrierTestEnv;
    use crate::task::test_utils::helper_make_local_actor;

    #[derive(Debug)]
    pub struct MockOutput {
        actor_id: ActorId,
        data: Arc<Mutex<Vec<DispatcherMessage>>>,
    }

    impl MockOutput {
        pub fn new(actor_id: ActorId, data: Arc<Mutex<Vec<DispatcherMessage>>>) -> Self {
            Self { actor_id, data }
        }
    }

    #[async_trait]
    impl Output for MockOutput {
        async fn send(&mut self, message: DispatcherMessage) -> StreamResult<()> {
            self.data.lock().unwrap().push(message);
            Ok(())
        }

        fn actor_id(&self) -> ActorId {
            self.actor_id
        }
    }

    // TODO: this test contains update being shuffled to different partitions, which is not
    // supported for now.
    #[tokio::test]
    async fn test_hash_dispatcher_complex() {
        test_hash_dispatcher_complex_inner().await
    }

    async fn test_hash_dispatcher_complex_inner() {
        // This test only works when vnode count is 256.
        assert_eq!(VirtualNode::COUNT_FOR_TEST, 256);

        let num_outputs = 2; // actor id ranges from 1 to 2
        let key_indices = &[0, 2];
        let output_data_vecs = (0..num_outputs)
            .map(|_| Arc::new(Mutex::new(Vec::new())))
            .collect::<Vec<_>>();
        let outputs = output_data_vecs
            .iter()
            .enumerate()
            .map(|(actor_id, data)| {
                Box::new(MockOutput::new(1 + actor_id as u32, data.clone())) as BoxedOutput
            })
            .collect::<Vec<_>>();
        let mut hash_mapping = (1..num_outputs + 1)
            .flat_map(|id| vec![id as ActorId; VirtualNode::COUNT_FOR_TEST / num_outputs])
            .collect_vec();
        hash_mapping.resize(VirtualNode::COUNT_FOR_TEST, num_outputs as u32);
        let mut hash_dispatcher = HashDataDispatcher::new(
            outputs,
            key_indices.to_vec(),
            vec![0, 1, 2],
            hash_mapping,
            0,
        );

        let chunk = StreamChunk::from_pretty(
            "  I I I
            +  4 6 8
            +  5 7 9
            +  0 0 0
            -  1 1 1 D
            U- 2 0 2
            U+ 2 0 2
            U- 3 3 2
            U+ 3 3 4",
        );
        hash_dispatcher.dispatch_data(chunk).await.unwrap();

        assert_eq!(
            *output_data_vecs[0].lock().unwrap()[0].as_chunk().unwrap(),
            StreamChunk::from_pretty(
                "  I I I
                +  4 6 8
                +  5 7 9
                +  0 0 0
                -  1 1 1 D
                U- 2 0 2
                U+ 2 0 2
                -  3 3 2 D  // Should rewrite UpdateDelete to Delete
                +  3 3 4    // Should rewrite UpdateInsert to Insert",
            )
        );
        assert_eq!(
            *output_data_vecs[1].lock().unwrap()[0].as_chunk().unwrap(),
            StreamChunk::from_pretty(
                "  I I I
                +  4 6 8 D
                +  5 7 9 D
                +  0 0 0 D
                -  1 1 1 D  // Should keep original invisible mark
                U- 2 0 2 D  // Should keep UpdateDelete
                U+ 2 0 2 D  // Should keep UpdateInsert
                -  3 3 2    // Should rewrite UpdateDelete to Delete
                +  3 3 4 D  // Should rewrite UpdateInsert to Insert",
            )
        );
    }

    #[tokio::test]
    async fn test_configuration_change() {
        let _schema = Schema { fields: vec![] };
        let (tx, rx) = channel_for_test();
        let actor_id = 233;
        let fragment_id = 666;
        let barrier_test_env = LocalBarrierTestEnv::for_test().await;
        let ctx = Arc::new(SharedContext::for_test());
        let metrics = Arc::new(StreamingMetrics::unused());

        let (untouched, old, new) = (234, 235, 238); // broadcast downstream actors
        let (old_simple, new_simple) = (114, 514); // simple downstream actors

        // 1. Register info in context.
        {
            let mut actor_infos = ctx.actor_infos.write();

            for local_actor_id in [actor_id, untouched, old, new, old_simple, new_simple] {
                actor_infos.insert(local_actor_id, helper_make_local_actor(local_actor_id));
            }
        }
        // actor_id -> untouched, old, new, old_simple, new_simple

        let broadcast_dispatcher_id = 666;
        let broadcast_dispatcher = DispatcherImpl::new(
            &ctx,
            actor_id,
            &PbDispatcher {
                r#type: DispatcherType::Broadcast as _,
                dispatcher_id: broadcast_dispatcher_id,
                downstream_actor_id: vec![untouched, old],
                ..Default::default()
            },
        )
        .unwrap();

        let simple_dispatcher_id = 888;
        let simple_dispatcher = DispatcherImpl::new(
            &ctx,
            actor_id,
            &PbDispatcher {
                r#type: DispatcherType::Simple as _,
                dispatcher_id: simple_dispatcher_id,
                downstream_actor_id: vec![old_simple],
                ..Default::default()
            },
        )
        .unwrap();

        let dispatcher_updates = maplit::hashmap! {
            actor_id => vec![PbDispatcherUpdate {
                actor_id,
                dispatcher_id: broadcast_dispatcher_id,
                added_downstream_actor_id: vec![new],
                removed_downstream_actor_id: vec![old],
                hash_mapping: Default::default(),
            }]
        };
        let b1 = Barrier::new_test_barrier(test_epoch(1)).with_mutation(Mutation::Update(
            UpdateMutation {
                dispatchers: dispatcher_updates,
                merges: Default::default(),
                vnode_bitmaps: Default::default(),
                dropped_actors: Default::default(),
                actor_splits: Default::default(),
                actor_new_dispatchers: Default::default(),
            },
        ));
        barrier_test_env.inject_barrier(&b1, [actor_id]);
        barrier_test_env
            .shared_context
            .local_barrier_manager
            .flush_all_events()
            .await;

        let input = Executor::new(
            Default::default(),
            ReceiverExecutor::for_test(actor_id, rx, barrier_test_env.shared_context.clone())
                .boxed(),
        );
        let executor = Box::new(DispatchExecutor::new(
            input,
            vec![broadcast_dispatcher, simple_dispatcher],
            actor_id,
            fragment_id,
            ctx.clone(),
            metrics,
            config::default::developer::stream_chunk_size(),
        ))
        .execute();
        pin_mut!(executor);

        // 2. Take downstream receivers.
        let mut rxs = [untouched, old, new, old_simple, new_simple]
            .into_iter()
            .map(|id| (id, ctx.take_receiver((actor_id, id)).unwrap()))
            .collect::<HashMap<_, _>>();
        macro_rules! try_recv {
            ($down_id:expr) => {
                rxs.get_mut(&$down_id).unwrap().try_recv()
            };
        }

        // 3. Send a chunk.
        tx.send(Message::Chunk(StreamChunk::default()))
            .await
            .unwrap();

        tx.send(Message::Barrier(b1.clone().into_dispatcher()))
            .await
            .unwrap();
        executor.next().await.unwrap().unwrap();

        // 5. Check downstream.
        try_recv!(untouched).unwrap().as_chunk().unwrap();
        try_recv!(untouched).unwrap().as_barrier().unwrap();

        try_recv!(old).unwrap().as_chunk().unwrap();
        try_recv!(old).unwrap().as_barrier().unwrap(); // It should still receive the barrier even if it's to be removed.

        try_recv!(new).unwrap().as_barrier().unwrap(); // Since it's just added, it won't receive the chunk.

        try_recv!(old_simple).unwrap().as_chunk().unwrap();
        try_recv!(old_simple).unwrap().as_barrier().unwrap(); // Untouched.

        // 6. Send another barrier.
        let b2 = Barrier::new_test_barrier(test_epoch(2));
        barrier_test_env.inject_barrier(&b2, [actor_id]);
        tx.send(Message::Barrier(b2.into_dispatcher()))
            .await
            .unwrap();
        executor.next().await.unwrap().unwrap();

        // 7. Check downstream.
        try_recv!(untouched).unwrap().as_barrier().unwrap();
        try_recv!(old).unwrap_err(); // Since it's stopped, we can't receive the new messages.
        try_recv!(new).unwrap().as_barrier().unwrap();

        try_recv!(old_simple).unwrap().as_barrier().unwrap(); // Untouched.
        try_recv!(new_simple).unwrap_err(); // Untouched.

        // 8. Send another chunk.
        tx.send(Message::Chunk(StreamChunk::default()))
            .await
            .unwrap();

        // 9. Send a configuration change barrier for simple dispatcher.
        let dispatcher_updates = maplit::hashmap! {
            actor_id => vec![PbDispatcherUpdate {
                actor_id,
                dispatcher_id: simple_dispatcher_id,
                added_downstream_actor_id: vec![new_simple],
                removed_downstream_actor_id: vec![old_simple],
                hash_mapping: Default::default(),
            }]
        };
        let b3 = Barrier::new_test_barrier(test_epoch(3)).with_mutation(Mutation::Update(
            UpdateMutation {
                dispatchers: dispatcher_updates,
                merges: Default::default(),
                vnode_bitmaps: Default::default(),
                dropped_actors: Default::default(),
                actor_splits: Default::default(),
                actor_new_dispatchers: Default::default(),
            },
        ));
        barrier_test_env.inject_barrier(&b3, [actor_id]);
        tx.send(Message::Barrier(b3.into_dispatcher()))
            .await
            .unwrap();
        executor.next().await.unwrap().unwrap();

        // 10. Check downstream.
        try_recv!(old_simple).unwrap().as_chunk().unwrap();
        try_recv!(old_simple).unwrap().as_barrier().unwrap(); // It should still receive the barrier even if it's to be removed.

        try_recv!(new_simple).unwrap().as_barrier().unwrap(); // Since it's just added, it won't receive the chunk.

        // 11. Send another barrier.
        let b4 = Barrier::new_test_barrier(test_epoch(4));
        barrier_test_env.inject_barrier(&b4, [actor_id]);
        tx.send(Message::Barrier(b4.into_dispatcher()))
            .await
            .unwrap();
        executor.next().await.unwrap().unwrap();

        // 12. Check downstream.
        try_recv!(old_simple).unwrap_err(); // Since it's stopped, we can't receive the new messages.
        try_recv!(new_simple).unwrap().as_barrier().unwrap();
    }

    #[tokio::test]
    async fn test_hash_dispatcher() {
        // This test only works when vnode count is 256.
        assert_eq!(VirtualNode::COUNT_FOR_TEST, 256);

        let num_outputs = 5; // actor id ranges from 1 to 5
        let cardinality = 10;
        let dimension = 4;
        let key_indices = &[0, 2];
        let output_data_vecs = (0..num_outputs)
            .map(|_| Arc::new(Mutex::new(Vec::new())))
            .collect::<Vec<_>>();
        let outputs = output_data_vecs
            .iter()
            .enumerate()
            .map(|(actor_id, data)| {
                Box::new(MockOutput::new(1 + actor_id as u32, data.clone())) as BoxedOutput
            })
            .collect::<Vec<_>>();
        let mut hash_mapping = (1..num_outputs + 1)
            .flat_map(|id| vec![id as ActorId; VirtualNode::COUNT_FOR_TEST / num_outputs])
            .collect_vec();
        hash_mapping.resize(VirtualNode::COUNT_FOR_TEST, num_outputs as u32);
        let mut hash_dispatcher = HashDataDispatcher::new(
            outputs,
            key_indices.to_vec(),
            (0..dimension).collect(),
            hash_mapping.clone(),
            0,
        );

        let mut ops = Vec::new();
        for idx in 0..cardinality {
            if idx % 2 == 0 {
                ops.push(Op::Insert);
            } else {
                ops.push(Op::Delete);
            }
        }

        let mut start = 19260817i32..;
        let mut builders = (0..dimension)
            .map(|_| I32ArrayBuilder::new(cardinality))
            .collect_vec();
        let mut output_cols = vec![vec![vec![]; dimension]; num_outputs];
        let mut output_ops = vec![vec![]; num_outputs];
        for op in &ops {
            let hash_builder = Crc32FastBuilder;
            let mut hasher = hash_builder.build_hasher();
            let one_row = (0..dimension).map(|_| start.next().unwrap()).collect_vec();
            for key_idx in key_indices {
                let val = one_row[*key_idx];
                let bytes = val.to_le_bytes();
                hasher.update(&bytes);
            }
            let output_idx =
                hash_mapping[hasher.finish() as usize % VirtualNode::COUNT_FOR_TEST] as usize - 1;
            for (builder, val) in builders.iter_mut().zip_eq_fast(one_row.iter()) {
                builder.append(Some(*val));
            }
            output_cols[output_idx]
                .iter_mut()
                .zip_eq_fast(one_row.iter())
                .for_each(|(each_column, val)| each_column.push(*val));
            output_ops[output_idx].push(op);
        }

        let columns = builders
            .into_iter()
            .map(|builder| {
                let array = builder.finish();
                array.into_ref()
            })
            .collect();

        let chunk = StreamChunk::new(ops, columns);
        hash_dispatcher.dispatch_data(chunk).await.unwrap();

        for (output_idx, output) in output_data_vecs.into_iter().enumerate() {
            let guard = output.lock().unwrap();
            // It is possible that there is no chunks, as a key doesn't belong to any hash bucket.
            assert!(guard.len() <= 1);
            if guard.is_empty() {
                assert!(output_cols[output_idx].iter().all(|x| { x.is_empty() }));
            } else {
                let message = guard.first().unwrap();
                let real_chunk = match message {
                    DispatcherMessage::Chunk(chunk) => chunk,
                    _ => panic!(),
                };
                real_chunk
                    .columns()
                    .iter()
                    .zip_eq_fast(output_cols[output_idx].iter())
                    .for_each(|(real_col, expect_col)| {
                        let real_vals = real_chunk
                            .visibility()
                            .iter_ones()
                            .map(|row_idx| real_col.as_int32().value_at(row_idx).unwrap())
                            .collect::<Vec<_>>();
                        assert_eq!(real_vals.len(), expect_col.len());
                        assert_eq!(real_vals, *expect_col);
                    });
            }
        }
    }
}