Skip to main content

dbsp/operator/dynamic/
trace.rs

1use crate::Runtime;
2use crate::circuit::circuit_builder::{StreamId, register_replay_stream};
3use crate::circuit::metadata::{INPUT_RECORDS_COUNT, MEMORY_ALLOCATIONS_COUNT, RETAINMENT_BOUNDS};
4use crate::dynamic::{Factory, Weight, WeightTrait};
5use crate::operator::require_persistent_id;
6use crate::trace::spine_async::WithSnapshot;
7use crate::trace::{BatchReaderFactories, Builder, GroupFilter, MergeCursor};
8use crate::{
9    Error, Timestamp,
10    circuit::{
11        Circuit, ExportId, ExportStream, FeedbackConnector, GlobalNodeId, OwnershipPreference,
12        Scope, Stream, WithClock,
13        metadata::{
14            ALLOCATED_MEMORY_BYTES, MetaItem, OperatorMeta, SHARED_MEMORY_BYTES,
15            STATE_RECORDS_COUNT, USED_MEMORY_BYTES,
16        },
17        operator_traits::{BinaryOperator, Operator, StrictOperator, StrictUnaryOperator},
18    },
19    circuit_cache_key,
20    dynamic::DataTrait,
21    trace::{Batch, BatchReader, Filter, Spine, SpineSnapshot, Trace},
22};
23use dyn_clone::clone_box;
24use feldera_storage::{FileCommitter, StoragePath};
25use ouroboros::self_referencing;
26use size_of::SizeOf;
27use std::any::TypeId;
28use std::collections::BTreeMap;
29use std::mem::transmute;
30use std::{
31    borrow::Cow,
32    cell::{Ref, RefCell},
33    cmp::Ordering,
34    fmt::Debug,
35    marker::PhantomData,
36    ops::Deref,
37    rc::Rc,
38    sync::Arc,
39};
40
41circuit_cache_key!(TraceId<C, D: BatchReader>(StreamId => Stream<C, D>));
42circuit_cache_key!(BoundsId<D: BatchReader>(StreamId => TraceBounds<<D as BatchReader>::Key, <D as BatchReader>::Val>));
43
44// Trace of a collection delayed by one step.
45circuit_cache_key!(DelayedTraceId<C, D>(StreamId => Stream<C, D>));
46
47/// Lower bound on keys or values in a trace.
48///
49/// Setting the bound to `None` is equivalent to setting it to
50/// `T::min_value()`, i.e., the contents of the trace will never
51/// get truncated.
52///
53/// The writer can update the value of the bound at each clock
54/// cycle.  The bound can only increase monotonically.
55#[repr(transparent)]
56pub struct TraceBound<T: ?Sized>(Rc<RefCell<Option<Box<T>>>>);
57
58impl<T: DataTrait + ?Sized> Clone for TraceBound<T> {
59    fn clone(&self) -> Self {
60        Self(self.0.clone())
61    }
62}
63
64impl<T: Debug + ?Sized> Debug for TraceBound<T> {
65    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
66        self.0.borrow().fmt(f)
67    }
68}
69
70impl<T: DataTrait + ?Sized> PartialEq for TraceBound<T> {
71    fn eq(&self, other: &Self) -> bool {
72        self.0 == other.0
73    }
74}
75
76impl<T: DataTrait + ?Sized> Eq for TraceBound<T> {}
77
78impl<T: DataTrait + ?Sized> PartialOrd for TraceBound<T> {
79    fn partial_cmp(&self, other: &Self) -> Option<Ordering> {
80        Some(self.cmp(other))
81    }
82}
83
84impl<T: DataTrait + ?Sized> Ord for TraceBound<T> {
85    fn cmp(&self, other: &Self) -> Ordering {
86        self.0.cmp(&other.0)
87    }
88}
89
90impl<K: DataTrait + ?Sized> Default for TraceBound<K> {
91    fn default() -> Self {
92        Self(Rc::new(RefCell::new(None)))
93    }
94}
95
96impl<K: DataTrait + ?Sized> TraceBound<K> {
97    pub fn new() -> Self {
98        Default::default()
99    }
100
101    /// Set the new value of the bound.
102    pub fn set(&self, bound: Box<K>) {
103        //debug_assert!(self.0.borrow().as_ref() <= Some(&bound));
104        *self.0.borrow_mut() = Some(bound);
105    }
106
107    /// Get the current value of the bound.
108    pub fn get(&self) -> Ref<'_, Option<Box<K>>> {
109        (*self.0).borrow()
110    }
111}
112
113/// Data structure that tracks key and value retainment policies for a
114/// trace.
115pub struct TraceBounds<K: ?Sized + 'static, V: DataTrait + ?Sized>(
116    Rc<RefCell<TraceBoundsInner<K, V>>>,
117);
118
119impl<K: DataTrait + ?Sized, V: DataTrait + ?Sized> Clone for TraceBounds<K, V> {
120    fn clone(&self) -> Self {
121        Self(self.0.clone())
122    }
123}
124
125impl<K: DataTrait + ?Sized, V: DataTrait + ?Sized> TraceBounds<K, V> {
126    /// Instantiate `TraceBounds` with empty sets of key and value bounds.
127    ///
128    /// The caller must add at least one key and one value bound before
129    /// running the circuit.
130    pub(crate) fn new() -> Self {
131        Self(Rc::new(RefCell::new(TraceBoundsInner {
132            key_predicate: Predicate::Bounds(Vec::new()),
133            unique_key_name: None,
134            val_predicate: GroupPredicate::Bounds(Vec::new()),
135            unique_val_name: None,
136        })))
137    }
138
139    /// Returns `TraceBounds` that prevent any values in the trace from
140    /// being truncated.
141    pub(crate) fn unbounded() -> Self {
142        Self(Rc::new(RefCell::new(TraceBoundsInner {
143            key_predicate: Predicate::Bounds(vec![TraceBound::new()]),
144            unique_key_name: None,
145            val_predicate: GroupPredicate::Bounds(vec![TraceBound::new()]),
146            unique_val_name: None,
147        })))
148    }
149
150    pub(crate) fn add_key_bound(&self, bound: TraceBound<K>) {
151        match &mut self.0.borrow_mut().key_predicate {
152            Predicate::Bounds(bounds) => bounds.push(bound),
153            Predicate::Filter(_) => {}
154        };
155    }
156
157    /// Set key retainment condition.  Disables any key bounds
158    /// set using [`Self::add_key_bound`].
159    pub(crate) fn set_key_filter(&self, filter: Filter<K>) {
160        self.0.borrow_mut().key_predicate = Predicate::Filter(filter);
161    }
162
163    pub(crate) fn set_unique_key_bound_name(&self, unique_name: Option<&str>) {
164        self.0.borrow_mut().unique_key_name = unique_name.map(str::to_string);
165    }
166
167    pub(crate) fn add_val_bound(&self, bound: TraceBound<V>) {
168        match &mut self.0.borrow_mut().val_predicate {
169            GroupPredicate::Bounds(bounds) => bounds.push(bound),
170            GroupPredicate::Filter(_) => {}
171        };
172    }
173
174    /// Set value retainment condition.  Disables any value bounds
175    /// set using [`Self::add_val_bound`].
176    pub(crate) fn set_val_filter(&self, filter: GroupFilter<V>) {
177        self.0.borrow_mut().val_predicate = GroupPredicate::Filter(filter);
178    }
179
180    pub(crate) fn set_unique_val_bound_name(&self, unique_name: Option<&str>) {
181        self.0.borrow_mut().unique_val_name = unique_name.map(str::to_string);
182    }
183
184    /// Returns effective key retention condition, computed as the
185    /// minimum bound installed using [`Self::add_val_bound`] or as the
186    /// condition installed using [`Self::set_val_filter`] (the latter
187    /// takes precedence).
188    pub(crate) fn effective_key_filter(&self) -> Option<Filter<K>> {
189        match &(*self.0).borrow().key_predicate {
190            Predicate::Bounds(bounds) => bounds
191                .iter()
192                .min()
193                .expect("At least one trace bound must be set")
194                .get()
195                .deref()
196                .as_ref()
197                .map(|bx| Arc::from(clone_box(bx.as_ref())))
198                .map(|bound: Arc<K>| {
199                    let metadata = MetaItem::String(format!("{bound:?}"));
200                    Filter::new(Box::new(move |k: &K| {
201                        bound.as_ref().cmp(k) != Ordering::Greater
202                    }))
203                    .with_metadata(metadata)
204                }),
205            Predicate::Filter(filter) => Some(filter.clone()),
206        }
207    }
208
209    /// Returns effective value retention condition, computed as the
210    /// minimum bound installed using [`Self::add_val_bound`] or as the
211    /// condition installed using [`Self::set_val_filter`] (the latter
212    /// takes precedence).
213    pub(crate) fn effective_val_filter(&self) -> Option<GroupFilter<V>> {
214        match &(*self.0).borrow().val_predicate {
215            GroupPredicate::Bounds(bounds) => bounds
216                .iter()
217                .min()
218                .expect("At least one trace bound must be set")
219                .get()
220                .deref()
221                .as_ref()
222                .map(|bx| Arc::from(clone_box(bx.as_ref())))
223                .map(|bound: Arc<V>| {
224                    let metadata = MetaItem::String(format!("{bound:?}"));
225                    GroupFilter::Simple(
226                        Filter::new(Box::new(move |v: &V| {
227                            bound.as_ref().cmp(v) != Ordering::Greater
228                        }))
229                        .with_metadata(metadata),
230                    )
231                }),
232            GroupPredicate::Filter(filter) => Some(filter.clone()),
233        }
234    }
235
236    pub(crate) fn metadata(&self) -> MetaItem {
237        self.0.borrow().metadata()
238    }
239}
240
241/// Value retainment predicate defined as either a set of bounds
242/// or a filter condition.
243///
244/// See [`Stream::dyn_integrate_trace_retain_keys`] for details.
245enum Predicate<V: ?Sized> {
246    Bounds(Vec<TraceBound<V>>),
247    Filter(Filter<V>),
248}
249
250impl<V: Debug + ?Sized> Predicate<V> {
251    pub fn metadata(&self) -> MetaItem {
252        match self {
253            Self::Bounds(bounds) => MetaItem::Array(
254                bounds
255                    .iter()
256                    .map(|b| MetaItem::String(format!("{b:?}")))
257                    .collect(),
258            ),
259            Self::Filter(filter) => filter.metadata().clone(),
260        }
261    }
262}
263
264enum GroupPredicate<V: DataTrait + ?Sized> {
265    /// Discard values that are less than at least one of the bounds in the vector.
266    Bounds(Vec<TraceBound<V>>),
267    /// Discard values that don't satisfy the filter.
268    Filter(GroupFilter<V>),
269}
270
271impl<V: DataTrait + ?Sized> GroupPredicate<V> {
272    pub fn metadata(&self) -> MetaItem {
273        match self {
274            Self::Bounds(bounds) => MetaItem::Array(
275                bounds
276                    .iter()
277                    .map(|b| MetaItem::String(format!("{b:?}")))
278                    .collect(),
279            ),
280            Self::Filter(filter) => filter.metadata().clone(),
281        }
282    }
283}
284
285struct TraceBoundsInner<K: ?Sized + 'static, V: DataTrait + ?Sized> {
286    /// Key bounds _or_ retainment condition.
287    key_predicate: Predicate<K>,
288    unique_key_name: Option<String>,
289    /// Value bounds _or_ retainment condition.
290    val_predicate: GroupPredicate<V>,
291    unique_val_name: Option<String>,
292}
293
294impl<K: Debug + ?Sized + 'static, V: DataTrait + ?Sized> TraceBoundsInner<K, V> {
295    pub fn metadata(&self) -> MetaItem {
296        MetaItem::Map(BTreeMap::from([
297            (Cow::Borrowed("key"), self.key_predicate.metadata()),
298            (Cow::Borrowed("value"), self.val_predicate.metadata()),
299        ]))
300    }
301}
302
303pub type TimedSpine<B, C> = Spine<<<C as WithClock>::Time as Timestamp>::TimedBatch<B>>;
304
305impl<C, B> Stream<C, B>
306where
307    C: Circuit,
308    B: Clone + Send + Sync + 'static,
309{
310    /// See [`Stream::trace`].
311    pub fn dyn_trace(
312        &self,
313        output_factories: &<TimedSpine<B, C> as BatchReader>::Factories,
314        batch_factories: &B::Factories,
315    ) -> Stream<C, TimedSpine<B, C>>
316    where
317        B: Batch<Time = ()>,
318    {
319        self.dyn_trace_with_bound(
320            output_factories,
321            batch_factories,
322            TraceBound::new(),
323            TraceBound::new(),
324        )
325    }
326
327    /// See [`Stream::trace_with_bound`].
328    pub fn dyn_trace_with_bound(
329        &self,
330        output_factories: &<TimedSpine<B, C> as BatchReader>::Factories,
331        batch_factories: &B::Factories,
332        lower_key_bound: TraceBound<B::Key>,
333        lower_val_bound: TraceBound<B::Val>,
334    ) -> Stream<C, TimedSpine<B, C>>
335    where
336        B: Batch<Time = ()>,
337    {
338        let bounds = self.trace_bounds_with_bound(lower_key_bound, lower_val_bound);
339
340        self.circuit()
341            .cache_get_or_insert_with(TraceId::new(self.stream_id()), || {
342                let circuit = self.circuit();
343
344                circuit.region("trace", || {
345                    let persistent_id = self.get_persistent_id();
346                    let z1 = Z1Trace::new(
347                        output_factories,
348                        batch_factories,
349                        false,
350                        circuit.root_scope(),
351                        bounds.clone(),
352                    );
353                    let (delayed_trace, z1feedback) = circuit.add_feedback_persistent(
354                        persistent_id
355                            .map(|name| format!("{name}.integral"))
356                            .as_deref(),
357                        z1,
358                    );
359
360                    let replay_stream = z1feedback.operator_mut().prepare_replay_stream(self);
361
362                    let trace = circuit.add_binary_operator_with_preference(
363                        <TraceAppend<TimedSpine<B, C>, B, C>>::new(
364                            output_factories,
365                            circuit.clone(),
366                        ),
367                        (&delayed_trace, OwnershipPreference::STRONGLY_PREFER_OWNED),
368                        (self, OwnershipPreference::PREFER_OWNED),
369                    );
370                    if self.is_sharded() {
371                        delayed_trace.mark_sharded();
372                        trace.mark_sharded();
373                    }
374
375                    z1feedback.connect_with_preference(
376                        &trace,
377                        OwnershipPreference::STRONGLY_PREFER_OWNED,
378                    );
379
380                    register_replay_stream(circuit, self, &replay_stream);
381
382                    circuit.cache_insert(DelayedTraceId::new(trace.stream_id()), delayed_trace);
383                    trace
384                })
385            })
386            .clone()
387    }
388
389    /// See [`Stream::integrate_trace_retain_keys`].
390    #[track_caller]
391    pub fn dyn_integrate_trace_retain_keys<TS>(
392        &self,
393        bounds_stream: &Stream<C, Box<TS>>,
394        retain_key_func: Box<dyn Fn(&TS) -> Filter<B::Key>>,
395    ) where
396        B: Batch<Time = ()>,
397        TS: DataTrait + ?Sized,
398        Box<TS>: Clone,
399    {
400        let bounds = self.trace_bounds();
401        bounds.set_unique_key_bound_name(bounds_stream.get_persistent_id().as_deref());
402        bounds_stream.inspect(move |ts| {
403            let filter = retain_key_func(ts.as_ref());
404            bounds.set_key_filter(filter);
405        });
406    }
407
408    /// See [`Stream::integrate_trace_retain_values`].
409    #[track_caller]
410    pub fn dyn_integrate_trace_retain_values<TS>(
411        &self,
412        bounds_stream: &Stream<C, Box<TS>>,
413        retain_val_func: Box<dyn Fn(&TS) -> Filter<B::Val>>,
414    ) where
415        B: Batch<Time = ()>,
416        TS: DataTrait + ?Sized,
417        Box<TS>: Clone,
418    {
419        let bounds = self.trace_bounds();
420        bounds.set_unique_val_bound_name(bounds_stream.get_persistent_id().as_deref());
421
422        bounds_stream.inspect(move |ts| {
423            let filter = GroupFilter::Simple(retain_val_func(ts.as_ref()));
424            bounds.set_val_filter(filter);
425        });
426    }
427
428    #[track_caller]
429    pub fn dyn_integrate_trace_retain_values_last_n<TS>(
430        &self,
431        bounds_stream: &Stream<C, Box<TS>>,
432        retain_val_func: Box<dyn Fn(&TS) -> Filter<B::Val>>,
433        n: usize,
434    ) where
435        B: Batch<Time = ()>,
436        TS: DataTrait + ?Sized,
437        Box<TS>: Clone,
438    {
439        let bounds = self.trace_bounds();
440        bounds.set_unique_val_bound_name(bounds_stream.get_persistent_id().as_deref());
441
442        bounds_stream.inspect(move |ts| {
443            let filter = GroupFilter::LastN(n, retain_val_func(ts.as_ref()));
444            bounds.set_val_filter(filter);
445        });
446    }
447
448    #[track_caller]
449    pub fn dyn_integrate_trace_retain_values_top_n<TS>(
450        &self,
451        val_factory: &'static dyn Factory<B::Val>,
452        bounds_stream: &Stream<C, Box<TS>>,
453        retain_val_func: Box<dyn Fn(&TS) -> Filter<B::Val>>,
454        n: usize,
455    ) where
456        B: Batch<Time = ()>,
457        TS: DataTrait + ?Sized,
458        Box<TS>: Clone,
459    {
460        let bounds = self.trace_bounds();
461        bounds.set_unique_val_bound_name(bounds_stream.get_persistent_id().as_deref());
462
463        bounds_stream.inspect(move |ts| {
464            let filter = GroupFilter::TopN(n, retain_val_func(ts.as_ref()), val_factory);
465            bounds.set_val_filter(filter);
466        });
467    }
468
469    /// Retrieves trace bounds for `self`, creating them if necessary.
470    ///
471    /// It's important that a single `TraceBounds` includes all of the bounds
472    /// relevant to a particular trace.  This can be tricky in the presence of
473    /// multiple versions of a stream that code tends to treat as the same.  We
474    /// manage it by mapping all of those versions to just one single version:
475    ///
476    /// * For a sharded version of some source stream, we use the source stream.
477    ///
478    /// * For a spilled version of some source stream, we use the source stream.
479    ///
480    /// Using the source stream is a safer choice than using the sharded (or
481    /// spilled) version, because it always exists, whereas the sharded version
482    /// might be created only *after* we get the trace bounds for the source
483    /// stream.
484    fn trace_bounds(&self) -> TraceBounds<B::Key, B::Val>
485    where
486        B: BatchReader,
487    {
488        // We handle moving from the sharded to unsharded stream directly here.
489        let stream_id = self.try_unsharded_version().stream_id();
490
491        self.circuit()
492            .cache_get_or_insert_with(BoundsId::<B>::new(stream_id), TraceBounds::new)
493            .clone()
494    }
495
496    /// Retrieves trace bounds for `self`, or a sharded version of `self` if it
497    /// exists, creating them if necessary, and adds bounds for
498    /// `lower_key_bound` and `lower_val_bound`.
499    fn trace_bounds_with_bound(
500        &self,
501        lower_key_bound: TraceBound<B::Key>,
502        lower_val_bound: TraceBound<B::Val>,
503    ) -> TraceBounds<B::Key, B::Val>
504    where
505        B: BatchReader,
506    {
507        let bounds = self.trace_bounds();
508        bounds.add_key_bound(lower_key_bound);
509        bounds.add_val_bound(lower_val_bound);
510        bounds
511    }
512
513    // TODO: this method should replace `Stream::integrate()`.
514    #[track_caller]
515    pub fn dyn_integrate_trace(&self, factories: &B::Factories) -> Stream<C, Spine<B>>
516    where
517        B: Batch<Time = ()>,
518        Spine<B>: SizeOf,
519    {
520        self.dyn_integrate_trace_with_bound(factories, TraceBound::new(), TraceBound::new())
521    }
522
523    pub fn dyn_integrate_trace_with_bound(
524        &self,
525        factories: &B::Factories,
526        lower_key_bound: TraceBound<B::Key>,
527        lower_val_bound: TraceBound<B::Val>,
528    ) -> Stream<C, Spine<B>>
529    where
530        B: Batch<Time = ()>,
531        Spine<B>: SizeOf,
532    {
533        self.integrate_trace_inner(
534            factories,
535            self.trace_bounds_with_bound(lower_key_bound, lower_val_bound),
536        )
537    }
538
539    #[allow(clippy::type_complexity)]
540    fn integrate_trace_inner(
541        &self,
542        input_factories: &B::Factories,
543        bounds: TraceBounds<B::Key, B::Val>,
544    ) -> Stream<C, Spine<B>>
545    where
546        B: Batch<Time = ()>,
547        Spine<B>: SizeOf,
548    {
549        self.circuit()
550            .cache_get_or_insert_with(TraceId::new(self.stream_id()), || {
551                let circuit = self.circuit();
552                let bounds = bounds.clone();
553
554                let persistent_id = self.get_persistent_id();
555
556                circuit.region("integrate_trace", || {
557                    let z1 = Z1Trace::new(
558                        input_factories,
559                        input_factories,
560                        true,
561                        circuit.root_scope(),
562                        bounds,
563                    );
564
565                    let (
566                        ExportStream {
567                            local: delayed_trace,
568                            export,
569                        },
570                        z1feedback,
571                    ) = circuit.add_feedback_with_export_persistent(
572                        persistent_id
573                            .map(|name| format!("{name}.integral"))
574                            .as_deref(),
575                        z1,
576                    );
577
578                    let replay_stream = z1feedback.operator_mut().prepare_replay_stream(self);
579
580                    let trace = circuit.add_binary_operator_with_preference(
581                        UntimedTraceAppend::<Spine<B>>::new(),
582                        (&delayed_trace, OwnershipPreference::STRONGLY_PREFER_OWNED),
583                        (self, OwnershipPreference::PREFER_OWNED),
584                    );
585
586                    if self.is_sharded() {
587                        delayed_trace.mark_sharded();
588                        trace.mark_sharded();
589                    }
590
591                    z1feedback.connect_with_preference(
592                        &trace,
593                        OwnershipPreference::STRONGLY_PREFER_OWNED,
594                    );
595
596                    register_replay_stream(circuit, self, &replay_stream);
597
598                    circuit.cache_insert(DelayedTraceId::new(trace.stream_id()), delayed_trace);
599                    circuit.cache_insert(ExportId::new(trace.stream_id()), export);
600
601                    trace
602                })
603            })
604            .clone()
605    }
606}
607
608/// See [`trait TraceFeedback`] documentation.
609pub struct TraceFeedbackConnector<C, T>
610where
611    C: Circuit,
612    T: Trace,
613{
614    feedback: FeedbackConnector<C, T, T, Z1Trace<C, T::Batch, T>>,
615    /// `delayed_trace` stream in the diagram in
616    /// [`trait TraceFeedback`] documentation.
617    pub delayed_trace: Stream<C, T>,
618    export_trace: Stream<C::Parent, T>,
619    bounds: TraceBounds<T::Key, T::Val>,
620}
621
622impl<C, T> TraceFeedbackConnector<C, T>
623where
624    T: Trace<Time = ()> + Clone,
625    C: Circuit,
626{
627    pub fn connect(self, stream: &Stream<C, T::Batch>) {
628        let circuit = self.delayed_trace.circuit();
629
630        let replay_stream = self.feedback.operator_mut().prepare_replay_stream(stream);
631
632        let trace = circuit.add_binary_operator_with_preference(
633            <UntimedTraceAppend<T>>::new(),
634            (
635                &self.delayed_trace,
636                OwnershipPreference::STRONGLY_PREFER_OWNED,
637            ),
638            (stream, OwnershipPreference::PREFER_OWNED),
639        );
640
641        if stream.is_sharded() {
642            self.delayed_trace.mark_sharded();
643            trace.mark_sharded();
644        }
645
646        self.feedback
647            .connect_with_preference(&trace, OwnershipPreference::STRONGLY_PREFER_OWNED);
648
649        register_replay_stream(circuit, stream, &replay_stream);
650
651        circuit.cache_insert(
652            DelayedTraceId::new(trace.stream_id()),
653            self.delayed_trace.clone(),
654        );
655
656        circuit.cache_insert(TraceId::new(stream.stream_id()), trace.clone());
657        circuit.cache_insert(
658            BoundsId::<T::Batch>::new(stream.stream_id()),
659            self.bounds.clone(),
660        );
661        circuit.cache_insert(ExportId::new(trace.stream_id()), self.export_trace);
662    }
663}
664
665/// Extension trait to trait [`Circuit`] that provides a convenience API
666/// to construct circuits of the following shape:
667///
668/// ```text
669///  external inputs    ┌───┐    output    ┌──────────────────┐
670/// ───────────────────►│ F ├─────────────►│UntimedTraceAppend├───┐
671///                     └───┘              └──────────────────┘   │
672///                       ▲                      ▲                │trace
673///                       │                      │                │
674///                       │    delayed_trace   ┌─┴──┐             │
675///                       └────────────────────┤Z^-1│◄────────────┘
676///                                            └────┘
677/// ```
678/// where `F` is an operator that consumes an integral of its own output
679/// stream.
680///
681/// Use this method to create a
682/// [`TraceFeedbackConnector`] struct.  The struct contains the `delayed_trace`
683/// stream, which can be used as input to instantiate `F` and the `output`
684/// stream.  Close the loop by calling
685/// `TraceFeedbackConnector::connect(output)`.
686pub trait TraceFeedback: Circuit {
687    fn add_integrate_trace_feedback<T>(
688        &self,
689        persistent_id: Option<&str>,
690        factories: &T::Factories,
691        bounds: TraceBounds<T::Key, T::Val>,
692    ) -> TraceFeedbackConnector<Self, T>
693    where
694        T: Trace<Time = ()> + Clone,
695    {
696        // We'll give `Z1Trace` a real name inside `TraceFeedbackConnector::connect`, where we have the name of the input stream.
697        let (ExportStream { local, export }, feedback) = self.add_feedback_with_export_persistent(
698            persistent_id
699                .map(|name| format!("{name}.integral"))
700                .as_deref(),
701            Z1Trace::new(
702                factories,
703                factories,
704                true,
705                self.root_scope(),
706                bounds.clone(),
707            ),
708        );
709
710        TraceFeedbackConnector {
711            feedback,
712            delayed_trace: local,
713            export_trace: export,
714            bounds,
715        }
716    }
717}
718
719impl<C: Circuit> TraceFeedback for C {}
720
721impl<C, B> Stream<C, Spine<B>>
722where
723    C: Circuit,
724    B: Batch,
725{
726    pub fn delay_trace(&self) -> Stream<C, SpineSnapshot<B>> {
727        // The delayed trace should be automatically created while the real trace is
728        // created via `.trace()` or a similar function
729        // FIXME: Create a trace if it doesn't exist
730        let delayed_trace = self
731            .circuit()
732            .cache_get_or_insert_with(DelayedTraceId::new(self.stream_id()), || {
733                panic!("called `.delay_trace()` on a stream without a previously created trace")
734            })
735            .deref()
736            .clone();
737        delayed_trace.apply(|spine: &Spine<B>| spine.ro_snapshot())
738    }
739}
740
741pub struct UntimedTraceAppend<T>
742where
743    T: Trace,
744{
745    // Total number of input tuples processed by the operator.
746    num_inputs: usize,
747
748    _phantom: PhantomData<T>,
749}
750
751impl<T> Default for UntimedTraceAppend<T>
752where
753    T: Trace,
754{
755    fn default() -> Self {
756        Self::new()
757    }
758}
759
760impl<T> UntimedTraceAppend<T>
761where
762    T: Trace,
763{
764    pub fn new() -> Self {
765        Self {
766            num_inputs: 0,
767            _phantom: PhantomData,
768        }
769    }
770}
771
772impl<T> Operator for UntimedTraceAppend<T>
773where
774    T: Trace + 'static,
775{
776    fn name(&self) -> Cow<'static, str> {
777        Cow::from("UntimedTraceAppend")
778    }
779
780    fn metadata(&self, meta: &mut OperatorMeta) {
781        meta.extend(metadata! {
782            INPUT_RECORDS_COUNT => MetaItem::Count(self.num_inputs),
783        });
784    }
785
786    fn fixedpoint(&self, _scope: Scope) -> bool {
787        true
788    }
789}
790
791impl<T> BinaryOperator<T, T::Batch, T> for UntimedTraceAppend<T>
792where
793    T: Trace + 'static,
794{
795    async fn eval(&mut self, _trace: &T, _batch: &T::Batch) -> T {
796        // Refuse to accept trace by reference.  This should not happen in a correctly
797        // constructed circuit.
798        panic!("UntimedTraceAppend::eval(): cannot accept trace by reference")
799    }
800
801    async fn eval_owned_and_ref(&mut self, mut trace: T, batch: &T::Batch) -> T {
802        self.num_inputs += batch.len();
803        trace.insert(batch.clone());
804        trace
805    }
806
807    async fn eval_ref_and_owned(&mut self, _trace: &T, _batch: T::Batch) -> T {
808        // Refuse to accept trace by reference.  This should not happen in a correctly
809        // constructed circuit.
810        panic!("UntimedTraceAppend::eval_ref_and_owned(): cannot accept trace by reference")
811    }
812
813    async fn eval_owned(&mut self, mut trace: T, batch: T::Batch) -> T {
814        self.num_inputs += batch.len();
815
816        trace.insert(batch);
817        trace
818    }
819
820    fn input_preference(&self) -> (OwnershipPreference, OwnershipPreference) {
821        (
822            OwnershipPreference::PREFER_OWNED,
823            OwnershipPreference::PREFER_OWNED,
824        )
825    }
826}
827
828pub struct TraceAppend<T: Trace, B: BatchReader, C> {
829    clock: C,
830    output_factories: T::Factories,
831
832    // Total number of input tuples processed by the operator.
833    num_inputs: usize,
834
835    _phantom: PhantomData<(T, B)>,
836}
837
838impl<T: Trace, B: BatchReader, C> TraceAppend<T, B, C> {
839    pub fn new(output_factories: &T::Factories, clock: C) -> Self {
840        Self {
841            clock,
842            output_factories: output_factories.clone(),
843            num_inputs: 0,
844            _phantom: PhantomData,
845        }
846    }
847}
848
849impl<T, B, Clk> Operator for TraceAppend<T, B, Clk>
850where
851    T: Trace,
852    B: BatchReader,
853    Clk: 'static,
854{
855    fn name(&self) -> Cow<'static, str> {
856        Cow::from("TraceAppend")
857    }
858    fn fixedpoint(&self, _scope: Scope) -> bool {
859        true
860    }
861
862    fn metadata(&self, meta: &mut OperatorMeta) {
863        meta.extend(metadata! {
864            INPUT_RECORDS_COUNT => MetaItem::Count(self.num_inputs),
865        });
866    }
867}
868
869impl<T, B, Clk> BinaryOperator<T, B, T> for TraceAppend<T, B, Clk>
870where
871    B: BatchReader<Time = ()>,
872    Clk: WithClock + 'static,
873    T: Trace<Key = B::Key, Val = B::Val, R = B::R, Time = Clk::Time>,
874{
875    async fn eval(&mut self, _trace: &T, _batch: &B) -> T {
876        // Refuse to accept trace by reference.  This should not happen in a correctly
877        // constructed circuit.
878        unimplemented!()
879    }
880
881    async fn eval_owned_and_ref(&mut self, mut trace: T, batch: &B) -> T {
882        // TODO: extend `trace` type to feed untimed batches directly
883        // (adding fixed timestamp on the fly).
884        self.num_inputs += batch.len();
885        trace.insert(T::Batch::from_batch(
886            batch,
887            &self.clock.time(),
888            &self.output_factories,
889        ));
890        trace
891    }
892
893    async fn eval_ref_and_owned(&mut self, _trace: &T, _batch: B) -> T {
894        // Refuse to accept trace by reference.  This should not happen in a correctly
895        // constructed circuit.
896        unimplemented!()
897    }
898
899    async fn eval_owned(&mut self, mut trace: T, batch: B) -> T {
900        self.num_inputs += batch.len();
901
902        if TypeId::of::<B>() == TypeId::of::<T::Batch>() {
903            let mut batch = Some(batch);
904            let batch = unsafe { transmute::<&mut Option<B>, &mut Option<T::Batch>>(&mut batch) };
905            trace.insert(batch.take().unwrap());
906        } else {
907            trace.insert(T::Batch::from_batch(
908                &batch,
909                &self.clock.time(),
910                &self.output_factories,
911            ));
912        }
913        trace
914    }
915
916    fn input_preference(&self) -> (OwnershipPreference, OwnershipPreference) {
917        (
918            OwnershipPreference::PREFER_OWNED,
919            OwnershipPreference::PREFER_OWNED,
920        )
921    }
922}
923
924#[self_referencing]
925struct ReplayState<T: Trace> {
926    trace: T,
927    #[borrows(trace)]
928    #[covariant]
929    cursor: Box<dyn MergeCursor<T::Key, T::Val, T::Time, T::R> + Send + 'this>,
930}
931
932impl<T: Trace> ReplayState<T> {
933    fn create(trace: T) -> Self {
934        ReplayStateBuilder {
935            trace,
936            cursor_builder: |trace| trace.merge_cursor(None, None),
937        }
938        .build()
939    }
940}
941
942pub struct Z1Trace<C: Circuit, B: Batch, T: Trace> {
943    // For error reporting.
944    global_id: GlobalNodeId,
945    time: T::Time,
946    trace: Option<T>,
947    replay_state: Option<ReplayState<T>>,
948    replay_mode: bool,
949    trace_factories: T::Factories,
950    // `dirty[scope]` is `true` iff at least one non-empty update was added to the trace
951    // since the previous clock cycle at level `scope`.
952    dirty: Vec<bool>,
953    root_scope: Scope,
954    reset_on_clock_start: bool,
955    bounds: TraceBounds<T::Key, T::Val>,
956
957    // Metrics maintained by the trace.
958    batch_factories: B::Factories,
959    // Stream whose integral this Z1 operator stores, if any.
960    delta_stream: Option<Stream<C, B>>,
961    flush: bool,
962}
963
964impl<C, B, T> Z1Trace<C, B, T>
965where
966    C: Circuit,
967    B: Batch,
968    T: Trace,
969{
970    pub fn new(
971        trace_factories: &T::Factories,
972        batch_factories: &B::Factories,
973        reset_on_clock_start: bool,
974        root_scope: Scope,
975        bounds: TraceBounds<T::Key, T::Val>,
976    ) -> Self {
977        Self {
978            global_id: GlobalNodeId::root(),
979            time: <T::Time as Timestamp>::clock_start(),
980            trace: None,
981            replay_state: None,
982            replay_mode: false,
983            trace_factories: trace_factories.clone(),
984            batch_factories: batch_factories.clone(),
985            dirty: vec![false; root_scope as usize + 1],
986            root_scope,
987            reset_on_clock_start,
988            bounds,
989            delta_stream: None,
990            flush: false,
991        }
992    }
993
994    /// Creates a stream that will be used to replay the contents of `stream`.
995    ///
996    /// Given a circuit that implements an integral, the Z-1 operator can be used
997    /// to replay the `delta` stream during bootstrapping.  This function sets this
998    /// up at circuit construction time. It creates a new stream (`replay_stream`)
999    /// that aliases `stream` internally.  In replay mode, Z-1 will send the contents
1000    /// of the integral to `replay_stream` chunk-by-chunk.
1001    ///
1002    ///   │stream
1003    ///   │
1004    ///   │
1005    ///   │◄............
1006    ///   │            .replay_stream
1007    ///   ▼            .
1008    /// ┌───┐        ┌───┐
1009    /// │ + ├───────►│Z-1│
1010    /// └───┘        └─┬─┘
1011    ///   ▲            │
1012    ///   │            │
1013    ///   └────────────┘
1014    ///
1015    /// Note that at most one of `stream` or `replay_stream` can be active at a time.
1016    /// During normal operation, `stream` is active and `replay_stream` is not.  During
1017    /// replay, `replay_stream` is active while the operator that normally write to
1018    /// stream is disabled.
1019    pub fn prepare_replay_stream(&mut self, stream: &Stream<C, B>) -> Stream<C, B> {
1020        let replay_stream = Stream::with_value(
1021            stream.circuit().clone(),
1022            self.global_id.local_node_id().unwrap(),
1023            stream.value(),
1024        );
1025
1026        self.delta_stream = Some(replay_stream.clone());
1027        replay_stream
1028    }
1029}
1030
1031impl<C, B, T> Operator for Z1Trace<C, B, T>
1032where
1033    C: Circuit,
1034    B: Batch,
1035    T: Trace,
1036{
1037    fn name(&self) -> Cow<'static, str> {
1038        Cow::from("Z1 (trace)")
1039    }
1040
1041    fn clock_start(&mut self, scope: Scope) {
1042        self.dirty[scope as usize] = false;
1043
1044        if scope == 0 && self.trace.is_none() {
1045            self.trace = Some(T::new(&self.trace_factories));
1046        }
1047    }
1048
1049    fn clock_end(&mut self, scope: Scope) {
1050        if scope + 1 == self.root_scope
1051            && !self.reset_on_clock_start
1052            && let Some(tr) = self.trace.as_mut()
1053        {
1054            tr.set_frontier(&self.time.epoch_start(scope));
1055        }
1056        self.time = self.time.advance(scope + 1);
1057    }
1058
1059    fn init(&mut self, global_id: &GlobalNodeId) {
1060        self.global_id = global_id.clone();
1061    }
1062
1063    fn metadata(&self, meta: &mut OperatorMeta) {
1064        let total_size = self
1065            .trace
1066            .as_ref()
1067            .map(|trace| trace.num_entries_deep())
1068            .unwrap_or(0);
1069
1070        let bytes = self
1071            .trace
1072            .as_ref()
1073            .map(|trace| trace.size_of())
1074            .unwrap_or_default();
1075
1076        meta.extend(metadata! {
1077            STATE_RECORDS_COUNT => MetaItem::Count(total_size),
1078            ALLOCATED_MEMORY_BYTES => MetaItem::bytes(bytes.total_bytes()),
1079            USED_MEMORY_BYTES => MetaItem::bytes(bytes.used_bytes()),
1080            MEMORY_ALLOCATIONS_COUNT => MetaItem::Count(bytes.distinct_allocations()),
1081            SHARED_MEMORY_BYTES => MetaItem::bytes(bytes.shared_bytes()),
1082            RETAINMENT_BOUNDS => self.bounds.metadata()
1083        });
1084
1085        if let Some(trace) = self.trace.as_ref() {
1086            trace.metadata(meta);
1087        }
1088    }
1089
1090    fn fixedpoint(&self, scope: Scope) -> bool {
1091        !self.dirty[scope as usize] && self.replay_state.is_none()
1092    }
1093
1094    fn checkpoint(
1095        &mut self,
1096        base: &StoragePath,
1097        pid: Option<&str>,
1098        files: &mut Vec<Arc<dyn FileCommitter>>,
1099    ) -> Result<(), Error> {
1100        let pid = require_persistent_id(pid, &self.global_id)?;
1101        self.trace
1102            .as_mut()
1103            .map(|trace| trace.save(base, pid, files))
1104            .unwrap_or(Ok(()))
1105    }
1106
1107    fn restore(&mut self, base: &StoragePath, pid: Option<&str>) -> Result<(), Error> {
1108        let pid = require_persistent_id(pid, &self.global_id)?;
1109
1110        self.trace
1111            .as_mut()
1112            .map(|trace| trace.restore(base, pid))
1113            .unwrap_or(Ok(()))
1114    }
1115
1116    fn clear_state(&mut self) -> Result<(), Error> {
1117        // println!("Z1Trace-{}::clear_state", &self.global_id);
1118        self.trace = Some(T::new(&self.trace_factories));
1119        self.replay_state = None;
1120        self.dirty = vec![false; self.root_scope as usize + 1];
1121
1122        Ok(())
1123    }
1124
1125    fn start_replay(&mut self) -> Result<(), Error> {
1126        // The second condition is necessary if `start_replay` is called twice, for the input
1127        // and output halves of Z1.
1128        // println!(
1129        //     "Z1Trace-{}::start_replay delta_stream: {:?}",
1130        //     &self.global_id,
1131        //     self.delta_stream.is_some()
1132        // );
1133        self.replay_mode = true;
1134        if self.delta_stream.is_some() && self.replay_state.is_none() {
1135            let trace = self.trace.take().expect("Z1Trace::start_replay: no trace");
1136            self.trace = Some(T::new(&self.trace_factories));
1137
1138            //println!("Z1Trace-{}::initializing replay_state", &self.global_id);
1139
1140            self.replay_state = Some(ReplayState::create(trace));
1141        }
1142
1143        Ok(())
1144    }
1145
1146    fn is_replay_complete(&self) -> bool {
1147        self.replay_state.is_none()
1148    }
1149
1150    fn end_replay(&mut self) -> Result<(), Error> {
1151        //println!("Z1Trace-{}::end_replay", &self.global_id);
1152        self.replay_mode = false;
1153        self.replay_state = None;
1154
1155        Ok(())
1156    }
1157
1158    fn flush(&mut self) {
1159        self.flush = true;
1160    }
1161}
1162
1163impl<C, B, T> StrictOperator<T> for Z1Trace<C, B, T>
1164where
1165    C: Circuit,
1166    B: Batch<Key = T::Key, Val = T::Val, Time = (), R = T::R>,
1167    T: Trace,
1168{
1169    fn get_output(&mut self) -> T {
1170        //println!("Z1-{}::get_output", &self.global_id);
1171        let replay_step_size = Runtime::replay_step_size();
1172
1173        if self.replay_mode {
1174            if let Some(replay) = &mut self.replay_state {
1175                //println!("Z1-{}::get_output: replaying", &self.global_id);
1176                let mut builder = <B::Builder as Builder<B>>::with_capacity(
1177                    &self.batch_factories,
1178                    replay_step_size,
1179                    replay_step_size,
1180                );
1181
1182                let mut num_values = 0;
1183                let mut weight = self.batch_factories.weight_factory().default_box();
1184
1185                while replay.borrow_cursor().key_valid() && num_values < replay_step_size {
1186                    let mut values_added = false;
1187                    while replay.borrow_cursor().val_valid() && num_values < replay_step_size {
1188                        weight.set_zero();
1189                        replay.with_cursor_mut(|cursor| {
1190                            cursor.map_times(&mut |_t, w| weight.add_assign(w))
1191                        });
1192
1193                        if !weight.is_zero() {
1194                            builder.push_val_diff(replay.borrow_cursor().val(), weight.as_ref());
1195                            values_added = true;
1196                            num_values += 1;
1197                        }
1198                        replay.with_cursor_mut(|cursor| cursor.step_val());
1199                    }
1200                    if values_added {
1201                        builder.push_key(replay.borrow_cursor().key());
1202                    }
1203                    if !replay.borrow_cursor().val_valid() {
1204                        replay.with_cursor_mut(|cursor| cursor.step_key());
1205                    }
1206                }
1207
1208                let batch = builder.done();
1209                self.delta_stream.as_ref().unwrap().value().put(batch);
1210                if !replay.borrow_cursor().key_valid() {
1211                    self.replay_state = None;
1212                }
1213            } else {
1214                // Continue producing empty outputs as long as the circuit is in the replay mode.
1215                self.delta_stream
1216                    .as_ref()
1217                    .unwrap()
1218                    .value()
1219                    .put(B::dyn_empty(&self.batch_factories));
1220            }
1221        }
1222
1223        let mut result = self.trace.take().unwrap();
1224        result.clear_dirty_flag();
1225        result
1226    }
1227
1228    fn get_final_output(&mut self) -> T {
1229        // We only create the operator using `add_feedback_with_export` if
1230        // `reset_on_clock_start` is true, so this should never get invoked
1231        // otherwise.
1232        assert!(self.reset_on_clock_start);
1233        self.get_output()
1234    }
1235}
1236
1237impl<C, B, T> StrictUnaryOperator<T, T> for Z1Trace<C, B, T>
1238where
1239    C: Circuit,
1240    B: Batch<Key = T::Key, Val = T::Val, Time = (), R = T::R>,
1241    T: Trace,
1242{
1243    async fn eval_strict(&mut self, _i: &T) {
1244        unimplemented!()
1245    }
1246
1247    async fn eval_strict_owned(&mut self, mut i: T) {
1248        // println!("Z1-{}::eval_strict_owned", &self.global_id);
1249
1250        if self.flush {
1251            self.time = self.time.advance(0);
1252            self.flush = false;
1253        }
1254
1255        let dirty = i.dirty();
1256
1257        if let Some(filter) = self.bounds.effective_key_filter() {
1258            i.retain_keys(filter);
1259        }
1260
1261        if let Some(filter) = self.bounds.effective_val_filter() {
1262            i.retain_values(filter);
1263        }
1264
1265        self.trace = Some(i);
1266
1267        self.dirty[0] = dirty;
1268        if dirty {
1269            self.dirty.fill(true);
1270        }
1271    }
1272
1273    fn input_preference(&self) -> OwnershipPreference {
1274        OwnershipPreference::PREFER_OWNED
1275    }
1276}
1277
1278#[cfg(test)]
1279mod test {
1280    use std::{
1281        cmp::max,
1282        collections::{BTreeMap, BTreeSet},
1283    };
1284
1285    use crate::{
1286        OrdIndexedZSet, Runtime, Stream, TypedBox, ZWeight,
1287        dynamic::DynData,
1288        typed_batch::{self, Spine},
1289        utils::Tup2,
1290    };
1291    use proptest::{collection::vec, prelude::*};
1292    use size_of::SizeOf;
1293
1294    fn quasi_monotone_batches(
1295        key_window_size: i32,
1296        key_window_step: i32,
1297        val_window_size: i32,
1298        val_window_step: i32,
1299        max_tuples: usize,
1300        batches: usize,
1301    ) -> impl Strategy<Value = Vec<Vec<((i32, i32), ZWeight)>>> {
1302        (0..batches)
1303            .map(|i| {
1304                vec(
1305                    (
1306                        (
1307                            i as i32 * key_window_step
1308                                ..i as i32 * key_window_step + key_window_size,
1309                            i as i32 * val_window_step
1310                                ..i as i32 * val_window_step + val_window_size,
1311                        ),
1312                        1..2i64,
1313                    ),
1314                    0..max_tuples,
1315                )
1316            })
1317            .collect::<Vec<_>>()
1318    }
1319
1320    /// Test that `integrate_trace_retain_XXX` functions don't discard more than
1321    /// they are supposed to.
1322    //
1323    // TODO: this test used to also check that the memory footprint of the collection
1324    // is bounded, but this check was flaky, since GC is lazy.
1325    fn test_integrate_trace_retain(
1326        batches: Vec<Vec<((i32, i32), ZWeight)>>,
1327        key_lateness: i32,
1328        val_lateness: i32,
1329    ) {
1330        let (mut dbsp, input_handle) = Runtime::init_circuit(4, move |circuit| {
1331            let (stream, handle) = circuit.add_input_indexed_zset::<i32, i32>();
1332            let stream = stream.shard();
1333            let watermark: Stream<_, TypedBox<(i32, i32), DynData>> = stream.waterline(
1334                || (i32::MIN, i32::MIN),
1335                |k, v| (*k, *v),
1336                |(ts1_left, ts2_left), (ts1_right, ts2_right)| {
1337                    (max(*ts1_left, *ts1_right), max(*ts2_left, *ts2_right))
1338                },
1339            );
1340
1341            // Track all records in `stream` (this integral is not GC'd).
1342            let trace = stream.integrate_trace();
1343
1344            // Test integrate_trace_retain_keys.
1345            let stream1 = stream.map_index(|(k, v)| (*k, *v)).shard();
1346            let trace1 = stream.integrate_trace();
1347            stream1.integrate_trace_retain_keys(&watermark, move |key, ts| {
1348                *key >= ts.0.saturating_sub(key_lateness)
1349            });
1350
1351            // trace1.apply(|trace| {
1352            //     // println!("retain_keys: {}bytes", trace.size_of().total_bytes());
1353            //     assert!(trace.size_of().total_bytes() < 100_000);
1354            // });
1355
1356            // Test `integrate_trace_retain_values`.
1357            let stream2 = stream.map_index(|(k, v)| (*k, *v)).shard();
1358
1359            let trace2 = stream2.integrate_trace();
1360            stream2.integrate_trace_retain_values(&watermark, move |val, ts| {
1361                *val >= ts.1.saturating_sub(val_lateness)
1362            });
1363
1364            trace2.apply(|trace| {
1365                // println!("retain_vals: {}bytes", trace.size_of().total_bytes());
1366                assert!(trace.size_of().total_bytes() < 100_000);
1367            });
1368
1369            // Test `integrate_trace_retain_values_last_n(1)`.
1370            let stream3 = stream.map_index(|(k, v)| (*k, *v)).shard();
1371
1372            let trace3 = stream3.integrate_trace();
1373            stream3.integrate_trace_retain_values_last_n(
1374                &watermark,
1375                move |val, ts| *val >= ts.1.saturating_sub(val_lateness),
1376                1,
1377            );
1378
1379            // Test `integrate_trace_retain_values_last_n(3)`.
1380            let stream4 = stream.map_index(|(k, v)| (*k, *v)).shard();
1381
1382            let trace4 = stream4.integrate_trace();
1383            stream4.integrate_trace_retain_values_last_n(
1384                &watermark,
1385                move |val, ts| *val >= ts.1.saturating_sub(val_lateness),
1386                3,
1387            );
1388
1389            // Test `integrate_trace_retain_values_top_n(1)`.
1390            let stream5 = stream.map_index(|(k, v)| (*k, *v)).shard();
1391
1392            let trace5 = stream5.integrate_trace();
1393            stream5.integrate_trace_retain_values_top_n(
1394                &watermark,
1395                move |val, ts| *val >= ts.1.saturating_sub(val_lateness),
1396                1,
1397            );
1398
1399            // Test `integrate_trace_retain_values_top_n(3)`.
1400            let stream6 = stream.map_index(|(k, v)| (*k, *v)).shard();
1401
1402            let trace6 = stream6.integrate_trace();
1403            stream6.integrate_trace_retain_values_top_n(
1404                &watermark,
1405                move |val, ts| *val >= ts.1.saturating_sub(val_lateness),
1406                1,
1407            );
1408
1409            // Validate outputs.
1410            trace.apply3(&trace1, &watermark, move |trace, trace1, ts| {
1411                let expected = typed_batch::IndexedZSetReader::iter(trace.as_ref())
1412                    .filter(|(k, _v, _w)| *k >= ts.0.saturating_sub(key_lateness))
1413                    .collect::<BTreeSet<_>>();
1414                let actual =
1415                    typed_batch::IndexedZSetReader::iter(trace1.as_ref()).collect::<BTreeSet<_>>();
1416
1417                let missing = expected.difference(&actual).collect::<Vec<_>>();
1418                assert!(
1419                    missing.is_empty(),
1420                    "missing tuples in trace1: {:?}",
1421                    missing
1422                );
1423            });
1424
1425            trace.apply3(&trace2, &watermark, move |trace, trace2, ts| {
1426                let expected = typed_batch::IndexedZSetReader::iter(trace.as_ref())
1427                    .filter(|(_k, v, _w)| *v >= ts.1.saturating_sub(val_lateness))
1428                    .collect::<BTreeSet<_>>();
1429                let actual =
1430                    typed_batch::IndexedZSetReader::iter(trace2.as_ref()).collect::<BTreeSet<_>>();
1431
1432                let missing = expected.difference(&actual).collect::<Vec<_>>();
1433                assert!(
1434                    missing.is_empty(),
1435                    "missing tuples in trace2: {:?}",
1436                    missing
1437                );
1438            });
1439
1440            fn test_retain_values_last_n(
1441                trace: &Spine<OrdIndexedZSet<i32, i32>>,
1442                watermark: &TypedBox<(i32, i32), DynData>,
1443                val_lateness: i32,
1444                n: usize,
1445            ) -> BTreeSet<(i32, i32, ZWeight)> {
1446                let mut all_tuples = BTreeMap::new();
1447                typed_batch::IndexedZSetReader::iter(trace).for_each(|(k, v, w)| {
1448                    all_tuples
1449                        .entry(k)
1450                        .or_insert_with(|| Vec::new())
1451                        .push((v, w))
1452                });
1453                let mut expected = BTreeSet::new();
1454                for (k, mut tuples) in all_tuples.into_iter() {
1455                    let index = tuples
1456                        .iter()
1457                        .position(|(v, _w)| *v >= watermark.1.saturating_sub(val_lateness))
1458                        .unwrap_or(tuples.len());
1459                    let first_index = index.saturating_sub(n);
1460                    tuples.drain(first_index..).for_each(|(v, w)| {
1461                        let _ = expected.insert((k, v, w));
1462                    });
1463                }
1464                expected
1465            }
1466
1467            fn test_retain_values_top_n(
1468                trace: &Spine<OrdIndexedZSet<i32, i32>>,
1469                watermark: &TypedBox<(i32, i32), DynData>,
1470                val_lateness: i32,
1471                n: usize,
1472            ) -> BTreeSet<(i32, i32, ZWeight)> {
1473                let mut all_tuples = BTreeMap::new();
1474                typed_batch::IndexedZSetReader::iter(trace).for_each(|(k, v, w)| {
1475                    all_tuples
1476                        .entry(k)
1477                        .or_insert_with(|| Vec::new())
1478                        .push((v, w))
1479                });
1480
1481                let mut expected = BTreeSet::new();
1482                for (k, mut tuples) in all_tuples.into_iter() {
1483                    tuples.retain(|(v, _w)| *v >= watermark.1.saturating_sub(val_lateness));
1484                    let first_index = tuples.len().saturating_sub(n);
1485                    tuples.drain(first_index..).for_each(|(v, w)| {
1486                        let _ = expected.insert((k, v, w));
1487                    });
1488                }
1489                expected
1490            }
1491
1492            trace.apply3(&trace3, &watermark, move |trace, trace3, ts| {
1493                let mut all_tuples = BTreeMap::new();
1494                typed_batch::IndexedZSetReader::iter(trace.as_ref()).for_each(|(k, v, w)| {
1495                    all_tuples
1496                        .entry(k)
1497                        .or_insert_with(|| Vec::new())
1498                        .push((v, w))
1499                });
1500
1501                let expected =
1502                    test_retain_values_last_n(trace.as_ref(), ts.as_ref(), val_lateness, 1);
1503
1504                let actual =
1505                    typed_batch::IndexedZSetReader::iter(trace3.as_ref()).collect::<BTreeSet<_>>();
1506
1507                let missing = expected.difference(&actual).collect::<Vec<_>>();
1508                assert!(
1509                    missing.is_empty(),
1510                    "missing tuples in trace3: {:?}",
1511                    missing
1512                );
1513            });
1514
1515            trace.apply3(&trace4, &watermark, move |trace, trace4, ts| {
1516                let mut all_tuples = BTreeMap::new();
1517                typed_batch::IndexedZSetReader::iter(trace.as_ref()).for_each(|(k, v, w)| {
1518                    all_tuples
1519                        .entry(k)
1520                        .or_insert_with(|| Vec::new())
1521                        .push((v, w))
1522                });
1523
1524                let expected =
1525                    test_retain_values_last_n(trace.as_ref(), ts.as_ref(), val_lateness, 3);
1526
1527                let actual =
1528                    typed_batch::IndexedZSetReader::iter(trace4.as_ref()).collect::<BTreeSet<_>>();
1529
1530                let missing = expected.difference(&actual).collect::<Vec<_>>();
1531                assert!(
1532                    missing.is_empty(),
1533                    "missing tuples in trace4: {:?}",
1534                    missing
1535                );
1536            });
1537
1538            trace.apply3(&trace5, &watermark, move |trace, trace5, ts| {
1539                let mut all_tuples = BTreeMap::new();
1540                typed_batch::IndexedZSetReader::iter(trace.as_ref()).for_each(|(k, v, w)| {
1541                    all_tuples
1542                        .entry(k)
1543                        .or_insert_with(|| Vec::new())
1544                        .push((v, w))
1545                });
1546
1547                let expected =
1548                    test_retain_values_top_n(trace.as_ref(), ts.as_ref(), val_lateness, 1);
1549
1550                let actual =
1551                    typed_batch::IndexedZSetReader::iter(trace5.as_ref()).collect::<BTreeSet<_>>();
1552
1553                let missing = expected.difference(&actual).collect::<Vec<_>>();
1554                assert!(
1555                    missing.is_empty(),
1556                    "missing tuples in trace5: {:?}",
1557                    missing
1558                );
1559            });
1560
1561            trace.apply3(&trace6, &watermark, move |trace, trace6, ts| {
1562                let mut all_tuples = BTreeMap::new();
1563                typed_batch::IndexedZSetReader::iter(trace.as_ref()).for_each(|(k, v, w)| {
1564                    all_tuples
1565                        .entry(k)
1566                        .or_insert_with(|| Vec::new())
1567                        .push((v, w))
1568                });
1569
1570                let expected =
1571                    test_retain_values_top_n(trace.as_ref(), ts.as_ref(), val_lateness, 3);
1572
1573                let actual =
1574                    typed_batch::IndexedZSetReader::iter(trace6.as_ref()).collect::<BTreeSet<_>>();
1575
1576                let missing = expected.difference(&actual).collect::<Vec<_>>();
1577                assert!(
1578                    missing.is_empty(),
1579                    "missing tuples in trace6: {:?}",
1580                    missing
1581                );
1582            });
1583
1584            Ok(handle)
1585        })
1586        .unwrap();
1587
1588        for batch in batches {
1589            let mut tuples = batch
1590                .into_iter()
1591                .map(|((k, v), r)| Tup2(k, Tup2(v, r)))
1592                .collect::<Vec<_>>();
1593            input_handle.append(&mut tuples);
1594            dbsp.transaction().unwrap();
1595        }
1596    }
1597
1598    /// Deterministic input that inserts and retracts records in order to simulate a situation where
1599    /// records present in a batch are not present in the trace because they were deleted in a subsequent batch.
1600    /// This triggered a bug in the initial implementation of LastN filters.
1601    #[test]
1602    fn test_integrate_trace_retain_with_deletions() {
1603        let mut batches: Vec<Vec<((i32, i32), ZWeight)>> = vec![];
1604        for i in 0..1000 {
1605            batches.push(vec![
1606                ((i, i), 1), // Keep this value, delete subsequent values
1607                ((i, i + 1), 1),
1608                ((i, i + 2), 1),
1609                ((i, i + 3), 1),
1610                ((i, i + 4), 1),
1611            ]);
1612            batches.push(vec![
1613                ((i, i + 1), -1),
1614                ((i, i + 2), -1),
1615                ((i, i + 3), -1),
1616                ((i, i + 4), -1),
1617            ]);
1618        }
1619
1620        test_integrate_trace_retain(batches, 10, 10);
1621    }
1622
1623    proptest! {
1624        #![proptest_config(ProptestConfig::with_cases(16))]
1625
1626        #[test]
1627        fn test_integrate_trace_retain_proptest(batches in quasi_monotone_batches(100, 20, 1000, 200, 100, 200)) {
1628            test_integrate_trace_retain(batches, 100, 1000);
1629        }
1630    }
1631}