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
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
//! Operators to organize time series data into windows.

use crate::{
    Error, Position, RootCircuit, Runtime,
    algebra::{IndexedZSet, NegByRef},
    circuit::{
        Circuit, GlobalNodeId, Scope, Stream,
        metadata::{BatchSizeStats, INPUT_BATCHES_STATS, OUTPUT_BATCHES_STATS, OperatorMeta},
        operator_traits::Operator,
        splitter_output_chunk_size,
    },
    dynamic::{
        ClonableTrait, DataTrait, DynData, WeightTrait,
        rkyv::{DeserializableDyn, SerializeDyn},
    },
    operator::{
        async_stream_operators::{StreamingTernaryOperator, StreamingTernaryWrapper},
        dynamic::{MonoIndexedZSet, trace::TraceBound},
        require_persistent_id,
    },
    storage::file::{SerializerInner, to_bytes},
    trace::{
        BatchFactories, BatchReader, BatchReaderFactories, Cursor, Spine, SpineSnapshot,
        spine_async::WithSnapshot,
    },
};
use async_stream::stream;
use feldera_storage::{FileCommitter, StoragePath};
use futures::Stream as AsyncStream;
use rkyv::Deserialize;
use std::{
    borrow::Cow,
    cell::{Cell, RefCell},
    marker::PhantomData,
    rc::Rc,
    sync::Arc,
};

impl Stream<RootCircuit, MonoIndexedZSet> {
    pub fn dyn_window_mono(
        &self,
        factories: &<MonoIndexedZSet as BatchReader>::Factories,
        inclusive: (bool, bool),
        bounds: &Stream<RootCircuit, (Box<DynData>, Box<DynData>)>,
    ) -> Stream<RootCircuit, MonoIndexedZSet> {
        self.dyn_window(factories, inclusive, bounds)
    }
}

impl<C, B> Stream<C, B>
where
    C: Circuit,
    B: IndexedZSet,
    Box<B::Key>: Clone,
{
    /// See [`Stream::window`].
    pub fn dyn_window(
        &self,
        factories: &B::Factories,
        inclusive: (bool, bool),
        bounds: &Stream<C, (Box<B::Key>, Box<B::Key>)>,
    ) -> Stream<C, B> {
        self.circuit()
            .region("window", || {
                let bound = TraceBound::new();
                let bound_clone = bound.clone();
                bounds.apply(move |(lower, _upper)| {
                    bound_clone.set(lower.clone());
                });
                let trace = self
                    .dyn_accumulate_integrate_trace_with_bound(factories, bound, TraceBound::new())
                    .accumulate_delay_trace();
                self.circuit().add_ternary_operator(
                    StreamingTernaryWrapper::new(<Window<B, SpineSnapshot<B>>>::new(
                        factories, inclusive,
                    )),
                    &trace,
                    &self.dyn_accumulate(factories),
                    bounds,
                )
            })
            .clone()
    }
}

/// A window that is serialized to a file.
#[derive(rkyv::Serialize, rkyv::Deserialize, rkyv::Archive)]
struct CommittedWindow {
    window: Option<(Vec<u8>, Vec<u8>)>,
}

impl<B: IndexedZSet, T: WithSnapshot<Batch = B>> From<&Window<B, T>> for CommittedWindow {
    fn from(value: &Window<B, T>) -> Self {
        // Transform the window bounds into a serialized form and store it as a byte vector.
        // This is necessary because the key type is not sized.
        let window = value.window.borrow().as_ref().map(|(a, b)| {
            let sa = SerializerInner::to_fbuf_with_thread_local(|s| a.serialize(s));
            let sb = SerializerInner::to_fbuf_with_thread_local(|s| b.serialize(s));
            (sa.into_vec(), sb.into_vec())
        });

        CommittedWindow { window }
    }
}

struct Window<B, T>
where
    B: IndexedZSet,
    T: WithSnapshot<Batch = B>,
{
    // For error reporting.
    global_id: GlobalNodeId,
    factories: B::Factories,
    left_inclusive: bool,
    right_inclusive: bool,
    // `None` means we're at the start of a clock epoch, no inputs
    // have been received yet, and window boundaries haven't been set.
    window: RefCell<Option<(Box<B::Key>, Box<B::Key>)>>,
    delta: RefCell<Option<SpineSnapshot<B>>>,
    flush: Cell<bool>,

    // Input batch sizes.
    input_batch_stats: RefCell<BatchSizeStats>,

    // Output batch sizes.
    output_batch_stats: RefCell<BatchSizeStats>,

    _phantom: PhantomData<(B, T)>,
}

impl<B, T> Window<B, T>
where
    B: IndexedZSet,
    T: WithSnapshot<Batch = B>,
{
    pub fn new(factories: &B::Factories, (left_inclusive, right_inclusive): (bool, bool)) -> Self {
        Self {
            global_id: GlobalNodeId::root(),
            factories: factories.clone(),
            left_inclusive,
            right_inclusive,
            window: RefCell::new(None),
            delta: RefCell::new(None),
            flush: Cell::new(false),
            input_batch_stats: RefCell::new(BatchSizeStats::new()),
            output_batch_stats: RefCell::new(BatchSizeStats::new()),

            _phantom: PhantomData,
        }
    }

    /// Return the absolute path of the file for a checkpointed Window.
    fn checkpoint_file(base: &StoragePath, persistent_id: &str) -> StoragePath {
        base.child(format!("window-{}.dat", persistent_id))
    }
}

impl<B, T> Operator for Window<B, T>
where
    B: IndexedZSet,
    T: WithSnapshot<Batch = B> + 'static,
{
    fn name(&self) -> Cow<'static, str> {
        Cow::from("Window")
    }

    fn init(&mut self, global_id: &GlobalNodeId) {
        self.global_id = global_id.clone();
    }

    fn clock_start(&mut self, _scope: Scope) {
        *self.window.borrow_mut() = None;
    }

    fn metadata(&self, meta: &mut OperatorMeta) {
        meta.extend(metadata! {
            INPUT_BATCHES_STATS => self.input_batch_stats.borrow().metadata(),
            OUTPUT_BATCHES_STATS => self.output_batch_stats.borrow().metadata(),
        });
    }

    fn fixedpoint(&self, _scope: Scope) -> bool {
        // Windows can currently only be used in top-level circuits.
        // Do we have meaningful examples of using windows inside nested scopes?
        panic!("'Window' operator used in fixedpoint iteration")
    }

    fn checkpoint(
        &mut self,
        base: &StoragePath,
        persistent_id: Option<&str>,
        files: &mut Vec<Arc<dyn FileCommitter>>,
    ) -> Result<(), Error> {
        let persistent_id = require_persistent_id(persistent_id, &self.global_id)?;
        let window_path = Self::checkpoint_file(base, persistent_id);

        let committed: CommittedWindow = (self as &Self).into();
        let as_bytes = to_bytes(&committed).expect("Serializing CommittedWindow should work.");
        files.push(
            Runtime::storage_backend()
                .unwrap()
                .write(&window_path, as_bytes)?,
        );
        Ok(())
    }

    fn restore(&mut self, base: &StoragePath, persistent_id: Option<&str>) -> Result<(), Error> {
        let persistent_id = require_persistent_id(persistent_id, &self.global_id)?;

        let window_path = Self::checkpoint_file(base, persistent_id);
        let content = Runtime::storage_backend().unwrap().read(&window_path)?;
        let archived = unsafe { rkyv::archived_root::<CommittedWindow>(&content) };
        let committed: CommittedWindow = archived.deserialize(&mut rkyv::Infallible).unwrap();

        *self.window.borrow_mut() = committed.window.map(|(a, b)| {
            // Serialize the window bounds back into the key.
            let mut boxed_a = self.factories.key_factory().default_box();
            let mut boxed_b = self.factories.key_factory().default_box();
            unsafe { boxed_a.deserialize_from_bytes(&a, 0) };
            unsafe { boxed_b.deserialize_from_bytes(&b, 0) };

            (boxed_a, boxed_b)
        });

        Ok(())
    }

    fn clear_state(&mut self) -> Result<(), Error> {
        *self.window.borrow_mut() = None;
        Ok(())
    }

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

/// `true` if cursor points to a key to the left of the interval.
fn before_start<K, V, T, R, C>(cursor: &C, start: &K, inclusive: bool) -> bool
where
    C: Cursor<K, V, T, R>,
    K: DataTrait + ?Sized,
    V: DataTrait + ?Sized,
    R: WeightTrait + ?Sized,
{
    if inclusive {
        cursor.key() < start
    } else {
        cursor.key() <= start
    }
}

/// `true` if cursor points to a key _not_ to the right of the interval.
fn before_end<K, V, T, R, C>(cursor: &C, end: &K, inclusive: bool) -> bool
where
    C: Cursor<K, V, T, R>,
    K: DataTrait + ?Sized,
    V: DataTrait + ?Sized,
    R: WeightTrait + ?Sized,
{
    if inclusive {
        cursor.key() <= end
    } else {
        cursor.key() < end
    }
}

/// Seek to the first location after the end of the interval.
fn seek_after_end<K, V, T, R, C>(cursor: &mut C, end: &K, inclusive: bool)
where
    C: Cursor<K, V, T, R>,
    K: DataTrait + ?Sized,
    V: DataTrait + ?Sized,
    R: WeightTrait + ?Sized,
{
    if inclusive {
        cursor.seek_key_with(&|key| key > end)
    } else {
        cursor.seek_key(end)
    }
}

/// Seek to the first location within the interval.
fn seek_start<K, V, T, R, C>(cursor: &mut C, start: &K, inclusive: bool)
where
    C: Cursor<K, V, T, R>,
    K: DataTrait + ?Sized,
    V: DataTrait + ?Sized,
    R: WeightTrait + ?Sized,
{
    if inclusive {
        cursor.seek_key(start)
    } else {
        cursor.seek_key_with(&|key| key > start)
    }
}

impl<B, T> StreamingTernaryOperator<T, Option<Spine<B>>, (Box<B::Key>, Box<B::Key>), B>
    for Window<B, T>
where
    B: IndexedZSet,
    T: WithSnapshot<Batch = B> + Clone + 'static,
    Box<B::Key>: Clone,
{
    /// * `batch` - input stream containing new time series data points indexed
    ///   by time.
    /// * `trace` - trace of the input stream up to, but not including current
    ///   clock cycle.
    /// * `bounds` - window bounds.  The lower bound must grow monotonically.
    // TODO: This can be optimized to add tuples in order, so we can use the
    // builder API to construct the output batch.  This requires processing
    // regions in order + extra care to iterate over `batch` and `trace` jointly
    // in region3.
    // TODO: Accurate progress tracking. We currently just yield `None` every time.
    fn eval(
        self: Rc<Self>,
        trace: Cow<'_, T>,
        delta: Cow<'_, Option<Spine<B>>>,
        bounds: Cow<'_, (Box<B::Key>, Box<B::Key>)>,
    ) -> impl AsyncStream<Item = (B, bool, Option<Position>)> + 'static {
        let chunk_size = splitter_output_chunk_size();

        if let Some(delta) = &*delta {
            assert!(self.delta.borrow().is_none());
            *self.delta.borrow_mut() = Some(delta.ro_snapshot());
        }

        let trace = if self.flush.get() {
            Some(trace.as_ref().ro_snapshot())
        } else {
            None
        };

        let (start1, end1) = bounds.into_owned();
        let bounds0: Option<_> = (*self.window.borrow()).clone();

        stream! {
            let delta = if self.flush.replace(false) {
                self.delta.take().unwrap()
            } else {
                yield (B::dyn_empty(&self.factories), true, None);
                return;
            };

            self.input_batch_stats.borrow_mut().add_batch(delta.len());

            //           ┌────────────────────────────────────────┐
            //           │       previous window                  │
            //           │                                        │             e1
            // ──────────┴────────────┬───────────────────────────┴──────────────┬─────►
            //          s0          s1│                          e0              │
            //                        │         new window                       │
            //                        └──────────────────────────────────────────┘
            //  region 1: [s0 .. s1)
            //  region 2: [s1 .. e0)
            //  region 3: [e0 .. e1)

            // println!("{:?}-{:?}", start1, end1);
            let trace = trace.as_ref();

            // TODO: In order to preallocate the buffer, we need to estimate the number of
            // keys in each component below.  For this, we need to extend
            // `Cursor::seek` to return the number of keys skipped over by the search.
            let mut tuples = self.factories.weighted_items_factory().default_box();
            tuples.reserve(chunk_size);

            let mut tuple = self.factories.weighted_item_factory().default_box();
            let mut key = self.factories.key_factory().default_box();

            let mut trace_cursor = trace.unwrap().cursor();
            let mut batch_cursor = delta.cursor();

            if let Some((start0, end0)) = bounds0 {
                // Retract tuples in `trace` that slid out of the window (region 1).
                seek_start(&mut trace_cursor, &start0, self.left_inclusive);
                while trace_cursor.key_valid()
                    && before_start(&trace_cursor, &start1, self.left_inclusive)
                    && before_end(&trace_cursor, &end0, self.right_inclusive)
                {
                    trace_cursor.key().clone_to(&mut *key);
                    while trace_cursor.val_valid(){
                        let (kv, w) = tuple.split_mut();
                        let (k, v) = kv.split_mut();
                        key.clone_to(k);
                        trace_cursor.val().clone_to(v);
                        **w = trace_cursor.weight().neg_by_ref();
                        tuples.push_val(&mut *tuple);

                        if tuples.len() >= chunk_size {
                            let result = B::dyn_from_tuples(&self.factories, (), &mut tuples);
                            self.output_batch_stats.borrow_mut().add_batch(result.len());
                            yield (result, false, None);
                            tuples = self.factories.weighted_items_factory().default_box();
                            tuples.reserve(chunk_size);
                        }

                        trace_cursor.step_val();
                    };
                    trace_cursor.step_key();
                }

                // If the window shrunk, retract values that dropped off the right end of the
                // window.
                if end1 < end0 {
                    seek_after_end(&mut trace_cursor, &end1, self.right_inclusive);

                    while trace_cursor.key_valid()
                        && before_end(&trace_cursor, &end0, self.right_inclusive)
                    {
                        trace_cursor.key().clone_to(&mut key);
                        while trace_cursor.val_valid()  {
                            let (kv, w) = tuple.split_mut();
                            let (k, v) = kv.split_mut();
                            key.clone_to(k);
                            trace_cursor.val().clone_to(v);
                            **w = trace_cursor.weight().neg_by_ref();

                            tuples.push_val(&mut *tuple);

                            if tuples.len() >= chunk_size {
                                let result = B::dyn_from_tuples(&self.factories, (), &mut tuples);
                                self.output_batch_stats.borrow_mut().add_batch(result.len());
                                yield (result, false, None);
                                tuples = self.factories.weighted_items_factory().default_box();
                                tuples.reserve(chunk_size);
                            }

                            trace_cursor.step_val();
                        };
                        trace_cursor.step_key();
                    }
                }

                // Add tuples in `trace` that slid into the window (region 3).
                seek_after_end(&mut trace_cursor, &end0, self.right_inclusive);

                // In case start1 > end0
                seek_start(&mut trace_cursor, &start1, self.left_inclusive);
                // trace_cursor.seek_key(max(end0, &start1));
                while trace_cursor.key_valid() && before_end(&trace_cursor, &end1, self.right_inclusive)
                {
                    trace_cursor.key().clone_to(&mut key);
                    while trace_cursor.val_valid() {
                        let (kv, w) = tuple.split_mut();
                        let (k, v) = kv.split_mut();
                        key.clone_to(k);
                        trace_cursor.val().clone_to(v);
                        trace_cursor.weight().clone_to(w);

                        tuples.push_val(&mut *tuple);

                        if tuples.len() >= chunk_size {
                            let result = B::dyn_from_tuples(&self.factories, (), &mut tuples);
                            self.output_batch_stats.borrow_mut().add_batch(result.len());
                            yield (result, false, None);
                            tuples = self.factories.weighted_items_factory().default_box();
                            tuples.reserve(chunk_size);
                        }

                        trace_cursor.step_val();
                    };
                    trace_cursor.step_key();
                }
            };

            // Insert tuples in `batch` that fall within the new window.
            seek_start(&mut batch_cursor, &start1, self.left_inclusive);
            // batch_cursor.seek_key(&start1);
            while batch_cursor.key_valid() && before_end(&batch_cursor, &end1, self.right_inclusive) {
                batch_cursor.key().clone_to(&mut key);
                while batch_cursor.val_valid() {
                    let (kv, w) = tuple.split_mut();
                    let (k, v) = kv.split_mut();
                    key.clone_to(k);
                    batch_cursor.val().clone_to(v);
                    batch_cursor.weight().clone_to(w);

                    tuples.push_val(&mut *tuple);

                    if tuples.len() >= chunk_size {
                        let result = B::dyn_from_tuples(&self.factories, (), &mut tuples);
                        self.output_batch_stats.borrow_mut().add_batch(result.len());
                        yield (result, false, None);
                        tuples = self.factories.weighted_items_factory().default_box();
                        tuples.reserve(chunk_size);
                    }

                    batch_cursor.step_val();
                };
                batch_cursor.step_key();
            }

            *self.window.borrow_mut() = Some((start1, end1));

            let result = B::dyn_from_tuples(&self.factories, (), &mut tuples);
            self.output_batch_stats.borrow_mut().add_batch(result.len());
            yield (result, true, None);
        }
    }
}

#[cfg(test)]
mod test {
    use crate::{
        Circuit, IndexedZSet, IndexedZSetHandle, InputHandle, OrdZSet, OutputHandle, RootCircuit,
        Runtime, Stream,
        circuit::mkconfig,
        dynamic::{DowncastTrait, DynData, Erase},
        indexed_zset,
        operator::dynamic::trace::TraceBound,
        typed_batch::{IndexedZSetReader, OrdIndexedZSet, SpineSnapshot, TypedBox},
        utils::{Tup2, Tup3},
    };
    use anyhow::Error as AnyError;
    use size_of::SizeOf;
    use std::vec;

    type Time = u64;

    // A simple, but inefficient implementation of `window` for testing.
    // (it's inefficient even for a non-incremental implementation).
    impl<C, B> Stream<C, B>
    where
        C: Circuit,
        B: IndexedZSet,
        Box<B::DynK>: Clone,
    {
        pub fn window_non_incremental(
            &self,
            (left_inclusive, right_inclusive): (bool, bool),
            bounds: &Stream<C, (TypedBox<B::Key, B::DynK>, TypedBox<B::Key, B::DynK>)>,
        ) -> Stream<C, B> {
            self.apply2(bounds, move |batch, (start, end)| {
                batch.filter(|k, _v| {
                    let left = if left_inclusive {
                        k >= start
                    } else {
                        k > start
                    };
                    let right = if right_inclusive { k <= end } else { k < end };

                    left && right
                })
            })
        }
    }

    // Test all combinations of open and closed intervals against reference implementation.
    fn compare_with_reference(
        stream: &Stream<RootCircuit, OrdIndexedZSet<u64, String>>,
        bounds: &Stream<RootCircuit, (TypedBox<Time, DynData>, TypedBox<Time, DynData>)>,
    ) {
        let closed_closed = stream.window((true, true), bounds).accumulate_integrate();
        let closed_open = stream.window((true, false), bounds).accumulate_integrate();
        let open_closed = stream.window((false, true), bounds).accumulate_integrate();
        let open_open = stream.window((false, false), bounds).accumulate_integrate();

        let closed_closed_expected = stream
            .circuit()
            .non_incremental(stream, |child, stream| {
                Ok(stream
                    .integrate()
                    .window_non_incremental((true, true), &bounds.delta0_non_iterative(child)))
            })
            .unwrap();
        let closed_open_expected = stream
            .circuit()
            .non_incremental(stream, |child, stream| {
                Ok(stream
                    .integrate()
                    .window_non_incremental((true, false), &bounds.delta0_non_iterative(child)))
            })
            .unwrap();
        let open_closed_expected = stream
            .circuit()
            .non_incremental(stream, |child, stream| {
                Ok(stream
                    .integrate()
                    .window_non_incremental((false, true), &bounds.delta0_non_iterative(child)))
            })
            .unwrap();
        let open_open_expected = stream
            .circuit()
            .non_incremental(stream, |child, stream| {
                Ok(stream
                    .integrate()
                    .window_non_incremental((false, false), &bounds.delta0_non_iterative(child)))
            })
            .unwrap();

        closed_closed.accumulate_apply2(&closed_closed_expected, |actual, expected| {
            assert_eq!(
                actual.iter().collect::<Vec<_>>(),
                expected.iter().collect::<Vec<_>>()
            )
        });

        closed_open.accumulate_apply2(&closed_open_expected, |actual, expected| {
            assert_eq!(
                actual.iter().collect::<Vec<_>>(),
                expected.iter().collect::<Vec<_>>()
            )
        });

        open_closed.accumulate_apply2(&open_closed_expected, |actual, expected| {
            assert_eq!(
                actual.iter().collect::<Vec<_>>(),
                expected.iter().collect::<Vec<_>>()
            )
        });

        open_open.accumulate_apply2(&open_open_expected, |actual, expected| {
            assert_eq!(
                actual.iter().collect::<Vec<_>>(),
                expected.iter().collect::<Vec<_>>()
            )
        });
    }

    fn window_test_circuit(
        circuit: &mut RootCircuit,
    ) -> Result<
        (
            InputHandle<(u64, u64)>,
            IndexedZSetHandle<Time, String>,
            OutputHandle<SpineSnapshot<OrdIndexedZSet<Time, String>>>,
        ),
        AnyError,
    > {
        let (bounds, bounds_handle) = circuit.add_input_stream::<(Time, Time)>();

        let (index1, index1_handle) = circuit.add_input_indexed_zset::<Time, String>();

        let bounds = &bounds.apply(|(start, end)| (TypedBox::new(*start), TypedBox::new(*end)));
        let output_handle = index1.window((true, false), bounds).accumulate_output();

        compare_with_reference(&index1, bounds);

        Ok((bounds_handle, index1_handle, output_handle))
    }

    #[test]
    fn sliding() {
        let mut input = vec![
            vec![
                // old value before the first window, should never appear in the output.
                Tup2(800, Tup2("800".to_string(), 1i64)),
                Tup2(900, Tup2("900".to_string(), 1)),
                Tup2(950, Tup2("950".to_string(), 1)),
                Tup2(999, Tup2("999".to_string(), 1)),
                // will appear in the next window
                Tup2(1000, Tup2("1000".to_string(), 1)),
            ],
            vec![
                // old value before the first window
                Tup2(700, Tup2("700".to_string(), 1)),
                // too late, the window already moved forward
                Tup2(900, Tup2("900".to_string(), 1)),
                Tup2(901, Tup2("901".to_string(), 1)),
                Tup2(999, Tup2("999".to_string(), 1)),
                Tup2(1000, Tup2("1000".to_string(), 1)),
                Tup2(1001, Tup2("1001".to_string(), 1)), // will appear in the next window
                Tup2(1002, Tup2("1002".to_string(), 1)), // will appear two windows later
                Tup2(1003, Tup2("1003".to_string(), 1)), // will appear three windows later
            ],
            vec![Tup2(1004, Tup2("1004".to_string(), 1))], // no new values in this window
            vec![],
            vec![],
            vec![],
        ]
        .into_iter();

        let mut expected_outputs = vec![
                indexed_zset! { 900 => {"900".to_string() => 1} , 950 => {"950".to_string() => 1} , 999 => {"999".to_string() => 1} },
                indexed_zset! { 900 => {"900".to_string() => -1} , 901 => {"901".to_string() => 1} , 999 => {"999".to_string() => 1} , 1000 => {"1000".to_string() => 2} },
                indexed_zset! { 901 => {"901".to_string() => -1} , 1001 => {"1001".to_string() => 1} },
                indexed_zset! { 1002 => {"1002".to_string() => 1} },
                indexed_zset! { 1003 => {"1003".to_string() => 1} },
                indexed_zset! { 1004 => {"1004".to_string() => 1} },
            ]
            .into_iter();

        let (mut circuit, (bounds_handle, index1_handle, output_handle)) =
            Runtime::init_circuit(1, move |circuit| window_test_circuit(circuit)).unwrap();

        for clock in 1000..1006 {
            bounds_handle.set_for_all(((clock - 100), clock));

            index1_handle.append(&mut input.next().unwrap());

            circuit.transaction().unwrap();

            let output = SpineSnapshot::<OrdIndexedZSet<u64, String>>::concat(
                &output_handle.take_from_all(),
            );

            assert_eq!(
                output.iter().collect::<Vec<_>>(),
                expected_outputs.next().unwrap().iter().collect::<Vec<_>>()
            );
        }
    }

    #[test]
    fn tumbling() {
        let mut input = vec![
            // window: 995..1000
            vec![
                Tup2(700, Tup2("700".to_string(), 1)),
                Tup2(995, Tup2("995".to_string(), 1)),
                Tup2(996, Tup2("996".to_string(), 1)),
                Tup2(999, Tup2("999".to_string(), 1)),
                Tup2(1000, Tup2("1000".to_string(), 1)),
            ],
            vec![
                Tup2(995, Tup2("995".to_string(), 1)),
                Tup2(1000, Tup2("1000".to_string(), 1)),
                Tup2(1001, Tup2("1001".to_string(), 1)),
            ],
            vec![Tup2(999, Tup2("999".to_string(), 1))],
            vec![Tup2(1002, Tup2("1002".to_string(), 1))],
            vec![Tup2(1003, Tup2("1003".to_string(), 1))],
            // window: 1000..1005
            vec![Tup2(996, Tup2("996".to_string(), 1))], // no longer within window
            vec![Tup2(999, Tup2("999".to_string(), 1))],
            vec![Tup2(1004, Tup2("1004".to_string(), 1))],
            vec![Tup2(1005, Tup2("1005".to_string(), 1))], // next window
            vec![Tup2(1010, Tup2("1010".to_string(), 1))],
            // window: 1005..1010
            vec![Tup2(1005, Tup2("1005".to_string(), 1))],
        ]
        .into_iter();

        let mut expected_outputs = vec![
                indexed_zset! { 995 => {"995".to_string() => 1} , 996 => {"996".to_string() => 1} , 999 => {"999".to_string() => 1} },
                indexed_zset! { 995 => {"995".to_string() => 1} },
                indexed_zset! { 999 => {"999".to_string() => 1} },
                indexed_zset! {},
                indexed_zset! {},
                indexed_zset! { 1000 => {"1000".to_string() => 2} , 1001 => {"1001".to_string() => 1} , 1002 => {"1002".to_string() => 1} , 1003 => {"1003".to_string() => 1} , 995 => {"995".to_string() => -2} , 996 => {"996".to_string() => -1} , 999 => {"999".to_string() => -2} },
                indexed_zset! {},
                indexed_zset! { 1004 => {"1004".to_string() => 1} },
                indexed_zset! {},
                indexed_zset! {},
                indexed_zset! { 1000 => {"1000".to_string() => -2} , 1001 => {"1001".to_string() => -1} , 1002 => {"1002".to_string() => -1} , 1003 => {"1003".to_string() => -1} , 1004 => {"1004".to_string() => -1} , 1005 => {"1005".to_string() => 2} },
            ]
            .into_iter();

        const WINDOW_SIZE: Time = 5;

        let (mut circuit, (bounds_handle, index1_handle, output_handle)) =
            Runtime::init_circuit(1, move |circuit| window_test_circuit(circuit)).unwrap();

        for clock in 1000..1011 {
            bounds_handle.set_for_all((
                (clock / WINDOW_SIZE) * WINDOW_SIZE - WINDOW_SIZE,
                (clock / WINDOW_SIZE) * WINDOW_SIZE,
            ));

            index1_handle.append(&mut input.next().unwrap());

            circuit.transaction().unwrap();

            let output = SpineSnapshot::<OrdIndexedZSet<u64, String>>::concat(
                &output_handle.take_from_all(),
            );

            assert_eq!(
                output.iter().collect::<Vec<_>>(),
                expected_outputs.next().unwrap().iter().collect::<Vec<_>>()
            );
        }
    }

    #[test]
    fn shrinking() {
        let mut input = vec![
            vec![
                Tup2(800u64, Tup2("800".to_string(), 1)),
                Tup2(900, Tup2("900".to_string(), 1)),
                Tup2(950, Tup2("950".to_string(), 1)),
                Tup2(990, Tup2("990".to_string(), 1)),
                Tup2(999, Tup2("999".to_string(), 1)),
                Tup2(1000, Tup2("1000".to_string(), 1)),
            ],
            vec![
                Tup2(700, Tup2("700".to_string(), 1)),
                Tup2(900, Tup2("900".to_string(), 1)),
                Tup2(901, Tup2("901".to_string(), 1)),
                Tup2(915, Tup2("915".to_string(), 1)),
                Tup2(940, Tup2("940".to_string(), 1)),
                Tup2(985, Tup2("985".to_string(), 1)),
                Tup2(999, Tup2("999".to_string(), 1)),
                Tup2(1000, Tup2("1000".to_string(), 1)),
                Tup2(1001, Tup2("1001".to_string(), 1)),
                Tup2(1002, Tup2("1002".to_string(), 1)),
                Tup2(1003, Tup2("1003".to_string(), 1)),
            ],
            vec![
                Tup2(1004, Tup2("1004".to_string(), 1)),
                Tup2(1010, Tup2("1010".to_string(), 1)),
                Tup2(1020, Tup2("1020".to_string(), 1)),
                Tup2(1039, Tup2("1039".to_string(), 1)),
            ],
            vec![],
            vec![],
            vec![],
            vec![],
        ]
        .into_iter();

        let mut expected_outputs = vec![
                indexed_zset! { Time => String: 900 => {"900".to_string() => 1} , 950 => {"950".to_string() => 1} , 990 => {"990".to_string() => 1}, 999 => {"999".to_string() => 1} },
                indexed_zset! { 900 => {"900".to_string() => -1} , 915 => {"915".to_string() => 1} , 940 => {"940".to_string() => 1} , 985 => {"985".to_string() => 1}, 990 => {"990".to_string() => -1}, 999 => {"999".to_string() => -1} },
                indexed_zset! { 915 => {"915".to_string() => -1} , 985 => {"985".to_string() => -1} },
                indexed_zset! { 1000 => {"1000".to_string() => 2}, 1001 => {"1001".to_string() => 1}, 1002 => {"1002".to_string() => 1}, 1003 => {"1003".to_string() => 1}, 1004 => {"1004".to_string() => 1}, 1010 => {"1010".to_string() => 1}, 1020 => {"1020".to_string() => 1}, 1039 => {"1039".to_string() => 1}, 985 => {"985".to_string() => 1}, 990 => {"990".to_string() => 1}, 999 => {"999".to_string() => 2} },
                indexed_zset! { 1039 => {"1039".to_string() => -1}, 940 => {"940".to_string() => -1} },
                indexed_zset! { 1020 => {"1020".to_string() => -1}, 950 => {"950".to_string() => -1} },
                indexed_zset! {}
            ]
            .into_iter();

        let mut windows = vec![
            // Shrink window from both ends.
            (900u64, 1000u64),
            (910, 990),
            (920, 980),
            // Jump forward and start shrinking again.
            (940, 1040),
            (950, 1030),
            (960, 1020),
            // Empty interval
            (1020, 1020),
        ]
        .into_iter();

        let (mut circuit, (bounds_handle, index1_handle, output_handle)) =
            Runtime::init_circuit(1, move |circuit| window_test_circuit(circuit)).unwrap();

        for _ in 1000..1006 {
            bounds_handle.set_for_all(windows.next().unwrap());

            index1_handle.append(&mut input.next().unwrap());

            circuit.transaction().unwrap();

            let output = SpineSnapshot::<OrdIndexedZSet<u64, String>>::concat(
                &output_handle.take_from_all(),
            );

            assert_eq!(
                output.iter().collect::<Vec<_>>(),
                expected_outputs.next().unwrap().iter().collect::<Vec<_>>()
            );
        }
    }

    // It's hard to make this test reliable with async background compactor threads.
    #[test]
    #[ignore]
    fn bounded_memory() {
        let (mut dbsp, input_handle) = Runtime::init_circuit(8, |circuit| {
            let (input, input_handle) = circuit.add_input_indexed_zset::<i64, i64>();
            let bounds =
                input
                    .waterline_monotonic(|| 0, |ts| *ts)
                    .apply(|ts: &TypedBox<i64, DynData>| {
                        (
                            TypedBox::new(*ts.inner().downcast_checked::<i64>() - 1000),
                            TypedBox::new(*ts.inner().downcast_checked::<i64>()),
                        )
                    });

            let bound = TraceBound::new();
            bound.set(Box::new(i64::MAX).erase_box());

            input.window((true, false), &bounds);

            input
                .integrate_trace_with_bound(bound, TraceBound::new())
                .apply(|trace| {
                    let trace_size = trace.size_of().total_bytes();
                    assert!(
                        trace_size < 70000,
                        "trace_size {trace_size} is larger than expected"
                    );
                });

            Ok(input_handle)
        })
        .unwrap();

        for i in 0..10000 {
            for j in i * 100..(i + 1) * 100 {
                input_handle.push(j, (1, 1));
            }
            dbsp.transaction().unwrap();
        }
    }

    /// This test shows how to implement a SQL query similar to
    ///
    /// SELECT
    ///   user,
    ///   COUNT(*)
    ///   FROM transactions
    ///   WHERE transactions.time >= now() - 1000
    /// GROUP BY user
    ///
    /// where now() is a monotonically increasing timestamp.
    /// By using `window` to implement the `WHERE` clause, we automatically GC old transactions
    /// and compute changes to the window incrementally.
    type Clock = i64;

    /// (time, user, amt)
    type Transaction = Tup3<Clock, String, u64>;

    #[test]
    fn aggregate_with_now() {
        let (mut dbsp, (now_handle, data_handle)) = Runtime::init_circuit(8, |circuit| {
            let (now, now_handle) = circuit.add_input_stream::<Clock>();

            let (data, data_handle) = circuit.add_input_zset::<Transaction>();
            let data_by_time = data.map_index(|x| (x.0, x.clone()));

            let bounds = now.apply(|ts: &Clock| (TypedBox::new(*ts - 1000), TypedBox::new(*ts)));

            let counts = data_by_time
                .window((true, false), &bounds)
                .map_index(|(_ts, x)| (x.1.clone(), x.clone()))
                .weighted_count();

            // Reference implementation: filter the entire collection wrt to now.
            let expected = circuit
                .non_incremental(&data, |child, data| {
                    Ok(data
                        .integrate()
                        .apply2(&now.delta0_non_iterative(child), |batch, ts| {
                            OrdZSet::from_keys(
                                (),
                                batch
                                    .iter()
                                    .filter_map(|(x, (), w)| {
                                        if x.0 <= *ts && x.0 >= *ts - 1000 {
                                            Some(Tup2(x.clone(), w))
                                        } else {
                                            None
                                        }
                                    })
                                    .collect::<Vec<_>>(),
                            )
                        })
                        .differentiate())
                })
                .unwrap();

            let expected = expected
                .map_index(|x| (x.1.clone(), x.clone()))
                .weighted_count();
            expected.accumulate_apply2(&counts, |expected, actual| {
                assert_eq!(
                    expected.iter().collect::<Vec<_>>(),
                    actual.iter().collect::<Vec<_>>()
                )
            });

            Ok((now_handle, data_handle))
        })
        .unwrap();

        for i in 1..1000 {
            now_handle.set_for_all(i * 10);
            for j in (i - 1) * 10..i * 10 {
                data_handle.push(Tup3(j, format!("{}", j % 10), j as u64), 1);
            }
            dbsp.transaction().unwrap();
        }
    }

    /// This test exercises the checkpoint/restore path of the Window operator.
    #[test]
    fn window_checkpointing() {
        let (_temp, mut cconf) = mkconfig();

        let mut input = vec![
            vec![
                // old value before the first window, should never appear in the output.
                Tup2(800, Tup2("800".to_string(), 1i64)),
                Tup2(900, Tup2("900".to_string(), 1)),
                Tup2(950, Tup2("950".to_string(), 1)),
                Tup2(999, Tup2("999".to_string(), 1)),
                // will appear in the next window
                Tup2(1000, Tup2("1000".to_string(), 1)),
            ],
            vec![
                // old value before the first window
                Tup2(700, Tup2("700".to_string(), 1)),
                // too late, the window already moved forward
                Tup2(900, Tup2("900".to_string(), 1)),
                Tup2(901, Tup2("901".to_string(), 1)),
                Tup2(999, Tup2("999".to_string(), 1)),
                Tup2(1000, Tup2("1000".to_string(), 1)),
                Tup2(1001, Tup2("1001".to_string(), 1)), // will appear in the next window
                Tup2(1002, Tup2("1002".to_string(), 1)), // will appear two windows later
                Tup2(1003, Tup2("1003".to_string(), 1)), // will appear three windows later
            ],
            vec![Tup2(1004, Tup2("1004".to_string(), 1))], // no new values in this window
            vec![],
            vec![],
            vec![],
        ]
        .into_iter();

        let mut expected_outputs = vec![
                indexed_zset! { 900 => {"900".to_string() => 1} , 950 => {"950".to_string() => 1} , 999 => {"999".to_string() => 1} },
                indexed_zset! { 900 => {"900".to_string() => -1} , 901 => {"901".to_string() => 1} , 999 => {"999".to_string() => 1} , 1000 => {"1000".to_string() => 2} },
                indexed_zset! { 901 => {"901".to_string() => -1} , 1001 => {"1001".to_string() => 1} },
                indexed_zset! { 1002 => {"1002".to_string() => 1} },
                indexed_zset! { 1003 => {"1003".to_string() => 1} },
                indexed_zset! { 1004 => {"1004".to_string() => 1} },
            ]
            .into_iter();

        for clock in 1000..1006 {
            println!("clock: {clock}");

            let (mut circuit, (bounds_handle, index1_handle, output_handle)) =
                Runtime::init_circuit(&cconf, move |circuit| {
                    let (bounds, bounds_handle) = circuit.add_input_stream::<(Time, Time)>();
                    let (index1, index1_handle) = circuit.add_input_indexed_zset::<Time, String>();

                    let bounds =
                        &bounds.apply(|(start, end)| (TypedBox::new(*start), TypedBox::new(*end)));
                    let output_handle = index1.window((true, false), bounds).accumulate_output();

                    Ok((bounds_handle, index1_handle, output_handle))
                })
                .unwrap();

            bounds_handle.set_for_all(((clock - 100), clock));

            index1_handle.append(&mut input.next().unwrap());

            circuit.transaction().unwrap();

            let output = SpineSnapshot::<OrdIndexedZSet<u64, String>>::concat(
                &output_handle.take_from_all(),
            );

            assert_eq!(
                output.iter().collect::<Vec<_>>(),
                expected_outputs.next().unwrap().iter().collect::<Vec<_>>()
            );

            let cpm = circuit.checkpoint().run().unwrap();
            cconf.storage.as_mut().unwrap().init_checkpoint = Some(cpm.uuid);
            circuit.kill().unwrap();
        }
    }
}