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
// Copyright 2018 ETH Zurich. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 <LICENSE-APACHE or
// http://www.apache.org/licenses/LICENSE-2.0> or the MIT license
// <LICENSE-MIT or http://opensource.org/licenses/MIT>, at your
// option. This file may not be copied, modified, or distributed
// except according to those terms.

#![deny(missing_docs)]

//! The Strymon client-library. In order to register a Timely Dataflow job with
//! Strymon, users are expected to link against this library and eventually
//! invoke `strymon_job::execute`.
//!
//! In addition, this library also provides the operaters needed to use the
//! publish-subscribe mechanism.
//!
//! # Examples
//!
//! ```rust,no_run
//! extern crate strymon_job;
//! extern crate timely;
//!
//! use timely::dataflow::operators::ToStream;
//! use strymon_job::operators::publish::Partition;
//!
//! fn main() {
//!     strymon_job::execute(|worker, coord| {
//!         worker.dataflow::<u64, _, _>(|scope| {
//!             let stream = (0..1000).to_stream(scope);
//!             coord.publish("numbers", &stream, Partition::Merge)
//!                 .expect("failed to publish topic");
//!         });
//!     }).unwrap();
//! }
//! ```

extern crate timely;
extern crate timely_communication;
#[macro_use]
extern crate futures;
extern crate serde;
extern crate slab;
extern crate tokio_core;
#[macro_use]
extern crate log;
extern crate typename;
#[macro_use]
extern crate serde_derive;

extern crate strymon_communication;

extern crate strymon_model;
extern crate strymon_rpc;

mod protocol;
mod publisher;
mod subscriber;
mod util;
pub mod operators;

use std::io;
use std::sync::Mutex;

use futures::Future;

use timely::dataflow::scopes::Root;

use timely_communication::Allocator;
use timely_communication::initialize::{Configuration, WorkerGuards};

use strymon_communication::Network;
use strymon_communication::rpc::Outgoing;

use strymon_rpc::coordinator::{JobToken, AddWorkerGroup, CoordinatorRPC};
use strymon_model::JobId;
use strymon_model::config::job::Process;

/// Handle to communicate with the Strymon coordinator.
///
/// The methods of this object can be used to publish or subscribe to
/// topics available in the catalog. In order to obtain a `Coordinator` instance,
/// users must register the current process with `strymon_job::execute`.
#[derive(Clone)]
pub struct Coordinator {
    token: JobToken,
    network: Network,
    tx: Outgoing,
}

impl Coordinator {
    /// Registers the local job at the coordinator at address `coord`.
    fn initialize(
        id: JobId,
        process: usize,
        coord: String,
        hostname: String,
    ) -> io::Result<Self> {
        let network = Network::new(Some(hostname))?;
        let (tx, _) = network.client::<CoordinatorRPC, _>(&*coord)?;

        let announce = tx.request(&AddWorkerGroup {
            job: id,
            group: process,
        });

        let token = announce
            .wait()
            .map_err(|err| {
                err.and_then::<(), _>(|err| {
                    let err = format!("failed to register: {:?}", err);
                    Err(io::Error::new(io::ErrorKind::Other, err))
                })
            })
            .map_err(Result::unwrap_err)?;

        Ok(Coordinator {
            tx: tx,
            network: network,
            token: token,
        })
    }
}

/// Executes a Timely dataflow within this Strymon job.
///
/// This function requires that the calling process has been spawned by an executor.
/// Upon successful registration with the Strymon coordinator, the closure `func`
/// is invoked for each requested worker hosted by the current process.
///
/// This function intentionally mirrors `timely::execute`, with the difference that
/// the worker configuration is provided by the parent executor and that the running
/// worker gains the ability to talk to the coordinator.
pub fn execute<T, F>(func: F) -> Result<WorkerGuards<T>, String>
where
    T: Send + 'static,
    F: Fn(&mut Root<Allocator>, Coordinator) -> T,
    F: Send + Sync + 'static,
{
    let config = Process::from_env().map_err(|err| {
        format!(
            concat!(
                "Failed to parse data from executor. ",
                "Has this binary been launched by an executor? ",
                "Error: {:?}"
            ),
            err
        )
    })?;

    // create timely configuration
    let timely_conf = if config.addrs.len() > 1 {
        info!(
            "Configuration:Cluster({}, {}/{})",
            config.threads,
            config.index,
            config.addrs.len() - 1
        );
        Configuration::Cluster(config.threads, config.index, config.addrs, true)
    } else if config.threads > 1 {
        info!("Configuration:Process({})", config.threads);
        Configuration::Process(config.threads)
    } else {
        info!("Configuration:Thread");
        Configuration::Thread
    };

    let coord = Coordinator::initialize(config.job_id, config.index, config.coord, config.hostname)
        .map_err(|err| format!("failed to connect to coordinator: {:?}", err))?;

    // wrap in mutex because the `Outgoing` is not `Sync` (due to std::mpsc)
    let coord = Mutex::new(coord);
    timely::execute(timely_conf, move |root| {
        let coord = coord.lock().unwrap().clone();
        func(root, coord)
    })
}

#[cfg(test)]
mod tests {
    use std::thread;
    use std::sync::Mutex;

    use futures::future::Future;
    use futures::stream::Stream;

    use timely;
    use timely::progress::timestamp::RootTimestamp;
    use timely::progress::nested::product::Product;
    use timely::dataflow::operators::{Delay, Map, ToStream};
    use timely::dataflow::operators::capture::{Capture, Extract};

    use strymon_communication::Network;

    use subscriber::{SubscriberGroup, SubscriptionEvent};
    use publisher::{Publisher, Addr};

    type ExampleTime = Product<RootTimestamp, u64>;

    fn publisher_thread(network: &Network) -> Addr {
        let (addr, publisher) = Publisher::<ExampleTime, String>::new(&network).unwrap();

        thread::spawn(move || {
            publisher.subscriber_barrier().expect(
                "publisher died unexpectedly",
            );

            let slot = Mutex::new(Some(publisher));
            timely::example(move |scope| {
                let publisher = slot.lock().unwrap().take().unwrap();
                (0..100)
                    .to_stream(scope)
                    .delay(|d, _| RootTimestamp::new((*d / 10) * 10))
                    .map(|d| d.to_string())
                    .capture_into(publisher);
            });
        });

        addr
    }

    #[test]
    fn raw_publisher() {
        let network = Network::new("localhost".to_string()).unwrap();
        let addr = publisher_thread(&network);

        use timely::dataflow::operators::generic::operator::source;
        use timely::dataflow::operators::CapabilitySet;

        let captured = timely::example(move |scope| {
            source(scope, "Source", |base_cap| {
                let mut capabilities = CapabilitySet::new();
                capabilities.insert(base_cap);

                let socket = network.connect((&*addr.0, addr.1)).unwrap();
                let connecting = SubscriberGroup::<ExampleTime, String>::new(Some(socket));
                let connected = connecting.wait().unwrap();
                let mut stream = connected.wait();

                move |output| {
                    // subscriber will drop remaining capabilities once the publisher is drained
                    if let Some(event) = stream.next() {
                        match event.unwrap() {
                            SubscriptionEvent::Data(time, data) => {
                                output.session(&capabilities.delayed(&time)).give_iterator(
                                    data.into_iter(),
                                );
                            }
                            SubscriptionEvent::FrontierUpdate => {
                                capabilities.downgrade(stream.get_ref().frontier());
                            }
                        }
                    }
                }
            }).capture()
        });

        let expected: Vec<(ExampleTime, Vec<String>)> = (0..10)
            .map(|t| t * 10)
            .map(|t| {
                (
                    RootTimestamp::new(t),
                    (0..10).map(|d| format!("{}", t + d)).collect(),
                )
            })
            .collect();
        assert_eq!(captured.extract(), expected);
    }
}