risingwave_stream/executor/
receiver.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
// 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 anyhow::Context;
use itertools::Itertools;
use tokio::sync::mpsc;
use tokio::time::Instant;

use super::exchange::input::BoxedInput;
use crate::executor::exchange::input::{
    assert_equal_dispatcher_barrier, new_input, process_dispatcher_msg,
};
use crate::executor::prelude::*;
use crate::executor::DispatcherMessage;
use crate::task::{FragmentId, SharedContext};

/// `ReceiverExecutor` is used along with a channel. After creating a mpsc channel,
/// there should be a `ReceiverExecutor` running in the background, so as to push
/// messages down to the executors.
pub struct ReceiverExecutor {
    /// Input from upstream.
    input: BoxedInput,

    /// The context of the actor.
    actor_context: ActorContextRef,

    /// Belonged fragment id.
    fragment_id: FragmentId,

    /// Upstream fragment id.
    upstream_fragment_id: FragmentId,

    /// Shared context of the stream manager.
    context: Arc<SharedContext>,

    /// Metrics
    metrics: Arc<StreamingMetrics>,

    barrier_rx: mpsc::UnboundedReceiver<Barrier>,
}

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

impl ReceiverExecutor {
    #[allow(clippy::too_many_arguments)]
    pub fn new(
        ctx: ActorContextRef,
        fragment_id: FragmentId,
        upstream_fragment_id: FragmentId,
        input: BoxedInput,
        context: Arc<SharedContext>,
        metrics: Arc<StreamingMetrics>,
        barrier_rx: mpsc::UnboundedReceiver<Barrier>,
    ) -> Self {
        Self {
            input,
            actor_context: ctx,
            upstream_fragment_id,
            metrics,
            fragment_id,
            context,
            barrier_rx,
        }
    }

    #[cfg(test)]
    pub fn for_test(
        actor_id: ActorId,
        input: super::exchange::permit::Receiver,
        shared_context: Arc<SharedContext>,
    ) -> Self {
        use super::exchange::input::LocalInput;
        use crate::executor::exchange::input::Input;

        let barrier_rx = shared_context
            .local_barrier_manager
            .subscribe_barrier(actor_id);

        Self::new(
            ActorContext::for_test(actor_id),
            514,
            1919,
            LocalInput::new(input, 0).boxed_input(),
            shared_context,
            StreamingMetrics::unused().into(),
            barrier_rx,
        )
    }
}

impl Execute for ReceiverExecutor {
    fn execute(mut self: Box<Self>) -> BoxedMessageStream {
        let actor_id = self.actor_context.id;

        let mut metrics = self.metrics.new_actor_input_metrics(
            actor_id,
            self.fragment_id,
            self.upstream_fragment_id,
        );

        let stream = #[try_stream]
        async move {
            let mut start_time = Instant::now();
            while let Some(msg) = self.input.next().await {
                metrics
                    .actor_input_buffer_blocking_duration_ns
                    .inc_by(start_time.elapsed().as_nanos() as u64);
                let msg: DispatcherMessage = msg?;
                let mut msg = process_dispatcher_msg(msg, &mut self.barrier_rx).await?;

                match &mut msg {
                    Message::Watermark(_) => {
                        // Do nothing.
                    }
                    Message::Chunk(chunk) => {
                        metrics.actor_in_record_cnt.inc_by(chunk.cardinality() as _);
                    }
                    Message::Barrier(barrier) => {
                        tracing::debug!(
                            target: "events::stream::barrier::path",
                            actor_id = actor_id,
                            "receiver receives barrier from path: {:?}",
                            barrier.passed_actors
                        );
                        barrier.passed_actors.push(actor_id);

                        if let Some(update) = barrier
                            .as_update_merge(self.actor_context.id, self.upstream_fragment_id)
                        {
                            let new_upstream_fragment_id = update
                                .new_upstream_fragment_id
                                .unwrap_or(self.upstream_fragment_id);
                            let added_upstream_actor_id = update.added_upstream_actor_id.clone();
                            let removed_upstream_actor_id: Vec<_> =
                                if update.new_upstream_fragment_id.is_some() {
                                    vec![self.input.actor_id()]
                                } else {
                                    update.removed_upstream_actor_id.clone()
                                };

                            assert_eq!(
                                removed_upstream_actor_id,
                                vec![self.input.actor_id()],
                                "the removed upstream actor should be the same as the current input"
                            );
                            let upstream_actor_id = *added_upstream_actor_id
                                .iter()
                                .exactly_one()
                                .expect("receiver should have exactly one upstream");

                            // Create new upstream receiver.
                            let mut new_upstream = new_input(
                                &self.context,
                                self.metrics.clone(),
                                self.actor_context.id,
                                self.fragment_id,
                                upstream_actor_id,
                                new_upstream_fragment_id,
                            )
                            .context("failed to create upstream input")?;

                            // Poll the first barrier from the new upstream. It must be the same as
                            // the one we polled from original upstream.
                            let new_barrier = expect_first_barrier(&mut new_upstream).await?;
                            assert_equal_dispatcher_barrier(barrier, &new_barrier);

                            // Replace the input.
                            self.input = new_upstream;

                            self.upstream_fragment_id = new_upstream_fragment_id;
                            metrics = self.metrics.new_actor_input_metrics(
                                actor_id,
                                self.fragment_id,
                                self.upstream_fragment_id,
                            );
                        }
                    }
                };

                yield msg;
                start_time = Instant::now();
            }
        };

        stream.boxed()
    }
}

#[cfg(test)]
mod tests {
    use std::collections::HashMap;

    use futures::{pin_mut, FutureExt};
    use risingwave_common::util::epoch::test_epoch;
    use risingwave_pb::stream_plan::update_mutation::MergeUpdate;

    use super::*;
    use crate::executor::{MessageInner as Message, UpdateMutation};
    use crate::task::barrier_test_utils::LocalBarrierTestEnv;
    use crate::task::test_utils::helper_make_local_actor;

    #[tokio::test]
    async fn test_configuration_change() {
        let actor_id = 233;
        let (old, new) = (114, 514); // old and new upstream actor id

        let barrier_test_env = LocalBarrierTestEnv::for_test().await;

        let ctx = barrier_test_env.shared_context.clone();
        let metrics = Arc::new(StreamingMetrics::unused());

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

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

        let (upstream_fragment_id, fragment_id) = (10, 18);

        // 4. Send a configuration change barrier.
        let merge_updates = maplit::hashmap! {
            (actor_id, upstream_fragment_id) => MergeUpdate {
                actor_id,
                upstream_fragment_id,
                new_upstream_fragment_id: None,
                added_upstream_actor_id: vec![new],
                removed_upstream_actor_id: vec![old],
            }
        };

        let b1 = Barrier::new_test_barrier(test_epoch(1)).with_mutation(Mutation::Update(
            UpdateMutation {
                dispatchers: Default::default(),
                merges: merge_updates,
                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 = new_input(
            &ctx,
            metrics.clone(),
            actor_id,
            fragment_id,
            old,
            upstream_fragment_id,
        )
        .unwrap();

        let receiver = ReceiverExecutor::new(
            ActorContext::for_test(actor_id),
            fragment_id,
            upstream_fragment_id,
            input,
            ctx.clone(),
            metrics.clone(),
            ctx.local_barrier_manager.subscribe_barrier(actor_id),
        )
        .boxed()
        .execute();

        pin_mut!(receiver);

        // 2. Take downstream receivers.
        let txs = [old, new]
            .into_iter()
            .map(|id| (id, ctx.take_sender(&(id, actor_id)).unwrap()))
            .collect::<HashMap<_, _>>();
        macro_rules! send {
            ($actors:expr, $msg:expr) => {
                for actor in $actors {
                    txs.get(&actor).unwrap().send($msg).await.unwrap();
                }
            };
        }
        macro_rules! send_error {
            ($actors:expr, $msg:expr) => {
                for actor in $actors {
                    txs.get(&actor).unwrap().send($msg).await.unwrap_err();
                }
            };
        }
        macro_rules! assert_recv_pending {
            () => {
                assert!(receiver
                    .next()
                    .now_or_never()
                    .flatten()
                    .transpose()
                    .unwrap()
                    .is_none());
            };
        }

        macro_rules! recv {
            () => {
                receiver.next().await.transpose().unwrap()
            };
        }

        // 3. Send a chunk.
        send!([old], Message::Chunk(StreamChunk::default()));
        recv!().unwrap().as_chunk().unwrap(); // We should be able to receive the chunk.
        assert_recv_pending!();

        send!([new], Message::Barrier(b1.clone().into_dispatcher()));
        assert_recv_pending!(); // We should not receive the barrier, as new is not the upstream.

        send!([old], Message::Barrier(b1.clone().into_dispatcher()));
        recv!().unwrap().as_barrier().unwrap(); // We should now receive the barrier.

        // 5. Send a chunk to the removed upstream.
        send_error!([old], Message::Chunk(StreamChunk::default()));
        assert_recv_pending!();

        // 6. Send a chunk to the added upstream.
        send!([new], Message::Chunk(StreamChunk::default()));
        recv!().unwrap().as_chunk().unwrap(); // We should be able to receive the chunk.
        assert_recv_pending!();
    }
}