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

use std::collections::BTreeMap;
use std::convert::Infallible;
use std::sync::Arc;

use differential_dataflow::AsCollection;
use futures::stream::StreamExt;
use mz_ore::cast::CastFrom;
use mz_ore::iter::IteratorExt;
use mz_repr::{Datum, Diff, Row};
use mz_storage_types::errors::DataflowError;
use mz_storage_types::sources::load_generator::{KeyValueLoadGenerator, LoadGeneratorOutput};
use mz_storage_types::sources::{MzOffset, SourceTimestamp};
use mz_timely_util::builder_async::{OperatorBuilder as AsyncOperatorBuilder, PressOnDropButton};
use mz_timely_util::containers::stack::AccountedStackBuilder;
use rand::rngs::StdRng;
use rand::{RngCore, SeedableRng};
use timely::container::CapacityContainerBuilder;
use timely::dataflow::operators::{Concat, ToStream};
use timely::dataflow::{Scope, Stream};
use timely::progress::Antichain;
use tracing::info;

use crate::healthcheck::{HealthStatusMessage, HealthStatusUpdate, StatusNamespace};
use crate::source::types::{ProgressStatisticsUpdate, SignaledFuture, StackedCollection};
use crate::source::{RawSourceCreationConfig, SourceMessage};

pub fn render<G: Scope<Timestamp = MzOffset>>(
    key_value: KeyValueLoadGenerator,
    scope: &mut G,
    config: RawSourceCreationConfig,
    committed_uppers: impl futures::Stream<Item = Antichain<MzOffset>> + 'static,
    start_signal: impl std::future::Future<Output = ()> + 'static,
    output_map: BTreeMap<LoadGeneratorOutput, Vec<usize>>,
) -> (
    StackedCollection<G, (usize, Result<SourceMessage, DataflowError>)>,
    Option<Stream<G, Infallible>>,
    Stream<G, HealthStatusMessage>,
    Stream<G, ProgressStatisticsUpdate>,
    Vec<PressOnDropButton>,
) {
    let (steady_state_stats_stream, stats_button) =
        render_statistics_operator(scope, config.clone(), committed_uppers);

    let mut builder = AsyncOperatorBuilder::new(config.name.clone(), scope.clone());

    let (data_output, stream) = builder.new_output::<AccountedStackBuilder<_>>();
    let (_progress_output, progress_stream) = builder.new_output::<CapacityContainerBuilder<_>>();
    let (stats_output, stats_stream) = builder.new_output::<CapacityContainerBuilder<_>>();

    let busy_signal = Arc::clone(&config.busy_signal);
    let button = builder.build(move |caps| {
        SignaledFuture::new(busy_signal, async move {
            let [mut cap, mut progress_cap, stats_cap]: [_; 3] = caps.try_into().unwrap();

            let resume_upper = Antichain::from_iter(
                config
                    .source_resume_uppers
                    .values()
                    .flat_map(|f| f.iter().map(MzOffset::decode_row)),
            );

            let Some(resume_offset) = resume_upper.into_option() else {
                return;
            };

            // The key-value load generator only has one 'output' stream, which is the default output
            // that needs to be emitted to all output indexes.
            let output_indexes = output_map
                .get(&LoadGeneratorOutput::Default)
                .expect("default output");

            info!(
                ?config.worker_id,
                "starting key-value load generator at {}",
                resume_offset.offset,
            );
            cap.downgrade(&resume_offset);
            progress_cap.downgrade(&resume_offset);
            start_signal.await;
            info!(?config.worker_id, "received key-value load generator start signal");

            let snapshotting = resume_offset.offset == 0;

            let mut local_partitions: Vec<_> = (0..key_value.partitions)
                .filter_map(|p| {
                    config
                        .responsible_for(p)
                        .then(|| TransactionalSnapshotProducer::new(p, key_value.clone()))
                })
                .collect();

            let stats_worker = config.responsible_for(0);

            if local_partitions.is_empty() {
                stats_output.give(
                    &stats_cap,
                    ProgressStatisticsUpdate::Snapshot {
                        records_known: 0,
                        records_staged: 0,
                    },
                );
                return;
            }

            let local_snapshot_size = (u64::cast_from(local_partitions.len()))
                * key_value.keys
                * key_value.transactional_snapshot_rounds()
                / key_value.partitions;

            // Re-usable buffers.
            let mut value_buffer: Vec<u8> = vec![0; usize::cast_from(key_value.value_size)];

            // snapshotting
            let mut upper_offset = if snapshotting {
                let snapshot_rounds = key_value.transactional_snapshot_rounds();

                if stats_worker {
                    stats_output.give(
                        &stats_cap,
                        ProgressStatisticsUpdate::SteadyState {
                            offset_known: snapshot_rounds,
                            offset_committed: 0,
                        },
                    );
                };

                // Downgrade to the snapshot frontier.
                progress_cap.downgrade(&MzOffset::from(snapshot_rounds));

                let mut emitted = 0;
                stats_output.give(
                    &stats_cap,
                    ProgressStatisticsUpdate::Snapshot {
                        records_known: local_snapshot_size,
                        records_staged: emitted,
                    },
                );
                while local_partitions.iter().any(|si| !si.finished()) {
                    for sp in local_partitions.iter_mut() {
                        for u in sp.produce_batch(&mut value_buffer, output_indexes) {
                            data_output.give_fueled(&cap, u).await;
                            emitted += 1;
                        }

                        stats_output.give(
                            &stats_cap,
                            ProgressStatisticsUpdate::Snapshot {
                                records_known: local_snapshot_size,
                                records_staged: emitted,
                            },
                        );
                    }
                }

                cap.downgrade(&MzOffset::from(snapshot_rounds));
                snapshot_rounds
            } else {
                cap.downgrade(&resume_offset);
                progress_cap.downgrade(&resume_offset);
                resume_offset.offset
            };

            let mut local_partitions: Vec<_> = (0..key_value.partitions)
                .filter_map(|p| {
                    config
                        .responsible_for(p)
                        .then(|| UpdateProducer::new(p, upper_offset, key_value.clone()))
                })
                .collect();
            if !local_partitions.is_empty()
                && (key_value.tick_interval.is_some() || !key_value.transactional_snapshot)
            {
                let mut interval = key_value.tick_interval.map(tokio::time::interval);

                loop {
                    if local_partitions.iter().all(|si| si.finished_quick()) {
                        if let Some(interval) = &mut interval {
                            interval.tick().await;
                        } else {
                            break;
                        }
                    }

                    for up in local_partitions.iter_mut() {
                        let (new_upper, iter) = up.produce_batch(&mut value_buffer, output_indexes);
                        upper_offset = new_upper;
                        for u in iter {
                            data_output.give_fueled(&cap, u).await;
                        }
                    }
                    cap.downgrade(&MzOffset::from(upper_offset));
                    progress_cap.downgrade(&MzOffset::from(upper_offset));
                }
            }

            std::future::pending::<()>().await;
        })
    });

    let status = [HealthStatusMessage {
        index: 0,
        namespace: StatusNamespace::Generator,
        update: HealthStatusUpdate::running(),
    }]
    .to_stream(scope);
    let stats_stream = stats_stream.concat(&steady_state_stats_stream);

    (
        stream.as_collection(),
        Some(progress_stream),
        status,
        stats_stream,
        vec![button.press_on_drop(), stats_button],
    )
}

/// An iterator that produces keys belonging to a partition.
struct PartitionKeyIterator {
    /// The partition.
    partition: u64,
    /// The number of partitions.
    partitions: u64,
    /// The key space.
    keys: u64,
    /// The next key.
    next: u64,
}

impl PartitionKeyIterator {
    fn new(partition: u64, partitions: u64, keys: u64, start_key: u64) -> Self {
        assert_eq!(keys % partitions, 0);
        PartitionKeyIterator {
            partition,
            partitions,
            keys,
            next: start_key,
        }
    }
}

impl Iterator for &mut PartitionKeyIterator {
    type Item = u64;

    fn next(&mut self) -> Option<Self::Item> {
        let ret = self.next;
        self.next = (self.next + self.partitions) % self.keys;
        Some(ret)
    }
}

/// Create `StdRng` seeded so identical values can be produced during resumption (during
/// snapshotting or after).
fn create_consistent_rng(source_seed: u64, offset: u64, partition: u64) -> StdRng {
    let mut seed = [0; 32];
    seed[0..8].copy_from_slice(&source_seed.to_le_bytes());
    seed[8..16].copy_from_slice(&offset.to_le_bytes());
    seed[16..24].copy_from_slice(&partition.to_le_bytes());
    StdRng::from_seed(seed)
}

/// A struct that produces batches of data for the snapshotting phase.
struct TransactionalSnapshotProducer {
    /// The key iterator for the partition.
    pi: PartitionKeyIterator,
    /// The batch size.
    batch_size: u64,
    /// The number of batches produced in the current round.
    produced_batches: u64,
    /// The expected number of batches per round.
    expected_batches: u64,
    /// The current round of the offset
    round: u64,
    /// The total number of rounds.
    snapshot_rounds: u64,
    /// The rng for the current round.
    rng: Option<StdRng>,
    /// The source-level seed for the rng.
    seed: u64,
    /// Whether to include the offset or not.
    include_offset: bool,
}

impl TransactionalSnapshotProducer {
    fn new(partition: u64, key_value: KeyValueLoadGenerator) -> Self {
        let snapshot_rounds = key_value.transactional_snapshot_rounds();

        let KeyValueLoadGenerator {
            partitions,
            keys,
            batch_size,
            seed,
            include_offset,
            ..
        } = key_value;

        assert_eq!((keys / partitions) % batch_size, 0);
        let pi = PartitionKeyIterator::new(
            partition, partitions, keys, // The first start key is the partition.
            partition,
        );
        TransactionalSnapshotProducer {
            pi,
            batch_size,
            produced_batches: 0,
            expected_batches: keys / partitions / batch_size,
            round: 0,
            snapshot_rounds,
            rng: None,
            seed,
            include_offset: include_offset.is_some(),
        }
    }

    /// If this partition is done snapshotting.
    fn finished(&self) -> bool {
        self.round >= self.snapshot_rounds
    }

    /// Produce a batch of message into `buffer`, of size `batch_size`. Advances the current
    /// batch and round counter.
    ///
    /// The output iterator must be consumed fully for this method to be used correctly.
    fn produce_batch<'a>(
        &'a mut self,
        value_buffer: &'a mut Vec<u8>,
        output_indexes: &'a [usize],
    ) -> impl Iterator<
        Item = (
            (usize, Result<SourceMessage, DataflowError>),
            MzOffset,
            Diff,
        ),
    > + 'a {
        let finished = self.finished();

        let rng = self
            .rng
            .get_or_insert_with(|| create_consistent_rng(self.seed, self.round, self.pi.partition));

        let partition: u64 = self.pi.partition;
        let iter_round: u64 = self.round;
        let include_offset: bool = self.include_offset;
        let iter = self
            .pi
            .take(if finished {
                0
            } else {
                usize::cast_from(self.batch_size)
            })
            .flat_map(move |key| {
                rng.fill_bytes(value_buffer.as_mut_slice());
                let msg = Ok(SourceMessage {
                    key: Row::pack_slice(&[Datum::UInt64(key)]),
                    value: Row::pack_slice(&[Datum::UInt64(partition), Datum::Bytes(value_buffer)]),
                    metadata: if include_offset {
                        Row::pack(&[Datum::UInt64(iter_round)])
                    } else {
                        Row::default()
                    },
                });
                output_indexes
                    .iter()
                    .repeat_clone(msg)
                    .map(move |(idx, msg)| ((*idx, msg), MzOffset::from(iter_round), 1))
            });

        if !finished {
            self.produced_batches += 1;

            if self.produced_batches == self.expected_batches {
                self.round += 1;
                self.produced_batches = 0;
            }
        }

        iter
    }
}

/// A struct that produces batches of data for the post-snapshotting phase.
struct UpdateProducer {
    /// The key iterator for the partition.
    pi: PartitionKeyIterator,
    /// The batch size.
    batch_size: u64,
    /// The next offset to produce updates at.
    next_offset: u64,
    /// The source-level seed for the rng.
    seed: u64,
    /// The number of offsets we expect to be part of the `transactional_snapshot`.
    expected_quick_offsets: u64,
    /// Whether to include the offset or not.
    include_offset: bool,
}

impl UpdateProducer {
    fn new(partition: u64, next_offset: u64, key_value: KeyValueLoadGenerator) -> Self {
        let snapshot_rounds = key_value.transactional_snapshot_rounds();
        let quick_rounds = key_value.non_transactional_snapshot_rounds();
        let KeyValueLoadGenerator {
            partitions,
            keys,
            batch_size,
            seed,
            include_offset,
            ..
        } = key_value;

        // Each snapshot _round_ is associated with an offset, starting at 0. Therefore
        // the `next_offset` - `snapshot_rounds` is the index of the next non-snapshot update
        // batches we must produce. The start key for the that batch is that index
        // multiplied by the batch size and the number of partitions.
        let start_key =
            (((next_offset - snapshot_rounds) * batch_size * partitions) + partition) % keys;

        // We expect to emit 1 batch per offset. A round is emitting the full set of keys (in the
        // partition. The number of keys divided by the batch size (and partitioned, so divided by
        // the partition count) is the number of offsets in each _round_. We also add the number of
        // snapshot rounds, which is also the number of offsets in the snapshot.
        let expected_quick_offsets =
            ((keys / partitions / batch_size) * quick_rounds) + snapshot_rounds;

        let pi = PartitionKeyIterator::new(partition, partitions, keys, start_key);
        UpdateProducer {
            pi,
            batch_size,
            next_offset,
            seed,
            expected_quick_offsets,
            include_offset: include_offset.is_some(),
        }
    }

    /// If this partition is done producing quick updates.
    fn finished_quick(&self) -> bool {
        self.next_offset >= self.expected_quick_offsets
    }

    /// Produce a batch of message into `buffer`, of size `batch_size`. Advances the current
    /// batch and round counter. Also returns the frontier after the batch.
    ///
    /// The output iterator must be consumed fully for this method to be used correctly.
    fn produce_batch<'a>(
        &'a mut self,
        value_buffer: &'a mut Vec<u8>,
        output_indexes: &'a [usize],
    ) -> (
        u64,
        impl Iterator<
                Item = (
                    (usize, Result<SourceMessage, DataflowError>),
                    MzOffset,
                    Diff,
                ),
            > + 'a,
    ) {
        let mut rng = create_consistent_rng(self.seed, self.next_offset, self.pi.partition);

        let partition: u64 = self.pi.partition;
        let iter_offset: u64 = self.next_offset;
        let include_offset: bool = self.include_offset;
        let iter = self
            .pi
            .take(usize::cast_from(self.batch_size))
            .flat_map(move |key| {
                rng.fill_bytes(value_buffer.as_mut_slice());
                let msg = Ok(SourceMessage {
                    key: Row::pack_slice(&[Datum::UInt64(key)]),
                    value: Row::pack_slice(&[Datum::UInt64(partition), Datum::Bytes(value_buffer)]),
                    metadata: if include_offset {
                        Row::pack(&[Datum::UInt64(iter_offset)])
                    } else {
                        Row::default()
                    },
                });
                output_indexes
                    .iter()
                    .repeat_clone(msg)
                    .map(move |(idx, msg)| ((*idx, msg), MzOffset::from(iter_offset), 1))
            });

        // Advance to the next offset.
        self.next_offset += 1;
        (self.next_offset, iter)
    }
}

pub fn render_statistics_operator<G: Scope<Timestamp = MzOffset>>(
    scope: &G,
    config: RawSourceCreationConfig,
    committed_uppers: impl futures::Stream<Item = Antichain<MzOffset>> + 'static,
) -> (Stream<G, ProgressStatisticsUpdate>, PressOnDropButton) {
    let id = config.id;
    let mut builder =
        AsyncOperatorBuilder::new(format!("key_value_loadgen_statistics:{id}"), scope.clone());

    let (stats_output, stats_stream) = builder.new_output();

    let button = builder.build(move |caps| async move {
        let [stats_cap]: [_; 1] = caps.try_into().unwrap();

        let offset_worker = config.responsible_for(0);

        if !offset_worker {
            // Emit 0, to mark this worker as having started up correctly.
            stats_output.give(
                &stats_cap,
                ProgressStatisticsUpdate::SteadyState {
                    offset_known: 0,
                    offset_committed: 0,
                },
            );
            return;
        }

        tokio::pin!(committed_uppers);
        loop {
            match committed_uppers.next().await {
                Some(frontier) => {
                    if let Some(offset) = frontier.as_option() {
                        stats_output.give(
                            &stats_cap,
                            ProgressStatisticsUpdate::SteadyState {
                                offset_known: offset.offset,
                                offset_committed: offset.offset,
                            },
                        );
                    }
                }
                None => return,
            }
        }
    });

    (stats_stream, button.press_on_drop())
}

#[cfg(test)]
mod test {
    use super::*;

    #[mz_ore::test]
    fn test_key_value_loadgen_resume_upper() {
        let up = UpdateProducer::new(
            1, // partition 1
            5, // resume upper of 5
            KeyValueLoadGenerator {
                keys: 126,
                snapshot_rounds: 2,
                transactional_snapshot: true,
                value_size: 1234,
                partitions: 3,
                tick_interval: None,
                batch_size: 2,
                seed: 1234,
                include_offset: None,
            },
        );

        // The first key 3 rounds after the 2 snapshot rounds would be the 7th key (3 rounds of 2
        // beforehand) produced for partition 1, so 13.
        assert_eq!(up.pi.next, 19);

        let up = UpdateProducer::new(
            1,           // partition 1
            5 + 126 / 2, // resume upper of 5 after a full set of keys has been produced.
            KeyValueLoadGenerator {
                keys: 126,
                snapshot_rounds: 2,
                transactional_snapshot: true,
                value_size: 1234,
                partitions: 3,
                tick_interval: None,
                batch_size: 2,
                seed: 1234,
                include_offset: None,
            },
        );

        assert_eq!(up.pi.next, 19);
    }

    #[mz_ore::test]
    fn test_key_value_loadgen_part_iter() {
        let mut pi = PartitionKeyIterator::new(
            1,   // partition 1
            3,   // of 3 partitions
            126, // 126 keys.
            1,   // Start at the beginning
        );

        assert_eq!(1, Iterator::next(&mut &mut pi).unwrap());
        assert_eq!(4, Iterator::next(&mut &mut pi).unwrap());

        // After a full round, ensure we wrap around correctly;
        let _ = pi.take((126 / 3) - 2).count();
        assert_eq!(pi.next, 1);
    }
}