Files
abomonation
abomonation_derive
ansi_term
async_trait
atty
bincode
bitflags
byteorder
bytes
cfg_if
chrono
clap
dirs
dirs_sys
erdos
fixedbitset
fnv
futures
futures_channel
futures_core
futures_executor
futures_io
futures_macro
futures_sink
futures_task
futures_util
async_await
future
io
lock
sink
stream
task
indexmap
iovec
lazy_static
libc
log
memchr
mio
net2
num_cpus
num_integer
num_traits
petgraph
pin_project_lite
pin_utils
proc_macro2
proc_macro_hack
proc_macro_nested
quote
rand
rand_chacha
rand_core
rand_hc
rand_isaac
rand_jitter
rand_os
rand_pcg
rand_xorshift
serde
serde_derive
sha1
slab
slog
slog_term
strsim
syn
synstructure
term
textwrap
thread_local
time
tokio
future
io
loom
macros
net
park
runtime
stream
sync
task
time
util
tokio_macros
tokio_serde
tokio_serde_bincode
tokio_util
unicode_width
unicode_xid
uuid
vec_map
  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
use async_trait::async_trait;
use serde::Deserialize;
use std::{any::Any, collections::HashMap, sync::Arc};
use tokio::sync::{mpsc, Mutex};

use crate::{
    communication::{Pusher, PusherT, RecvEndpoint, SendEndpoint},
    dataflow::{
        graph::{Channel, Graph, Vertex},
        stream::StreamId,
        Data, Message,
    },
    node::NodeId,
    scheduler::endpoints_manager::{ChannelsToReceivers, ChannelsToSenders},
};

#[async_trait]
pub trait StreamEndpointsT: Send {
    fn as_any(&mut self) -> &mut dyn Any;

    /// Creates a new inter-thread channel for the stream.
    ///
    /// It creates a `mpsc::Channel` and adds the sender and receiver to the
    /// corresponding endpoints.
    fn add_inter_thread_channel(&mut self);

    /// Adds a `SendEndpoint` to the other node.
    ///
    /// Assumes that `channels_to_senders` already stores a `mpsc::Sender` to the
    /// network sender to the other node.
    async fn add_inter_node_send_endpoint(
        &mut self,
        other_node_id: NodeId,
        channels_to_senders: Arc<Mutex<ChannelsToSenders>>,
    ) -> Result<(), String>;
    fn add_inter_node_recv_endpoint(
        &mut self,
        receiver_pushers: &mut HashMap<StreamId, Box<dyn PusherT>>,
    ) -> Result<(), String>;
}

pub struct StreamEndpoints<D>
where
    for<'a> D: Data + Deserialize<'a>,
{
    /// The id of the stream.
    stream_id: StreamId,
    /// The receive endopoints of the stream.
    recv_endpoints: Vec<RecvEndpoint<Arc<Message<D>>>>,
    /// The send endpoints of the stream.
    send_endpoints: Vec<SendEndpoint<Arc<Message<D>>>>,
}

impl<D> StreamEndpoints<D>
where
    for<'a> D: Data + Deserialize<'a>,
{
    pub fn new(stream_id: StreamId) -> Self {
        Self {
            stream_id,
            recv_endpoints: Vec::new(),
            send_endpoints: Vec::new(),
        }
    }

    /// Takes a `RecvEndpoint` out of the stream.
    fn take_recv_endpoint(&mut self) -> Result<RecvEndpoint<Arc<Message<D>>>, &'static str> {
        match self.recv_endpoints.pop() {
            Some(recv_endpoint) => Ok(recv_endpoint),
            None => Err("No more recv endpoints available"),
        }
    }

    /// Returns a cloned list of the `SendEndpoint`s the stream has.
    fn get_send_endpoints(&mut self) -> Result<Vec<SendEndpoint<Arc<Message<D>>>>, &'static str> {
        let mut result: Vec<SendEndpoint<Arc<Message<D>>>> = Vec::new();
        result.append(&mut self.send_endpoints);
        Ok(result)
    }

    fn add_send_endpoint(&mut self, endpoint: SendEndpoint<Arc<Message<D>>>) {
        self.send_endpoints.push(endpoint);
    }

    fn add_recv_endpoint(&mut self, endpoint: RecvEndpoint<Arc<Message<D>>>) {
        self.recv_endpoints.push(endpoint);
    }
}

#[async_trait]
impl<D> StreamEndpointsT for StreamEndpoints<D>
where
    for<'a> D: Data + Deserialize<'a>,
{
    fn as_any(&mut self) -> &mut dyn Any {
        self
    }

    fn add_inter_thread_channel(&mut self) {
        let (tx, rx) = mpsc::unbounded_channel();
        self.add_send_endpoint(SendEndpoint::InterThread(tx));
        self.add_recv_endpoint(RecvEndpoint::InterThread(rx));
    }

    async fn add_inter_node_send_endpoint(
        &mut self,
        other_node_id: NodeId,
        channels_to_senders: Arc<Mutex<ChannelsToSenders>>,
    ) -> Result<(), String> {
        let channels_to_senders = channels_to_senders.lock().await;
        if let Some(tx) = channels_to_senders.clone_channel(other_node_id) {
            self.add_send_endpoint(SendEndpoint::InterProcess(self.stream_id, tx));
            Ok(())
        } else {
            Err(format!("Unable to clone channel to node {}", other_node_id))
        }
    }

    fn add_inter_node_recv_endpoint(
        &mut self,
        receiver_pushers: &mut HashMap<StreamId, Box<dyn PusherT>>,
    ) -> Result<(), String> {
        let pusher: &mut Box<dyn PusherT> = receiver_pushers
            .entry(self.stream_id)
            .or_insert_with(|| Box::new(Pusher::<Arc<Message<D>>>::new()));
        if let Some(pusher) = pusher.as_any().downcast_mut::<Pusher<Arc<Message<D>>>>() {
            let (tx, rx) = mpsc::unbounded_channel();
            pusher.add_endpoint(SendEndpoint::InterThread(tx));
            self.add_recv_endpoint(RecvEndpoint::InterThread(rx));
            Ok(())
        } else {
            Err(format!(
                "Error casting pusher when adding inter node recv endpoint for stream {}",
                self.stream_id
            ))
        }
    }
}

/// Data structure that stores information needed to set up dataflow channels
/// by constructing individual transport channels.
pub struct ChannelManager {
    /// The node to which the [`ChannelManager`] belongs.
    node_id: NodeId,
    /// The dataflow graph.
    graph: Graph,
    /// Stores a `StreamEndpoints` for each stream id.
    stream_entries: HashMap<StreamId, Box<dyn StreamEndpointsT>>,
}

impl ChannelManager {
    /// Creates transport channels between connected operators on this node, transport channels
    /// for operators with streams containing dataflow channels to other nodes, and transport
    /// channels from TCP receivers to operators that are connected to streams originating on
    /// other nodes.
    pub async fn new(
        graph: &Graph,
        node_id: NodeId,
        channels_to_receivers: Arc<Mutex<ChannelsToReceivers>>,
        channels_to_senders: Arc<Mutex<ChannelsToSenders>>,
    ) -> Self {
        let mut channel_manager = Self {
            node_id,
            graph: graph.clone(),
            stream_entries: HashMap::new(),
        };

        let mut receiver_pushers: HashMap<StreamId, Box<dyn PusherT>> = HashMap::new();

        let node_vertices = graph.get_vertices_on(node_id);
        for stream_metadata in graph.get_streams() {
            if node_vertices.contains(&stream_metadata.get_source()) {
                let stream_endpoint_t = channel_manager
                    .stream_entries
                    .entry(stream_metadata.get_id())
                    .or_insert_with(|| stream_metadata.to_stream_endpoints_t());
                for channel in stream_metadata.get_channels() {
                    match channel {
                        Channel::InterNode(channel_metadata) => {
                            let other_node_id = match channel_metadata.sink {
                                Vertex::Driver(id) => id,
                                Vertex::Operator(op_id) => {
                                    graph.get_operator(op_id).unwrap().node_id
                                }
                            };
                            stream_endpoint_t
                                .add_inter_node_send_endpoint(
                                    other_node_id,
                                    Arc::clone(&channels_to_senders),
                                )
                                .await
                                .unwrap();
                        }
                        Channel::InterThread(_) => {
                            stream_endpoint_t.add_inter_thread_channel();
                        }
                        Channel::Unscheduled(cm) => eprintln!("Unscheduled channel: {:?}", cm),
                    }
                }
            } else {
                for channel in stream_metadata.get_channels() {
                    if let Channel::InterNode(channel_metadata) = channel {
                        if node_vertices.contains(&channel_metadata.sink) {
                            let stream_endpoint_t = channel_manager
                                .stream_entries
                                .entry(stream_metadata.get_id())
                                .or_insert_with(|| stream_metadata.to_stream_endpoints_t());
                            stream_endpoint_t
                                .add_inter_node_recv_endpoint(&mut receiver_pushers)
                                .unwrap();
                        }
                    }
                }
            }
        }

        // Send pushers to the DataReceiver which publishes received messages from TCP
        // on the proper transport channel.
        for (k, v) in receiver_pushers.into_iter() {
            channels_to_receivers.lock().await.send(k, v);
        }
        channel_manager
    }

    pub fn node_id(&self) -> NodeId {
        self.node_id
    }

    /// Takes a `RecvEnvpoint` from a given stream.
    pub fn take_recv_endpoint<D>(
        &mut self,
        stream_id: StreamId,
    ) -> Result<RecvEndpoint<Arc<Message<D>>>, String>
    where
        for<'a> D: Data + Deserialize<'a>,
    {
        let stream_id = self.graph.resolve_stream_id(stream_id);

        if let Some(stream_entry_t) = self.stream_entries.get_mut(&stream_id) {
            if let Some(stream_entry) = stream_entry_t.as_any().downcast_mut::<StreamEndpoints<D>>()
            {
                match stream_entry.take_recv_endpoint() {
                    Ok(recv_endpoint) => Ok(recv_endpoint),
                    Err(msg) => Err(format!(
                        "Could not get recv endpoint with id {}: {}",
                        stream_id, msg
                    )),
                }
            } else {
                Err(format!(
                    "Type mismatch for recv endpoint with ID {}",
                    stream_id
                ))
            }
        } else {
            Err(format!("No recv endpoints found with ID {}", stream_id))
        }
    }

    /// Returns a cloned vector of the `SendEndpoint`s for a given stream.
    pub fn get_send_endpoints<D>(
        &mut self,
        stream_id: StreamId,
    ) -> Result<Vec<SendEndpoint<Arc<Message<D>>>>, String>
    where
        for<'a> D: Data + Deserialize<'a>,
    {
        if let Some(stream_entry_t) = self.stream_entries.get_mut(&stream_id) {
            if let Some(stream_entry) = stream_entry_t.as_any().downcast_mut::<StreamEndpoints<D>>()
            {
                match stream_entry.get_send_endpoints() {
                    Ok(send_endpoints) => Ok(send_endpoints),
                    Err(msg) => Err(format!(
                        "Could not get recv endpoint with id {}: {}",
                        stream_id, msg
                    )),
                }
            } else {
                Err(format!(
                    "Type mismatch for recv endpoint with ID {}",
                    stream_id
                ))
            }
        } else {
            Err(format!("No recv endpoints found with ID {}", stream_id))
        }
    }
}