declarative-dataflow 0.2.0

A reactive query engine built on Differential Dataflow.
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
//! Server logic for driving the library via commands.

use std::cell::RefCell;
use std::collections::{HashMap, HashSet};
use std::hash::Hash;
use std::rc::Rc;
use std::time::{Duration, Instant};

use timely::communication::Allocate;
use timely::dataflow::operators::capture::event::link::EventLink;
use timely::dataflow::{ProbeHandle, Scope};
use timely::logging::{BatchLogger, TimelyEvent};
use timely::progress::Timestamp;
use timely::worker::Worker;

use differential_dataflow::collection::Collection;
use differential_dataflow::lattice::Lattice;
use differential_dataflow::logging::DifferentialEvent;

use crate::domain::Domain;
use crate::logging::DeclarativeEvent;
use crate::plan::ImplContext;
use crate::sinks::Sink;
use crate::sources::{Source, Sourceable, SourcingContext};
use crate::Rule;
use crate::{implement, implement_neu, AttributeConfig, RelationHandle, ShutdownHandle};
use crate::{Aid, Error, Rewind, Time, TxData, Value};
use crate::{TraceKeyHandle, TraceValHandle};

pub mod scheduler;
use self::scheduler::Scheduler;

/// Server configuration.
#[derive(Clone, Debug, Serialize, Deserialize)]
pub struct Configuration {
    /// Automatic domain tick interval.
    pub tick: Option<Duration>,
    /// Do clients have to call AdvanceDomain explicitely?
    pub manual_advance: bool,
    /// Should logging streams be created?
    pub enable_logging: bool,
    /// Should queries use the optimizer during implementation?
    pub enable_optimizer: bool,
}

impl Default for Configuration {
    fn default() -> Self {
        Configuration {
            tick: None,
            manual_advance: false,
            enable_logging: false,
            enable_optimizer: false,
        }
    }
}

#[cfg(feature = "getopts")]
impl Configuration {
    /// Returns a `getopts::Options` struct describing all available
    /// configuration options.
    pub fn options() -> getopts::Options {
        let mut opts = getopts::Options::new();

        opts.optopt(
            "",
            "tick",
            "advance domain at a regular interval",
            "SECONDS",
        );
        opts.optflag(
            "",
            "manual-advance",
            "forces clients to call AdvanceDomain explicitely",
        );
        opts.optflag("", "enable-logging", "enable log event sources");
        opts.optflag("", "enable-optimizer", "enable WCO queries");
        opts.optflag("", "enable-meta", "enable queries on the query graph");

        opts
    }

    /// Parses configuration options from the provided arguments.
    pub fn from_args<I: Iterator<Item = String>>(args: I) -> Result<Self, String> {
        let default: Self = Default::default();
        let opts = Self::options();

        let matches = opts.parse(args)?;

        let tick: Option<Duration> = matches
            .opt_str("tick")
            .map(|x| Duration::from_secs(x.parse().expect("failed to parse tick duration")));

        Self {
            tick,
            manual_advance: matches.opt_present("manual-advance"),
            enable_logging: matches.opt_present("enable-logging"),
            enable_optimizer: matches.opt_present("enable-optimizer"),
        }
    }
}

/// Transaction ids.
pub type TxId = u64;

/// A request expressing interest in receiving results published under
/// the specified name.
#[derive(Hash, PartialEq, Eq, PartialOrd, Ord, Clone, Debug, Serialize, Deserialize)]
pub struct Interest {
    /// The name of a previously registered dataflow.
    pub name: String,
    /// Granularity at which to send results. None indicates no delay.
    pub granularity: Option<Time>,
    /// An optional sink configuration.
    pub sink: Option<Sink>,
    /// Whether or not to log events from this dataflow.
    pub disable_logging: Option<bool>,
}

impl std::convert::From<&Interest> for crate::sinks::SinkingContext {
    fn from(interest: &Interest) -> Self {
        Self {
            name: interest.name.clone(),
            granularity: interest.granularity.clone(),
        }
    }
}

/// A request with the intent of synthesising one or more new rules
/// and optionally publishing one or more of them.
#[derive(Hash, PartialEq, Eq, PartialOrd, Ord, Clone, Debug, Serialize, Deserialize)]
pub struct Register {
    /// A list of rules to synthesise in order.
    pub rules: Vec<Rule>,
    /// The names of rules that should be published.
    pub publish: Vec<String>,
}

/// A request with the intent of creating a new named, globally
/// available input that can be transacted upon.
#[derive(Hash, PartialEq, Eq, PartialOrd, Ord, Clone, Debug, Serialize, Deserialize)]
pub struct CreateAttribute {
    /// A globally unique name under which to publish data sent via
    /// this input.
    pub name: String,
    /// Semantics enforced on this attribute by 3DF.
    pub config: AttributeConfig,
}

/// Possible request types.
#[derive(Hash, PartialEq, Eq, PartialOrd, Ord, Clone, Debug, Serialize, Deserialize)]
pub enum Request {
    /// Sends inputs via one or more registered handles.
    Transact(Vec<TxData>),
    /// Expresses interest in a named relation.
    Interest(Interest),
    /// Expresses that the interest in a named relation has
    /// stopped. Once all interested clients have sent this, the
    /// dataflow can be cleaned up.
    Uninterest(String),
    /// Registers one or more named relations.
    Register(Register),
    /// A request with the intent of attaching to an external data
    /// source that publishes one or more attributes and relations.
    RegisterSource(Source),
    /// Creates a named input handle that can be `Transact`ed upon.
    CreateAttribute(CreateAttribute),
    /// Advances the specified domain to the specified time.
    AdvanceDomain(Option<String>, Time),
    /// Requests a domain advance to whatever epoch the server
    /// determines is *now*. Used by clients to enforce a minimum
    /// granularity of responses, if inputs happen only infrequently.
    Tick,
    /// Closes a named input handle.
    CloseInput(String),
    /// Client has disconnected.
    Disconnect,
    /// Requests any setup logic that needs to be executed
    /// deterministically across all workers.
    Setup,
    /// Requests a heartbeat containing status information.
    Status,
    /// Requests orderly shutdown of the system.
    Shutdown,
}

/// Server context maintaining globally registered arrangements and
/// input handles.
pub struct Server<T, Token>
where
    T: Timestamp + Lattice,
    Token: Hash + Eq + Copy,
{
    /// Server configuration.
    pub config: Configuration,
    /// A timer started at the initation of the timely computation
    /// (copied from worker).
    pub t0: Instant,
    /// Implementation context.
    pub context: Context<T>,
    /// Mapping from query names to interested client tokens.
    pub interests: HashMap<String, HashSet<Token>>,
    // Mapping from query names to their shutdown handles.
    shutdown_handles: HashMap<String, ShutdownHandle>,
    /// Probe keeping track of overall dataflow progress.
    pub probe: ProbeHandle<T>,
    /// Scheduler managing deferred operator activations.
    pub scheduler: Rc<RefCell<Scheduler>>,
    // Link to replayable Timely logging events.
    timely_events: Option<Rc<EventLink<Duration, (Duration, usize, TimelyEvent)>>>,
    // Link to replayable Differential logging events.
    differential_events: Option<Rc<EventLink<Duration, (Duration, usize, DifferentialEvent)>>>,
}

/// Implementation context.
pub struct Context<T>
where
    T: Timestamp + Lattice,
{
    /// Representation of named rules.
    pub rules: HashMap<Aid, Rule>,
    /// Set of rules known to be underconstrained.
    pub underconstrained: HashSet<Aid>,
    /// Internal domain of command sequence numbers.
    pub internal: Domain<T>,
}

impl<T> ImplContext<T> for Context<T>
where
    T: Timestamp + Lattice,
{
    fn rule(&self, name: &str) -> Option<&Rule> {
        self.rules.get(name)
    }

    fn global_arrangement(&mut self, name: &str) -> Option<&mut RelationHandle<T>> {
        self.internal.arrangements.get_mut(name)
    }

    fn has_attribute(&self, name: &str) -> bool {
        self.internal.attributes.contains_key(name)
    }

    fn forward_count(&mut self, name: &str) -> Option<&mut TraceKeyHandle<Value, T, isize>> {
        self.internal.forward_count.get_mut(name)
    }

    fn forward_propose(
        &mut self,
        name: &str,
    ) -> Option<&mut TraceValHandle<Value, Value, T, isize>> {
        self.internal.forward_propose.get_mut(name)
    }

    fn forward_validate(
        &mut self,
        name: &str,
    ) -> Option<&mut TraceKeyHandle<(Value, Value), T, isize>> {
        self.internal.forward_validate.get_mut(name)
    }

    fn reverse_count(&mut self, name: &str) -> Option<&mut TraceKeyHandle<Value, T, isize>> {
        self.internal.reverse_count.get_mut(name)
    }

    fn reverse_propose(
        &mut self,
        name: &str,
    ) -> Option<&mut TraceValHandle<Value, Value, T, isize>> {
        self.internal.reverse_propose.get_mut(name)
    }

    fn reverse_validate(
        &mut self,
        name: &str,
    ) -> Option<&mut TraceKeyHandle<(Value, Value), T, isize>> {
        self.internal.reverse_validate.get_mut(name)
    }

    fn is_underconstrained(&self, _name: &str) -> bool {
        // self.underconstrained.contains(name)
        true
    }
}

impl<T, Token> Server<T, Token>
where
    T: Timestamp + Lattice + Default + Rewind,
    Token: Hash + Eq + Copy,
{
    /// Creates a new server state from a configuration.
    pub fn new(config: Configuration) -> Self {
        Server::new_at(config, Instant::now())
    }

    /// Creates a new server state from a configuration with an
    /// additionally specified beginning of the computation: an
    /// instant in relation to which all durations will be measured.
    pub fn new_at(config: Configuration, t0: Instant) -> Self {
        let timely_events = Some(Rc::new(EventLink::new()));
        let differential_events = Some(Rc::new(EventLink::new()));

        Server {
            config,
            t0,
            context: Context {
                rules: HashMap::new(),
                internal: Domain::new(Default::default()),
                underconstrained: HashSet::new(),
            },
            interests: HashMap::new(),
            shutdown_handles: HashMap::new(),
            probe: ProbeHandle::new(),
            scheduler: Rc::new(RefCell::new(Scheduler::new())),
            timely_events,
            differential_events,
        }
    }

    /// Returns commands to install built-in plans.
    pub fn builtins() -> Vec<Request> {
        vec![
            // Request::CreateAttribute(CreateAttribute {
            //     name: "df.pattern/e".to_string(),
            //     semantics: InputSemantics::Raw,
            // }),
            // Request::CreateAttribute(CreateAttribute {
            //     name: "df.pattern/a".to_string(),
            //     semantics: InputSemantics::Raw,
            // }),
            // Request::CreateAttribute(CreateAttribute {
            //     name: "df.pattern/v".to_string(),
            //     semantics: InputSemantics::Raw,
            // }),
        ]
    }

    /// Drops all shutdown handles associated with the specified
    /// query, resulting in its dataflow getting cleaned up.
    fn shutdown_query(&mut self, name: &str) {
        info!("Shutting down {}", name);
        self.shutdown_handles.remove(name);
    }

    /// Handle a Transact request.
    pub fn transact(
        &mut self,
        tx_data: Vec<TxData>,
        owner: usize,
        worker_index: usize,
    ) -> Result<(), Error> {
        // only the owner should actually introduce new inputs
        if owner == worker_index {
            self.context.internal.transact(tx_data)
        } else {
            Ok(())
        }
    }

    /// Handles an Interest request.
    pub fn interest<S: Scope<Timestamp = T>>(
        &mut self,
        name: &str,
        scope: &mut S,
    ) -> Result<Collection<S, Vec<Value>, isize>, Error> {
        // We need to do a `contains_key` here to avoid taking
        // a mut ref on context.
        if self.context.internal.arrangements.contains_key(name) {
            // Rule is already implemented.
            let relation = self
                .context
                .global_arrangement(name)
                .unwrap()
                .import_named(scope, name)
                .as_collection(|tuple, _| tuple.clone());

            Ok(relation)
        } else {
            let (mut rel_map, shutdown_handle) = if self.config.enable_optimizer {
                implement_neu(name, scope, &mut self.context)?
            } else {
                implement(name, scope, &mut self.context)?
            };

            // @TODO when do we actually want to register result traces for re-use?
            // for (name, relation) in rel_map.into_iter() {
            // let trace = relation.map(|t| (t, ())).arrange_named(name).trace;
            //     self.context.register_arrangement(name, config, trace);
            // }

            match rel_map.remove(name) {
                None => Err(Error::fault(format!(
                    "Relation of interest ({}) wasn't actually implemented.",
                    name
                ))),
                Some(relation) => {
                    self.shutdown_handles
                        .insert(name.to_string(), shutdown_handle);

                    Ok(relation)
                }
            }
        }
    }

    /// Handle a Register request.
    pub fn register(&mut self, req: Register) -> Result<(), Error> {
        let Register { rules, .. } = req;

        for rule in rules.into_iter() {
            if self.context.rules.contains_key(&rule.name) {
                // @TODO panic if hashes don't match
                // panic!("Attempted to re-register a named relation");
                continue;
            } else {
                // if self.config.enable_meta {
                //     let mut data = rule.plan.datafy();
                //     let tx_data: Vec<TxData> = data
                //         .drain(..)
                //         .map(|(e, a, v)| TxData(1, Value::Eid(e), a, v, None))
                //         .collect();

                //     self.transact(tx_data, 0, 0)?;
                // }

                self.context.rules.insert(rule.name.to_string(), rule);
            }
        }

        Ok(())
    }

    /// Handle a RegisterSource request.
    pub fn register_source<S: Scope<Timestamp = T>>(
        &mut self,
        source: Box<dyn Sourceable<S>>,
        scope: &mut S,
    ) -> Result<(), Error> {
        // use timely::logging::Logger;
        // let timely_logger = scope.log_register().remove("timely");

        // let differential_logger = scope.log_register().remove("differential/arrange");

        let context = SourcingContext {
            t0: self.t0,
            scheduler: Rc::downgrade(&self.scheduler),
            domain_probe: self.context.internal.domain_probe().clone(),
            timely_events: self.timely_events.clone().unwrap(),
            differential_events: self.differential_events.clone().unwrap(),
        };

        // self.timely_events = None;
        // self.differential_events = None;

        let mut attribute_streams = source.source(scope, context);

        for (aid, config, datoms) in attribute_streams.drain(..) {
            self.context
                .internal
                .create_sourced_attribute(&aid, config, &datoms)?;
        }

        // if let Some(logger) = timely_logger {
        //     if let Ok(logger) = logger.downcast::<Logger<TimelyEvent>>() {
        //         scope
        //             .log_register()
        //             .insert_logger::<TimelyEvent>("timely", *logger);
        //     }
        // }

        // if let Some(logger) = differential_logger {
        //     if let Ok(logger) = logger.downcast::<Logger<DifferentialEvent>>() {
        //         scope
        //             .log_register()
        //             .insert_logger::<DifferentialEvent>("differential/arrange", *logger);
        //     }
        // }

        Ok(())
    }

    /// Handle an AdvanceDomain request.
    pub fn advance_domain(&mut self, name: Option<String>, next: T) -> Result<(), Error> {
        match name {
            None => self.context.internal.advance_epoch(next),
            Some(_) => Err(Error::unsupported("Named domains are not yet supported.")),
        }
    }

    /// Handles an Uninterest request, possibly cleaning up dataflows
    /// that are no longer interesting to any client.
    pub fn uninterest(&mut self, client: Token, name: &str) -> Result<(), Error> {
        // All workers keep track of every client's interests, s.t. they
        // know when to clean up unused dataflows.
        if let Some(entry) = self.interests.get_mut(name) {
            entry.remove(&client);

            if entry.is_empty() {
                self.shutdown_query(name);
                self.interests.remove(name);
            }
        }

        Ok(())
    }

    /// Cleans up all bookkeeping state for the specified client.
    pub fn disconnect_client(&mut self, client: Token) -> Result<(), Error> {
        let names: Vec<String> = self.interests.keys().cloned().collect();

        for query_name in names.iter() {
            self.uninterest(client, query_name)?
        }

        Ok(())
    }

    /// Returns true iff the probe is behind any input handle. Mostly
    /// used as a convenience method during testing. Using this within
    /// `step_while` is not safe in general and might lead to stalls.
    pub fn is_any_outdated(&self) -> bool {
        self.probe
            .with_frontier(|out_frontier| self.context.internal.dominates(out_frontier))
    }

    /// Helper for registering, publishing, and indicating interest in
    /// a single, named query. Used for testing.
    pub fn test_single<S: Scope<Timestamp = T>>(
        &mut self,
        scope: &mut S,
        rule: Rule,
    ) -> Collection<S, Vec<Value>, isize> {
        let interest_name = rule.name.clone();
        let publish_name = rule.name.clone();

        self.register(Register {
            rules: vec![rule],
            publish: vec![publish_name],
        })
        .unwrap();

        match self.interest(&interest_name, scope) {
            Err(error) => panic!("{:?}", error),
            Ok(relation) => relation.probe_with(&mut self.probe),
        }
    }
}

impl<Token> Server<Duration, Token>
where
    Token: Hash + Eq + Copy,
{
    /// Registers loggers for use in the various logging sources.
    pub fn enable_logging<A: Allocate>(&self, worker: &mut Worker<A>) -> Result<(), Error> {
        let mut timely_logger = BatchLogger::new(self.timely_events.clone().unwrap());
        worker
            .log_register()
            .insert::<TimelyEvent, _>("timely", move |time, data| {
                timely_logger.publish_batch(time, data)
            });

        let mut differential_logger = BatchLogger::new(self.differential_events.clone().unwrap());
        worker
            .log_register()
            .insert::<DifferentialEvent, _>("differential/arrange", move |time, data| {
                differential_logger.publish_batch(time, data)
            });

        Ok(())
    }

    /// Unregisters loggers.
    pub fn shutdown_logging<A: Allocate>(&self, worker: &mut Worker<A>) -> Result<(), Error> {
        worker
            .log_register()
            .insert::<TimelyEvent, _>("timely", move |_time, _data| {});

        worker
            .log_register()
            .insert::<DifferentialEvent, _>("differential/arrange", move |_time, _data| {});

        worker
            .log_register()
            .insert::<DeclarativeEvent, _>("declarative", move |_time, _data| {});

        Ok(())
    }
}