dbsp 0.287.0

Continuous streaming analytics engine
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
//! The scheduling framework controls the execution of a circuit at runtime.

#![allow(async_fn_in_trait)]

use super::{Circuit, GlobalNodeId, NodeId, trace::SchedulerEvent};
use crate::{DetailedError, Position};
use feldera_types::transaction::CommitProgressSummary;
use itertools::Itertools;
use serde::Serialize;
use std::{
    borrow::Cow,
    collections::{BTreeMap, BTreeSet},
    error::Error as StdError,
    fmt::{Display, Error as FmtError, Formatter},
    future::Future,
    pin::Pin,
    string::ToString,
};

mod dynamic_scheduler;
pub use dynamic_scheduler::DynamicScheduler;

/// Scheduler errors.
#[derive(Clone, Debug, Eq, PartialEq, Serialize)]
#[serde(untagged)]
pub enum Error {
    /// `origin` node has more than one strong successors who insist on
    /// consuming its output by value
    /// (`OwnershipPreference::STRONGLY_PREFER_OWNED` or higher).
    OwnershipConflict {
        origin: GlobalNodeId,
        consumers: Vec<GlobalNodeId>,
    },
    /// Ownership constraints introduce a cycle in the circuit graph.
    CyclicCircuit {
        node_id: GlobalNodeId,
    },
    CommitWithoutTransaction,
    StepWithoutTransaction,
    /// Execution of the circuit interrupted by the user (via
    /// [`RuntimeHandle::kill`](`crate::circuit::RuntimeHandle::kill`)).
    Killed,
    TokioError {
        error: String,
    },
    /// Replay info conflict.
    ///
    /// All workers should enter the replay mode with identical sets of replay and
    /// backfill nodes. If this is not the case, this indicates a bug or a corrupted
    /// checkpoint.
    ReplayInfoConflict {
        error: String,
    },
}

impl DetailedError for Error {
    fn error_code(&self) -> Cow<'static, str> {
        match self {
            Self::OwnershipConflict { .. } => Cow::from("OwnershipConflict"),
            Self::CyclicCircuit { .. } => Cow::from("CyclicCircuit"),
            Self::CommitWithoutTransaction => Cow::from("CommitWithoutTransaction"),
            Self::StepWithoutTransaction => Cow::from("StepWithoutTransaction"),
            Self::Killed => Cow::from("Killed"),
            Self::TokioError { .. } => Cow::from("TokioError"),
            Self::ReplayInfoConflict { .. } => Cow::from("ReplayInfoConflict"),
        }
    }
}

impl Display for Error {
    fn fmt(&self, f: &mut Formatter<'_>) -> Result<(), FmtError> {
        match self {
            Self::OwnershipConflict { origin, consumers } => {
                write!(
                    f,
                    "ownership conflict: output of node '{origin}' is consumed by value by the following nodes: [{}]",
                    consumers.iter().map(ToString::to_string).format(",")
                )
            }
            Self::CyclicCircuit { node_id } => {
                write!(
                    f,
                    "unschedulable circuit due to a cyclic topology: cycle through node '{node_id}'"
                )
            }
            Error::CommitWithoutTransaction => {
                f.write_str("commit invoked outside of a transaction")
            }
            Error::StepWithoutTransaction => f.write_str("step called outside of a transaction"),
            Self::Killed => f.write_str("circuit has been killed by the user"),
            Self::TokioError { error } => write!(f, "tokio error: {error}"),
            Self::ReplayInfoConflict { error } => {
                write!(f, "replay info conflict: {error}")
            }
        }
    }
}

impl StdError for Error {}

/// Progress toward committing a transaction.
#[derive(Debug)]
pub struct CommitProgress {
    /// Nodes that have been fully flushed along with the latest positions when flush completed.
    completed: BTreeMap<NodeId, Option<Position>>,

    /// Nodes that are currently being flushed, along with the current positions.
    in_progress: BTreeMap<NodeId, Option<Position>>,

    /// Nodes that are yet to be flushed.
    remaining: BTreeSet<NodeId>,
}

impl Default for CommitProgress {
    fn default() -> Self {
        Self::new()
    }
}

impl CommitProgress {
    pub fn new() -> Self {
        Self {
            completed: BTreeMap::new(),
            in_progress: BTreeMap::new(),
            remaining: BTreeSet::new(),
        }
    }

    pub fn add_remaining(&mut self, node_id: NodeId) {
        self.remaining.insert(node_id);
    }

    pub fn add_completed(&mut self, node_id: NodeId, progress: Option<Position>) {
        debug_assert!(!self.completed.contains_key(&node_id));
        debug_assert!(!self.in_progress.contains_key(&node_id));
        debug_assert!(!self.remaining.contains(&node_id));

        self.completed.insert(node_id, progress);
    }

    pub fn add_in_progress(&mut self, node_id: NodeId, progress: Option<Position>) {
        debug_assert!(!self.completed.contains_key(&node_id));
        debug_assert!(!self.in_progress.contains_key(&node_id));
        debug_assert!(!self.remaining.contains(&node_id));

        self.in_progress.insert(node_id, progress);
    }

    pub fn get_in_progress(&self) -> &BTreeMap<NodeId, Option<Position>> {
        &self.in_progress
    }

    pub fn summary(&self) -> CommitProgressSummary {
        let completed = self.completed.len() as u64;
        let in_progress = self.in_progress.len() as u64;
        let remaining = self.remaining.len() as u64;
        let in_progress_processed_records = self
            .in_progress
            .values()
            .map(|progress| progress.as_ref().map(|p| p.offset).unwrap_or_default())
            .sum();

        let in_progress_total_records = self
            .in_progress
            .values()
            .map(|progress| progress.as_ref().map(|p| p.total).unwrap_or_default())
            .sum();

        CommitProgressSummary {
            completed,
            in_progress,
            remaining,
            in_progress_processed_records,
            in_progress_total_records,
        }
    }
}

/// A scheduler defines the order in which nodes in a circuit are evaluated at
/// runtime.
///
/// Schedulers are per-[`Circuit`], not per-[`Runtime`](crate::Runtime).  They
/// decide the order in which a step through the circuit evaluates operators.
/// They do not affect process or thread scheduling.
///
/// This API supports two-level scheduling of circuits: **steps** and **transactions**.
///
/// ## Steps
///
/// During a step, the scheduler evaluates each node exactly once, after all of its
/// upstream nodes have been evaluated.  Note that this works for circuits with logical
/// cycles, as all such cycles must contain a [strict
/// operator](`crate::circuit::operator_traits::StrictOperator`), which maps
/// into a pair of source and sink nodes, so that the resulting circuit is still
/// acyclic and output of the strict operator is evaluated before feed input to
/// it.  In addition, the scheduler must wait for an async operator to be in a
/// ready state before evaluating it (see
/// [`Operator::is_async`](`crate::circuit::operator_traits::Operator`)).
///
/// ## Transactions
///
/// A transaction is a sequence of steps that evaluate a set of inputs for a single logical
/// timestamp to completion.
///
/// Transaction lifecycle:
///
/// ```text
///                              is_commit_complete() = true
///    ┌────────────────────────────────────────────────────────────────────────────────────┐
///    ▼                                                                                    │
/// ┌───────┐      start_transaction()      ┌───────────┐ start_commit_transaction()  ┌─────┴────┐
/// │ idle  ├──────────────────────────────►│  started  ├────────────────────────────►│committing│
/// └───────┘                               └────────┬──┘                             └─────────┬┘
///                                           ▲      │                                    ▲     │
///                                           └──────┘                                    └─────┘
///                                            step()                                      step()
/// ```
///
/// During the in-progress phase, each operator gets to decide how much of the input to process.
/// Some operators may accumulate inputs to process them later.
///
/// During the committing phase, the scheduler forces operators to process their inputs to completion
/// by invoking `flush` on each operator. Once all predecessor of an operator have finished processing
/// inputs for the current transaction, the scheduler invokes `flush` of the operator. It tracks the
/// frontier of flushed operators and reports `is_commit_complete()` as true once all operators have
/// been flushed.
pub trait Scheduler
where
    Self: Sized,
{
    fn new() -> Self;

    /// Initialize the scheduler for the circuit.
    ///
    /// Invoked before running the circuit to validate that the circuit is schedulable,
    /// e.g., doesn't contain circular scheduling dependencies, and to initialize any
    /// internal scheduler state.
    ///
    /// This function can be invoked multiple times, e.g., once before running the circuit
    /// in the backfill mode, and once before running the circuit in the normal mode.
    ///
    /// # Arguments
    ///
    /// * `nodes` - when specified, only the nodes in this set must be scheduled. All
    ///   other nodes remain idle.
    fn prepare<C>(&mut self, circuit: &C, nodes: Option<&BTreeSet<NodeId>>) -> Result<(), Error>
    where
        C: Circuit;

    /// Start a transaction.
    async fn start_transaction<C>(&self, circuit: &C) -> Result<(), Error>
    where
        C: Circuit;

    /// Start committing the current transaction.
    fn start_commit_transaction(&self) -> Result<(), Error>;

    /// Check if the current transaction is complete.
    ///
    /// Must be invoked after every `step` in the `committing` phase of the transaction.
    fn is_commit_complete(&self) -> bool;

    /// Estimated commit progress.
    fn commit_progress(&self) -> CommitProgress;

    /// Evaluate the circuit at runtime.
    ///
    /// Evaluates each node in the circuit exactly once in an order that
    /// respects (1) its dependency graph, and (2) the
    /// [`ready`](`crate::circuit::operator_traits::Operator::ready`) status
    /// of async operators.
    ///
    /// # Arguments
    ///
    /// * `circuit` - circuit to schedule, this must be the same circuit for
    ///   which the schedule was computed.
    async fn step<C>(&self, circuit: &C) -> Result<(), Error>
    where
        C: Circuit;

    async fn transaction<C>(&self, circuit: &C) -> Result<(), Error>
    where
        C: Circuit,
    {
        self.start_transaction(circuit).await?;
        self.start_commit_transaction()?;
        while !self.is_commit_complete() {
            self.step(circuit).await?;
        }

        Ok(())
    }

    fn flush(&self);

    fn is_flush_complete(&self) -> bool;
}

/// An executor executes a circuit by evaluating all of its operators using a
/// `Scheduler`. It can run the circuit exactly once or multiple times, until
/// some termination condition is reached.
pub trait Executor<C>: 'static {
    fn prepare(&mut self, circuit: &C, nodes: Option<&BTreeSet<NodeId>>) -> Result<(), Error>;

    /// Start a transaction.
    ///
    /// Only called on the executor in the top-level circuit.
    fn start_transaction<'a>(
        &'a self,
        circuit: &'a C,
    ) -> Pin<Box<dyn Future<Output = Result<(), Error>> + 'a>>;

    /// Start committing the current transaction.
    ///
    /// Only called on the executor in the top-level circuit.
    fn start_commit_transaction(&self) -> Result<(), Error>;

    /// Check if the current transaction is complete.
    ///
    /// Only called on the executor in the top-level circuit.
    fn is_commit_complete(&self) -> bool;

    /// Get the commit progress.
    ///
    /// Only called on the executor in the top-level circuit.
    fn commit_progress(&self) -> CommitProgress;

    fn step<'a>(&'a self, circuit: &'a C) -> Pin<Box<dyn Future<Output = Result<(), Error>> + 'a>>;

    fn transaction<'a>(
        &'a self,
        circuit: &'a C,
    ) -> Pin<Box<dyn Future<Output = Result<(), Error>> + 'a>>;

    /// The ChildNode operator that wraps this circuit is being flushed.
    ///
    /// Only called on nested circuits.
    fn flush(&self);

    /// Check if the current flush operation is complete.
    ///
    /// Only called on nested circuits.
    fn is_flush_complete(&self) -> bool;
}

/// An iterative executor evaluates the circuit until the `termination_check`
/// callback returns true.  Every time the executor is invoked, it first sends
/// the `clock_start` notification to all operators in the circuit. It then
/// evaluates the circuit until the termination condition is satisfied (but at
/// least once), and finally calls `clock_end` on it.
pub(crate) struct IterativeExecutor<F, S> {
    termination_check: F,
    scheduler: S,
}

impl<F, S> IterativeExecutor<F, S> {
    pub(crate) fn new(termination_check: F) -> Self
    where
        S: Scheduler,
    {
        Self {
            termination_check,
            scheduler: <S as Scheduler>::new(),
        }
    }
}

impl<C, F, S> Executor<C> for IterativeExecutor<F, S>
where
    F: AsyncFn() -> Result<bool, Error> + 'static,
    C: Circuit,
    S: Scheduler + 'static,
{
    /// Only called on the executor in the top-level circuit.
    fn start_transaction<'a>(
        &'a self,
        _circuit: &C,
    ) -> Pin<Box<dyn Future<Output = Result<(), Error>> + 'a>> {
        unimplemented!()
    }

    /// Only called on the executor in the top-level circuit.
    fn start_commit_transaction(&self) -> Result<(), Error> {
        unimplemented!()
    }

    /// Only called on the executor in the top-level circuit.
    fn commit_progress(&self) -> CommitProgress {
        unimplemented!()
    }

    /// Only called on the executor in the top-level circuit.
    fn is_commit_complete(&self) -> bool {
        unimplemented!()
    }

    fn step<'a>(&'a self, circuit: &'a C) -> Pin<Box<dyn Future<Output = Result<(), Error>> + 'a>> {
        let circuit = circuit.clone();
        Box::pin(async move { self.scheduler.step(&circuit).await })
    }

    fn transaction<'a>(
        &'a self,
        circuit: &C,
    ) -> Pin<Box<dyn Future<Output = Result<(), Error>> + 'a>> {
        let circuit = circuit.clone();
        Box::pin(async move {
            circuit.log_scheduler_event(&SchedulerEvent::clock_start());
            circuit.clock_start(0);

            loop {
                self.scheduler.transaction(&circuit).await?;
                if (self.termination_check)().await? {
                    break;
                }
            }

            circuit.log_scheduler_event(&SchedulerEvent::clock_end());
            circuit.clock_end(0);
            Ok(())
        })
    }

    fn prepare(&mut self, circuit: &C, nodes: Option<&BTreeSet<NodeId>>) -> Result<(), Error> {
        self.scheduler.prepare(circuit, nodes)
    }

    fn flush(&self) {
        self.scheduler.flush();
    }

    fn is_flush_complete(&self) -> bool {
        self.scheduler.is_flush_complete()
    }
}

/// An executor that evaluates the circuit exactly once every time it is
/// invoked.
pub(crate) struct OnceExecutor<S> {
    scheduler: S,
}

impl<S> OnceExecutor<S>
where
    S: Scheduler,
    Self: Sized,
{
    pub(crate) fn new() -> Self {
        Self {
            scheduler: <S as Scheduler>::new(),
        }
    }
}

impl<C, S> Executor<C> for OnceExecutor<S>
where
    C: Circuit,
    S: Scheduler + 'static,
{
    fn start_transaction<'a>(
        &'a self,
        circuit: &'a C,
    ) -> Pin<Box<dyn Future<Output = Result<(), Error>> + 'a>> {
        Box::pin(async { self.scheduler.start_transaction(circuit).await })
    }

    fn step<'a>(&'a self, circuit: &'a C) -> Pin<Box<dyn Future<Output = Result<(), Error>> + 'a>> {
        Box::pin(async { self.scheduler.step(circuit).await })
    }

    fn transaction<'a>(
        &'a self,
        circuit: &'a C,
    ) -> Pin<Box<dyn Future<Output = Result<(), Error>> + 'a>> {
        Box::pin(async { self.scheduler.transaction(circuit).await })
    }

    fn prepare(&mut self, circuit: &C, nodes: Option<&BTreeSet<NodeId>>) -> Result<(), Error> {
        self.scheduler.prepare(circuit, nodes)
    }

    fn start_commit_transaction(&self) -> Result<(), Error> {
        self.scheduler.start_commit_transaction()
    }

    fn is_commit_complete(&self) -> bool {
        self.scheduler.is_commit_complete()
    }

    fn commit_progress(&self) -> CommitProgress {
        self.scheduler.commit_progress()
    }

    /// Only called on nested circuits.
    fn flush(&self) {
        unimplemented!()
    }

    /// Only called on nested circuits.
    fn is_flush_complete(&self) -> bool {
        unimplemented!()
    }
}

/// Some useful tools for developing schedulers.
mod util {

    use crate::circuit::{
        Circuit, GlobalNodeId, NodeId, OwnershipPreference, circuit_builder::StreamId,
        schedule::Error,
    };
    use petgraph::graphmap::DiGraphMap;
    use std::{collections::HashMap, ops::Deref};

    /// Dump circuit topology as a graph.
    pub(crate) fn circuit_graph<C>(circuit: &C) -> DiGraphMap<NodeId, ()>
    where
        C: Circuit,
    {
        let mut g = DiGraphMap::<NodeId, ()>::new();

        for node_id in circuit.node_ids().into_iter() {
            g.add_node(node_id);
        }

        for edge in circuit.edges().deref().iter() {
            g.add_edge(edge.from, edge.to, ());
        }

        g
    }

    /// Helper function used by schedulers to enforce ownership preferences.
    ///
    /// Individual schedulers can implement their own algorithms to enforce (or
    /// ignore) ownership preferences.  This helper function can optionally be
    /// used by schedulers that wish to implement one particular approach.
    /// The idea is to treat **strong** ownership preferences
    /// (`OwnershipPreference::STRONGLY_PREFER_OWNED` and above) as scheduling
    /// constraints: assuming an operator has exactly one successor with a
    /// strong ownership preference ("strong successor"), we can enforce
    /// this preference by scheduling this successor last. This scheduling
    /// constraint can in turn be enforced by adding a dependency edge from
    /// all other successors to the strong successor node to the circuit graph.
    /// This function computes the set of dependency edges needed to enforce
    /// all such constraints in the circuit.
    ///
    /// # Caveat
    ///
    /// The additional edges computed by this function can introduce cycles to
    /// the circuit graph making it unschedulable.  Schedulers must check
    /// for cycles before adding these constraints and either fail or drop
    /// some of the constraints to eliminate cycles.
    ///
    /// (Current scheduler implementations fail if there is a cycle).
    ///
    /// # Errors
    ///
    /// The function fails with [`Error::OwnershipConflict`] if the circuit has
    /// at least one node with multiple strong successors.
    pub(crate) fn ownership_constraints<C>(circuit: &C) -> Result<Vec<(NodeId, NodeId)>, Error>
    where
        C: Circuit,
    {
        // Compute successors of each node in the circuit.  Note: we index successors by
        // origin id, not local node id, since the former uniquely identifies a
        // stream, but the latter doesn't, since a subcircuit node can have
        // multiple output streams.
        let num_nodes = circuit.num_nodes();
        let mut successors: HashMap<
            (GlobalNodeId, StreamId),
            Vec<(NodeId, Option<OwnershipPreference>)>,
        > = HashMap::with_capacity(num_nodes);

        for edge in circuit.edges().deref().iter() {
            let Some(stream_id) = edge.stream_id() else {
                continue;
            };
            let origin = edge.origin.clone();

            successors
                .entry((origin, stream_id))
                .or_default()
                .push((edge.to, edge.ownership_preference));
        }

        let mut constraints = Vec::new();

        for ((origin, _), succ) in successors.into_iter() {
            // Find all strong successors of a node.
            let strong_successors: Vec<_> = succ
                .iter()
                .enumerate()
                .filter(|(_i, (_, pref))| {
                    pref.is_some() && pref.unwrap() >= OwnershipPreference::STRONGLY_PREFER_OWNED
                })
                .collect();

            // Declare conflict if there's more than one strong successor.
            if strong_successors.len() > 1 {
                return Err(Error::OwnershipConflict {
                    origin,
                    consumers: strong_successors
                        .into_iter()
                        .map(|(_, (suc, _))| GlobalNodeId::child_of(circuit, *suc))
                        .collect(),
                });
            };

            // No strong successors -- nothing to do for this node.
            if strong_successors.is_empty() {
                continue;
            }

            // A unique strong successor found; add edges from all other successors to it.
            let strong_successor_index = strong_successors[0].0;
            for (i, successor) in succ.iter().enumerate() {
                // Ignore dependency edges.
                if i != strong_successor_index && successor.1.is_some() {
                    constraints.push((successor.0, succ[strong_successor_index].0));
                }
            }
        }

        Ok(constraints)
    }
}