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
// 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.

//! Management of dataflow-local state, like arrangements, while building a
//! dataflow.

use std::collections::BTreeMap;

use dataflow_types::plan::AvailableCollections;
use differential_dataflow::lattice::Lattice;
use differential_dataflow::operators::arrange::Arrange;
use differential_dataflow::operators::arrange::Arranged;
use differential_dataflow::trace::wrappers::enter::TraceEnter;
use differential_dataflow::trace::wrappers::frontier::TraceFrontier;
use differential_dataflow::trace::BatchReader;
use differential_dataflow::trace::{Cursor, TraceReader};
use differential_dataflow::Collection;
use differential_dataflow::Data;
use timely::communication::message::RefOrMut;
use timely::dataflow::channels::pact::Pipeline;
use timely::dataflow::{Scope, ScopeParent};
use timely::progress::timestamp::Refines;
use timely::progress::{Antichain, Timestamp};

use crate::arrangement::manager::{ErrSpine, RowSpine, TraceErrHandle, TraceRowHandle};
use crate::operator::CollectionExt;
use dataflow_types::{DataflowDescription, DataflowError};
use expr::{GlobalId, Id, MapFilterProject, MirScalarExpr};
use repr::{DatumVec, Diff, Row, RowArena};

// Local type definition to avoid the horror in signatures.
pub type Arrangement<S, V> = Arranged<S, TraceRowHandle<V, V, <S as ScopeParent>::Timestamp, Diff>>;
pub type ErrArrangement<S> =
    Arranged<S, TraceErrHandle<DataflowError, <S as ScopeParent>::Timestamp, Diff>>;
pub type ArrangementImport<S, V, T> = Arranged<
    S,
    TraceEnter<TraceFrontier<TraceRowHandle<V, V, T, Diff>>, <S as ScopeParent>::Timestamp>,
>;
pub type ErrArrangementImport<S, T> = Arranged<
    S,
    TraceEnter<
        TraceFrontier<TraceErrHandle<DataflowError, T, Diff>>,
        <S as ScopeParent>::Timestamp,
    >,
>;

/// Dataflow-local collections and arrangements.
///
/// A context means to wrap available data assets and present them in an easy-to-use manner.
/// These assets include dataflow-local collections and arrangements, as well as imported
/// arrangements from outside the dataflow.
///
/// Context has two timestamp types, one from `S::Timestamp` and one from `T`, where the
/// former must refine the latter. The former is the timestamp used by the scope in question,
/// and the latter is the timestamp of imported traces. The two may be different in the case
/// of regions or iteration.
pub struct Context<S: Scope, V: Data, T>
where
    T: Timestamp + Lattice,
    S::Timestamp: Lattice + Refines<T>,
{
    /// The debug name of the dataflow associated with this context.
    pub debug_name: String,
    /// The Timely ID of the dataflow associated with this context.
    pub dataflow_id: usize,
    /// Indicates a frontier that can be used to compact input timestamps
    /// without affecting the results. We *should* apply it, to sources and
    /// imported traces, both because it improves performance, and because
    /// potentially incorrect results are visible in sinks.
    pub as_of_frontier: Antichain<repr::Timestamp>,
    /// Bindings of identifiers to collections.
    pub bindings: BTreeMap<Id, CollectionBundle<S, V, T>>,
}

impl<S: Scope, V: Data, T> Context<S, V, T>
where
    T: Timestamp + Lattice,
    S::Timestamp: Lattice + Refines<T>,
{
    /// Creates a new empty Context.
    pub fn for_dataflow<Plan>(dataflow: &DataflowDescription<Plan>, dataflow_id: usize) -> Self {
        let as_of_frontier = dataflow
            .as_of
            .clone()
            .unwrap_or_else(|| Antichain::from_elem(0));

        Self {
            debug_name: dataflow.debug_name.clone(),
            dataflow_id,
            as_of_frontier,
            bindings: BTreeMap::new(),
        }
    }

    /// Insert a collection bundle by an identifier.
    ///
    /// This is expected to be used to install external collections (sources, indexes, other views),
    /// as well as for `Let` bindings of local collections.
    pub fn insert_id(
        &mut self,
        id: Id,
        collection: CollectionBundle<S, V, T>,
    ) -> Option<CollectionBundle<S, V, T>> {
        self.bindings.insert(id, collection)
    }
    /// Remove a collection bundle by an identifier.
    ///
    /// The primary use of this method is uninstalling `Let` bindings.
    pub fn remove_id(&mut self, id: Id) -> Option<CollectionBundle<S, V, T>> {
        self.bindings.remove(&id)
    }
    /// Melds a collection bundle to whatever exists.
    pub fn update_id(&mut self, id: Id, collection: CollectionBundle<S, V, T>) {
        if !self.bindings.contains_key(&id) {
            self.bindings.insert(id, collection);
        } else {
            let binding = self
                .bindings
                .get_mut(&id)
                .expect("Binding verified to exist");
            if collection.collection.is_some() {
                binding.collection = collection.collection;
            }
            for (key, flavor) in collection.arranged.into_iter() {
                binding.arranged.insert(key, flavor);
            }
        }
    }
    /// Look up a collection bundle by an identifier.
    pub fn lookup_id(&self, id: Id) -> Option<CollectionBundle<S, V, T>> {
        self.bindings.get(&id).cloned()
    }
}

/// Describes flavor of arrangement: local or imported trace.
#[derive(Clone)]
pub enum ArrangementFlavor<S: Scope, V: Data, T: Lattice>
where
    T: Timestamp + Lattice,
    S::Timestamp: Lattice + Refines<T>,
{
    /// A dataflow-local arrangement.
    Local(Arrangement<S, V>, ErrArrangement<S>),
    /// An imported trace from outside the dataflow.
    ///
    /// The `GlobalId` identifier exists so that exports of this same trace
    /// can refer back to and depend on the original instance.
    Trace(
        GlobalId,
        ArrangementImport<S, V, T>,
        ErrArrangementImport<S, T>,
    ),
}

impl<S: Scope, T> ArrangementFlavor<S, Row, T>
where
    T: Timestamp + Lattice,
    S::Timestamp: Lattice + Refines<T>,
{
    /// Presents `self` as a stream of updates.
    ///
    /// This method presents the contents as they are, without further computation.
    /// If you have logic that could be applied to each record, consider using the
    /// `flat_map` methods which allows this and can reduce the work done.
    pub fn as_collection(&self) -> (Collection<S, Row, Diff>, Collection<S, DataflowError, Diff>) {
        let mut datum_vec = DatumVec::new();
        let mut row_builder = Row::default();
        match &self {
            ArrangementFlavor::Local(oks, errs) => (
                oks.as_collection(move |k, v| {
                    let borrow = datum_vec.borrow_with_many(&[k, v]);
                    row_builder.extend(&*borrow);
                    row_builder.finish_and_reuse()
                }),
                errs.as_collection(|k, &()| k.clone()),
            ),
            ArrangementFlavor::Trace(_, oks, errs) => (
                oks.as_collection(move |k, v| {
                    let borrow = datum_vec.borrow_with_many(&[k, v]);
                    row_builder.extend(&*borrow);
                    row_builder.finish_and_reuse()
                }),
                errs.as_collection(|k, &()| k.clone()),
            ),
        }
    }

    /// Constructs and applies logic to elements of `self` and returns the results.
    ///
    /// `constructor` takes a permutation and produces the logic to apply on elements. The logic
    /// conceptually receives `(&Row, &Row)` pairs in the form of a slice. Only after borrowing
    /// the elements and applying the permutation the datums will be in the expected order.
    ///
    /// If `key` is set, this is a promise that `logic` will produce no results on
    /// records for which the key does not evaluate to the value. This is used to
    /// leap directly to exactly those records.
    pub fn flat_map<I, C, L>(
        &self,
        key: Option<Row>,
        constructor: C,
    ) -> (
        timely::dataflow::Stream<S, I::Item>,
        Collection<S, DataflowError, Diff>,
    )
    where
        I: IntoIterator,
        I::Item: Data,
        C: FnOnce() -> L,
        L: for<'a, 'b> FnMut(&'a [&'b RefOrMut<'b, Row>], &'a S::Timestamp, &'a Diff) -> I
            + 'static,
    {
        // Set a number of tuples after which the operator should yield.
        // This allows us to remain responsive even when enumerating a substantial
        // arrangement, as well as provides time to accumulate our produced output.
        let refuel = 1000000;

        match &self {
            ArrangementFlavor::Local(oks, errs) => {
                let mut logic = constructor();
                let oks = CollectionBundle::<S, Row, T>::flat_map_core(
                    &oks,
                    key,
                    move |k, v, t, d| logic(&[&k, &v], t, d),
                    refuel,
                );
                let errs = errs.as_collection(|k, &()| k.clone());
                return (oks, errs);
            }
            ArrangementFlavor::Trace(_, oks, errs) => {
                let mut logic = constructor();
                let oks = CollectionBundle::<S, Row, T>::flat_map_core(
                    &oks,
                    key,
                    move |k, v, t, d| logic(&[&k, &v], t, d),
                    refuel,
                );
                let errs = errs.as_collection(|k, &()| k.clone());
                return (oks, errs);
            }
        }
    }
}

/// A bundle of the various ways a collection can be represented.
///
/// This type maintains the invariant that it does contain at least one valid
/// source of data, either a collection or at least one arrangement.
#[derive(Clone)]
pub struct CollectionBundle<S: Scope, V: Data, T: Lattice>
where
    T: Timestamp + Lattice,
    S::Timestamp: Lattice + Refines<T>,
{
    pub collection: Option<(Collection<S, V, Diff>, Collection<S, DataflowError, Diff>)>,
    pub arranged: BTreeMap<Vec<MirScalarExpr>, ArrangementFlavor<S, V, T>>,
}

impl<S: Scope, V: Data, T: Lattice> CollectionBundle<S, V, T>
where
    T: Timestamp + Lattice,
    S::Timestamp: Lattice + Refines<T>,
{
    /// Construct a new collection bundle from update streams.
    pub fn from_collections(
        oks: Collection<S, V, Diff>,
        errs: Collection<S, DataflowError, Diff>,
    ) -> Self {
        Self {
            collection: Some((oks, errs)),
            arranged: BTreeMap::default(),
        }
    }

    /// Inserts arrangements by the expressions on which they are keyed.
    pub fn from_expressions(
        exprs: Vec<MirScalarExpr>,
        arrangements: ArrangementFlavor<S, V, T>,
    ) -> Self {
        let mut arranged = BTreeMap::new();
        arranged.insert(exprs, arrangements);
        Self {
            collection: None,
            arranged,
        }
    }

    /// Inserts arrangements by the columns on which they are keyed.
    pub fn from_columns<I: IntoIterator<Item = usize>>(
        columns: I,
        arrangements: ArrangementFlavor<S, V, T>,
    ) -> Self {
        let mut keys = Vec::new();
        for column in columns {
            keys.push(MirScalarExpr::Column(column));
        }
        Self::from_expressions(keys, arrangements)
    }
}

impl<S: Scope, T: Lattice> CollectionBundle<S, Row, T>
where
    T: Timestamp + Lattice,
    S::Timestamp: Lattice + Refines<T>,
{
    /// Asserts that the arrangement for a specific key
    /// (or the raw collection for no key) exists,
    /// and returns the corresponding collection.
    ///
    /// This returns the collection as-is, without
    /// doing any unthinning transformation.
    /// Therefore, it should be used when the appropriate transformation
    /// was planned as part of a following MFP.
    pub fn as_specific_collection(
        &self,
        key: Option<&[MirScalarExpr]>,
    ) -> (Collection<S, Row, Diff>, Collection<S, DataflowError, Diff>) {
        // Any operator that uses this method was told to use a particular
        // collection during LIR planning, where we should have made
        // sure that that collection exists.
        //
        // If it doesn't, we panic.
        match key {
            None => self
                .collection
                .clone()
                .expect("The unarranged collection doesn't exist."),
            Some(key) => self
                .arranged
                .get(key)
                .unwrap_or_else(|| panic!("The collection arranged by {:?} doesn't exist.", key))
                .as_collection(),
        }
    }

    /// Constructs and applies logic to elements of a collection and returns the results.
    ///
    /// `constructor` takes a permutation and produces the logic to apply on elements. The logic
    /// conceptually receives `(&Row, &Row)` pairs in the form of a slice. Only after borrowing
    /// the elements and applying the permutation the datums will be in the expected order.
    ///
    /// If `key_val` is set, this is a promise that `logic` will produce no results on
    /// records for which the key does not evaluate to the value. This is used when we
    /// have an arrangement by that key to leap directly to exactly those records.
    /// It is important that `logic` still guard against data that does not satisfy
    /// this constraint, as this method does not statically know that it will have
    /// that arrangement.
    pub fn flat_map<I, C, L>(
        &self,
        key_val: Option<(Vec<MirScalarExpr>, Option<Row>)>,
        constructor: C,
    ) -> (
        timely::dataflow::Stream<S, I::Item>,
        Collection<S, DataflowError, Diff>,
    )
    where
        I: IntoIterator,
        I::Item: Data,
        C: FnOnce() -> L,
        L: for<'a, 'b> FnMut(&'a [&'b RefOrMut<'b, Row>], &'a S::Timestamp, &'a Diff) -> I
            + 'static,
    {
        // If `key_val` is set, we should have use the corresponding arrangement.
        // If there isn't one, that implies an error in the contract between
        // key-production and available arrangements.
        if let Some((key, val)) = key_val {
            let flavor = self
                .arrangement(&key)
                .expect("Should have ensured during planning that this arrangement exists.");
            flavor.flat_map(val, constructor)
        } else {
            use timely::dataflow::operators::Map;
            let (oks, errs) = self
                .collection
                .clone()
                .expect("Invariant violated: CollectionBundle contains no collection.");
            let mut logic = constructor();
            (
                oks.inner
                    .flat_map(move |(mut v, t, d)| logic(&[&RefOrMut::Mut(&mut v)], &t, &d)),
                errs,
            )
        }
    }

    /// Factored out common logic for using literal keys in general traces.
    ///
    /// This logic is sufficiently interesting that we want to write it only
    /// once, and thereby avoid any skew in the two uses of the logic.
    ///
    /// The function presents the contents of the trace as `(key, value, time, delta)` tuples,
    /// where key and value are rows.
    fn flat_map_core<Tr, I, L>(
        trace: &Arranged<S, Tr>,
        key: Option<Row>,
        mut logic: L,
        refuel: usize,
    ) -> timely::dataflow::Stream<S, I::Item>
    where
        Tr: TraceReader<Key = Row, Val = Row, Time = S::Timestamp, R = repr::Diff>
            + Clone
            + 'static,
        Tr::Batch: BatchReader<Row, Tr::Val, S::Timestamp, repr::Diff> + 'static,
        Tr::Cursor: Cursor<Row, Tr::Val, S::Timestamp, repr::Diff> + 'static,
        I: IntoIterator,
        I::Item: Data,
        L: for<'a, 'b> FnMut(
                RefOrMut<'b, Row>,
                RefOrMut<'b, Row>,
                &'a S::Timestamp,
                &'a repr::Diff,
            ) -> I
            + 'static,
    {
        let mode = if key.is_some() { "index" } else { "scan" };
        let name = format!("ArrangementFlatMap({})", mode);
        use timely::dataflow::operators::Operator;
        trace.stream.unary(Pipeline, &name, move |_, info| {
            // Acquire an activator to reschedule the operator when it has unfinished work.
            use timely::scheduling::Activator;
            let activations = trace.stream.scope().activations();
            let activator = Activator::new(&info.address[..], activations);
            // Maintain a list of work to do, cursor to navigate and process.
            let mut todo = std::collections::VecDeque::new();
            move |input, output| {
                // First, dequeue all batches.
                input.for_each(|time, data| {
                    let capability = time.retain();
                    for batch in data.iter() {
                        // enqueue a capability, cursor, and batch.
                        todo.push_back(PendingWork::new(
                            capability.clone(),
                            batch.cursor(),
                            batch.clone(),
                        ));
                    }
                });

                // Second, make progress on `todo`.
                let mut fuel = refuel;
                while !todo.is_empty() && fuel > 0 {
                    todo.front_mut()
                        .unwrap()
                        .do_work(&key, &mut logic, &mut fuel, output);
                    if fuel > 0 {
                        todo.pop_front();
                    }
                }
                // If we have not finished all work, re-activate the operator.
                if !todo.is_empty() {
                    activator.activate();
                }
            }
        })
    }

    /// Look up an arrangement by the expressions that form the key.
    ///
    /// The result may be `None` if no such arrangement exists, or it may be one of many
    /// "arrangement flavors" that represent the types of arranged data we might have.
    pub fn arrangement(&self, key: &[MirScalarExpr]) -> Option<ArrangementFlavor<S, Row, T>> {
        self.arranged.get(key).map(|x| x.clone())
    }
}

impl<S> CollectionBundle<S, repr::Row, repr::Timestamp>
where
    S: Scope<Timestamp = repr::Timestamp>,
{
    /// Presents `self` as a stream of updates, having been subjected to `mfp`.
    ///
    /// This operator is able to apply the logic of `mfp` early, which can substantially
    /// reduce the amount of data produced when `mfp` is non-trivial.
    ///
    /// The `key_val` argument, when present, indicates that a specific arrangement should
    /// be used, and if, in addition, the `val` component is present,
    /// that we can seek to the supplied row.
    pub fn as_collection_core(
        &self,
        mut mfp: MapFilterProject,
        key_val: Option<(Vec<MirScalarExpr>, Option<Row>)>,
    ) -> (
        Collection<S, repr::Row, Diff>,
        Collection<S, DataflowError, Diff>,
    ) {
        mfp.optimize();
        let mfp_plan = mfp.into_plan().unwrap();

        // If the MFP is trivial, we can just call `as_collection`.
        // In the case that we weren't going to apply the `key_val` optimization,
        // this path results in a slightly smaller and faster
        // dataflow graph, and is intended to fix
        // https://github.com/MaterializeInc/materialize/issues/10507
        let has_key_val = if let Some((_key, Some(_val))) = &key_val {
            true
        } else {
            false
        };

        if mfp_plan.is_identity() && !has_key_val {
            let key = key_val.map(|(k, _v)| k);
            return self.as_specific_collection(key.as_deref());
        }
        let (stream, errors) = self.flat_map(key_val, || {
            let mut row_builder = Row::default();
            let mut datum_vec = DatumVec::new();

            move |row_parts, time, diff| {
                let temp_storage = RowArena::new();
                let mut datums_local = datum_vec.borrow_with_many(row_parts);
                mfp_plan.evaluate(
                    &mut datums_local,
                    &temp_storage,
                    time.clone(),
                    diff.clone(),
                    &mut row_builder,
                )
            }
        });

        use timely::dataflow::operators::ok_err::OkErr;
        let (oks, errs) = stream.ok_err(|x| x);

        use differential_dataflow::AsCollection;
        let oks = oks.as_collection();
        let errs = errs.as_collection();
        (oks, errors.concat(&errs))
    }
    pub fn ensure_collections(
        mut self,
        collections: AvailableCollections,
        input_key: Option<Vec<MirScalarExpr>>,
        input_mfp: MapFilterProject,
    ) -> Self {
        if collections == Default::default() {
            return self;
        }
        // Cache collection to avoid reforming it each time.
        //
        // TODO(mcsherry): In theory this could be faster run out of another arrangement,
        // as the `map_fallible` that follows could be run against an arrangement itself.
        //
        // Note(btv): If we ever do that, we would then only need to make the raw collection here
        // if `collections.raw` is true.

        // We need the collection if either (1) it is explicitly demanded, or (2) we are going to render any arrangement
        let form_raw_collection = collections.raw
            || collections
                .arranged
                .iter()
                .any(|(key, _, _)| !self.arranged.contains_key(key));
        if form_raw_collection && self.collection.is_none() {
            self.collection =
                Some(self.as_collection_core(input_mfp, input_key.map(|k| (k, None))));
        }
        for (key, _, thinning) in collections.arranged {
            if !self.arranged.contains_key(&key) {
                // TODO: Consider allowing more expressive names.
                let name = format!("ArrangeBy[{:?}]", key);
                let key2 = key.clone();
                let (oks, errs) = self
                    .collection
                    .clone()
                    .expect("Collection constructed above");

                let mut row_packer = Row::default();

                let mut datums = DatumVec::new();
                let (oks_keyed, errs_keyed) = oks.map_fallible("FormArrangementKey", move |row| {
                    // TODO: Consider reusing the `row` allocation; probably in *next* invocation.
                    let datums = datums.borrow_with(&row);
                    let temp_storage = RowArena::new();
                    row_packer.try_extend(key2.iter().map(|k| k.eval(&datums, &temp_storage)))?;
                    let key_row = row_packer.finish_and_reuse();
                    row_packer.extend(thinning.iter().map(|c| datums[*c]));
                    let val_row = row_packer.finish_and_reuse();
                    Ok::<(Row, Row), DataflowError>((key_row, val_row))
                });

                let oks = oks_keyed.arrange_named::<RowSpine<Row, Row, _, _>>(&name);
                let errs = errs
                    .concat(&errs_keyed)
                    .arrange_named::<ErrSpine<_, _, _>>(&format!("{}-errors", name));
                self.arranged
                    .insert(key, ArrangementFlavor::Local(oks, errs));
            }
        }
        self
    }
}

use timely::dataflow::operators::generic::OutputHandle;
use timely::dataflow::operators::Capability;
struct PendingWork<K, V, T: Timestamp, R, C: Cursor<K, V, T, R>> {
    capability: Capability<T>,
    cursor: C,
    batch: C::Storage,
}

impl<K: PartialEq, V, T: Timestamp, R, C: Cursor<K, V, T, R>> PendingWork<K, V, T, R, C> {
    /// Create a new bundle of pending work, from the capability, cursor, and backing storage.
    fn new(capability: Capability<T>, cursor: C, batch: C::Storage) -> Self {
        Self {
            capability,
            cursor,
            batch,
        }
    }
    /// Perform roughly `fuel` work through the cursor, applying `logic` and sending results to `output`.
    fn do_work<I, L>(
        &mut self,
        key: &Option<K>,
        logic: &mut L,
        fuel: &mut usize,
        output: &mut OutputHandle<
            '_,
            T,
            I::Item,
            timely::dataflow::channels::pushers::Tee<T, I::Item>,
        >,
    ) where
        I: IntoIterator,
        I::Item: Data,
        L: for<'a, 'b> FnMut(RefOrMut<'b, K>, RefOrMut<'b, V>, &'a T, &'a R) -> I + 'static,
    {
        // Attempt to make progress on this batch.
        let mut work: usize = 0;
        let mut session = output.session(&self.capability);
        if let Some(key) = key {
            if self.cursor.get_key(&self.batch) != Some(key) {
                self.cursor.seek_key(&self.batch, key);
            }
            if self.cursor.get_key(&self.batch) == Some(key) {
                while let Some(val) = self.cursor.get_val(&self.batch) {
                    self.cursor.map_times(&self.batch, |time, diff| {
                        for datum in logic(RefOrMut::Ref(key), RefOrMut::Ref(val), time, diff) {
                            session.give(datum);
                            work += 1;
                        }
                    });
                    self.cursor.step_val(&self.batch);
                    if work >= *fuel {
                        *fuel = 0;
                        return;
                    }
                }
            }
        } else {
            while let Some(key) = self.cursor.get_key(&self.batch) {
                while let Some(val) = self.cursor.get_val(&self.batch) {
                    self.cursor.map_times(&self.batch, |time, diff| {
                        for datum in logic(RefOrMut::Ref(key), RefOrMut::Ref(val), time, diff) {
                            session.give(datum);
                            work += 1;
                        }
                    });
                    self.cursor.step_val(&self.batch);
                    if work >= *fuel {
                        *fuel = 0;
                        return;
                    }
                }
                self.cursor.step_key(&self.batch);
            }
        }
        *fuel -= work;
    }
}