@@ -16,18 +16,14 @@ use std::collections::VecDeque;
16
16
use std:: pin:: Pin ;
17
17
use std:: task:: { Context , Poll } ;
18
18
19
- use anyhow:: Context as _;
20
- use futures:: future:: try_join_all;
21
19
use risingwave_common:: array:: StreamChunkBuilder ;
22
20
use risingwave_common:: config:: MetricLevel ;
23
21
use tokio:: sync:: mpsc;
24
22
use tokio:: time:: Instant ;
25
23
26
24
use super :: exchange:: input:: BoxedActorInput ;
27
25
use super :: * ;
28
- use crate :: executor:: exchange:: input:: {
29
- assert_equal_dispatcher_barrier, new_input, process_dispatcher_msg,
30
- } ;
26
+ use crate :: executor:: exchange:: input:: { apply_dispatcher_barrier, assert_equal_dispatcher_barrier} ;
31
27
use crate :: executor:: prelude:: * ;
32
28
use crate :: task:: LocalBarrierManager ;
33
29
@@ -242,31 +238,43 @@ impl MergeExecutor {
242
238
// Channels that're blocked by the barrier to align.
243
239
let mut start_time = Instant :: now ( ) ;
244
240
pin_mut ! ( select_all) ;
245
- while let Some ( msg) = select_all. next ( ) . await {
241
+
242
+ let mut barrier_buffer = DispatchBarrierBuffer {
243
+ buffer : VecDeque :: new ( ) ,
244
+ curr_fragment_id : self . upstream_fragment_id ,
245
+ barrier_rx : self . barrier_rx ,
246
+ actor_id,
247
+ local_barrier_manager : self . local_barrier_manager ,
248
+ metrics : self . metrics . clone ( ) ,
249
+ fragment_id : self . fragment_id ,
250
+ } ;
251
+
252
+ loop {
253
+ let msg = barrier_buffer. select_once ( & mut select_all) . await ?;
246
254
metrics
247
255
. actor_input_buffer_blocking_duration_ns
248
256
. inc_by ( start_time. elapsed ( ) . as_nanos ( ) as u64 ) ;
249
- let msg: DispatcherMessage = msg?;
250
- let mut msg: Message = process_dispatcher_msg ( msg, & mut self . barrier_rx ) . await ?;
251
257
252
- match & mut msg {
253
- Message :: Watermark ( _) => {
254
- // Do nothing.
255
- }
256
- Message :: Chunk ( chunk) => {
258
+ let msg = match msg {
259
+ DispatcherMessage :: Watermark ( watermark) => Message :: Watermark ( watermark) ,
260
+ DispatcherMessage :: Chunk ( chunk) => {
257
261
metrics. actor_in_record_cnt . inc_by ( chunk. cardinality ( ) as _ ) ;
262
+ Message :: Chunk ( chunk)
258
263
}
259
- Message :: Barrier ( barrier) => {
264
+ DispatcherMessage :: Barrier ( barrier) => {
260
265
tracing:: debug!(
261
266
target: "events::stream::barrier::path" ,
262
267
actor_id = actor_id,
263
268
"receiver receives barrier from path: {:?}" ,
264
269
barrier. passed_actors
265
270
) ;
266
- barrier. passed_actors . push ( actor_id) ;
271
+ let ( mut recv_barrier, new_inputs) =
272
+ barrier_buffer. pop_barrier_with_inputs ( ) . await ?;
273
+ apply_dispatcher_barrier ( & mut recv_barrier, barrier) ;
274
+ recv_barrier. passed_actors . push ( actor_id) ;
267
275
268
276
if let Some ( Mutation :: Update ( UpdateMutation { dispatchers, .. } ) ) =
269
- barrier . mutation . as_deref ( )
277
+ recv_barrier . mutation . as_deref ( )
270
278
&& select_all
271
279
. upstream_input_ids ( )
272
280
. any ( |actor_id| dispatchers. contains_key ( & actor_id) )
@@ -275,8 +283,8 @@ impl MergeExecutor {
275
283
select_all. flush_buffered_watermarks ( ) ;
276
284
}
277
285
278
- if let Some ( update) =
279
- barrier . as_update_merge ( self . actor_context . id , self . upstream_fragment_id )
286
+ if let Some ( update) = recv_barrier
287
+ . as_update_merge ( self . actor_context . id , self . upstream_fragment_id )
280
288
{
281
289
let new_upstream_fragment_id = update
282
290
. new_upstream_fragment_id
@@ -291,28 +299,12 @@ impl MergeExecutor {
291
299
// `Watermark` of upstream may become stale after upstream scaling.
292
300
select_all. flush_buffered_watermarks ( ) ;
293
301
294
- if !update. added_upstream_actors . is_empty ( ) {
295
- // Create new upstreams receivers.
296
- let mut new_upstreams: Vec < _ > = try_join_all (
297
- update. added_upstream_actors . iter ( ) . map ( |upstream_actor| {
298
- new_input (
299
- & self . local_barrier_manager ,
300
- self . metrics . clone ( ) ,
301
- self . actor_context . id ,
302
- self . fragment_id ,
303
- upstream_actor,
304
- new_upstream_fragment_id,
305
- )
306
- } ) ,
307
- )
308
- . await
309
- . context ( "failed to create upstream receivers" ) ?;
310
-
302
+ if let Some ( mut new_upstreams) = new_inputs {
311
303
// Poll the first barrier from the new upstreams. It must be the same as
312
304
// the one we polled from original upstreams.
313
305
for upstream in & mut new_upstreams {
314
306
let new_barrier = expect_first_barrier ( upstream) . await ?;
315
- assert_equal_dispatcher_barrier ( barrier , & new_barrier) ;
307
+ assert_equal_dispatcher_barrier ( & recv_barrier , & new_barrier) ;
316
308
}
317
309
318
310
// Add the new upstreams to select.
@@ -332,12 +324,16 @@ impl MergeExecutor {
332
324
) ;
333
325
}
334
326
335
- if barrier. is_stop ( actor_id) {
327
+ let is_stop = recv_barrier. is_stop ( actor_id) ;
328
+ let msg = Message :: Barrier ( recv_barrier) ;
329
+ if is_stop {
336
330
yield msg;
337
331
break ;
338
332
}
333
+
334
+ msg
339
335
}
340
- }
336
+ } ;
341
337
342
338
yield msg;
343
339
start_time = Instant :: now ( ) ;
0 commit comments