1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
// 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::borrow::Cow;
use std::collections::VecDeque;
use std::fmt::Debug;
use std::marker::PhantomData;
use std::pin::pin;
use std::sync::Arc;
use std::time::{Duration, Instant};

use anyhow::anyhow;
use differential_dataflow::difference::Semigroup;
use differential_dataflow::lattice::Lattice;
use differential_dataflow::trace::Description;
use futures_util::{StreamExt, TryFutureExt};
use mz_dyncfg::Config;
use mz_ore::cast::CastFrom;
use mz_ore::error::ErrorExt;
use mz_persist::indexed::columnar::ColumnarRecords;
use mz_persist::indexed::encoding::BlobTraceUpdates;
use mz_persist::location::Blob;
use mz_persist_types::{Codec, Codec64};
use timely::progress::{Antichain, Timestamp};
use timely::PartialOrder;
use tokio::sync::mpsc::Sender;
use tokio::sync::{mpsc, oneshot, TryAcquireError};
use tracing::{debug, debug_span, error, trace, warn, Instrument, Span};

use crate::async_runtime::IsolatedRuntime;
use crate::batch::{BatchBuilderConfig, BatchBuilderInternal, BatchParts, PartDeletes};
use crate::cfg::{
    MiB, COMPACTION_HEURISTIC_MIN_INPUTS, COMPACTION_HEURISTIC_MIN_PARTS,
    COMPACTION_HEURISTIC_MIN_UPDATES, COMPACTION_MEMORY_BOUND_BYTES,
    GC_BLOB_DELETE_CONCURRENCY_LIMIT,
};
use crate::fetch::FetchBatchFilter;
use crate::internal::encoding::Schemas;
use crate::internal::gc::GarbageCollector;
use crate::internal::machine::Machine;
use crate::internal::maintenance::RoutineMaintenance;
use crate::internal::metrics::ShardMetrics;
use crate::internal::state::{HollowBatch, RunMeta, RunOrder, RunPart};
use crate::internal::trace::{ApplyMergeResult, FueledMergeRes};
use crate::iter::{CodecSort, Consolidator, StructuredSort};
use crate::{Metrics, PersistConfig, ShardId};

/// A request for compaction.
///
/// This is similar to FueledMergeReq, but intentionally a different type. If we
/// move compaction to an rpc server, this one will become a protobuf; the type
/// parameters will become names of codecs to look up in some registry.
#[derive(Debug, Clone)]
pub struct CompactReq<T> {
    /// The shard the input and output batches belong to.
    pub shard_id: ShardId,
    /// A description for the output batch.
    pub desc: Description<T>,
    /// The updates to include in the output batch. Any data in these outside of
    /// the output descriptions bounds should be ignored.
    pub inputs: Vec<HollowBatch<T>>,
}

/// A response from compaction.
#[derive(Debug)]
pub struct CompactRes<T> {
    /// The compacted batch.
    pub output: HollowBatch<T>,
}

/// A snapshot of dynamic configs to make it easier to reason about an
/// individual run of compaction.
#[derive(Debug, Clone)]
pub struct CompactConfig {
    pub(crate) compaction_memory_bound_bytes: usize,
    pub(crate) compaction_yield_after_n_updates: usize,
    pub(crate) version: semver::Version,
    pub(crate) batch: BatchBuilderConfig,
}

impl CompactConfig {
    /// Initialize the compaction config from Persist configuration.
    pub fn new(value: &PersistConfig, shard_id: ShardId) -> Self {
        CompactConfig {
            compaction_memory_bound_bytes: COMPACTION_MEMORY_BOUND_BYTES.get(value),
            compaction_yield_after_n_updates: value.compaction_yield_after_n_updates,
            version: value.build_version.clone(),
            batch: BatchBuilderConfig::new(value, shard_id),
        }
    }
}

/// A service for performing physical and logical compaction.
///
/// This will possibly be called over RPC in the future. Physical compaction is
/// merging adjacent batches. Logical compaction is advancing timestamps to a
/// new since and consolidating the resulting updates.
#[derive(Debug)]
pub struct Compactor<K, V, T, D> {
    cfg: PersistConfig,
    metrics: Arc<Metrics>,
    sender: Sender<(
        Instant,
        CompactReq<T>,
        Machine<K, V, T, D>,
        oneshot::Sender<Result<ApplyMergeResult, anyhow::Error>>,
    )>,
    _phantom: PhantomData<fn() -> D>,
}

impl<K, V, T, D> Clone for Compactor<K, V, T, D> {
    fn clone(&self) -> Self {
        Compactor {
            cfg: self.cfg.clone(),
            metrics: Arc::clone(&self.metrics),
            sender: self.sender.clone(),
            _phantom: Default::default(),
        }
    }
}

/// In Compactor::compact_and_apply_background, the minimum amount of time to
/// allow a compaction request to run before timing it out. A request may be
/// given a timeout greater than this value depending on the inputs' size
pub(crate) const COMPACTION_MINIMUM_TIMEOUT: Config<Duration> = Config::new(
    "persist_compaction_minimum_timeout",
    Duration::from_secs(90),
    "\
    The minimum amount of time to allow a persist compaction request to run \
    before timing it out (Materialize).",
);

pub(crate) const COMPACTION_USE_MOST_RECENT_SCHEMA: Config<bool> = Config::new(
    "persist_compaction_use_most_recent_schema",
    true,
    "\
    Use the most recent schema from all the Runs that are currently being \
    compacted, instead of the schema on the current write handle (Materialize).
    ",
);

pub(crate) const COMPACTION_CHECK_PROCESS_FLAG: Config<bool> = Config::new(
    "persist_compaction_check_process_flag",
    true,
    "Whether Compactor will obey the process_requests flag in PersistConfig, \
        which allows dynamically disabling compaction. If false, all compaction requests will be processed.",
);

impl<K, V, T, D> Compactor<K, V, T, D>
where
    K: Debug + Codec,
    V: Debug + Codec,
    T: Timestamp + Lattice + Codec64 + Sync,
    D: Semigroup + Ord + Codec64 + Send + Sync,
{
    pub fn new(
        cfg: PersistConfig,
        metrics: Arc<Metrics>,
        write_schemas: Schemas<K, V>,
        gc: GarbageCollector<K, V, T, D>,
    ) -> Self {
        let (compact_req_sender, mut compact_req_receiver) = mpsc::channel::<(
            Instant,
            CompactReq<T>,
            Machine<K, V, T, D>,
            oneshot::Sender<Result<ApplyMergeResult, anyhow::Error>>,
        )>(cfg.compaction_queue_size);
        let concurrency_limit = Arc::new(tokio::sync::Semaphore::new(
            cfg.compaction_concurrency_limit,
        ));
        let check_process_requests = COMPACTION_CHECK_PROCESS_FLAG.handle(&cfg.configs);
        let process_requests = Arc::clone(&cfg.compaction_process_requests);

        // spin off a single task responsible for executing compaction requests.
        // work is enqueued into the task through a channel
        let _worker_handle = mz_ore::task::spawn(|| "PersistCompactionScheduler", async move {
            while let Some((enqueued, req, machine, completer)) = compact_req_receiver.recv().await
            {
                assert_eq!(req.shard_id, machine.shard_id());
                let metrics = Arc::clone(&machine.applier.metrics);

                // Only allow skipping compaction requests if the dyncfg is enabled.
                if check_process_requests.get()
                    && !process_requests.load(std::sync::atomic::Ordering::Relaxed)
                {
                    // Respond to the requester, track in our metrics, and log
                    // that compaction is disabled.
                    let _ = completer.send(Err(anyhow::anyhow!("compaction disabled")));
                    metrics.compaction.disabled.inc();
                    tracing::warn!(shard_id = ?req.shard_id, "Dropping compaction request on the floor.");

                    continue;
                }

                let permit = {
                    let inner = Arc::clone(&concurrency_limit);
                    // perform a non-blocking attempt to acquire a permit so we can
                    // record how often we're ever blocked on the concurrency limit
                    match inner.try_acquire_owned() {
                        Ok(permit) => permit,
                        Err(TryAcquireError::NoPermits) => {
                            metrics.compaction.concurrency_waits.inc();
                            Arc::clone(&concurrency_limit)
                                .acquire_owned()
                                .await
                                .expect("semaphore is never closed")
                        }
                        Err(TryAcquireError::Closed) => {
                            // should never happen in practice. the semaphore is
                            // never explicitly closed, nor will it close on Drop
                            warn!("semaphore for shard {} is closed", machine.shard_id());
                            continue;
                        }
                    }
                };
                metrics
                    .compaction
                    .queued_seconds
                    .inc_by(enqueued.elapsed().as_secs_f64());

                let write_schemas = write_schemas.clone();

                let compact_span =
                    debug_span!(parent: None, "compact::apply", shard_id=%machine.shard_id());
                compact_span.follows_from(&Span::current());
                let gc = gc.clone();
                mz_ore::task::spawn(|| "PersistCompactionWorker", async move {
                    let res = Self::compact_and_apply(&machine, req, write_schemas)
                        .instrument(compact_span)
                        .await;
                    let res = res.map(|(res, maintenance)| {
                        maintenance.start_performing(&machine, &gc);
                        res
                    });

                    // we can safely ignore errors here, it's possible the caller
                    // wasn't interested in waiting and dropped their receiver
                    let _ = completer.send(res);

                    // moves `permit` into async scope so it can be dropped upon completion
                    drop(permit);
                });
            }
        });

        Compactor {
            cfg,
            metrics,
            sender: compact_req_sender,
            _phantom: PhantomData,
        }
    }

    /// Enqueues a [CompactReq] to be consumed by the compaction background task when available.
    ///
    /// Returns a receiver that indicates when compaction has completed. The receiver can be
    /// safely dropped at any time if the caller does not wish to wait on completion.
    pub fn compact_and_apply_background(
        &self,
        req: CompactReq<T>,
        machine: &Machine<K, V, T, D>,
    ) -> Option<oneshot::Receiver<Result<ApplyMergeResult, anyhow::Error>>> {
        // Run some initial heuristics to ignore some requests for compaction.
        // We don't gain much from e.g. compacting two very small batches that
        // were just written, but it does result in non-trivial blob traffic
        // (especially in aggregate). This heuristic is something we'll need to
        // tune over time.
        let should_compact = req.inputs.len() >= COMPACTION_HEURISTIC_MIN_INPUTS.get(&self.cfg)
            || req.inputs.iter().map(|x| x.part_count()).sum::<usize>()
                >= COMPACTION_HEURISTIC_MIN_PARTS.get(&self.cfg)
            || req.inputs.iter().map(|x| x.len).sum::<usize>()
                >= COMPACTION_HEURISTIC_MIN_UPDATES.get(&self.cfg);
        if !should_compact {
            self.metrics.compaction.skipped.inc();
            return None;
        }

        let (compaction_completed_sender, compaction_completed_receiver) = oneshot::channel();
        let new_compaction_sender = self.sender.clone();

        self.metrics.compaction.requested.inc();
        // NB: we intentionally pass along the input machine, as it ought to come from the
        // writer that generated the compaction request / maintenance. this machine has a
        // spine structure that generated the request, so it has a much better chance of
        // merging and committing the result than a machine kept up-to-date through state
        // diffs, which may have a different spine structure less amenable to merging.
        let send = new_compaction_sender.try_send((
            Instant::now(),
            req,
            machine.clone(),
            compaction_completed_sender,
        ));
        if let Err(_) = send {
            self.metrics.compaction.dropped.inc();
            return None;
        }

        Some(compaction_completed_receiver)
    }

    pub(crate) async fn compact_and_apply(
        machine: &Machine<K, V, T, D>,
        req: CompactReq<T>,
        write_schemas: Schemas<K, V>,
    ) -> Result<(ApplyMergeResult, RoutineMaintenance), anyhow::Error> {
        let metrics = Arc::clone(&machine.applier.metrics);
        metrics.compaction.started.inc();
        let start = Instant::now();

        // pick a timeout for our compaction request proportional to the amount
        // of data that must be read (with a minimum set by PersistConfig)
        let total_input_bytes = req
            .inputs
            .iter()
            .map(|batch| batch.encoded_size_bytes())
            .sum::<usize>();
        let timeout = Duration::max(
            // either our minimum timeout
            COMPACTION_MINIMUM_TIMEOUT.get(&machine.applier.cfg),
            // or 1s per MB of input data
            Duration::from_secs(u64::cast_from(total_input_bytes / MiB)),
        );
        // always use most recent schema from all the Runs we're compacting to prevent Compactors
        // created before the schema was evolved, from trying to "de-evolve" a Part.
        let compaction_schema_id = req
            .inputs
            .iter()
            .flat_map(|batch| batch.run_meta.iter())
            .filter_map(|run_meta| run_meta.schema)
            // It's an invariant that SchemaIds are ordered.
            .max();
        let maybe_compaction_schema = match compaction_schema_id {
            Some(id) => machine
                .get_schema(id)
                .map(|(key_schema, val_schema)| (id, key_schema, val_schema)),
            None => None,
        };
        let use_most_recent_schema = COMPACTION_USE_MOST_RECENT_SCHEMA.get(&machine.applier.cfg);

        let compaction_schema = match maybe_compaction_schema {
            Some((id, key_schema, val_schema)) if use_most_recent_schema => {
                metrics.compaction.schema_selection.recent_schema.inc();
                Schemas {
                    id: Some(id),
                    key: Arc::new(key_schema),
                    val: Arc::new(val_schema),
                }
            }
            Some(_) => {
                metrics.compaction.schema_selection.disabled.inc();
                write_schemas
            }
            None => {
                metrics.compaction.schema_selection.no_schema.inc();
                write_schemas
            }
        };

        trace!(
            "compaction request for {}MBs ({} bytes), with timeout of {}s, and schema {:?}.",
            total_input_bytes / MiB,
            total_input_bytes,
            timeout.as_secs_f64(),
            compaction_schema.id,
        );

        let compact_span = debug_span!("compact::consolidate");
        let res = tokio::time::timeout(
            timeout,
            // Compaction is cpu intensive, so be polite and spawn it on the isolated runtime.
            machine
                .isolated_runtime
                .spawn_named(
                    || "persist::compact::consolidate",
                    Self::compact(
                        CompactConfig::new(&machine.applier.cfg, machine.shard_id()),
                        Arc::clone(&machine.applier.state_versions.blob),
                        Arc::clone(&metrics),
                        Arc::clone(&machine.applier.shard_metrics),
                        Arc::clone(&machine.isolated_runtime),
                        req,
                        compaction_schema,
                    )
                    .instrument(compact_span),
                )
                .map_err(|e| anyhow!(e)),
        )
        .await;

        let res = match res {
            Ok(res) => res,
            Err(err) => {
                metrics.compaction.timed_out.inc();
                Err(anyhow!(err))
            }
        };

        metrics
            .compaction
            .seconds
            .inc_by(start.elapsed().as_secs_f64());

        match res {
            Ok(Ok(res)) => {
                let res = FueledMergeRes { output: res.output };
                let (apply_merge_result, maintenance) = machine.merge_res(&res).await;
                match &apply_merge_result {
                    ApplyMergeResult::AppliedExact => {
                        metrics.compaction.applied.inc();
                        metrics.compaction.applied_exact_match.inc();
                        machine.applier.shard_metrics.compaction_applied.inc();
                        Ok((apply_merge_result, maintenance))
                    }
                    ApplyMergeResult::AppliedSubset => {
                        metrics.compaction.applied.inc();
                        metrics.compaction.applied_subset_match.inc();
                        machine.applier.shard_metrics.compaction_applied.inc();
                        Ok((apply_merge_result, maintenance))
                    }
                    ApplyMergeResult::NotAppliedNoMatch
                    | ApplyMergeResult::NotAppliedInvalidSince
                    | ApplyMergeResult::NotAppliedTooManyUpdates => {
                        if let ApplyMergeResult::NotAppliedTooManyUpdates = &apply_merge_result {
                            metrics.compaction.not_applied_too_many_updates.inc();
                        }
                        metrics.compaction.noop.inc();
                        let mut part_deletes = PartDeletes::default();
                        for part in res.output.parts {
                            part_deletes.add(&part);
                        }
                        let () = part_deletes
                            .delete(
                                machine.applier.state_versions.blob.as_ref(),
                                machine.shard_id(),
                                GC_BLOB_DELETE_CONCURRENCY_LIMIT.get(&machine.applier.cfg),
                                &*metrics,
                                &metrics.retries.external.compaction_noop_delete,
                            )
                            .await;
                        Ok((apply_merge_result, maintenance))
                    }
                }
            }
            Ok(Err(err)) | Err(err) => {
                metrics.compaction.failed.inc();
                debug!(
                    "compaction for {} failed: {}",
                    machine.shard_id(),
                    err.display_with_causes()
                );
                Err(err)
            }
        }
    }

    /// Compacts input batches in bounded memory.
    ///
    /// The memory bound is broken into pieces:
    ///     1. in-progress work
    ///     2. fetching parts from runs
    ///     3. additional in-flight requests to Blob
    ///
    /// 1. In-progress work is bounded by 2 * [BatchBuilderConfig::blob_target_size]. This
    ///    usage is met at two mutually exclusive moments:
    ///   * When reading in a part, we hold the columnar format in memory while writing its
    ///     contents into a heap.
    ///   * When writing a part, we hold a temporary updates buffer while encoding/writing
    ///     it into a columnar format for Blob.
    ///
    /// 2. When compacting runs, only 1 part from each one is held in memory at a time.
    ///    Compaction will determine an appropriate number of runs to compact together
    ///    given the memory bound and accounting for the reservation in (1). A minimum
    ///    of 2 * [BatchBuilderConfig::blob_target_size] of memory is expected, to be
    ///    able to at least have the capacity to compact two runs together at a time,
    ///    and more runs will be compacted together if more memory is available.
    ///
    /// 3. If there is excess memory after accounting for (1) and (2), we increase the
    ///    number of outstanding parts we can keep in-flight to Blob.
    pub async fn compact(
        cfg: CompactConfig,
        blob: Arc<dyn Blob>,
        metrics: Arc<Metrics>,
        shard_metrics: Arc<ShardMetrics>,
        isolated_runtime: Arc<IsolatedRuntime>,
        req: CompactReq<T>,
        write_schemas: Schemas<K, V>,
    ) -> Result<CompactRes<T>, anyhow::Error> {
        let () = Self::validate_req(&req)?;

        // We introduced a fast-path optimization in https://github.com/MaterializeInc/materialize/pull/15363
        // but had to revert it due to a very scary bug. Here we count how many of our compaction reqs
        // could be eligible for the optimization to better understand whether it's worth trying to
        // reintroduce it.
        let mut single_nonempty_batch = None;
        for batch in &req.inputs {
            if batch.len > 0 {
                match single_nonempty_batch {
                    None => single_nonempty_batch = Some(batch),
                    Some(_previous_nonempty_batch) => {
                        single_nonempty_batch = None;
                        break;
                    }
                }
            }
        }
        if let Some(single_nonempty_batch) = single_nonempty_batch {
            if single_nonempty_batch.run_splits.len() == 0
                && single_nonempty_batch.desc.since() != &Antichain::from_elem(T::minimum())
            {
                metrics.compaction.fast_path_eligible.inc();
            }
        }

        // compaction needs memory enough for at least 2 runs and 2 in-progress parts
        assert!(cfg.compaction_memory_bound_bytes >= 4 * cfg.batch.blob_target_size);
        // reserve space for the in-progress part to be held in-mem representation and columnar
        let in_progress_part_reserved_memory_bytes = 2 * cfg.batch.blob_target_size;
        // then remaining memory will go towards pulling down as many runs as we can
        let run_reserved_memory_bytes =
            cfg.compaction_memory_bound_bytes - in_progress_part_reserved_memory_bytes;

        let mut all_parts = vec![];
        let mut all_run_splits = vec![];
        let mut all_run_meta = vec![];
        let mut len = 0;

        let ordered_runs =
            Self::order_runs(&req, cfg.batch.preferred_order, &*blob, &*metrics).await?;
        for (runs, run_chunk_max_memory_usage) in
            Self::chunk_runs(&ordered_runs, &cfg, &*metrics, run_reserved_memory_bytes)
        {
            metrics.compaction.chunks_compacted.inc();
            metrics
                .compaction
                .runs_compacted
                .inc_by(u64::cast_from(runs.len()));

            // given the runs we actually have in our batch, we might have extra memory
            // available. we reserved enough space to always have 1 in-progress part in
            // flight, but if we have excess, we can use it to increase our write parallelism
            let extra_outstanding_parts = (run_reserved_memory_bytes
                .saturating_sub(run_chunk_max_memory_usage))
                / cfg.batch.blob_target_size;
            let mut run_cfg = cfg.clone();
            run_cfg.batch.batch_builder_max_outstanding_parts = 1 + extra_outstanding_parts;
            let batch = Self::compact_runs(
                &run_cfg,
                &req.shard_id,
                &req.desc,
                runs,
                Arc::clone(&blob),
                Arc::clone(&metrics),
                Arc::clone(&shard_metrics),
                Arc::clone(&isolated_runtime),
                write_schemas.clone(),
            )
            .await?;
            let (parts, run_splits, run_meta, updates) =
                (batch.parts, batch.run_splits, batch.run_meta, batch.len);
            assert!(
                (updates == 0 && parts.len() == 0) || (updates > 0 && parts.len() > 0),
                "updates={}, parts={}",
                updates,
                parts.len(),
            );

            if updates == 0 {
                continue;
            }
            // merge together parts and runs from each compaction round.
            // parts are appended onto our existing vec, and then we shift
            // the latest run offsets to account for prior parts.
            //
            // e.g. if we currently have 3 parts and 2 runs (including the implicit one from 0):
            //         parts: [k0, k1, k2]
            //         runs:  [    1     ]
            //
            // and we merge in another result with 2 parts and 2 runs:
            //         parts: [k3, k4]
            //         runs:  [    1]
            //
            // we our result will contain 5 parts and 4 runs:
            //         parts: [k0, k1, k2, k3, k4]
            //         runs:  [    1       3   4 ]
            let run_offset = all_parts.len();
            if all_parts.len() > 0 {
                all_run_splits.push(run_offset);
            }
            all_run_splits.extend(run_splits.iter().map(|run_start| run_start + run_offset));
            all_run_meta.extend(run_meta);
            all_parts.extend(parts);
            len += updates;
        }

        Ok(CompactRes {
            output: HollowBatch::new(
                req.desc.clone(),
                all_parts,
                len,
                all_run_meta,
                all_run_splits,
            ),
        })
    }

    /// Sorts and groups all runs from the inputs into chunks, each of which has been determined
    /// to consume no more than `run_reserved_memory_bytes` at a time, unless the input parts
    /// were written with a different target size than this build. Uses [Self::order_runs] to
    /// determine the order in which runs are selected.
    fn chunk_runs<'a>(
        ordered_runs: &'a [(&'a Description<T>, &'a RunMeta, Cow<'a, [RunPart<T>]>)],
        cfg: &CompactConfig,
        metrics: &Metrics,
        run_reserved_memory_bytes: usize,
    ) -> Vec<(
        Vec<(&'a Description<T>, &'a RunMeta, &'a [RunPart<T>])>,
        usize,
    )> {
        let mut ordered_runs = ordered_runs.into_iter().peekable();

        let mut chunks = vec![];
        let mut current_chunk = vec![];
        let mut current_chunk_max_memory_usage = 0;
        while let Some((desc, meta, run)) = ordered_runs.next() {
            let run_greatest_part_size = run
                .iter()
                .map(|x| x.max_part_bytes())
                .max()
                .unwrap_or(cfg.batch.blob_target_size);
            current_chunk.push((*desc, *meta, &**run));
            current_chunk_max_memory_usage += run_greatest_part_size;

            if let Some((_next_desc, _next_meta, next_run)) = ordered_runs.peek() {
                let next_run_greatest_part_size = next_run
                    .iter()
                    .map(|x| x.max_part_bytes())
                    .max()
                    .unwrap_or(cfg.batch.blob_target_size);

                // if we can fit the next run in our chunk without going over our reserved memory, we should do so
                if current_chunk_max_memory_usage + next_run_greatest_part_size
                    <= run_reserved_memory_bytes
                {
                    continue;
                }

                // NB: There's an edge case where we cannot fit at least 2 runs into a chunk
                // with our reserved memory. This could happen if blobs were written with a
                // larger target size than the current build. When this happens, we violate
                // our memory requirement and force chunks to be at least length 2, so that we
                // can be assured runs are merged and converge over time.
                if current_chunk.len() == 1 {
                    // in the steady state we expect this counter to be 0, and would only
                    // anticipate it being temporarily nonzero if we changed target blob size
                    // or our memory requirement calculations
                    metrics.compaction.memory_violations.inc();
                    continue;
                }
            }

            chunks.push((
                std::mem::take(&mut current_chunk),
                current_chunk_max_memory_usage,
            ));
            current_chunk_max_memory_usage = 0;
        }

        chunks
    }

    /// With bounded memory where we cannot compact all runs/parts together, the groupings
    /// in which we select runs to compact together will affect how much we're able to
    /// consolidate updates.
    ///
    /// This approach orders the input runs by cycling through each batch, selecting the
    /// head element until all are consumed. It assumes that it is generally more effective
    /// to prioritize compacting runs from different batches, rather than runs from within
    /// a single batch.
    ///
    /// ex.
    /// ```text
    ///        inputs                                        output
    ///     b0 runs=[A, B]
    ///     b1 runs=[C]                           output=[A, C, D, B, E, F]
    ///     b2 runs=[D, E, F]
    /// ```
    async fn order_runs<'a>(
        req: &'a CompactReq<T>,
        target_order: RunOrder,
        blob: &'a dyn Blob,
        metrics: &'a Metrics,
    ) -> anyhow::Result<Vec<(&'a Description<T>, &'a RunMeta, Cow<'a, [RunPart<T>]>)>> {
        let total_number_of_runs = req
            .inputs
            .iter()
            .map(|x| x.run_splits.len() + 1)
            .sum::<usize>();

        let mut batch_runs: VecDeque<_> = req
            .inputs
            .iter()
            .map(|batch| (&batch.desc, batch.runs()))
            .collect();

        let mut ordered_runs = Vec::with_capacity(total_number_of_runs);

        while let Some((desc, mut runs)) = batch_runs.pop_front() {
            if let Some((meta, run)) = runs.next() {
                let same_order = meta.order.unwrap_or(RunOrder::Codec) == target_order;
                if same_order {
                    ordered_runs.push((desc, meta, Cow::Borrowed(run)));
                } else {
                    // The downstream consolidation step will handle a long run that's not in
                    // the desired order by splitting it up into many single-element runs. This preserves
                    // correctness, but it means that we may end up needing to iterate through
                    // many more parts concurrently than expected, increasing memory use. Instead,
                    // we break up those runs into individual batch parts, fetching hollow runs as
                    // necessary, before they're grouped together to be passed to consolidation.
                    // The downside is that this breaks the usual property that compaction produces
                    // fewer runs than it takes in. This should generally be resolved by future
                    // runs of compaction.
                    for part in run {
                        let mut batch_parts = pin!(part.part_stream(req.shard_id, blob, metrics));
                        while let Some(part) = batch_parts.next().await {
                            ordered_runs.push((
                                desc,
                                meta,
                                Cow::Owned(vec![RunPart::Single(part?.into_owned())]),
                            ));
                        }
                    }
                }
                batch_runs.push_back((desc, runs));
            }
        }

        Ok(ordered_runs)
    }

    /// Compacts runs together. If the input runs are sorted, a single run will be created as output.
    ///
    /// Maximum possible memory usage is `(# runs + 2) * [crate::PersistConfig::blob_target_size]`
    pub(crate) async fn compact_runs(
        cfg: &CompactConfig,
        shard_id: &ShardId,
        desc: &Description<T>,
        runs: Vec<(&Description<T>, &RunMeta, &[RunPart<T>])>,
        blob: Arc<dyn Blob>,
        metrics: Arc<Metrics>,
        shard_metrics: Arc<ShardMetrics>,
        isolated_runtime: Arc<IsolatedRuntime>,
        write_schemas: Schemas<K, V>,
    ) -> Result<HollowBatch<T>, anyhow::Error> {
        // TODO: Figure out a more principled way to allocate our memory budget.
        // Currently, we give any excess budget to write parallelism. If we had
        // to pick between 100% towards writes vs 100% towards reads, then reads
        // is almost certainly better, but the ideal is probably somewhere in
        // between the two.
        //
        // For now, invent some some extra budget out of thin air for prefetch.
        let prefetch_budget_bytes = 2 * cfg.batch.blob_target_size;

        let mut timings = Timings::default();

        let mut batch_cfg = cfg.batch.clone();

        // Use compaction as a method of getting inline writes out of state, to
        // make room for more inline writes. We could instead do this at the end
        // of compaction by flushing out the batch, but doing it here based on
        // the config allows BatchBuilder to do its normal pipelining of writes.
        batch_cfg.inline_writes_single_max_bytes = 0;

        let parts = BatchParts::new_ordered(
            batch_cfg,
            cfg.batch.preferred_order,
            Arc::clone(&metrics),
            Arc::clone(&shard_metrics),
            *shard_id,
            Arc::clone(&blob),
            Arc::clone(&isolated_runtime),
            &metrics.compaction.batch,
        );
        let mut batch = BatchBuilderInternal::<K, V, T, D>::new(
            cfg.batch.clone(),
            parts,
            Arc::clone(&metrics),
            write_schemas.clone(),
            Arc::clone(&blob),
            shard_id.clone(),
            cfg.version.clone(),
        );

        // Duplicating a large codepath here during the migration.
        // TODO(database-issues#7188): dedup once the migration is complete.
        if cfg.batch.preferred_order == RunOrder::Structured {
            // If we're not writing down the record metadata, we must always use the old compaction
            // order. (Since that's the default when the metadata's not present.)
            let mut consolidator = Consolidator::new(
                format!(
                    "{}[lower={:?},upper={:?}]",
                    shard_id,
                    desc.lower().elements(),
                    desc.upper().elements()
                ),
                *shard_id,
                StructuredSort::<K, V, T, D>::new(write_schemas.clone()),
                blob,
                Arc::clone(&metrics),
                shard_metrics,
                metrics.read.compaction.clone(),
                FetchBatchFilter::Compaction {
                    since: desc.since().clone(),
                },
                prefetch_budget_bytes,
            );

            for (desc, meta, parts) in runs {
                consolidator.enqueue_run(desc, meta, parts.iter().cloned());
            }

            let remaining_budget = consolidator.start_prefetches();
            if remaining_budget.is_none() {
                metrics.compaction.not_all_prefetched.inc();
            }

            loop {
                let mut chunks = vec![];
                let mut total_bytes = 0;
                // We attempt to pull chunks out of the consolidator that match our target size,
                // but it's possible that we may get smaller chunks... for example, if not all
                // parts have been fetched yet. Loop until we've got enough data to justify flushing
                // it out to blob (or we run out of data.)
                while total_bytes < cfg.batch.blob_target_size {
                    let fetch_start = Instant::now();
                    let Some(chunk) = consolidator
                        .next_chunk(
                            cfg.compaction_yield_after_n_updates,
                            cfg.batch.blob_target_size - total_bytes,
                        )
                        .await?
                    else {
                        break;
                    };
                    timings.part_fetching += fetch_start.elapsed();
                    total_bytes += chunk.goodbytes();
                    chunks.push(chunk);
                    tokio::task::yield_now().await;
                }

                if chunks.is_empty() {
                    break;
                }

                // In the hopefully-common case of a single chunk, this will not copy.
                let updates = BlobTraceUpdates::concat::<K, V>(
                    chunks,
                    write_schemas.key.as_ref(),
                    write_schemas.val.as_ref(),
                    &metrics.columnar,
                )?;
                batch.flush_part(desc.clone(), updates).await;
            }
        } else {
            let mut consolidator = Consolidator::<T, D>::new(
                format!(
                    "{}[lower={:?},upper={:?}]",
                    shard_id,
                    desc.lower().elements(),
                    desc.upper().elements()
                ),
                *shard_id,
                CodecSort::default(),
                blob,
                Arc::clone(&metrics),
                shard_metrics,
                metrics.read.compaction.clone(),
                FetchBatchFilter::Compaction {
                    since: desc.since().clone(),
                },
                prefetch_budget_bytes,
            );

            for (desc, meta, parts) in runs {
                consolidator.enqueue_run(desc, meta, parts.iter().cloned());
            }

            let remaining_budget = consolidator.start_prefetches();
            if remaining_budget.is_none() {
                metrics.compaction.not_all_prefetched.inc();
            }

            loop {
                let mut chunks = vec![];
                let mut total_bytes = 0;
                // We attempt to pull chunks out of the consolidator that match our target size,
                // but it's possible that we may get smaller chunks... for example, if not all
                // parts have been fetched yet. Loop until we've got enough data to justify flushing
                // it out to blob (or we run out of data.)
                while total_bytes < cfg.batch.blob_target_size {
                    let fetch_start = Instant::now();
                    let Some(chunk) = consolidator
                        .next_chunk(
                            cfg.compaction_yield_after_n_updates,
                            cfg.batch.blob_target_size - total_bytes,
                        )
                        .await?
                    else {
                        break;
                    };
                    timings.part_fetching += fetch_start.elapsed();
                    total_bytes += chunk.goodbytes();
                    chunks.push(chunk.records().clone());
                    tokio::task::yield_now().await;
                }

                if chunks.is_empty() {
                    break;
                }

                // In the hopefully-common case of a single chunk, this will not copy.
                let updates = ColumnarRecords::concat(&chunks, &metrics.columnar);
                batch
                    .flush_part(desc.clone(), BlobTraceUpdates::Row(updates))
                    .await;
            }
        }
        let mut batch = batch.finish(desc.clone()).await?;

        // We use compaction as a method of getting inline writes out of state,
        // to make room for more inline writes. This happens in
        // `CompactConfig::new` by overriding the inline writes threshold
        // config. This is a bit action-at-a-distance, so defensively detect if
        // this breaks here and log and correct it if so.
        let has_inline_parts = batch.batch.parts.iter().any(|x| x.is_inline());
        if has_inline_parts {
            error!(%shard_id, ?cfg, "compaction result unexpectedly had inline writes");
            let () = batch
                .flush_to_blob(
                    &cfg.batch,
                    &metrics.compaction.batch,
                    &isolated_runtime,
                    &write_schemas,
                )
                .await;
        }

        timings.record(&metrics);
        Ok(batch.into_hollow_batch())
    }

    fn validate_req(req: &CompactReq<T>) -> Result<(), anyhow::Error> {
        let mut frontier = req.desc.lower();
        for input in req.inputs.iter() {
            if PartialOrder::less_than(req.desc.since(), input.desc.since()) {
                return Err(anyhow!(
                    "output since {:?} must be at or in advance of input since {:?}",
                    req.desc.since(),
                    input.desc.since()
                ));
            }
            if frontier != input.desc.lower() {
                return Err(anyhow!(
                    "invalid merge of non-consecutive batches {:?} vs {:?}",
                    frontier,
                    input.desc.lower()
                ));
            }
            frontier = input.desc.upper();
        }
        if frontier != req.desc.upper() {
            return Err(anyhow!(
                "invalid merge of non-consecutive batches {:?} vs {:?}",
                frontier,
                req.desc.upper()
            ));
        }
        Ok(())
    }
}

#[derive(Debug, Default)]
struct Timings {
    part_fetching: Duration,
    heap_population: Duration,
}

impl Timings {
    fn record(self, metrics: &Metrics) {
        // intentionally deconstruct so we don't forget to consider each field
        let Timings {
            part_fetching,
            heap_population,
        } = self;

        metrics
            .compaction
            .steps
            .part_fetch_seconds
            .inc_by(part_fetching.as_secs_f64());
        metrics
            .compaction
            .steps
            .heap_population_seconds
            .inc_by(heap_population.as_secs_f64());
    }
}

#[cfg(test)]
mod tests {
    use mz_dyncfg::ConfigUpdates;
    use mz_ore::{assert_contains, assert_err};
    use mz_persist_types::codec_impls::StringSchema;
    use timely::order::Product;
    use timely::progress::Antichain;

    use crate::batch::BLOB_TARGET_SIZE;
    use crate::tests::{all_ok, expect_fetch_part, new_test_client_cache};
    use crate::PersistLocation;

    use super::*;

    // A regression test for a bug caught during development of materialize#13160 (never
    // made it to main) where batches written by compaction would always have a
    // since of the minimum timestamp.
    #[mz_persist_proc::test(tokio::test)]
    #[cfg_attr(miri, ignore)] // unsupported operation: returning ready events from epoll_wait is not yet implemented
    async fn regression_minimum_since(dyncfgs: ConfigUpdates) {
        let data = vec![
            (("0".to_owned(), "zero".to_owned()), 0, 1),
            (("0".to_owned(), "zero".to_owned()), 1, -1),
            (("1".to_owned(), "one".to_owned()), 1, 1),
        ];

        let cache = new_test_client_cache(&dyncfgs);
        cache.cfg.set_config(&BLOB_TARGET_SIZE, 100);
        let (mut write, _) = cache
            .open(PersistLocation::new_in_mem())
            .await
            .expect("client construction failed")
            .expect_open::<String, String, u64, i64>(ShardId::new())
            .await;
        let b0 = write
            .expect_batch(&data[..1], 0, 1)
            .await
            .into_hollow_batch();
        let b1 = write
            .expect_batch(&data[1..], 1, 2)
            .await
            .into_hollow_batch();

        let req = CompactReq {
            shard_id: write.machine.shard_id(),
            desc: Description::new(
                b0.desc.lower().clone(),
                b1.desc.upper().clone(),
                Antichain::from_elem(10u64),
            ),
            inputs: vec![b0, b1],
        };
        let schemas = Schemas {
            id: None,
            key: Arc::new(StringSchema),
            val: Arc::new(StringSchema),
        };
        let res = Compactor::<String, String, u64, i64>::compact(
            CompactConfig::new(&write.cfg, write.shard_id()),
            Arc::clone(&write.blob),
            Arc::clone(&write.metrics),
            write.metrics.shards.shard(&write.machine.shard_id(), ""),
            Arc::new(IsolatedRuntime::default()),
            req.clone(),
            schemas.clone(),
        )
        .await
        .expect("compaction failed");

        assert_eq!(res.output.desc, req.desc);
        assert_eq!(res.output.len, 1);
        assert_eq!(res.output.part_count(), 1);
        let part = res.output.parts[0].expect_hollow_part();
        let (part, updates) = expect_fetch_part(
            write.blob.as_ref(),
            &part.key.complete(&write.machine.shard_id()),
            &write.metrics,
            &schemas,
        )
        .await;
        assert_eq!(part.desc, res.output.desc);
        assert_eq!(updates, all_ok(&data, 10));
    }

    #[mz_persist_proc::test(tokio::test)]
    #[cfg_attr(miri, ignore)] // unsupported operation: returning ready events from epoll_wait is not yet implemented
    async fn compaction_partial_order(dyncfgs: ConfigUpdates) {
        let data = vec![
            (("0".to_owned(), "zero".to_owned()), Product::new(0, 10), 1),
            (("1".to_owned(), "one".to_owned()), Product::new(10, 0), 1),
        ];

        let cache = new_test_client_cache(&dyncfgs);
        cache.cfg.set_config(&BLOB_TARGET_SIZE, 100);
        let (mut write, _) = cache
            .open(PersistLocation::new_in_mem())
            .await
            .expect("client construction failed")
            .expect_open::<String, String, Product<u32, u32>, i64>(ShardId::new())
            .await;
        let b0 = write
            .batch(
                &data[..1],
                Antichain::from_elem(Product::new(0, 0)),
                Antichain::from_iter([Product::new(0, 11), Product::new(10, 0)]),
            )
            .await
            .expect("invalid usage")
            .into_hollow_batch();

        let b1 = write
            .batch(
                &data[1..],
                Antichain::from_iter([Product::new(0, 11), Product::new(10, 0)]),
                Antichain::from_elem(Product::new(10, 1)),
            )
            .await
            .expect("invalid usage")
            .into_hollow_batch();

        let req = CompactReq {
            shard_id: write.machine.shard_id(),
            desc: Description::new(
                b0.desc.lower().clone(),
                b1.desc.upper().clone(),
                Antichain::from_elem(Product::new(10, 0)),
            ),
            inputs: vec![b0, b1],
        };
        let schemas = Schemas {
            id: None,
            key: Arc::new(StringSchema),
            val: Arc::new(StringSchema),
        };
        let res = Compactor::<String, String, Product<u32, u32>, i64>::compact(
            CompactConfig::new(&write.cfg, write.shard_id()),
            Arc::clone(&write.blob),
            Arc::clone(&write.metrics),
            write.metrics.shards.shard(&write.machine.shard_id(), ""),
            Arc::new(IsolatedRuntime::default()),
            req.clone(),
            schemas.clone(),
        )
        .await
        .expect("compaction failed");

        assert_eq!(res.output.desc, req.desc);
        assert_eq!(res.output.len, 2);
        assert_eq!(res.output.part_count(), 1);
        let part = res.output.parts[0].expect_hollow_part();
        let (part, updates) = expect_fetch_part(
            write.blob.as_ref(),
            &part.key.complete(&write.machine.shard_id()),
            &write.metrics,
            &schemas,
        )
        .await;
        assert_eq!(part.desc, res.output.desc);
        assert_eq!(updates, all_ok(&data, Product::new(10, 0)));
    }

    #[mz_persist_proc::test(tokio::test)]
    #[cfg_attr(miri, ignore)] // unsupported operation: returning ready events from epoll_wait is not yet implemented
    async fn disable_compaction(dyncfgs: ConfigUpdates) {
        let data = [
            (("0".to_owned(), "zero".to_owned()), 0, 1),
            (("0".to_owned(), "zero".to_owned()), 1, -1),
            (("1".to_owned(), "one".to_owned()), 1, 1),
        ];

        let cache = new_test_client_cache(&dyncfgs);
        cache.cfg.set_config(&BLOB_TARGET_SIZE, 100);
        let (mut write, _) = cache
            .open(PersistLocation::new_in_mem())
            .await
            .expect("client construction failed")
            .expect_open::<String, String, u64, i64>(ShardId::new())
            .await;
        let b0 = write
            .expect_batch(&data[..1], 0, 1)
            .await
            .into_hollow_batch();
        let b1 = write
            .expect_batch(&data[1..], 1, 2)
            .await
            .into_hollow_batch();

        let req = CompactReq {
            shard_id: write.machine.shard_id(),
            desc: Description::new(
                b0.desc.lower().clone(),
                b1.desc.upper().clone(),
                Antichain::from_elem(10u64),
            ),
            inputs: vec![b0, b1],
        };
        write.cfg.set_config(&COMPACTION_HEURISTIC_MIN_INPUTS, 1);
        let compactor = write.compact.as_ref().expect("compaction hard disabled");

        write.cfg.disable_compaction();
        let result = compactor
            .compact_and_apply_background(req.clone(), &write.machine)
            .expect("listener")
            .await
            .expect("channel closed");
        assert_err!(result);
        assert_contains!(result.unwrap_err().to_string(), "compaction disabled");

        write.cfg.enable_compaction();
        compactor
            .compact_and_apply_background(req, &write.machine)
            .expect("listener")
            .await
            .expect("channel closed")
            .expect("compaction success");

        // Make sure our CYA dyncfg works.
        let data2 = [
            (("2".to_owned(), "two".to_owned()), 2, 1),
            (("2".to_owned(), "two".to_owned()), 3, -1),
            (("3".to_owned(), "three".to_owned()), 3, 1),
        ];

        let b2 = write
            .expect_batch(&data2[..1], 2, 3)
            .await
            .into_hollow_batch();
        let b3 = write
            .expect_batch(&data2[1..], 3, 4)
            .await
            .into_hollow_batch();

        let req = CompactReq {
            shard_id: write.machine.shard_id(),
            desc: Description::new(
                b2.desc.lower().clone(),
                b3.desc.upper().clone(),
                Antichain::from_elem(20u64),
            ),
            inputs: vec![b2, b3],
        };
        let compactor = write.compact.as_ref().expect("compaction hard disabled");

        // When the dyncfg is set to false we should ignore the process flag.
        write.cfg.set_config(&COMPACTION_CHECK_PROCESS_FLAG, false);
        write.cfg.disable_compaction();
        // Compaction still succeeded!
        compactor
            .compact_and_apply_background(req, &write.machine)
            .expect("listener")
            .await
            .expect("channel closed")
            .expect("compaction success");
    }
}