arcon 0.2.1

A runtime for writing streaming applications
Documentation
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
/// Common code between node types
pub mod common;
/// Debug version of [Node]
pub mod debug;
/// SourceNode components that drives the execution of sources
pub mod source;
/// Event-time timer implementation
pub(crate) mod timer;

#[cfg(feature = "metrics")]
use metrics::{
    gauge, histogram, increment_counter, register_counter, register_gauge, register_histogram,
};

#[cfg(all(feature = "hardware_counters", target_os = "linux", not(test)))]
use perf_event::{Builder, Group};

use crate::application::conf::logger::ArconLogger;
use crate::{
    data::{flight_serde::reliable_remote::ReliableSerde, RawArconMessage, *},
    dataflow::builder::KeyBuilder,
    dataflow::dfg::GlobalNodeId,
    error::{ArconResult, *},
    index::{AppenderIndex, ArconState, EagerAppender, IndexOps},
    manager::epoch::EpochEvent,
    manager::node::*,
    manager::snapshot::{Snapshot, SnapshotEvent},
    reportable_error,
    stream::{
        channel::strategy::ChannelStrategy,
        operator::{Operator, OperatorContext},
    },
};
use arcon_macros::ArconState;
use arcon_state::Backend;
use fxhash::*;
use kompact::prelude::*;
use std::{
    cell::{RefCell, UnsafeCell},
    sync::Arc,
};

use self::timer::Timer;

/// Type alias for a Node description
pub type NodeDescriptor = String;

#[cfg(all(feature = "hardware_counters", target_os = "linux", not(test)))]
use crate::metrics::perf_event::PerfEvents;

#[cfg(feature = "metrics")]
use crate::metrics::runtime_metrics::NodeMetrics;

#[cfg(feature = "metrics")]
use std::time::Instant;

#[derive(ArconState)]
pub struct NodeState<OP: Operator + 'static, B: Backend> {
    /// Durable message buffer used for blocked channels
    message_buffer: EagerAppender<RawArconMessage<OP::IN>, B>,
    /// Map of senders and their corresponding Watermark
    #[ephemeral]
    watermarks: FxHashMap<NodeID, Watermark>,
    /// Map of blocked senders
    #[ephemeral]
    blocked_channels: FxHashSet<NodeID>,
    /// Current Watermark value for the Node
    #[ephemeral]
    current_watermark: Watermark,
    /// Current Epoch value for the Node
    #[ephemeral]
    current_epoch: Epoch,
    /// Vector of expected senders
    ///
    /// Used to validate message and separate channels
    #[ephemeral]
    in_channels: Vec<NodeID>,
    /// Identifier for the Node
    #[ephemeral]
    id: NodeID,
}

impl<OP: Operator + 'static, B: Backend> NodeState<OP, B> {
    pub fn new(id: NodeID, in_channels: Vec<NodeID>, backend: Arc<B>) -> Self {
        let message_buffer = EagerAppender::new("_messagebuffer", backend);

        // initialise watermarks
        let mut watermarks: FxHashMap<NodeID, Watermark> = FxHashMap::default();
        for sender in &in_channels {
            watermarks.insert(*sender, Watermark::new(0));
        }

        Self {
            message_buffer,
            watermarks,
            blocked_channels: FxHashSet::default(),
            current_watermark: Watermark::new(0),
            current_epoch: Epoch::new(0),
            in_channels,
            id,
        }
    }
}

/// A Node is a [kompact] component that drives the execution of streaming operators
#[derive(ComponentDefinition)]
pub struct Node<OP, B>
where
    OP: Operator + 'static,
    B: Backend,
{
    /// Component context
    ctx: ComponentContext<Self>,
    /// Port for NodeManager
    pub(crate) node_manager_port: RequiredPort<NodeManagerPort>,
    /// Node descriptor
    descriptor: NodeDescriptor,
    /// Channel Strategy used by the Node
    channel_strategy: UnsafeCell<ChannelStrategy<OP::OUT>>,
    /// User-defined Operator
    operator: OP,
    /// Context for the Operator of this Node
    operator_context: RefCell<OperatorContext<OP::TimerState, OP::OperatorState>>,
    /// Internal Node State
    node_state: NodeState<OP, B>,
    /// Node's backing state backend
    backend: Arc<B>,
    /// Reference to logger
    logger: ArconLogger,
    /// Actor Reference to the EpochManager
    epoch_manager: ActorRefStrong<EpochEvent>,
    #[cfg(all(feature = "hardware_counters", target_os = "linux", not(test)))]
    /// Configured hardware counters
    perf_events: PerfEvents,
    #[cfg(feature = "metrics")]
    /// Struct holding metrics information
    node_metrics: NodeMetrics,
    pub node_id: GlobalNodeId,
    in_key_builder: Option<KeyBuilder<OP::IN>>,
}

impl<OP, B> Node<OP, B>
where
    OP: Operator + 'static,
    B: Backend,
{
    /// Creates a new Node
    #[allow(clippy::too_many_arguments)]
    pub(crate) fn new(
        descriptor: NodeDescriptor,
        channel_strategy: ChannelStrategy<OP::OUT>,
        operator: OP,
        operator_state: OP::OperatorState,
        node_state: NodeState<OP, B>,
        backend: Arc<B>,
        logger: ArconLogger,
        epoch_manager: ActorRefStrong<EpochEvent>,
        #[cfg(all(feature = "hardware_counters", target_os = "linux"))]
        #[cfg(not(test))]
        perf_events: PerfEvents,
        node_id: GlobalNodeId,
        in_key_builder: Option<KeyBuilder<OP::IN>>,
    ) -> Self {
        let timer_id = format!("_{}_timer", descriptor);
        let timer = Timer::new(timer_id, backend.clone());

        let operator_context = OperatorContext::new(
            Box::new(timer),
            operator_state,
            logger.clone(),
            #[cfg(feature = "metrics")]
            descriptor.clone(),
        );

        #[cfg(feature = "metrics")]
        {
            register_gauge!("inbound_throughput", "node" => descriptor.clone());
            register_gauge!("last_watermark_timestamp", "node" => descriptor.clone());
            register_counter!("epoch_counter", "node" => descriptor.clone());
            register_counter!("watermark_counter", "node" => descriptor.clone());
            register_histogram!("batch_execution_time","execution time per events batch","node" => descriptor.clone());
        }

        #[cfg(all(feature = "hardware_counters", target_os = "linux", not(test)))]
        {
            for value in perf_events.counters.iter() {
                register_histogram!(value.to_string(),"node" => descriptor.clone());
            }
        }

        Node {
            ctx: ComponentContext::uninitialised(),
            node_manager_port: RequiredPort::uninitialised(),
            descriptor,
            channel_strategy: UnsafeCell::new(channel_strategy),
            operator,
            operator_context: RefCell::new(operator_context),
            node_state,
            backend,
            logger,
            epoch_manager,
            #[cfg(all(feature = "hardware_counters", target_os = "linux", not(test)))]
            perf_events,
            #[cfg(feature = "metrics")]
            node_metrics: NodeMetrics::new(),
            node_id,
            in_key_builder,
        }
    }

    /// Message handler for both locally and remote sent messages
    #[inline]
    fn handle_message(&mut self, message: MessageContainer<OP::IN>) -> ArconResult<()> {
        #[cfg(feature = "metrics")]
        self.node_metrics
            .inbound_throughput
            .mark_n(message.total_events());

        if !self.node_state.in_channels.contains(message.sender()) {
            error!(
                self.logger,
                "Message from invalid sender id {:?}",
                message.sender()
            );
            return Ok(());
        }

        if self.sender_blocked(message.sender()) {
            self.node_state.message_buffer().append(message.raw())?;
            return Ok(());
        }

        #[cfg(all(feature = "hardware_counters", target_os = "linux", not(test)))]
        let (mut group, counters) = {
            let mut group = Group::new()?;
            let mut counters = Vec::with_capacity(self.perf_events.counters.len());
            for hardware_counter in self.perf_events.counters.iter() {
                let counter = Builder::new()
                    .group(&mut group)
                    .kind(hardware_counter.get_hardware_kind())
                    .build()?;

                counters.push((hardware_counter.to_string(), counter));
            }
            (group, counters)
        };

        #[cfg(feature = "metrics")]
        let start_time = Instant::now();

        #[cfg(all(feature = "hardware_counters", target_os = "linux", not(test)))]
        group.enable()?;

        match message {
            MessageContainer::Raw(r) => self.handle_events(r.sender, r.events)?,
            MessageContainer::Local(l) => self.handle_events(l.sender, l.events)?,
        }

        #[cfg(all(feature = "hardware_counters", target_os = "linux", not(test)))]
        group.disable()?;

        #[cfg(feature = "metrics")]
        {
            let elapsed = start_time.elapsed();
            histogram!("batch_execution_time", elapsed.as_micros() as f64,"node" => self.descriptor.clone());
        }

        #[cfg(feature = "metrics")]
        gauge!("inbound_throughput", self.node_metrics.inbound_throughput.get_one_min_rate(), "node" => self.descriptor.clone());

        #[cfg(all(feature = "hardware_counters", target_os = "linux", not(test)))]
        {
            let counts = group.read()?;
            for (metric_name, counter) in counters.iter() {
                histogram!(String::from(metric_name), counts[counter] as f64, "node" => self.descriptor.clone());
            }
        }

        Ok(())
    }

    #[inline(always)]
    fn sender_blocked(&mut self, sender: &NodeID) -> bool {
        self.node_state.blocked_channels().contains(sender)
    }

    /// Iterate over a batch of ArconEvent's
    #[inline]
    fn handle_events<I>(&mut self, sender: NodeID, events: I) -> ArconResult<()>
    where
        I: IntoIterator<Item = ArconEventWrapper<OP::IN>>,
    {
        'event_loop: for event in events.into_iter() {
            match event.unwrap() {
                ArconEvent::Element(e) => {
                    let watermark = match self.node_state.watermarks().get(&sender) {
                        Some(wm) => wm,
                        None => return reportable_error!("Uninitialised watermark"),
                    };

                    if e.timestamp <= watermark.timestamp {
                        continue 'event_loop;
                    }
                    self.handle_element(e)?;
                }
                ArconEvent::Watermark(w) => {
                    self.handle_watermark(w, sender)?;
                }
                ArconEvent::Epoch(e) => {
                    self.handle_epoch(e, sender)?;
                }
                ArconEvent::Death(s) => {
                    // We are instructed to shutdown....
                    self.add_outgoing_event(ArconEvent::Death(s))?;
                    self.ctx.suicide(); // TODO: is suicide enough?
                }
            }
        }

        Ok(())
    }

    fn get_in_key(&self, e: &OP::IN) -> u64 {
        if let Some(key_builder) = &self.in_key_builder {
            key_builder.get_key(e)
        } else {
            0
        }
    }

    #[inline(always)]
    fn handle_element(&mut self, e: ArconElement<OP::IN>) -> ArconResult<()> {
        self.set_context(self.get_in_key(&e.data));
        for elem in self
            .operator
            .handle_element(e, &mut self.operator_context.borrow_mut())?
        {
            self.add_outgoing_event(ArconEvent::Element(elem))?;
        }
        Ok(())
    }

    #[inline]
    fn set_context(&mut self, key: u64) {
        let mut context = self.operator_context.borrow_mut();
        context.current_key = key;
    }

    #[inline]
    fn handle_watermark(&mut self, w: Watermark, sender: NodeID) -> ArconResult<()> {
        let watermark = match self.node_state.watermarks().get(&sender) {
            Some(wm) => wm,
            None => return reportable_error!("Uninitialised watermark"),
        };
        if w <= *watermark {
            return Ok(());
        }

        // Insert the watermark and try early return
        if let Some(old) = self.node_state.watermarks().insert(sender, w) {
            if old > self.node_state.current_watermark {
                return Ok(());
            }
        }

        // A different early return
        if w <= self.node_state.current_watermark {
            return Ok(());
        }

        let new_watermark = *self.node_state.watermarks().values().min().unwrap();

        if new_watermark.timestamp > self.node_state.current_watermark.timestamp {
            #[cfg(feature = "metrics")]
            gauge!("last_watermark_timestamp", new_watermark.timestamp as f64, "node" => self.descriptor.clone());

            self.node_state.current_watermark = new_watermark;

            let timeouts = self
                .operator_context
                .borrow_mut()
                .timer
                .advance_to(new_watermark.timestamp)?;

            for timer_entry in timeouts {
                self.set_context(timer_entry.key());
                if let Some(elems) = self
                    .operator
                    .handle_timeout(timer_entry.value(), &mut self.operator_context.borrow_mut())?
                {
                    for elem in elems {
                        self.add_outgoing_event(ArconEvent::Element(elem))?;
                    }
                }
            }

            #[cfg(feature = "metrics")]
            increment_counter!("watermark_counter", "node" => self.descriptor.clone());

            // Forward the watermark
            self.add_outgoing_event(ArconEvent::Watermark(new_watermark))?;
        }
        Ok(())
    }

    fn handle_epoch(&mut self, e: Epoch, sender: NodeID) -> ArconResult<()> {
        debug!(self.logger, "Got Epoch {:?}", e);

        // early return if the epoch is lower
        if e < self.node_state.current_epoch {
            return Ok(());
        }

        // Add the sender to the blocked set.
        self.node_state.blocked_channels().insert(sender);

        // If all senders blocked we can transition to new Epoch
        if self.node_state.blocked_channels().len() == self.node_state.in_channels.len() {
            // Forward the Epoch
            self.add_outgoing_event(ArconEvent::Epoch(self.node_state.current_epoch))?;

            // persist internal node state for this node
            self.node_state.persist()?;

            // persist possible operator state..
            self.operator_context.borrow_mut().state.persist()?;

            // Perform the actual checkpoint
            self.checkpoint()?;

            // Send Ack to EpochManager
            self.epoch_manager.tell(EpochEvent::Ack(
                self.descriptor.clone(),
                self.node_state.current_epoch,
            ));

            // Update current epoch
            self.node_state.current_epoch.epoch += 1;

            #[cfg(feature = "metrics")]
            increment_counter!("epoch_counter", "node" => self.descriptor.clone());

            // flush the blocked_channels list
            self.node_state.blocked_channels().clear();

            // Iterate over the message-buffer until empty
            for message in self.node_state.message_buffer().consume()? {
                self.handle_events(message.sender, message.events)?;
            }
        }

        Ok(())
    }

    #[inline]
    fn add_outgoing_event(&self, event: ArconEvent<OP::OUT>) -> ArconResult<()> {
        let strategy = unsafe { &mut *self.channel_strategy.get() };
        common::add_outgoing_event(event, strategy, self)
    }

    fn checkpoint(&mut self) -> ArconResult<()> {
        if let Some(base_dir) = &self.ctx.config()["checkpoint_dir"].as_string() {
            let checkpoint_dir = format!(
                "{}/{}/checkpoint_{id}_{epoch}",
                base_dir,
                self.descriptor,
                id = self.descriptor,
                epoch = self.node_state.current_epoch.epoch,
            );

            #[cfg(feature = "metrics")]
            let start_time = Instant::now();

            self.backend.checkpoint(checkpoint_dir.as_ref())?;

            #[cfg(feature = "metrics")]
            {
                let elapsed = start_time.elapsed();
                histogram!("checkpoint_execution_time_ms", elapsed.as_millis() as f64, "node" => self.descriptor.clone());
            }

            #[cfg(feature = "metrics")]
            {
                let metadata = std::fs::metadata(checkpoint_dir.clone())?;
                gauge!("last_checkpoint_size", metadata.len() as f64, "node" => self.descriptor.clone());
            }

            let snapshot = Snapshot::new(
                std::any::type_name::<B>().to_string(),
                self.node_state.current_epoch.epoch,
                checkpoint_dir.clone(),
            );
            self.node_manager_port.trigger(NodeManagerEvent::Checkpoint(
                self.node_state.id,
                SnapshotEvent::Snapshot(self.descriptor.clone(), snapshot),
            ));

            debug!(
                self.logger,
                "Completed a Checkpoint to path {}", checkpoint_dir
            );
        } else {
            return reportable_error!("Failed to fetch checkpoint_dir from Config");
        }

        Ok(())
    }
}

impl<OP, B> ComponentLifecycle for Node<OP, B>
where
    OP: Operator + 'static,
    B: Backend,
{
    fn on_start(&mut self) -> Handled {
        debug!(
            self.logger,
            "Started Arcon Node {} with Node ID {:?}", self.descriptor, self.node_state.id
        );

        // create directory that keeps checkpoints
        match self.ctx.config()["checkpoint_dir"].as_string() {
            Some(base_dir) => {
                let checkpoint_dir = format!("{}/{}", base_dir, self.descriptor,);
                std::fs::create_dir_all(checkpoint_dir).unwrap();
            }
            None => {
                // NOTE: just logging for now, should in the future order a shutdown for the whole application.
                error!(self.logger, "Failed to locate checkpoint_dir config");
            }
        }

        self.epoch_manager
            .tell(EpochEvent::Register(self.descriptor.clone()));

        if self
            .operator
            .on_start(&mut self.operator_context.borrow_mut())
            .is_err()
        {
            // NOTE: just logging for now, should in the future order a shutdown for the whole application.
            error!(self.logger, "Failed to run startup code");
        }

        Handled::Ok
    }
}

impl<OP, B> Require<NodeManagerPort> for Node<OP, B>
where
    OP: Operator + 'static,
    B: Backend,
{
    fn handle(&mut self, _: ()) -> Handled {
        Handled::Ok
    }
}

impl<OP, B> Provide<NodeManagerPort> for Node<OP, B>
where
    OP: Operator + 'static,
    B: Backend,
{
    fn handle(&mut self, e: NodeManagerEvent) -> Handled {
        trace!(self.logger, "Ignoring node event: {:?}", e);
        Handled::Ok
    }
}

impl<OP, B> Actor for Node<OP, B>
where
    OP: Operator + 'static,
    B: Backend,
{
    type Message = ArconMessage<OP::IN>;

    fn receive_local(&mut self, msg: Self::Message) -> Handled {
        if let Err(err) = self.handle_message(MessageContainer::Local(msg)) {
            error!(self.logger, "Failed to handle message: {}", err);
        }
        Handled::Ok
    }
    fn receive_network(&mut self, msg: NetMessage) -> Handled {
        let arcon_msg = match *msg.ser_id() {
            id if id == OP::IN::RELIABLE_SER_ID => msg
                .try_deserialise::<RawArconMessage<OP::IN>, ReliableSerde<OP::IN>>()
                .map_err(|e| Error::Unsupported {
                    msg: format!("Failed to unpack reliable ArconMessage with err {:?}", e),
                }),
            id => reportable_error!("Unexpected deserialiser with id {}", id),
        };

        match arcon_msg {
            Ok(m) => {
                if let Err(err) = self.handle_message(MessageContainer::Raw(m)) {
                    error!(self.logger, "Failed to handle node message: {}", err);
                }
            }
            Err(e) => error!(self.logger, "Error ArconNetworkMessage: {:?}", e),
        }
        Handled::Ok
    }
}

#[cfg(test)]
mod tests {
    // Tests the message logic of Node.
    use super::*;

    #[cfg(all(feature = "hardware_counters", target_os = "linux"))]
    #[cfg(not(test))]
    use crate::metrics::perf_event::HardwareCounter;
    use crate::{
        application::Application,
        dataflow::builder::OperatorBuilder,
        index::EmptyState,
        stream::{
            channel::{strategy::forward::Forward, Channel},
            node::debug::DebugNode,
            operator::function::Filter,
        },
    };
    use std::{sync::Arc, thread, time};

    fn node_test_setup() -> (ActorRef<ArconMessage<i32>>, Arc<Component<DebugNode<i32>>>) {
        fn filter_fn(x: &i32) -> bool {
            *x >= 0
        }

        let builder = OperatorBuilder::<_> {
            operator: Arc::new(|| Filter::new(&filter_fn)),
            state: Arc::new(|_backend| EmptyState),
            conf: Default::default(),
        };

        fn setup<OP: Operator<IN = i32, OUT = i32> + 'static, B: Backend>(
            builder: OperatorBuilder<OP, B>,
        ) -> (ActorRef<ArconMessage<i32>>, Arc<Component<DebugNode<i32>>>) {
            // Returns a filter Node with input channels: sender1..sender3
            // And a debug sink receiving its results
            let app = Application::default();
            let pool_info = app.get_pool_info();
            let epoch_manager_ref = app.epoch_manager();

            let sink = app.data_system().create(DebugNode::<i32>::new);

            app.data_system()
                .start_notify(&sink)
                .wait_timeout(std::time::Duration::from_millis(1000))
                .expect("started");

            // Construct Channel to the Debug sink
            let actor_ref: ActorRefStrong<ArconMessage<i32>> =
                sink.actor_ref().hold().expect("Failed to fetch");
            let channel = Channel::Local(actor_ref);
            let channel_strategy: ChannelStrategy<i32> =
                ChannelStrategy::Forward(Forward::new(channel, NodeID::new(0), pool_info));

            // Set up  NodeManager
            let backend = Arc::new(crate::test_utils::temp_backend::<B>());
            let descriptor = String::from("node_");
            let in_channels = vec![1.into(), 2.into(), 3.into()];

            let operator = builder.operator.clone();
            let operator_state = builder.state.clone();

            #[cfg(not(test))]
            let mut perf_events = PerfEvents::new();

            let nm = NodeManager::<OP, B>::new(
                descriptor.clone(),
                app.data_system().clone(),
                in_channels.clone(),
                app.arcon_logger.clone(),
                Arc::new(builder),
            );
            let node_manager_comp = app.ctrl_system().create(|| nm);

            app.ctrl_system()
                .start_notify(&node_manager_comp)
                .wait_timeout(std::time::Duration::from_millis(1000))
                .expect("started");

            let node = Node::<OP, _>::new(
                descriptor,
                channel_strategy,
                operator(),
                operator_state(backend.clone()),
                NodeState::new(NodeID::new(0), in_channels, backend.clone()),
                backend,
                app.arcon_logger.clone(),
                epoch_manager_ref,
                #[cfg(not(test))]
                perf_events,
                GlobalNodeId::null(),
                None,
            );

            let filter_comp = app.data_system().create(|| node);
            let required_ref = filter_comp.on_definition(|cd| cd.node_manager_port.share());

            biconnect_components::<NodeManagerPort, _, _>(&node_manager_comp, &filter_comp)
                .expect("connection");

            app.data_system()
                .start_notify(&filter_comp)
                .wait_timeout(std::time::Duration::from_millis(1000))
                .expect("started");

            let filter_ref = filter_comp.actor_ref();

            node_manager_comp.on_definition(|cd| {
                // Insert the created Node into the NodeManager
                cd.nodes
                    .insert(GlobalNodeId::null(), (filter_comp, required_ref));
            });

            (filter_ref, sink)
        }

        setup(builder)
    }

    fn watermark(time: u64, sender: u32) -> ArconMessage<i32> {
        ArconMessage::watermark(time, sender.into())
    }

    fn element(data: i32, time: u64, sender: u32) -> ArconMessage<i32> {
        ArconMessage::element(data, time, sender.into())
    }

    fn epoch(epoch: u64, sender: u32) -> ArconMessage<i32> {
        ArconMessage::epoch(epoch, sender.into())
    }
    fn death(sender: u32) -> ArconMessage<i32> {
        ArconMessage::death(String::from("die"), sender.into())
    }

    fn wait(time: u64) {
        thread::sleep(time::Duration::from_secs(time));
    }

    #[test]
    fn node_no_watermark() {
        let (node_ref, sink) = node_test_setup();
        node_ref.tell(watermark(1, 1));

        wait(1);
        sink.on_definition(|cd| {
            let data_len = cd.data.len();
            let watermark_len = cd.watermarks.len();
            assert_eq!(watermark_len, 0);
            assert_eq!(data_len, 0);
        });
    }

    #[test]
    fn node_one_watermark() {
        let (node_ref, sink) = node_test_setup();
        node_ref.tell(watermark(1, 1));
        node_ref.tell(watermark(1, 2));
        node_ref.tell(watermark(1, 3));

        wait(1);
        sink.on_definition(|cd| {
            let data_len = cd.data.len();
            let watermark_len = cd.watermarks.len();
            assert_eq!(watermark_len, 1);
            assert_eq!(data_len, 0);
        });
    }

    #[test]
    fn node_outoforder_watermarks() {
        let (node_ref, sink) = node_test_setup();
        node_ref.tell(watermark(1, 1));
        node_ref.tell(watermark(3, 1));
        node_ref.tell(watermark(1, 2));
        node_ref.tell(watermark(2, 2));
        node_ref.tell(watermark(4, 3));

        wait(1);
        sink.on_definition(|cd| {
            let watermark_len = cd.watermarks.len();
            assert_eq!(watermark_len, 1);
            assert_eq!(cd.watermarks[0].timestamp, 2u64);
        });
    }

    #[test]
    fn node_epoch_block() {
        let (node_ref, sink) = node_test_setup();
        node_ref.tell(element(1, 1, 1));
        node_ref.tell(epoch(3, 1));
        // should be blocked:
        node_ref.tell(element(2, 1, 1));
        // should not be blocked
        node_ref.tell(element(3, 1, 2));

        node_ref.tell(death(2)); // send death marker on unblocked channel to flush

        wait(1);
        sink.on_definition(|cd| {
            let data_len = cd.data.len();
            let epoch_len = cd.epochs.len();
            assert_eq!(epoch_len, 0);
            assert_eq!(cd.data[0].data, 1i32);
            assert_eq!(cd.data[1].data, 3i32);
            assert_eq!(data_len, 2);
        });
    }

    #[test]
    fn node_epoch_no_continue() {
        let (node_ref, sink) = node_test_setup();
        node_ref.tell(element(11, 1, 1)); // not blocked
        node_ref.tell(epoch(1, 1)); // sender1 blocked
        node_ref.tell(element(12, 1, 1)); // blocked
        node_ref.tell(element(21, 1, 2)); // not blocked
        node_ref.tell(epoch(2, 1)); // blocked
        node_ref.tell(epoch(1, 2)); // sender2 blocked
        node_ref.tell(epoch(2, 2)); // blocked
        node_ref.tell(element(23, 1, 2)); // blocked
        node_ref.tell(element(31, 1, 3)); // not blocked

        node_ref.tell(death(3)); // send death marker on unblocked channel to flush
        wait(1);
        sink.on_definition(|cd| {
            let data_len = cd.data.len();
            let epoch_len = cd.epochs.len();
            assert_eq!(epoch_len, 0); // no epochs should've completed
            assert_eq!(cd.data[0].data, 11i32);
            assert_eq!(cd.data[1].data, 21i32);
            assert_eq!(cd.data[2].data, 31i32);
            assert_eq!(data_len, 3);
        });
    }

    #[test]
    fn node_epoch_continue() {
        // Same test as previous but we finnish it by sending the required epochs
        let (node_ref, sink) = node_test_setup();
        node_ref.tell(element(11, 1, 1)); // not blocked
        node_ref.tell(epoch(1, 1)); // sender1 blocked
        node_ref.tell(element(12, 1, 1)); // blocked
        node_ref.tell(element(21, 1, 2)); // not blocked
        node_ref.tell(epoch(2, 1)); // blocked
        node_ref.tell(element(13, 1, 1)); // blocked
        node_ref.tell(epoch(1, 2)); // sender2 blocked
        node_ref.tell(epoch(2, 2)); // blocked
        node_ref.tell(element(22, 1, 2)); // blocked
        node_ref.tell(element(31, 1, 3)); // not blocked
        node_ref.tell(epoch(1, 3)); // Complete our epochs
        node_ref.tell(epoch(2, 3));
        // All the elements should now have been delivered in specific order

        wait(3);
        node_ref.tell(death(3)); // send death marker on unblocked channel to flush
        wait(3);

        sink.on_definition(|cd| {
            let data_len = cd.data.len();
            let epoch_len = cd.epochs.len();
            assert_eq!(epoch_len, 2); // 3 epochs should've completed
            assert_eq!(cd.data[0].data, 11i32);
            assert_eq!(cd.data[1].data, 21i32);
            assert_eq!(cd.data[2].data, 31i32);
            assert_eq!(cd.data[3].data, 12i32); // First message in epoch1
            assert_eq!(cd.data[4].data, 13i32); // First message in epoch2
            assert_eq!(cd.data[5].data, 22i32); // 2nd message in epoch2
            assert_eq!(data_len, 6);
        });
    }
}