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
// Copyright Materialize, Inc. and contributors. All rights reserved.
//
// Use of this software is governed by the Business Source License
// included in the LICENSE file.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0.

//! TopK execution logic.
//!
//! Consult [TopKPlan] documentation for details.

use std::cell::RefCell;
use std::collections::BTreeMap;
use std::rc::Rc;

use differential_dataflow::hashable::Hashable;
use differential_dataflow::lattice::Lattice;
use differential_dataflow::operators::arrange::{Arranged, TraceAgent};
use differential_dataflow::trace::cursor::IntoOwned;
use differential_dataflow::trace::{Batch, Builder, Trace, TraceReader};
use differential_dataflow::{AsCollection, Collection};
use mz_compute_types::plan::top_k::{
    BasicTopKPlan, MonotonicTop1Plan, MonotonicTopKPlan, TopKPlan,
};
use mz_expr::func::CastUint64ToInt64;
use mz_expr::{BinaryFunc, EvalError, MirScalarExpr, UnaryFunc};
use mz_ore::cast::CastFrom;
use mz_ore::soft_assert_or_log;
use mz_repr::{Datum, DatumVec, Diff, Row, ScalarType, SharedRow};
use mz_storage_types::errors::DataflowError;
use mz_timely_util::operator::CollectionExt;
use timely::container::columnation::Columnation;
use timely::container::{CapacityContainerBuilder, PushInto};
use timely::dataflow::channels::pact::Pipeline;
use timely::dataflow::operators::Operator;
use timely::dataflow::Scope;
use timely::Container;

use crate::extensions::arrange::{ArrangementSize, KeyCollection, MzArrange};
use crate::extensions::reduce::MzReduce;
use crate::render::context::{CollectionBundle, Context};
use crate::render::errors::MaybeValidatingRow;
use crate::render::Pairer;
use crate::row_spine::{
    DatumSeq, RowBatcher, RowBuilder, RowRowBatcher, RowRowBuilder, RowValBuilder, RowValSpine,
};
use crate::typedefs::{KeyBatcher, RowRowSpine, RowSpine};

// The implementation requires integer timestamps to be able to delay feedback for monotonic inputs.
impl<G> Context<G>
where
    G: Scope,
    G::Timestamp: crate::render::RenderTimestamp,
{
    pub(crate) fn render_topk(
        &self,
        input: CollectionBundle<G>,
        top_k_plan: TopKPlan,
    ) -> CollectionBundle<G> {
        let (ok_input, err_input) = input.as_specific_collection(None);

        // We create a new region to compartmentalize the topk logic.
        let (ok_result, err_collection) = ok_input.scope().region_named("TopK", |inner| {
            let ok_input = ok_input.enter_region(inner);
            let mut err_collection = err_input.enter_region(inner);

            // Determine if there should be errors due to limit evaluation; update `err_collection`.
            // TODO(vmarcos): We evaluate the limit expression below for each input update. There
            // is an opportunity to do so for every group key instead if the error handling is
            // integrated with: 1. The intra-timestamp thinning step in monotonic top-k, e.g., by
            // adding an error output there; 2. The validating reduction on basic top-k
            // (database-issues#7108).
            let limit_err = match &top_k_plan {
                TopKPlan::MonotonicTop1(MonotonicTop1Plan { .. }) => None,
                TopKPlan::MonotonicTopK(MonotonicTopKPlan { limit, .. }) => Some(limit),
                TopKPlan::Basic(BasicTopKPlan { limit, .. }) => Some(limit),
            };
            if let Some(limit) = limit_err {
                if let Some(expr) = limit {
                    // Produce errors from limit selectors that error or are
                    // negative, and nothing from limit selectors that do
                    // not. Note that even if expr.could_error() is false,
                    // the expression might still return a negative limit and
                    // thus needs to be checked.
                    let expr = expr.clone();
                    let mut datum_vec = mz_repr::DatumVec::new();
                    let errors = ok_input.flat_map(move |row| {
                        let temp_storage = mz_repr::RowArena::new();
                        let datums = datum_vec.borrow_with(&row);
                        match expr.eval(&datums[..], &temp_storage) {
                            Ok(l) if l != Datum::Null && l.unwrap_int64() < 0 => {
                                Some(EvalError::NegLimit.into())
                            }
                            Ok(_) => None,
                            Err(e) => Some(e.into()),
                        }
                    });
                    err_collection = err_collection.concat(&errors);
                }
            }

            let ok_result = match top_k_plan {
                TopKPlan::MonotonicTop1(MonotonicTop1Plan {
                    group_key,
                    order_key,
                    must_consolidate,
                }) => {
                    let (oks, errs) = self.render_top1_monotonic(
                        ok_input,
                        group_key,
                        order_key,
                        must_consolidate,
                    );
                    err_collection = err_collection.concat(&errs);
                    oks
                }
                TopKPlan::MonotonicTopK(MonotonicTopKPlan {
                    order_key,
                    group_key,
                    arity,
                    mut limit,
                    must_consolidate,
                }) => {
                    // Must permute `limit` to reference `group_key` elements as if in order.
                    if let Some(expr) = limit.as_mut() {
                        let mut map = BTreeMap::new();
                        for (index, column) in group_key.iter().enumerate() {
                            map.insert(*column, index);
                        }
                        expr.permute_map(&map);
                    }

                    // Map the group key along with the row and consolidate if required to do so.
                    let mut datum_vec = mz_repr::DatumVec::new();
                    let collection = ok_input
                        .map(move |row| {
                            let group_row = {
                                let datums = datum_vec.borrow_with(&row);
                                SharedRow::pack(group_key.iter().map(|i| datums[*i]))
                            };
                            (group_row, row)
                        })
                        .consolidate_named_if::<KeyBatcher<_, _, _>>(
                            must_consolidate,
                            "Consolidated MonotonicTopK input",
                        );

                    // It should be now possible to ensure that we have a monotonic collection.
                    let error_logger = self.error_logger();
                    let (collection, errs) = collection.ensure_monotonic(move |data, diff| {
                        error_logger.log(
                            "Non-monotonic input to MonotonicTopK",
                            &format!("data={data:?}, diff={diff}"),
                        );
                        let m = "tried to build monotonic top-k on non-monotonic input".into();
                        (DataflowError::from(EvalError::Internal(m)), 1)
                    });
                    err_collection = err_collection.concat(&errs);

                    // For monotonic inputs, we are able to thin the input relation in two stages:
                    // 1. First, we can do an intra-timestamp thinning which has the advantage of
                    //    being computed in a streaming fashion, even for the initial snapshot.
                    // 2. Then, we can do inter-timestamp thinning by feeding back negations for
                    //    any records that have been invalidated.
                    let collection = if let Some(limit) = limit.clone() {
                        render_intra_ts_thinning(collection, order_key.clone(), limit)
                    } else {
                        collection
                    };

                    let pairer = Pairer::new(1);
                    let collection = collection.map(move |(group_row, row)| {
                        let hash = row.hashed();
                        let hash_key = pairer.merge(std::iter::once(Datum::from(hash)), &group_row);
                        (hash_key, row)
                    });

                    // For monotonic inputs, we are able to retract inputs that can no longer be produced
                    // as outputs. Any inputs beyond `offset + limit` will never again be produced as
                    // outputs, and can be removed. The simplest form of this is when `offset == 0` and
                    // these removable records are those in the input not produced in the output.
                    // TODO: consider broadening this optimization to `offset > 0` by first filtering
                    // down to `offset = 0` and `limit = offset + limit`, followed by a finishing act
                    // of `offset` and `limit`, discarding only the records not produced in the intermediate
                    // stage.
                    use differential_dataflow::operators::iterate::Variable;
                    let delay = std::time::Duration::from_secs(10);
                    let retractions = Variable::new(
                        &mut ok_input.scope(),
                        <G::Timestamp as crate::render::RenderTimestamp>::system_delay(
                            delay.try_into().expect("must fit"),
                        ),
                    );
                    let thinned = collection.concat(&retractions.negate());

                    // As an additional optimization, we can skip creating the full topk hierachy
                    // here since we now have an upper bound on the number records due to the
                    // intra-ts thinning. The maximum number of records per timestamp is
                    // (num_workers * limit), which we expect to be a small number and so we render
                    // a single topk stage.
                    let (result, errs) =
                        self.build_topk_stage(thinned, order_key, 1u64, 0, limit, arity, false);
                    // Consolidate the output of `build_topk_stage` because it's not guaranteed to be.
                    let result = result.consolidate_named::<KeyBatcher<_, _, _>>(
                        "Monotonic TopK final consolidate",
                    );
                    retractions.set(&collection.concat(&result.negate()));
                    soft_assert_or_log!(
                        errs.is_none(),
                        "requested no validation, but received error collection"
                    );

                    result.map(|(_key_hash, row)| row)
                }
                TopKPlan::Basic(BasicTopKPlan {
                    group_key,
                    order_key,
                    offset,
                    mut limit,
                    arity,
                    buckets,
                }) => {
                    // Must permute `limit` to reference `group_key` elements as if in order.
                    if let Some(expr) = limit.as_mut() {
                        let mut map = BTreeMap::new();
                        for (index, column) in group_key.iter().enumerate() {
                            map.insert(*column, index);
                        }
                        expr.permute_map(&map);
                    }

                    let (oks, errs) = self.build_topk(
                        ok_input, group_key, order_key, offset, limit, arity, buckets,
                    );
                    err_collection = err_collection.concat(&errs);
                    oks
                }
            };

            // Extract the results from the region.
            (ok_result.leave_region(), err_collection.leave_region())
        });

        CollectionBundle::from_collections(ok_result, err_collection)
    }

    /// Constructs a TopK dataflow subgraph.
    fn build_topk<S>(
        &self,
        collection: Collection<S, Row, Diff>,
        group_key: Vec<usize>,
        order_key: Vec<mz_expr::ColumnOrder>,
        offset: usize,
        limit: Option<mz_expr::MirScalarExpr>,
        arity: usize,
        buckets: Vec<u64>,
    ) -> (Collection<S, Row, Diff>, Collection<S, DataflowError, Diff>)
    where
        S: Scope<Timestamp = G::Timestamp>,
    {
        let pairer = Pairer::new(1);
        let mut datum_vec = mz_repr::DatumVec::new();
        let mut collection = collection.map({
            move |row| {
                let group_row = {
                    let row_hash = row.hashed();
                    let datums = datum_vec.borrow_with(&row);
                    let iterator = group_key.iter().map(|i| datums[*i]);
                    pairer.merge(std::iter::once(Datum::from(row_hash)), iterator)
                };
                (group_row, row)
            }
        });

        let mut validating = true;
        let mut err_collection: Option<Collection<S, _, _>> = None;

        if let Some(mut limit) = limit.clone() {
            // We may need a new `limit` that reflects the addition of `offset`.
            // Ideally we compile it down to a literal if at all possible.
            if offset > 0 {
                let new_limit = (|| {
                    let limit = limit.as_literal_int64()?;
                    let offset = i64::try_from(offset).ok()?;
                    limit.checked_add(offset)
                })();

                if let Some(new_limit) = new_limit {
                    limit = MirScalarExpr::literal_ok(Datum::Int64(new_limit), ScalarType::Int64);
                } else {
                    limit = limit.call_binary(
                        MirScalarExpr::literal_ok(
                            Datum::UInt64(u64::cast_from(offset)),
                            ScalarType::UInt64,
                        )
                        .call_unary(UnaryFunc::CastUint64ToInt64(CastUint64ToInt64)),
                        BinaryFunc::AddInt64,
                    );
                }
            }

            // These bucket values define the shifts that happen to the 64 bit hash of the
            // record, and should have the properties that 1. there are not too many of them,
            // and 2. each has a modest difference to the next.
            for bucket in buckets.into_iter() {
                // here we do not apply `offset`, but instead restrict ourself with a limit
                // that includes the offset. We cannot apply `offset` until we perform the
                // final, complete reduction.
                let (oks, errs) = self.build_topk_stage(
                    collection,
                    order_key.clone(),
                    bucket,
                    0,
                    Some(limit.clone()),
                    arity,
                    validating,
                );
                collection = oks;
                if validating {
                    err_collection = errs;
                    validating = false;
                }
            }
        }

        // We do a final step, both to make sure that we complete the reduction, and to correctly
        // apply `offset` to the final group, as we have not yet been applying it to the partially
        // formed groups.
        let (oks, errs) = self.build_topk_stage(
            collection, order_key, 1u64, offset, limit, arity, validating,
        );
        // Consolidate the output of `build_topk_stage` because it's not guaranteed to be.
        let oks = oks.consolidate_named::<KeyBatcher<_, _, _>>("TopK final consolidate");
        collection = oks;
        if validating {
            err_collection = errs;
        }
        (
            collection.map(|(_key_hash, row)| row),
            err_collection.expect("at least one stage validated its inputs"),
        )
    }

    /// To provide a robust incremental orderby-limit experience, we want to avoid grouping *all*
    /// records (or even large groups) and then applying the ordering and limit. Instead, a more
    /// robust approach forms groups of bounded size and applies the offset and limit to each,
    /// and then increases the sizes of the groups.
    ///
    /// Builds a "stage", which uses a finer grouping than is required to reduce the volume of
    /// updates, and to reduce the amount of work on the critical path for updates. The cost is
    /// a larger number of arrangements when this optimization does nothing beneficial.
    ///
    /// The function accepts a collection of the form `(hash_key, row)`, a modulus it applies to the
    /// `hash_key`'s hash datum, an `offset` for returning results, and a `limit` to restrict the
    /// output size. `arity` represents the number of columns in the input data, and
    /// if `validating` is true, we check for negative multiplicities, which indicate
    /// an error in the input data.
    ///
    /// The output of this function is _not consolidated_.
    ///
    /// The dataflow fragment has the following shape:
    /// ```text
    ///     | input
    ///     |
    ///   arrange
    ///     |\
    ///     | \
    ///     |  reduce
    ///     |  |
    ///     concat
    ///     |
    ///     | output
    /// ```
    /// There are additional map/flat_map operators as well as error demuxing operators, but we're
    /// omitting them here for the sake of simplicity.
    fn build_topk_stage<S>(
        &self,
        collection: Collection<S, (Row, Row), Diff>,
        order_key: Vec<mz_expr::ColumnOrder>,
        modulus: u64,
        offset: usize,
        limit: Option<mz_expr::MirScalarExpr>,
        arity: usize,
        validating: bool,
    ) -> (
        Collection<S, (Row, Row), Diff>,
        Option<Collection<S, DataflowError, Diff>>,
    )
    where
        S: Scope<Timestamp = G::Timestamp>,
    {
        // Form appropriate input by updating the `hash` column (first datum in `hash_key`) by
        // applying `modulus`.
        let input = collection.map(move |(hash_key, row)| {
            let mut hash_key_iter = hash_key.iter();
            let hash = hash_key_iter.next().unwrap().unwrap_uint64() % modulus;
            let hash_key = SharedRow::pack(std::iter::once(hash.into()).chain(hash_key_iter));
            (hash_key, row)
        });

        // If validating: demux errors, otherwise we cannot produce errors.
        let (input, oks, errs) = if validating {
            // Build topk stage, produce errors for invalid multiplicities.
            let (input, stage) = build_topk_negated_stage::<
                S,
                _,
                RowValBuilder<_, _, _>,
                RowValSpine<Result<Row, Row>, _, _>,
            >(&input, order_key, offset, limit, arity);
            let stage = stage.as_collection(|k, v| (k.into_owned(), v.clone()));

            // Demux oks and errors.
            let error_logger = self.error_logger();
            type CB<C> = CapacityContainerBuilder<C>;
            let (oks, errs) = stage.map_fallible::<CB<_>, CB<_>, _, _, _>(
                "Demuxing Errors",
                move |(hk, result)| match result {
                    Err(v) => {
                        let mut hk_iter = hk.iter();
                        let h = hk_iter.next().unwrap().unwrap_uint64();
                        let k = SharedRow::pack(hk_iter);
                        let message = "Negative multiplicities in TopK";
                        error_logger.log(message, &format!("k={k:?}, h={h}, v={v:?}"));
                        Err(EvalError::Internal(message.into()).into())
                    }
                    Ok(t) => Ok((hk, t)),
                },
            );
            (input, oks, Some(errs))
        } else {
            // Build non-validating topk stage.
            let (input, stage) =
                build_topk_negated_stage::<S, _, RowRowBuilder<_, _>, RowRowSpine<_, _>>(
                    &input, order_key, offset, limit, arity,
                );
            // Turn arrangement into collection.
            let stage = stage.as_collection(|k, v| (k.into_owned(), v.into_owned()));

            (input, stage, None)
        };
        let input = input.as_collection(|k, v| (k.into_owned(), v.into_owned()));
        (oks.concat(&input), errs)
    }

    fn render_top1_monotonic<S>(
        &self,
        collection: Collection<S, Row, Diff>,
        group_key: Vec<usize>,
        order_key: Vec<mz_expr::ColumnOrder>,
        must_consolidate: bool,
    ) -> (Collection<S, Row, Diff>, Collection<S, DataflowError, Diff>)
    where
        S: Scope<Timestamp = G::Timestamp>,
    {
        // We can place our rows directly into the diff field, and only keep the relevant one
        // corresponding to evaluating our aggregate, instead of having to do a hierarchical
        // reduction. We start by mapping the group key along with the row and consolidating
        // if required to do so.
        let collection = collection
            .map({
                let mut datum_vec = mz_repr::DatumVec::new();
                move |row| {
                    // Scoped to allow borrow of `row` to drop.
                    let group_key = {
                        let datums = datum_vec.borrow_with(&row);
                        SharedRow::pack(group_key.iter().map(|i| datums[*i]))
                    };
                    (group_key, row)
                }
            })
            .consolidate_named_if::<KeyBatcher<_, _, _>>(
                must_consolidate,
                "Consolidated MonotonicTop1 input",
            );

        // It should be now possible to ensure that we have a monotonic collection and process it.
        let error_logger = self.error_logger();
        let (partial, errs) = collection.ensure_monotonic(move |data, diff| {
            error_logger.log(
                "Non-monotonic input to MonotonicTop1",
                &format!("data={data:?}, diff={diff}"),
            );
            let m = "tried to build monotonic top-1 on non-monotonic input".into();
            (EvalError::Internal(m).into(), 1)
        });
        let partial: KeyCollection<_, _, _> = partial
            .explode_one(move |(group_key, row)| {
                (
                    group_key,
                    monoids::Top1Monoid {
                        row,
                        order_key: order_key.clone(),
                    },
                )
            })
            .into();
        let result = partial
            .mz_arrange::<RowBatcher<_, _>, RowBuilder<_, _>, RowSpine<_, _>>(
                "Arranged MonotonicTop1 partial [val: empty]",
            )
            .mz_reduce_abelian::<_, _, _, RowRowBuilder<_, _>, RowRowSpine<_, _>>(
                "MonotonicTop1",
                move |_key, input, output| {
                    let accum: &monoids::Top1Monoid = &input[0].1;
                    output.push((accum.row.clone(), 1));
                },
            );
        // TODO(database-issues#2288): Here we discard the arranged output.
        (result.as_collection(|_k, v| v.into_owned()), errs)
    }
}

/// Build a stage of a topk reduction. Maintains the _retractions_ of the output instead of emitted
/// rows. This has the benefit that we have to maintain state proportionally to size of the output
/// instead of the size of the input.
///
/// Returns two arrangements:
/// * The arranged input data without modifications, and
/// * the maintained negated output data.
fn build_topk_negated_stage<G, V, Bu, Tr>(
    input: &Collection<G, (Row, Row), Diff>,
    order_key: Vec<mz_expr::ColumnOrder>,
    offset: usize,
    limit: Option<mz_expr::MirScalarExpr>,
    arity: usize,
) -> (
    Arranged<G, TraceAgent<RowRowSpine<G::Timestamp, Diff>>>,
    Arranged<G, TraceAgent<Tr>>,
)
where
    G: Scope,
    G::Timestamp: Lattice + Columnation,
    V: MaybeValidatingRow<Row, Row>,
    Bu: Builder<Time = G::Timestamp, Output = Tr::Batch>,
    Bu::Input: Container + PushInto<((Row, V), G::Timestamp, Diff)>,
    Tr: Trace
        + for<'a> TraceReader<Key<'a> = DatumSeq<'a>, Time = G::Timestamp, Diff = Diff>
        + 'static,
    for<'a> Tr::Val<'a>: IntoOwned<'a, Owned = V>,
    Tr::Batch: Batch,
    Arranged<G, TraceAgent<Tr>>: ArrangementSize,
{
    let mut datum_vec = mz_repr::DatumVec::new();

    // We only want to arrange parts of the input that are not part of the actual output
    // such that `input.concat(&negated_output)` yields the correct TopK
    // NOTE(vmarcos): The arranged input operator name below is used in the tuning advice
    // built-in view mz_introspection.mz_expected_group_size_advice.
    let arranged = input.mz_arrange::<RowRowBatcher<_, _>, RowRowBuilder<_, _>, RowRowSpine<_, _>>(
        "Arranged TopK input",
    );

    let reduced = arranged.mz_reduce_abelian::<_, _, _, Bu, Tr>("Reduced TopK input", {
        move |mut hash_key, source, target: &mut Vec<(V, Diff)>| {
            // Unpack the limit, either into an integer literal or an expression to evaluate.
            let limit: Option<i64> = limit.as_ref().map(|l| {
                if let Some(l) = l.as_literal_int64() {
                    l
                } else {
                    // Unpack `key` after skipping the hash and determine the limit.
                    // If the limit errors, use a zero limit; errors are surfaced elsewhere.
                    let temp_storage = mz_repr::RowArena::new();
                    let _hash = hash_key.next();
                    let mut key_datums = datum_vec.borrow();
                    key_datums.extend(hash_key);
                    let datum_limit = l
                        .eval(&key_datums, &temp_storage)
                        .unwrap_or(Datum::Int64(0));
                    if datum_limit == Datum::Null {
                        i64::MAX
                    } else {
                        datum_limit.unwrap_int64()
                    }
                }
            });

            if let Some(err) = V::into_error() {
                for (datums, diff) in source.iter() {
                    if diff.is_positive() {
                        continue;
                    }
                    target.push((err((*datums).into_owned()), 1));
                    return;
                }
            }

            // Determine if we must actually shrink the result set.
            let must_shrink = offset > 0
                || limit
                    .map(|l| source.iter().map(|(_, d)| *d).sum::<Diff>() > l)
                    .unwrap_or(false);
            if !must_shrink {
                return;
            }

            // First go ahead and emit all records. Note that we ensure target
            // has the capacity to hold at least these records, and avoid any
            // dependencies on the user-provided (potentially unbounded) limit.
            target.reserve(source.len());
            for (datums, diff) in source.iter() {
                target.push((V::ok((*datums).into_owned()), -diff));
            }
            // local copies that may count down to zero.
            let mut offset = offset;
            let mut limit = limit;

            // The order in which we should produce rows.
            let mut indexes = (0..source.len()).collect::<Vec<_>>();
            // We decode the datums once, into a common buffer for efficiency.
            // Each row should contain `arity` columns; we should check that.
            let mut buffer = datum_vec.borrow();
            for (index, (datums, _)) in source.iter().enumerate() {
                buffer.extend(*datums);
                assert_eq!(buffer.len(), arity * (index + 1));
            }
            let width = buffer.len() / source.len();

            //todo: use arrangements or otherwise make the sort more performant?
            indexes.sort_by(|left, right| {
                let left = &buffer[left * width..][..width];
                let right = &buffer[right * width..][..width];
                // Note: source was originally ordered by the u8 array representation
                // of rows, but left.cmp(right) uses Datum::cmp.
                mz_expr::compare_columns(&order_key, left, right, || left.cmp(right))
            });

            // We now need to lay out the data in order of `buffer`, but respecting
            // the `offset` and `limit` constraints.
            for index in indexes.into_iter() {
                let (datums, mut diff) = source[index];
                if !diff.is_positive() {
                    continue;
                }
                // If we are still skipping early records ...
                if offset > 0 {
                    let to_skip = std::cmp::min(offset, usize::try_from(diff).unwrap());
                    offset -= to_skip;
                    diff -= Diff::try_from(to_skip).unwrap();
                }
                // We should produce at most `limit` records.
                if let Some(limit) = &mut limit {
                    diff = std::cmp::min(diff, Diff::cast_from(*limit));
                    *limit -= diff;
                }
                // Output the indicated number of rows.
                if diff > 0 {
                    // Emit retractions for the elements actually part of
                    // the set of TopK elements.
                    target.push((V::ok(datums.into_owned()), diff));
                }
            }
        }
    });
    (arranged, reduced)
}

fn render_intra_ts_thinning<S>(
    collection: Collection<S, (Row, Row), Diff>,
    order_key: Vec<mz_expr::ColumnOrder>,
    limit: mz_expr::MirScalarExpr,
) -> Collection<S, (Row, Row), Diff>
where
    S: Scope,
    S::Timestamp: Lattice,
{
    let mut datum_vec = mz_repr::DatumVec::new();

    let mut aggregates = BTreeMap::new();
    let shared = Rc::new(RefCell::new(monoids::Top1MonoidShared {
        order_key,
        left: DatumVec::new(),
        right: DatumVec::new(),
    }));
    collection
        .inner
        .unary_notify(
            Pipeline,
            "TopKIntraTimeThinning",
            [],
            move |input, output, notificator| {
                while let Some((time, data)) = input.next() {
                    let agg_time = aggregates
                        .entry(time.time().clone())
                        .or_insert_with(BTreeMap::new);
                    for ((grp_row, row), record_time, diff) in data.drain(..) {
                        let monoid = monoids::Top1MonoidLocal {
                            row,
                            shared: Rc::clone(&shared),
                        };

                        // Evalute the limit, first as a constant and then against the key if needed.
                        let limit = if let Some(l) = limit.as_literal_int64() {
                            l
                        } else {
                            let temp_storage = mz_repr::RowArena::new();
                            let key_datums = datum_vec.borrow_with(&grp_row);
                            // Unpack `key` and determine the limit.
                            // If the limit errors, use a zero limit; errors are surfaced elsewhere.
                            let datum_limit = limit
                                .eval(&key_datums, &temp_storage)
                                .unwrap_or(mz_repr::Datum::Int64(0));
                            if datum_limit == Datum::Null {
                                i64::MAX
                            } else {
                                datum_limit.unwrap_int64()
                            }
                        };

                        let topk = agg_time
                            .entry((grp_row, record_time))
                            .or_insert_with(move || topk_agg::TopKBatch::new(limit));
                        topk.update(monoid, diff);
                    }
                    notificator.notify_at(time.retain());
                }

                notificator.for_each(|time, _, _| {
                    if let Some(aggs) = aggregates.remove(time.time()) {
                        let mut session = output.session(&time);
                        for ((grp_row, record_time), topk) in aggs {
                            session.give_iterator(topk.into_iter().map(|(monoid, diff)| {
                                (
                                    (grp_row.clone(), monoid.into_row()),
                                    record_time.clone(),
                                    diff,
                                )
                            }))
                        }
                    }
                });
            },
        )
        .as_collection()
}

/// Types for in-place intra-ts aggregation of monotonic streams.
pub mod topk_agg {
    use differential_dataflow::consolidation;
    use smallvec::SmallVec;

    // TODO: This struct looks a lot like ChangeBatch and indeed its code is a modified version of
    // that. It would be nice to find a way to reuse some or all of the code from there.
    //
    // Additionally, because we're calling into DD's consolidate method we are forced to work with
    // the `Ord` trait which for the usage we do above means that we need to clone the `order_key`
    // for each record. It would be nice to also remove the need for cloning that piece of data
    pub struct TopKBatch<T> {
        updates: SmallVec<[(T, i64); 16]>,
        clean: usize,
        limit: i64,
    }

    impl<T: Ord> TopKBatch<T> {
        pub fn new(limit: i64) -> Self {
            Self {
                updates: SmallVec::new(),
                clean: 0,
                limit,
            }
        }

        /// Adds a new update, for `item` with `value`.
        ///
        /// This could be optimized to perform compaction when the number of "dirty" elements exceeds
        /// half the length of the list, which would keep the total footprint within reasonable bounds
        /// even under an arbitrary number of updates. This has a cost, and it isn't clear whether it
        /// is worth paying without some experimentation.
        #[inline]
        pub fn update(&mut self, item: T, value: i64) {
            self.updates.push((item, value));
            self.maintain_bounds();
        }

        /// Compact the internal representation.
        ///
        /// This method sort `self.updates` and consolidates elements with equal item, discarding
        /// any whose accumulation is zero. It is optimized to only do this if the number of dirty
        /// elements is non-zero.
        #[inline]
        pub fn compact(&mut self) {
            if self.clean < self.updates.len() && self.updates.len() > 1 {
                let len = consolidation::consolidate_slice(&mut self.updates);
                self.updates.truncate(len);

                // We can now retain only the first K records and throw away everything else
                let mut limit = self.limit;
                self.updates.retain(|x| {
                    if limit > 0 {
                        limit -= x.1;
                        true
                    } else {
                        false
                    }
                });
                // By the end of the loop above `limit` will either be:
                // (a) Positive, in which case all updates were retained;
                // (b) Zero, in which case we discarded all updates after limit became zero;
                // (c) Negative, in which case the last record we retained had more copies
                // than necessary. In this latter case, we need to do one final adjustment
                // of the diff field of the last record so that the total sum of the diffs
                // in the batch is K.
                if limit < 0 {
                    if let Some(item) = self.updates.last_mut() {
                        // We are subtracting the limit *negated*, therefore we are subtracting a value
                        // that is *greater* than or equal to zero, which represents the excess.
                        item.1 -= -limit;
                    }
                }
            }
            self.clean = self.updates.len();
        }

        /// Maintain the bounds of pending (non-compacted) updates versus clean (compacted) data.
        /// This function tries to minimize work by only compacting if enough work has accumulated.
        fn maintain_bounds(&mut self) {
            // if we have more than 32 elements and at least half of them are not clean, compact
            if self.updates.len() > 32 && self.updates.len() >> 1 >= self.clean {
                self.compact()
            }
        }
    }

    impl<T: Ord> IntoIterator for TopKBatch<T> {
        type Item = (T, i64);
        type IntoIter = smallvec::IntoIter<[(T, i64); 16]>;

        fn into_iter(mut self) -> Self::IntoIter {
            self.compact();
            self.updates.into_iter()
        }
    }
}

/// Monoids for in-place compaction of monotonic streams.
pub mod monoids {
    use std::cell::RefCell;
    use std::cmp::Ordering;
    use std::hash::{Hash, Hasher};
    use std::rc::Rc;

    use differential_dataflow::difference::{IsZero, Multiply, Semigroup};
    use mz_expr::ColumnOrder;
    use mz_repr::{DatumVec, Diff, Row};
    use serde::{Deserialize, Serialize};
    use timely::container::columnation::{Columnation, Region};

    /// A monoid containing a row and an ordering.
    #[derive(Eq, PartialEq, Debug, Clone, Serialize, Deserialize, Hash)]
    pub struct Top1Monoid {
        pub row: Row,
        pub order_key: Vec<ColumnOrder>,
    }

    impl Multiply<Diff> for Top1Monoid {
        type Output = Self;

        fn multiply(self, factor: &Diff) -> Self {
            // Multiplication in Top1Monoid is idempotent, and its
            // users must ascertain its monotonicity beforehand
            // (typically with ensure_monotonic) since it has no zero
            // value for us to use here.
            assert!(factor.is_positive());
            self
        }
    }

    impl Ord for Top1Monoid {
        fn cmp(&self, other: &Self) -> Ordering {
            debug_assert_eq!(self.order_key, other.order_key);

            // It might be nice to cache this row decoding like the non-monotonic codepath, but we'd
            // have to store the decoded Datums in the same struct as the Row, which gets tricky.
            let left: Vec<_> = self.row.unpack();
            let right: Vec<_> = other.row.unpack();
            mz_expr::compare_columns(&self.order_key, &left, &right, || left.cmp(&right))
        }
    }
    impl PartialOrd for Top1Monoid {
        fn partial_cmp(&self, other: &Self) -> Option<Ordering> {
            Some(self.cmp(other))
        }
    }

    impl Semigroup for Top1Monoid {
        fn plus_equals(&mut self, rhs: &Self) {
            let cmp = (*self).cmp(rhs);
            // NB: Reminder that TopK returns the _minimum_ K items.
            if cmp == Ordering::Greater {
                self.clone_from(rhs);
            }
        }
    }

    impl IsZero for Top1Monoid {
        fn is_zero(&self) -> bool {
            false
        }
    }

    impl Columnation for Top1Monoid {
        type InnerRegion = Top1MonoidRegion;
    }

    #[derive(Default)]
    pub struct Top1MonoidRegion {
        row_region: <Row as Columnation>::InnerRegion,
        order_key_region: <Vec<ColumnOrder> as Columnation>::InnerRegion,
    }

    impl Region for Top1MonoidRegion {
        type Item = Top1Monoid;

        unsafe fn copy(&mut self, item: &Self::Item) -> Self::Item {
            let row = self.row_region.copy(&item.row);
            let order_key = self.order_key_region.copy(&item.order_key);
            Self::Item { row, order_key }
        }

        fn clear(&mut self) {
            self.row_region.clear();
            self.order_key_region.clear();
        }

        fn reserve_items<'a, I>(&mut self, items1: I)
        where
            Self: 'a,
            I: Iterator<Item = &'a Self::Item> + Clone,
        {
            let items2 = items1.clone();
            self.row_region
                .reserve_items(items1.into_iter().map(|s| &s.row));
            self.order_key_region
                .reserve_items(items2.into_iter().map(|s| &s.order_key));
        }

        fn reserve_regions<'a, I>(&mut self, regions1: I)
        where
            Self: 'a,
            I: Iterator<Item = &'a Self> + Clone,
        {
            let regions2 = regions1.clone();
            self.row_region
                .reserve_regions(regions1.into_iter().map(|s| &s.row_region));
            self.order_key_region
                .reserve_regions(regions2.into_iter().map(|s| &s.order_key_region));
        }

        fn heap_size(&self, mut callback: impl FnMut(usize, usize)) {
            self.row_region.heap_size(&mut callback);
            self.order_key_region.heap_size(callback);
        }
    }

    /// A shared portion of a thread-local top-1 monoid implementation.
    #[derive(Debug)]
    pub struct Top1MonoidShared {
        pub order_key: Vec<ColumnOrder>,
        pub left: DatumVec,
        pub right: DatumVec,
    }

    /// A monoid containing a row and a shared pointer to a shared structure.
    /// Only suitable for thread-local aggregations.
    #[derive(Debug, Clone)]
    pub struct Top1MonoidLocal {
        pub row: Row,
        pub shared: Rc<RefCell<Top1MonoidShared>>,
    }

    impl Top1MonoidLocal {
        pub fn into_row(self) -> Row {
            self.row
        }
    }

    impl PartialEq for Top1MonoidLocal {
        fn eq(&self, other: &Self) -> bool {
            self.row.eq(&other.row)
        }
    }

    impl Eq for Top1MonoidLocal {}

    impl Hash for Top1MonoidLocal {
        fn hash<H: Hasher>(&self, state: &mut H) {
            self.row.hash(state);
        }
    }

    impl Ord for Top1MonoidLocal {
        fn cmp(&self, other: &Self) -> Ordering {
            debug_assert!(Rc::ptr_eq(&self.shared, &other.shared));
            let Top1MonoidShared {
                left,
                right,
                order_key,
            } = &mut *self.shared.borrow_mut();

            let left = left.borrow_with(&self.row);
            let right = right.borrow_with(&other.row);
            mz_expr::compare_columns(order_key, &left, &right, || left.cmp(&right))
        }
    }

    impl PartialOrd for Top1MonoidLocal {
        fn partial_cmp(&self, other: &Self) -> Option<Ordering> {
            Some(self.cmp(other))
        }
    }

    impl Semigroup for Top1MonoidLocal {
        fn plus_equals(&mut self, rhs: &Self) {
            let cmp = (*self).cmp(rhs);
            // NB: Reminder that TopK returns the _minimum_ K items.
            if cmp == Ordering::Greater {
                self.clone_from(rhs);
            }
        }
    }

    impl IsZero for Top1MonoidLocal {
        fn is_zero(&self) -> bool {
            false
        }
    }
}