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
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
//! Dynamic scheduler.
//!
//! The dynamic scheduler picks the next circuit node to evaluate dynamically
//! based on three criteria:
//! 1. A node can only be evaluated after its predecessors.
//! 2. An async node can only be evaluated in a ready state (see
//!    [`Operator::ready`](`crate::circuit::operator_traits::Operator::ready`)).
//! 3. Pick a highest priority node among nodes that satisfy the first two
//!    conditions.
//!
//! Unlike [`StaticScheduler`](`crate::circuit::schedule::StaticScheduler`),
//! the dynamic scheduler blocks when there are no runnable nodes instead of
//! busy waiting.
//!
//! # Design
//!
//! ## Tasks
//!
//! We model the scheduling problem as a set of tasks.  Each task represents a
//! node in the circuit along with some metadata.  At runtime, the scheduler
//! tracks for each task the number of predecessor nodes that are yet to be
//! scheduled, and, for async nodes, their ready status.  Once criteria 1 and 2
//! above are satisfied, the scheduler moves the task to the run queue.
//!
//! ## Run queue
//!
//! The run queue is organized as a priority queue, with the scheduler picking
//! one of the highest-priority runnable tasks to run next.  Priority assignment
//! is heuristic.
//!
//! ## Notification processing
//!
//! The scheduler relies on notifications to determine when an async operator
//! becomes ready.  In order to minimize expensive synchronization when
//! processing notifications, we adopt an approach inspired by RCU locks.  The
//! event notifier simply records each notification in a set and unparks the
//! scheduler thread. Whenever the runnable queue becomes empty, the scheduler
//! scans the ready set marking operators ready and moving them to the runnable
//! queue when necessary. If the runnable queue is still empty, the scheduler
//! thread parks itself waiting for the next ready notification.
//!
//! ## Transactions
//!
//! As commit time, the scheduler tracks the number of unflushed predecessors for
//! each operator and flushes the operator when the number reaches zero. Once all
//! operators have been flushed, the scheduler waits for all parallel workers to
//! complete flushing their circuit before declaring the transaction committed.

use std::{
    cell::{Ref, RefCell, RefMut},
    collections::{BTreeSet, HashMap, HashSet},
    panic,
    sync::{Arc, Mutex},
};

use crate::{
    Position,
    circuit::{
        Circuit, GlobalNodeId, NodeId,
        circuit_builder::CircuitMetadata,
        runtime::{Broadcast, Runtime},
        schedule::{
            CommitProgress, Error, Scheduler,
            util::{circuit_graph, ownership_constraints},
        },
        trace::SchedulerEvent,
    },
};
use petgraph::algo::toposort;
use tokio::{select, sync::Notify, task::JoinSet};

#[derive(Debug)]
pub enum FlushState {
    /// The operator is waiting for all predecessors to be flushed.
    UnflushedDependencies(usize),
    /// `flush` has been called, but `flush_complete` hasn't yet returned
    /// `true`.
    Started(Option<Position>),
    /// Flush has been completed for the current transaction.
    Completed(Option<Position>),
}

/// A task is a unit of work scheduled by the dynamic scheduler.
/// It contains a reference to a node in the circuit and associated metadata.
struct Task {
    // Immutable fields (initialized once when preparing the scheduler).
    /// Circuit node to be scheduled.
    node_id: NodeId,

    /// The number of predecessors of the node in the circuit graph.
    /// All predecessors must be evaluated before the node can be evaluated.
    num_predecessors: usize,

    /// Successors of the node in the circuit graph.
    successors: Vec<NodeId>,

    /// `true` if this is an async node.  The node can only be evaluated in a
    /// ready state.
    is_async: bool,

    // Mutable fields reset on each step.
    /// Number of predecessors not yet evaluated.  Set to `num_predecessors`
    /// at the start of each step.
    unsatisfied_dependencies: usize,

    /// `true` if the async node is known to be in a ready state.  Always
    /// `true` for non-async nodes.
    is_ready: bool,

    /// Task has been scheduled (put on the run queue) in the current clock
    /// cycle.
    scheduled: bool,

    // Mutable fields reset on each transaction.
    flush_state: FlushState,
}

/// The set of async nodes for which the scheduler has received ready
/// notifications.
#[derive(Clone)]
struct Notifications {
    /// Nodes that received notifications.
    nodes: Arc<Mutex<HashSet<NodeId>>>,

    /// Notifier to wake up the scheduler thread when a notification arrives.
    notify: Arc<Notify>,
}

impl Notifications {
    fn new(size: usize) -> Self {
        Self {
            nodes: Arc::new(Mutex::new(HashSet::with_capacity(size))),
            notify: Arc::new(Notify::new()),
        }
    }

    /// Add a new notification.
    fn notify(&self, node_id: NodeId) {
        self.nodes.lock().unwrap().insert(node_id);
        self.notify.notify_one();
    }

    /// Wait for at least one notification.
    async fn wait(&self) {
        self.notify.notified().await
    }
}

enum TransactionPhase {
    /// Not started
    Idle,

    /// Started, but not yet committing.
    Started,

    /// Committing, but not yet complete.
    /// The value is the number of unflushed operators.
    Committing(usize),

    /// This scheduler, and all parallel workers have finished flushing
    /// all operators in the circuit.
    CommitComplete,
}

/// Dynamic scheduler internals.
struct Inner {
    // Immutable fields (initialized once when preparing the scheduler).
    /// Tasks that must be evaluated at each clock cycle.
    tasks: HashMap<NodeId, Task>,

    // Mutable fields.
    /// Ready notifications received while the scheduler was busy or sleeping.
    notifications: Notifications,

    /// Currently running tasks. Each task returns node id along with status,
    /// so that the scheduler can match it back to the circuit node.
    handles: JoinSet<(NodeId, Result<Option<Position>, Error>)>,

    /// True when the scheduler is waiting for at least one task to become ready.
    waiting: bool,

    transaction_phase: TransactionPhase,

    /// Used to synchronize commit completion and flush completion across all workers.
    ///
    /// This could be implemented using two separate consensus objects, but we combine them
    /// to reduce the number of communication cycles.
    ///
    /// The first bool is true if the commit is complete for the current transaction for
    /// the local circuit.
    ///
    /// The second bool is true if the circuit has received a flush request from the current
    /// _parent_ circuit transaction. Once all workers have completed the local transaction and
    /// report flush=true, the circuit can report is_flush_complete=true to the parent.
    global_commit_consensus: Broadcast<(bool, bool)>,

    /// Broadcast object used to exchange metadata with peers.
    metadata_broadcast: Broadcast<CircuitMetadata>,

    // True before the circuit has executed any steps.
    before_first_step: bool,

    flush_state: bool,
}

impl Inner {
    /// Called when task `node_id` has completed to update unsatisfied dependencies of
    /// all successors and spawn any tasks that are ready to run.
    fn schedule_successors<C>(&mut self, circuit: &C, node_id: NodeId, flush_complete: bool)
    where
        C: Circuit,
    {
        // Don't use iterator, as we will borrow `tasks` again below.
        for i in 0..self.tasks[&node_id].successors.len() {
            let succ_id = self.tasks[&node_id].successors[i];
            debug_assert!(self.tasks.contains_key(&succ_id));
            let successor = self.tasks.get_mut(&succ_id).unwrap();
            debug_assert_ne!(successor.unsatisfied_dependencies, 0);
            successor.unsatisfied_dependencies -= 1;
            if flush_complete {
                let FlushState::UnflushedDependencies(n) = &mut successor.flush_state else {
                    panic!(
                        "Internal scheduler error: node {node_id} is in state {:?} while it still has unflushed dependencies",
                        successor.flush_state
                    );
                };
                debug_assert!(*n > 0);
                *n -= 1;
            }
            if successor.unsatisfied_dependencies == 0 && successor.is_ready {
                self.spawn_task(circuit, succ_id);
            }
        }
    }

    /// Process and dequeue new notifications.
    fn process_notifications<C>(&mut self, circuit: &C)
    where
        C: Circuit,
    {
        // Don't hold the mutex during iteration below to avoid reentering the lock.
        let mut nodes = std::mem::take(&mut *self.notifications.nodes.lock().unwrap());

        // False positive via rust-clippy/#8963
        #[allow(unknown_lints)]
        #[allow(clippy::significant_drop_in_scrutinee)]
        for id in nodes.drain() {
            let task = self.tasks.get_mut(&id).unwrap();
            debug_assert!(task.is_async);

            // Ignore duplicate notifications.
            if task.is_ready {
                continue;
            }

            // Ignore spurious notifications.
            if circuit.ready(id) {
                task.is_ready = true;

                let node_id = task.node_id;

                // We can see a notification for an already scheduled task
                // indicating that it's become ready again.
                // This notification should take effect at the next clock
                // cycle.
                if task.unsatisfied_dependencies == 0 && !task.scheduled {
                    self.spawn_task(circuit, node_id);
                }
            }
        }
    }

    fn prepare<C>(circuit: &C, nodes: Option<&BTreeSet<NodeId>>) -> Result<Self, Error>
    where
        C: Circuit,
    {
        let nodes = nodes
            .map(|nodes| nodes.iter().cloned().collect::<BTreeSet<_>>())
            .unwrap_or_else(|| BTreeSet::from_iter(circuit.node_ids()));

        // Check that ownership constraints don't introduce cycles.
        let mut g: petgraph::prelude::GraphMap<NodeId, (), petgraph::Directed> =
            circuit_graph(circuit);

        // println!("g: {g:#?}");

        let extra_constraints = ownership_constraints(circuit)?;

        for (from, to) in extra_constraints.iter() {
            g.add_edge(*from, *to, ());
        }

        // `toposort` fails if the graph contains cycles.
        toposort(&g, None).map_err(|e| Error::CyclicCircuit {
            node_id: GlobalNodeId::child_of(circuit, e.node_id()),
        })?;

        let num_nodes = nodes.len();
        let mut successors: HashMap<NodeId, Vec<NodeId>> = HashMap::with_capacity(num_nodes);
        let mut predecessors: HashMap<NodeId, Vec<NodeId>> = HashMap::with_capacity(num_nodes);
        circuit.edges().iter().for_each(|edge| {
            if let Some(stream) = &edge.stream {
                // println!("clearing consumer count for stream {}", stream.stream_id());
                stream.clear_consumer_count();
            }
        });

        for edge in circuit.edges().iter() {
            if nodes.contains(&edge.to) && nodes.contains(&edge.from) {
                successors.entry(edge.from).or_default().push(edge.to);

                predecessors.entry(edge.to).or_default().push(edge.from);
                if let Some(stream) = &edge.stream {
                    // println!(
                    //     "Registering {} as consumer for stream {}",
                    //     edge.to,
                    //     stream.stream_id()
                    // );
                    stream.register_consumer();
                }
            }
        }

        // Add ownership constraints to the graph.
        for (from, to) in extra_constraints.into_iter() {
            if nodes.contains(&to) && nodes.contains(&from) {
                successors.entry(from).or_default().push(to);
                predecessors.entry(to).or_default().push(from);
            }
        }

        let mut tasks = HashMap::new();
        let mut num_async_nodes = 0;

        for &node_id in nodes.iter() {
            let num_predecessors = predecessors.entry(node_id).or_default().len();

            let is_async = circuit.is_async_node(node_id);
            if is_async {
                num_async_nodes += 1;
            }

            tasks.insert(
                node_id,
                Task {
                    node_id,
                    num_predecessors,
                    successors: successors.entry(node_id).or_default().clone(),
                    is_async,
                    unsatisfied_dependencies: num_predecessors,
                    is_ready: !is_async,
                    scheduled: false,
                    flush_state: FlushState::UnflushedDependencies(num_predecessors),
                },
            );
        }

        let scheduler = Self {
            tasks,
            notifications: Notifications::new(num_async_nodes),
            handles: JoinSet::new(),
            waiting: false,
            transaction_phase: TransactionPhase::Idle,
            global_commit_consensus: Broadcast::new(),
            metadata_broadcast: Broadcast::new(),
            before_first_step: true,
            flush_state: false,
        };

        // Setup scheduler callbacks.
        for &node_id in nodes.iter() {
            if circuit.is_async_node(node_id) {
                let notifications = scheduler.notifications.clone();
                circuit.register_ready_callback(
                    node_id,
                    Box::new(move || notifications.notify(node_id)),
                );

                // Since we missed any earlier notifications, generate one for
                // each ready node.
                if circuit.ready(node_id) {
                    scheduler.notifications.notify(node_id);
                }
            }
        }

        if circuit.root_scope() == 0 {
            circuit.balancer().prepare(circuit);
        }

        Ok(scheduler)
    }

    fn commit_complete(&self) -> bool {
        matches!(self.transaction_phase, TransactionPhase::CommitComplete)
    }

    fn transaction_started(&self) -> bool {
        matches!(self.transaction_phase, TransactionPhase::Started)
    }

    fn transaction_in_progress(&self) -> bool {
        matches!(
            self.transaction_phase,
            TransactionPhase::Started | TransactionPhase::Committing(_)
        )
    }

    /// Spawn a tokio task to evaluate `node_id`.
    fn spawn_task<C>(&mut self, circuit: &C, node_id: NodeId)
    where
        C: Circuit,
    {
        let task = self.tasks.get_mut(&node_id).unwrap();
        debug_assert_eq!(task.unsatisfied_dependencies, 0);
        debug_assert!(task.is_ready);
        debug_assert!(!task.scheduled);

        task.scheduled = true;

        if self.handles.is_empty() && self.waiting {
            self.waiting = false;
            circuit.log_scheduler_event(&SchedulerEvent::wait_end(circuit.global_id()));
        }

        let circuit = circuit.clone();

        let committing = matches!(self.transaction_phase, TransactionPhase::Committing(_));

        if committing && matches!(task.flush_state, FlushState::UnflushedDependencies(0)) {
            circuit.flush_node(node_id);
            task.flush_state = FlushState::Started(None);
        }

        self.handles.spawn_local(async move {
            let result = circuit.eval_node(node_id).await;
            (node_id, result)
        });
    }

    async fn abort(&mut self) {
        // Wait for all started tasks to abort.
        // TODO: we could use self.handles.abort_all() instead to terminate any slow (or stuck),
        // IO operations, but that requires all operators to handle cancellation gracefully.
        while !self.handles.is_empty() {
            let _ = self.handles.join_next().await;
        }
    }

    fn start_transaction<C>(&mut self, circuit: &C)
    where
        C: Circuit,
    {
        // Reset unflushed dependencies.
        for task in self.tasks.values_mut() {
            task.flush_state = FlushState::UnflushedDependencies(task.num_predecessors);
        }
        self.transaction_phase = TransactionPhase::Started;

        circuit.notify_start_transaction();

        if circuit.root_scope() == 0 {
            circuit.balancer().start_transaction();
        }
    }

    fn start_commit_transaction(&mut self) -> Result<(), Error> {
        if !self.transaction_started() {
            return Err(Error::CommitWithoutTransaction);
        }
        self.transaction_phase = TransactionPhase::Committing(self.tasks.len());

        Ok(())
    }

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

    fn commit_progress(&self) -> CommitProgress {
        let mut commit_progress = CommitProgress::new();

        for (node_id, task) in self.tasks.iter() {
            match &task.flush_state {
                FlushState::UnflushedDependencies(_) => commit_progress.add_remaining(*node_id),
                FlushState::Completed(progress) => {
                    commit_progress.add_completed(*node_id, progress.clone())
                }
                FlushState::Started(progress) => {
                    commit_progress.add_in_progress(*node_id, progress.clone())
                }
            }
        }
        // if Runtime::worker_index() == 0 {
        //     println!(
        //         "{}",
        //         serde_json::to_string(&circuit.metadata_exchange().get_global_metadata()).unwrap()
        //     );
        // }

        commit_progress
    }

    async fn exchange_metadata<C>(&mut self, circuit: &C) -> Result<(), Error>
    where
        C: Circuit,
    {
        if circuit.root_scope() != 0 {
            return Ok(());
        }

        let metadata = circuit.metadata_exchange().local_metadata().clone();
        let global_metadata = self.metadata_broadcast.collect(metadata).await?;
        circuit
            .metadata_exchange()
            .set_global_metadata(global_metadata);
        Ok(())
    }

    fn flush(&mut self) {
        self.flush_state = true;
    }

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

    async fn step<C>(&mut self, circuit: &C) -> Result<(), Error>
    where
        C: Circuit,
    {
        if !self.transaction_in_progress() {
            return Err(Error::StepWithoutTransaction);
        }

        circuit.log_scheduler_event(&SchedulerEvent::step_start(circuit.global_id()));

        if self.before_first_step {
            self.before_first_step = false;
            self.exchange_metadata(circuit).await?;
            if circuit.root_scope() == 0 {
                circuit.balancer().update_metadata();
            }
        }
        if circuit.root_scope() == 0 {
            circuit.balancer().start_step();
        }
        let result = self.do_step(circuit).await;

        // Exchange metadata with peers.
        self.exchange_metadata(circuit).await?;
        if circuit.root_scope() == 0 {
            circuit.balancer().update_metadata();
        }

        if let TransactionPhase::Committing(unflushed_operators) = &self.transaction_phase {
            let statuses = self
                .global_commit_consensus
                .collect((*unflushed_operators == 0, self.flush_state))
                .await?;

            let commit_complete = statuses
                .iter()
                .all(|(commit_complete, _flush_complete)| *commit_complete);

            if commit_complete {
                self.transaction_phase = TransactionPhase::CommitComplete;
                let flush_complete = statuses
                    .iter()
                    .all(|(_commit_complete, flush_complete)| *flush_complete);
                if flush_complete {
                    self.flush_state = false;
                }

                if circuit.root_scope() == 0 {
                    circuit.balancer().transaction_committed();
                }
            }
        }

        circuit.log_scheduler_event(&SchedulerEvent::step_end(circuit.global_id()));
        if self.commit_complete() {
            circuit.tick();
        }
        result
    }

    async fn do_step<C>(&mut self, circuit: &C) -> Result<(), Error>
    where
        C: Circuit,
    {
        let mut completed_tasks = 0;
        self.waiting = false;

        // Special case: empty circuit.
        if self.tasks.is_empty() {
            return Ok(());
        }

        let mut spawn = Vec::with_capacity(self.tasks.len());

        // Reset unsatisfied dependencies, initialize runnable queue.
        for task in self.tasks.values_mut() {
            task.unsatisfied_dependencies = task.num_predecessors;
            task.scheduled = false;

            if task.unsatisfied_dependencies == 0 && task.is_ready {
                spawn.push(task.node_id);
            }
        }

        for node_id in spawn.into_iter() {
            self.spawn_task(circuit, node_id);
        }

        loop {
            select! {
                ret = self.handles.join_next(), if !self.handles.is_empty() => {
                    completed_tasks += 1;

                    // The !self.handles.is_empty() check above should guarantee that this
                    // cannot be None.
                    let result = ret.expect("JoinSet::join_next returned None on a non-empty join set.");

                    let (node_id, task_result) = match result {
                        Err(error) => {
                            self.abort().await;
                            if error.is_panic() {
                                // Give our panic handler a chance to handle this.
                                panic::resume_unwind(error.into_panic());
                            } else {
                                return Err(Error::TokioError { error: error.to_string() });
                            }
                        },
                        Ok(result) => result
                    };

                    if self.tasks[&node_id].is_async {
                        self.tasks.get_mut(&node_id).unwrap().is_ready = false;
                    }

                    let progress = match task_result {
                        Ok(progress) => progress,
                        Err(e) => {
                            self.abort().await;
                            return Err(e);
                        }
                    };

                    if Runtime::kill_in_progress() {
                        self.abort().await;
                        return Err(Error::Killed);
                    }

                    let flush_complete = match &mut self.tasks.get_mut(&node_id).unwrap().flush_state {
                        flush_state@FlushState::Started(_) if circuit.is_flush_complete(node_id) => {
                            *flush_state = FlushState::Completed(progress);
                            let TransactionPhase::Committing(ref mut unflushed_operators) = self.transaction_phase else {
                                panic!("Internal scheduler error: flush called while not committing");
                            };
                            *unflushed_operators -= 1;
                            true
                        }
                        FlushState::Started(prog) => {
                            *prog = progress;
                            false
                        }
                        _ => false
                    };

                    // Are we done?
                    debug_assert!(completed_tasks <= self.tasks.len());
                    if completed_tasks == self.tasks.len() {
                        return Ok(());
                    }

                    // Spawn any new tasks that are now ready to run.
                    self.schedule_successors(circuit, node_id, flush_complete);

                    // No tasks are ready to run -- account any time until we have something to run as wait time
                    if self.handles.is_empty() {
                        self.waiting = true;
                        circuit.log_scheduler_event(
                            &SchedulerEvent::wait_start(circuit.global_id()));
                    }
                }
                _ = self.notifications.wait() => {
                    self.process_notifications(circuit);
                }
            }
        }
    }
}

pub struct DynamicScheduler(Option<RefCell<Inner>>);

impl DynamicScheduler {
    fn inner(&self) -> Ref<'_, Inner> {
        self.0
            .as_ref()
            .expect("DynamicScheduler: prepare() must be called before running the circuit")
            .borrow()
    }

    fn inner_mut(&self) -> RefMut<'_, Inner> {
        self.0
            .as_ref()
            .expect("DynamicScheduler: prepare() must be called before running the circuit")
            .borrow_mut()
    }
}

impl Scheduler for DynamicScheduler {
    fn new() -> Self {
        Self(None)
    }

    fn prepare<C>(&mut self, circuit: &C, nodes: Option<&BTreeSet<NodeId>>) -> Result<(), Error>
    where
        C: Circuit,
    {
        self.0 = Some(RefCell::new(Inner::prepare(circuit, nodes)?));
        Ok(())
    }

    async fn start_transaction<C>(&self, circuit: &C) -> Result<(), Error>
    where
        C: Circuit,
    {
        let inner = &mut *self.inner_mut();

        inner.start_transaction(circuit);

        Ok(())
    }

    #[allow(clippy::await_holding_refcell_ref)]
    async fn step<C>(&self, circuit: &C) -> Result<(), Error>
    where
        C: Circuit,
    {
        let inner = &mut *self.inner_mut();

        inner.step(circuit).await
    }

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

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

    fn commit_progress(&self) -> super::CommitProgress {
        self.inner().commit_progress()
    }

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

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