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
/// Arrow related types
pub mod arrow;
/// Serialisers and Deserialiser for in-flight data
pub mod flight_serde;
#[allow(dead_code)]
pub mod partition;
/// Known Serialisation IDs for Arcon Types
mod ser_id;

use crate::buffer::event::BufferReader;
use kompact::prelude::*;
use prost::{Message, Oneof};
use std::{
    fmt,
    hash::{Hash, Hasher},
    ops::Deref,
};

/// A type alias for registered state within Arcon
pub type StateID = String;

/// A type alias for an ArconType version id
pub type VersionId = u32;

pub trait ArconTypeBounds:
    Clone + fmt::Debug + Sync + Send + prost::Message + Default + 'static
{
}
impl<T> ArconTypeBounds for T where
    T: Clone + fmt::Debug + Sync + Send + prost::Message + Default + 'static
{
}

/// Type that can be passed through the Arcon runtime
pub trait ArconType: ArconTypeBounds
where
    Self: std::marker::Sized,
{
    /// Serialisation ID for Arcon's Reliable In-flight serde
    const RELIABLE_SER_ID: SerId;
    /// Current version of this ArconType
    const VERSION_ID: VersionId;
}

/// An Enum containing all possible stream events that may occur in an execution
#[derive(Oneof, Clone)]
pub enum ArconEvent<A: ArconType> {
    /// A stream element containing some data of type [ArconType] and an optional timestamp [u64]
    #[prost(message, tag = "1")]
    Element(ArconElement<A>),
    /// A [Watermark] message
    #[prost(message, tag = "2")]
    Watermark(Watermark),
    /// An [Epoch] marker message
    #[prost(message, tag = "3")]
    Epoch(Epoch),
    /// A death message
    #[prost(message, tag = "4")]
    Death(String),
}

// The struct below is required because of the peculiarity of prost/protobuf - you cannot have
// repeated (like, a Vec<_>) oneof fields, so we wrap ArconEvent in a struct which implements
// prost::Message. Unfortunately protobuf also doesn't allow for required oneof fields, so the inner
// value has to be optional. In practice we expect it to always be Some.

#[derive(Message, Clone)]
pub struct ArconEventWrapper<A: ArconType> {
    #[prost(oneof = "ArconEvent::<A>", tags = "1, 2, 3, 4")]
    inner: Option<ArconEvent<A>>,
}

impl<A: ArconType> ArconEventWrapper<A> {
    pub fn unwrap(self) -> ArconEvent<A> {
        self.inner
            .expect("ArconEventWrapper.inner is None. Prost deserialization error?")
    }

    pub fn unwrap_ref(&self) -> &ArconEvent<A> {
        self.inner
            .as_ref()
            .expect("ArconEventWrapper.inner is None. Prost deserialization error?")
    }
}

impl<A: ArconType> From<ArconEvent<A>> for ArconEventWrapper<A> {
    fn from(inner: ArconEvent<A>) -> Self {
        ArconEventWrapper { inner: Some(inner) }
    }
}

/// A Stream element containing some data and timestamp
#[derive(Message, Clone)]
pub struct ArconElement<A: ArconType> {
    #[prost(message, required, tag = "1")]
    pub data: A,
    #[prost(uint64, tag = "2")]
    pub timestamp: u64,
}

impl<A: ArconType> ArconElement<A> {
    /// Creates an ArconElement without a timestamp
    pub fn new(data: A) -> Self {
        ArconElement { data, timestamp: 0 }
    }

    /// Creates an ArconElement with a timestamp
    pub fn with_timestamp(data: A, timestamp: u64) -> Self {
        ArconElement { data, timestamp }
    }
}

/// Watermark message containing a [u64] timestamp
#[derive(Message, Clone, Copy, Ord, PartialOrd, Eq, PartialEq)]
pub struct Watermark {
    #[prost(uint64, tag = "1")]
    pub timestamp: u64,
}

impl Watermark {
    pub fn new(timestamp: u64) -> Self {
        Watermark { timestamp }
    }
}

/// Epoch marker message
#[derive(Message, Clone, Hash, Copy, Ord, PartialOrd, Eq, PartialEq)]
pub struct Epoch {
    #[prost(uint64, tag = "1")]
    pub epoch: u64,
}

impl Epoch {
    /// Creates a new Epoch
    pub fn new(epoch: u64) -> Self {
        Epoch { epoch }
    }
}

/// Container that holds two possible variants of messages
pub enum MessageContainer<A: ArconType> {
    /// Batch of events backed Rust's system allocator
    ///
    /// Used when receiving events from the network or when restoring messages from a state backend
    Raw(RawArconMessage<A>),
    /// Batch of events backed by Arcon's allocator
    Local(ArconMessage<A>),
}

impl<A: ArconType> MessageContainer<A> {
    /// Return a reference to the sender ID of this message
    #[inline]
    pub fn sender(&self) -> &NodeID {
        match self {
            MessageContainer::Raw(r) => &r.sender,
            MessageContainer::Local(l) => &l.sender,
        }
    }
    /// Return number of events in the message

    #[inline]
    #[cfg(feature = "metrics")]
    pub fn total_events(&self) -> u64 {
        match self {
            MessageContainer::Raw(r) => r.events.len() as u64,
            MessageContainer::Local(l) => l.events.len() as u64,
        }
    }
    /// Consumes the container and returns a RawArconMessage
    pub fn raw(self) -> RawArconMessage<A> {
        match self {
            MessageContainer::Raw(r) => r,
            MessageContainer::Local(l) => l.into(),
        }
    }
}

/// An ArconMessage backed by a reusable EventBuffer
#[derive(Debug, Clone)]
pub struct ArconMessage<A: ArconType> {
    /// Batch of ArconEvents backed by an EventBuffer
    pub events: BufferReader<ArconEventWrapper<A>>,
    /// ID identifying where the message is sent from
    pub sender: NodeID,
}

/// A raw ArconMessage for serialisation
#[derive(Message, Clone)]
pub struct RawArconMessage<A: ArconType> {
    /// Batch of ArconEvents
    #[prost(message, repeated, tag = "1")]
    pub events: Vec<ArconEventWrapper<A>>,
    /// ID identifying where the message is sent from
    #[prost(message, required, tag = "2")]
    pub sender: NodeID,
}

impl<A: ArconType> From<ArconMessage<A>> for RawArconMessage<A> {
    fn from(msg: ArconMessage<A>) -> Self {
        RawArconMessage {
            events: msg.events.to_vec(),
            sender: msg.sender,
        }
    }
}

/// Convenience methods
#[cfg(test)]
impl<A: ArconType> ArconMessage<A> {
    /// Creates an ArconMessage with a single [ArconEvent::Watermark] event
    ///
    /// This function should only be used for development and test purposes.
    pub fn watermark(timestamp: u64, sender: NodeID) -> ArconMessage<A> {
        ArconMessage {
            events: vec![ArconEvent::<A>::Watermark(Watermark { timestamp }).into()].into(),
            sender,
        }
    }
    /// Creates an ArconMessage with a single [ArconEvent::Epoch] event
    ///
    /// This function should only be used for development and test purposes.
    pub fn epoch(epoch: u64, sender: NodeID) -> ArconMessage<A> {
        ArconMessage {
            events: vec![ArconEvent::<A>::Epoch(Epoch { epoch }).into()].into(),
            sender,
        }
    }
    /// Creates an ArconMessage with a single [ArconEvent::Death] event
    ///
    /// This function should only be used for development and test purposes.
    pub fn death(msg: String, sender: NodeID) -> ArconMessage<A> {
        ArconMessage {
            events: vec![ArconEvent::<A>::Death(msg).into()].into(),
            sender,
        }
    }
    /// Creates an ArconMessage with a single [ArconEvent::Element] event
    ///
    /// This function should only be used for development and test purposes.
    pub fn element(data: A, timestamp: u64, sender: NodeID) -> ArconMessage<A> {
        ArconMessage {
            events: vec![ArconEvent::Element(ArconElement { data, timestamp }).into()].into(),
            sender,
        }
    }
}

/// A NodeID is used to identify a message sender
#[derive(Message, PartialEq, Eq, PartialOrd, Ord, Hash, Copy, Clone)]
pub struct NodeID {
    #[prost(uint32, tag = "1")]
    pub id: u32,
}

impl NodeID {
    pub fn new(new_id: u32) -> NodeID {
        NodeID { id: new_id }
    }
}

impl From<u32> for NodeID {
    fn from(id: u32) -> Self {
        NodeID::new(id)
    }
}

// Implement ArconType for primitives.
// NOTE: This is mainly for testing and development. In practice,
// an ArconType is always a struct or enum.

impl ArconType for u32 {
    const RELIABLE_SER_ID: SerId = ser_id::RELIABLE_U32_ID;
    const VERSION_ID: VersionId = 1;
}
impl ArconType for u64 {
    const RELIABLE_SER_ID: SerId = ser_id::RELIABLE_U64_ID;
    const VERSION_ID: VersionId = 1;
}
impl ArconType for i32 {
    const RELIABLE_SER_ID: SerId = ser_id::RELIABLE_I32_ID;
    const VERSION_ID: VersionId = 1;
}
impl ArconType for i64 {
    const RELIABLE_SER_ID: SerId = ser_id::RELIABLE_I64_ID;
    const VERSION_ID: VersionId = 1;
}
impl ArconType for ArconF32 {
    const RELIABLE_SER_ID: SerId = ser_id::RELIABLE_F32_ID;
    const VERSION_ID: VersionId = 1;
}
impl ArconType for ArconF64 {
    const RELIABLE_SER_ID: SerId = ser_id::RELIABLE_F64_ID;
    const VERSION_ID: VersionId = 1;
}
impl ArconType for bool {
    const RELIABLE_SER_ID: SerId = ser_id::RELIABLE_BOOLEAN_ID;
    const VERSION_ID: VersionId = 1;
}
impl ArconType for String {
    const RELIABLE_SER_ID: SerId = ser_id::RELIABLE_STRING_ID;
    const VERSION_ID: VersionId = 1;
}

/// Float wrapper for f32 in order to impl Hash [std::hash::Hash]
///
/// The `Hash` impl rounds the floats down to an integer and then hashes it.
#[derive(Clone, Message)]
#[repr(transparent)]
pub struct ArconF32 {
    #[prost(float, tag = "1")]
    pub value: f32,
}

impl ArconF32 {
    pub fn new(value: f32) -> ArconF32 {
        ArconF32 { value }
    }
}

impl Hash for ArconF32 {
    fn hash<H: Hasher>(&self, state: &mut H) {
        let s: u64 = self.value.trunc() as u64;
        s.hash(state);
    }
}

impl From<f32> for ArconF32 {
    fn from(value: f32) -> Self {
        ArconF32::new(value)
    }
}

impl std::str::FromStr for ArconF32 {
    type Err = ::std::num::ParseFloatError;
    fn from_str(s: &str) -> std::result::Result<Self, Self::Err> {
        let f: f32 = s.parse::<f32>()?;
        Ok(ArconF32::new(f))
    }
}
impl Deref for ArconF32 {
    type Target = f32;

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

impl PartialEq for ArconF32 {
    fn eq(&self, other: &Self) -> bool {
        self.value == other.value
    }
}

/// Float wrapper for f64 in order to impl Hash [std::hash::Hash]
///
/// The `Hash` impl rounds the floats down to an integer and then hashes it.
#[derive(Clone, Message)]
#[repr(transparent)]
pub struct ArconF64 {
    #[prost(double, tag = "1")]
    pub value: f64,
}

impl ArconF64 {
    pub fn new(value: f64) -> ArconF64 {
        ArconF64 { value }
    }
}

impl Hash for ArconF64 {
    fn hash<H: Hasher>(&self, state: &mut H) {
        let s: u64 = self.value.trunc() as u64;
        s.hash(state);
    }
}

impl From<f64> for ArconF64 {
    fn from(value: f64) -> Self {
        ArconF64::new(value)
    }
}

impl std::str::FromStr for ArconF64 {
    type Err = ::std::num::ParseFloatError;
    fn from_str(s: &str) -> std::result::Result<Self, Self::Err> {
        let f: f64 = s.parse::<f64>()?;
        Ok(ArconF64::new(f))
    }
}
impl Deref for ArconF64 {
    type Target = f64;

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

impl PartialEq for ArconF64 {
    fn eq(&self, other: &Self) -> bool {
        self.value == other.value
    }
}

/// Arcon variant of the `Never` (or `!`) type which fulfills `ArconType` requirements
#[derive(Clone, Copy, PartialEq, Eq)]
pub enum ArconNever {}
impl ArconNever {
    pub const IS_UNREACHABLE: &'static str = "The ArconNever type cannot be instantiated!";
}
impl ArconType for ArconNever {
    const RELIABLE_SER_ID: SerId = ser_id::NEVER_ID;
    const VERSION_ID: VersionId = 1;
}
impl fmt::Debug for ArconNever {
    fn fmt(&self, _f: &mut fmt::Formatter<'_>) -> fmt::Result {
        unreachable!("{}", ArconNever::IS_UNREACHABLE);
    }
}
impl prost::Message for ArconNever {
    fn encoded_len(&self) -> usize {
        unreachable!("{}", ArconNever::IS_UNREACHABLE);
    }

    fn clear(&mut self) {
        unreachable!("{}", ArconNever::IS_UNREACHABLE);
    }

    fn encode_raw<B>(&self, _: &mut B)
    where
        B: bytes::buf::BufMut,
    {
        unreachable!("{}", ArconNever::IS_UNREACHABLE);
    }
    fn merge_field<B>(
        &mut self,
        _: u32,
        _: prost::encoding::WireType,
        _: &mut B,
        _: prost::encoding::DecodeContext,
    ) -> std::result::Result<(), prost::DecodeError>
    where
        B: bytes::buf::Buf,
    {
        unreachable!("{}", ArconNever::IS_UNREACHABLE);
    }
}

impl Default for ArconNever {
    fn default() -> Self {
        unreachable!("{}", ArconNever::IS_UNREACHABLE);
    }
}