datafusion-execution 54.0.0

Execution configuration support for DataFusion query engine
Documentation
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
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements.  See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership.  The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License.  You may obtain a copy of the License at
//
//   http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied.  See the License for the
// specific language governing permissions and limitations
// under the License.

use crate::memory_pool::{
    MemoryConsumer, MemoryLimit, MemoryPool, MemoryReservation, human_readable_size,
};
use datafusion_common::HashMap;
use datafusion_common::{DataFusionError, Result, resources_datafusion_err};
use log::debug;
use parking_lot::Mutex;
use std::fmt::{Display, Formatter};
use std::{
    num::NonZeroUsize,
    sync::atomic::{AtomicUsize, Ordering},
};

/// A [`MemoryPool`] that enforces no limit
#[derive(Debug, Default)]
pub struct UnboundedMemoryPool {
    used: AtomicUsize,
}

impl MemoryPool for UnboundedMemoryPool {
    fn name(&self) -> &str {
        "unbounded"
    }

    fn grow(&self, _reservation: &MemoryReservation, additional: usize) {
        self.used.fetch_add(additional, Ordering::Relaxed);
    }

    fn shrink(&self, _reservation: &MemoryReservation, shrink: usize) {
        self.used.fetch_sub(shrink, Ordering::Relaxed);
    }

    fn try_grow(&self, reservation: &MemoryReservation, additional: usize) -> Result<()> {
        self.grow(reservation, additional);
        Ok(())
    }

    fn reserved(&self) -> usize {
        self.used.load(Ordering::Relaxed)
    }

    fn memory_limit(&self) -> MemoryLimit {
        MemoryLimit::Infinite
    }
}

impl Display for UnboundedMemoryPool {
    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
        let used = self.used.load(Ordering::Relaxed);
        write!(f, "{}(used: {})", &self.name(), human_readable_size(used))
    }
}

/// A [`MemoryPool`] that implements a greedy first-come first-serve limit.
///
/// This pool works well for queries that do not need to spill or have
/// a single spillable operator. See [`FairSpillPool`] if there are
/// multiple spillable operators that all will spill.
#[derive(Debug)]
pub struct GreedyMemoryPool {
    pool_size: usize,
    used: AtomicUsize,
}

impl GreedyMemoryPool {
    /// Create a new pool that can allocate up to `pool_size` bytes
    pub fn new(pool_size: usize) -> Self {
        debug!("Created new GreedyMemoryPool(pool_size={pool_size})");
        Self {
            pool_size,
            used: AtomicUsize::new(0),
        }
    }
}

impl MemoryPool for GreedyMemoryPool {
    fn name(&self) -> &str {
        "greedy"
    }

    fn grow(&self, _reservation: &MemoryReservation, additional: usize) {
        self.used.fetch_add(additional, Ordering::Relaxed);
    }

    fn shrink(&self, _reservation: &MemoryReservation, shrink: usize) {
        self.used.fetch_sub(shrink, Ordering::Relaxed);
    }

    fn try_grow(&self, reservation: &MemoryReservation, additional: usize) -> Result<()> {
        self.used
            .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |used| {
                let new_used = used + additional;
                (new_used <= self.pool_size).then_some(new_used)
            })
            .map_err(|used| {
                insufficient_capacity_err(
                    reservation,
                    additional,
                    self.pool_size.saturating_sub(used),
                    self,
                )
            })?;
        Ok(())
    }

    fn reserved(&self) -> usize {
        self.used.load(Ordering::Relaxed)
    }

    fn memory_limit(&self) -> MemoryLimit {
        MemoryLimit::Finite(self.pool_size)
    }
}

impl Display for GreedyMemoryPool {
    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
        let used = self.used.load(Ordering::Relaxed);
        write!(
            f,
            "{}(used: {}, pool_size: {})",
            &self.name(),
            human_readable_size(used),
            human_readable_size(self.pool_size)
        )
    }
}

/// A [`MemoryPool`] that prevents spillable reservations from using more than
/// an even fraction of the available memory sans any unspillable reservations
/// (i.e. `(pool_size - unspillable_memory) / num_spillable_reservations`)
///
/// This pool works best when you know beforehand the query has
/// multiple spillable operators that will likely all need to
/// spill. Sometimes it will cause spills even when there was
/// sufficient memory (reserved for other operators) to avoid doing
/// so.
///
/// ```text
///    ┌───────────────────────z──────────────────────z───────────────┐
///    │                       z                      z               │
///    │                       z                      z               │
///    │       Spillable       z       Unspillable    z     Free      │
///    │        Memory         z        Memory        z    Memory     │
///    │                       z                      z               │
///    │                       z                      z               │
///    └───────────────────────z──────────────────────z───────────────┘
/// ```
///
/// Unspillable memory is allocated in a first-come, first-serve fashion
#[derive(Debug)]
pub struct FairSpillPool {
    /// The total memory limit
    pool_size: usize,

    state: Mutex<FairSpillPoolState>,
}

#[derive(Debug)]
struct FairSpillPoolState {
    /// The number of consumers that can spill
    num_spill: usize,

    /// The total amount of memory reserved that can be spilled
    spillable: usize,

    /// The total amount of memory reserved by consumers that cannot spill
    unspillable: usize,
}

impl FairSpillPool {
    /// Allocate up to `limit` bytes
    pub fn new(pool_size: usize) -> Self {
        debug!("Created new FairSpillPool(pool_size={pool_size})");
        Self {
            pool_size,
            state: Mutex::new(FairSpillPoolState {
                num_spill: 0,
                spillable: 0,
                unspillable: 0,
            }),
        }
    }
}

impl MemoryPool for FairSpillPool {
    fn name(&self) -> &str {
        "fair"
    }

    fn register(&self, consumer: &MemoryConsumer) {
        if consumer.can_spill {
            self.state.lock().num_spill += 1;
        }
    }

    fn unregister(&self, consumer: &MemoryConsumer) {
        if consumer.can_spill {
            let mut state = self.state.lock();
            state.num_spill = state.num_spill.checked_sub(1).unwrap();
        }
    }

    fn grow(&self, reservation: &MemoryReservation, additional: usize) {
        let mut state = self.state.lock();
        match reservation.registration.consumer.can_spill {
            true => state.spillable += additional,
            false => state.unspillable += additional,
        }
    }

    fn shrink(&self, reservation: &MemoryReservation, shrink: usize) {
        let mut state = self.state.lock();
        match reservation.registration.consumer.can_spill {
            true => state.spillable -= shrink,
            false => state.unspillable -= shrink,
        }
    }

    fn try_grow(&self, reservation: &MemoryReservation, additional: usize) -> Result<()> {
        let mut state = self.state.lock();

        match reservation.registration.consumer.can_spill {
            true => {
                // The total amount of memory available to spilling consumers
                let spill_available = self.pool_size.saturating_sub(state.unspillable);

                // No spiller may use more than their fraction of the memory available
                let available = spill_available
                    .checked_div(state.num_spill)
                    .unwrap_or(spill_available);

                if reservation.size() + additional > available {
                    return Err(insufficient_capacity_err(
                        reservation,
                        additional,
                        available,
                        self,
                    ));
                }
                state.spillable += additional;
            }
            false => {
                let available = self
                    .pool_size
                    .saturating_sub(state.unspillable + state.spillable);

                if available < additional {
                    return Err(insufficient_capacity_err(
                        reservation,
                        additional,
                        available,
                        self,
                    ));
                }
                state.unspillable += additional;
            }
        }
        Ok(())
    }

    fn reserved(&self) -> usize {
        let state = self.state.lock();
        state.spillable + state.unspillable
    }

    fn memory_limit(&self) -> MemoryLimit {
        MemoryLimit::Finite(self.pool_size)
    }
}

impl Display for FairSpillPool {
    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
        write!(
            f,
            "{}(pool_size: {})",
            &self.name(),
            human_readable_size(self.pool_size),
        )
    }
}

/// Constructs a resources error based upon the individual [`MemoryReservation`].
///
/// The error references the `bytes already allocated` for the reservation,
/// and not the total within the collective [`MemoryPool`],
/// nor the total across multiple reservations with the same [`MemoryConsumer`].
#[inline(always)]
fn insufficient_capacity_err(
    reservation: &MemoryReservation,
    additional: usize,
    available: usize,
    pool: &impl MemoryPool,
) -> DataFusionError {
    resources_datafusion_err!(
        "Failed to allocate additional {} for {} with {} already allocated for this reservation - {} remain available for the total memory pool: {}",
        human_readable_size(additional),
        reservation.registration.consumer.name,
        human_readable_size(reservation.size()),
        human_readable_size(available),
        pool
    )
}

#[derive(Debug)]
struct TrackedConsumer {
    name: String,
    can_spill: bool,
    reserved: AtomicUsize,
    peak: AtomicUsize,
}

impl TrackedConsumer {
    /// Shorthand to return the currently reserved value
    fn reserved(&self) -> usize {
        self.reserved.load(Ordering::Relaxed)
    }

    /// Return the peak value
    fn peak(&self) -> usize {
        self.peak.load(Ordering::Relaxed)
    }

    /// Grows the tracked consumer's reserved size,
    /// should be called after the pool has successfully performed the grow().
    fn grow(&self, additional: usize) {
        self.reserved.fetch_add(additional, Ordering::Relaxed);
        self.peak.fetch_max(self.reserved(), Ordering::Relaxed);
    }

    /// Reduce the tracked consumer's reserved size,
    /// should be called after the pool has successfully performed the shrink().
    fn shrink(&self, shrink: usize) {
        self.reserved.fetch_sub(shrink, Ordering::Relaxed);
    }
}

/// A point-in-time snapshot of a tracked memory consumer's state.
///
/// Returned by [`TrackConsumersPool::metrics()`].
#[derive(Debug, Clone)]
pub struct MemoryConsumerMetrics {
    /// The name of the memory consumer
    pub name: String,
    /// Whether this consumer can spill to disk
    pub can_spill: bool,
    /// The number of bytes currently reserved by this consumer
    pub reserved: usize,
    /// The peak number of bytes reserved by this consumer
    pub peak: usize,
}

impl From<&TrackedConsumer> for MemoryConsumerMetrics {
    fn from(tracked: &TrackedConsumer) -> Self {
        Self {
            name: tracked.name.clone(),
            can_spill: tracked.can_spill,
            reserved: tracked.reserved(),
            peak: tracked.peak(),
        }
    }
}

/// A [`MemoryPool`] that tracks the consumers that have
/// reserved memory within the inner memory pool.
///
/// By tracking memory reservations more carefully this pool
/// can provide better error messages on the largest memory users
/// when memory allocation fails.
///
/// Tracking is per hashed [`MemoryConsumer`], not per [`MemoryReservation`].
/// The same consumer can have multiple reservations.
///
/// # Automatic Usage via [`RuntimeEnvBuilder`]
///
/// The easiest way to use `TrackConsumersPool` is via
/// [`RuntimeEnvBuilder::with_memory_limit()`].
///
/// [`RuntimeEnvBuilder`]: crate::runtime_env::RuntimeEnvBuilder
/// [`RuntimeEnvBuilder::with_memory_limit()`]: crate::runtime_env::RuntimeEnvBuilder::with_memory_limit
///
/// # Usage Examples
///
/// For more examples of using `TrackConsumersPool`, see the [memory_pool_tracking.rs] example
///
/// [memory_pool_tracking.rs]: https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/execution_monitoring/memory_pool_tracking.rs
/// [memory_pool_execution_plan.rs]: https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/execution_monitoring/memory_pool_execution_plan.rs
#[derive(Debug)]
pub struct TrackConsumersPool<I> {
    /// The wrapped memory pool that actually handles reservation logic
    inner: I,
    /// The amount of consumers to report(ordered top to bottom by reservation size)
    top: NonZeroUsize,
    /// Maps consumer_id --> TrackedConsumer
    tracked_consumers: Mutex<HashMap<usize, TrackedConsumer>>,
}

impl<I: MemoryPool> Display for TrackConsumersPool<I> {
    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
        write!(
            f,
            "{}(inner_pool: {}, num_of_top_consumers: {})",
            &self.name(),
            &self.inner,
            &self.top,
        )
    }
}

impl<I: MemoryPool> TrackConsumersPool<I> {
    /// Creates a new [`TrackConsumersPool`].
    ///
    /// # Arguments
    /// * `inner` - The underlying memory pool that handles actual memory allocation
    /// * `top` - The number of top memory consumers to include in error messages
    ///
    /// # Note
    /// In most cases, you should use [`RuntimeEnvBuilder::with_memory_limit()`](crate::runtime_env::RuntimeEnvBuilder::with_memory_limit)
    /// instead of creating this pool manually, as it automatically sets up tracking with
    /// sensible defaults (top 5 consumers).
    ///
    /// # Example
    ///
    /// ```rust
    /// use datafusion_execution::memory_pool::{
    ///     FairSpillPool, GreedyMemoryPool, TrackConsumersPool,
    /// };
    /// use std::num::NonZeroUsize;
    ///
    /// // Create with a greedy pool backend, reporting top 3 consumers in error messages
    /// let tracked_greedy = TrackConsumersPool::new(
    ///     GreedyMemoryPool::new(1024 * 1024), // 1MB limit
    ///     NonZeroUsize::new(3).unwrap(),
    /// );
    ///
    /// // Create with a fair spill pool backend, reporting top 5 consumers in error messages
    /// let tracked_fair = TrackConsumersPool::new(
    ///     FairSpillPool::new(2 * 1024 * 1024), // 2MB limit
    ///     NonZeroUsize::new(5).unwrap(),
    /// );
    /// ```
    ///
    /// # Impact on Error Messages
    ///
    /// The `top` determines how many Top K [`MemoryConsumer`]s to include
    /// in the reported [`DataFusionError::ResourcesExhausted`].
    pub fn new(inner: I, top: NonZeroUsize) -> Self {
        Self {
            inner,
            top,
            tracked_consumers: Default::default(),
        }
    }

    /// Returns a reference to the wrapped inner [`MemoryPool`].
    pub fn inner(&self) -> &I {
        &self.inner
    }

    /// Returns a snapshot of all currently tracked consumers.
    pub fn metrics(&self) -> Vec<MemoryConsumerMetrics> {
        self.tracked_consumers
            .lock()
            .values()
            .map(Into::into)
            .collect()
    }

    /// Returns a formatted string with the top memory consumers.
    pub fn report_top(&self, top: usize) -> String {
        let mut consumers = self
            .tracked_consumers
            .lock()
            .iter()
            .map(|(consumer_id, tracked_consumer)| {
                (
                    (
                        *consumer_id,
                        tracked_consumer.name.to_owned(),
                        tracked_consumer.can_spill,
                        tracked_consumer.peak(),
                    ),
                    tracked_consumer.reserved(),
                )
            })
            .collect::<Vec<_>>();
        consumers.sort_by_key(|consumer| std::cmp::Reverse(consumer.1));

        consumers[0..std::cmp::min(top, consumers.len())]
            .iter()
            .map(|((id, name, can_spill, peak), size)| {
                format!(
                    "  {name}#{id}(can spill: {can_spill}) consumed {}, peak {}",
                    human_readable_size(*size),
                    human_readable_size(*peak),
                )
            })
            .collect::<Vec<_>>()
            .join(",\n")
            + "."
    }
}

impl<I: MemoryPool> MemoryPool for TrackConsumersPool<I> {
    fn name(&self) -> &str {
        "track_consumers"
    }

    fn register(&self, consumer: &MemoryConsumer) {
        self.inner.register(consumer);

        let mut guard = self.tracked_consumers.lock();
        let existing = guard.insert(
            consumer.id(),
            TrackedConsumer {
                name: consumer.name().to_string(),
                can_spill: consumer.can_spill(),
                reserved: Default::default(),
                peak: Default::default(),
            },
        );

        debug_assert!(
            existing.is_none(),
            "Registered was called twice on the same consumer"
        );
    }

    fn unregister(&self, consumer: &MemoryConsumer) {
        self.inner.unregister(consumer);
        self.tracked_consumers.lock().remove(&consumer.id());
    }

    fn grow(&self, reservation: &MemoryReservation, additional: usize) {
        self.inner.grow(reservation, additional);
        self.tracked_consumers
            .lock()
            .entry(reservation.consumer().id())
            .and_modify(|tracked_consumer| {
                tracked_consumer.grow(additional);
            });
    }

    fn shrink(&self, reservation: &MemoryReservation, shrink: usize) {
        self.inner.shrink(reservation, shrink);
        self.tracked_consumers
            .lock()
            .entry(reservation.consumer().id())
            .and_modify(|tracked_consumer| {
                tracked_consumer.shrink(shrink);
            });
    }

    fn try_grow(&self, reservation: &MemoryReservation, additional: usize) -> Result<()> {
        self.inner
            .try_grow(reservation, additional)
            .map_err(|e| match e {
                DataFusionError::ResourcesExhausted(e) => {
                    // wrap OOM message in top consumers
                    DataFusionError::ResourcesExhausted(
                        provide_top_memory_consumers_to_error_msg(
                            &reservation.consumer().name,
                            &e,
                            &self.report_top(self.top.into()),
                        ),
                    )
                }
                _ => e,
            })?;

        self.tracked_consumers
            .lock()
            .entry(reservation.consumer().id())
            .and_modify(|tracked_consumer| {
                tracked_consumer.grow(additional);
            });
        Ok(())
    }

    fn reserved(&self) -> usize {
        self.inner.reserved()
    }

    fn memory_limit(&self) -> MemoryLimit {
        self.inner.memory_limit()
    }
}

fn provide_top_memory_consumers_to_error_msg(
    consumer_name: &str,
    error_msg: &str,
    top_consumers: &str,
) -> String {
    format!(
        "Additional allocation failed for {consumer_name} with top memory consumers (across reservations) as:\n{top_consumers}\nError: {error_msg}"
    )
}

#[cfg(test)]
mod tests {
    use super::*;
    use insta::{Settings, allow_duplicates, assert_snapshot, with_settings};
    use std::sync::Arc;

    fn make_settings() -> Settings {
        let mut settings = Settings::clone_current();
        settings.add_filter(
            r"([^\s]+)\#\d+\(can spill: (true|false)\)",
            "$1#[ID](can spill: $2)",
        );
        settings
    }

    #[test]
    fn test_fair() {
        let pool = Arc::new(FairSpillPool::new(100)) as _;

        let r1 = MemoryConsumer::new("unspillable").register(&pool);
        // Can grow beyond capacity of pool
        r1.grow(2000);
        assert_eq!(pool.reserved(), 2000);

        let r2 = MemoryConsumer::new("r2")
            .with_can_spill(true)
            .register(&pool);
        // Can grow beyond capacity of pool
        r2.grow(2000);

        assert_eq!(pool.reserved(), 4000);

        let err = r2.try_grow(1).unwrap_err().strip_backtrace();
        assert_snapshot!(err, @"Resources exhausted: Failed to allocate additional 1.0 B for r2 with 2000.0 B already allocated for this reservation - 0.0 B remain available for the total memory pool: fair(pool_size: 100.0 B)");

        let err = r2.try_grow(1).unwrap_err().strip_backtrace();
        assert_snapshot!(err, @"Resources exhausted: Failed to allocate additional 1.0 B for r2 with 2000.0 B already allocated for this reservation - 0.0 B remain available for the total memory pool: fair(pool_size: 100.0 B)");

        r1.shrink(1990);
        r2.shrink(2000);

        assert_eq!(pool.reserved(), 10);

        r1.try_grow(10).unwrap();
        assert_eq!(pool.reserved(), 20);

        // Can grow r2 to 80 as only spilling consumer
        r2.try_grow(80).unwrap();
        assert_eq!(pool.reserved(), 100);

        r2.shrink(70);

        assert_eq!(r1.size(), 20);
        assert_eq!(r2.size(), 10);
        assert_eq!(pool.reserved(), 30);

        let r3 = MemoryConsumer::new("r3")
            .with_can_spill(true)
            .register(&pool);

        let err = r3.try_grow(70).unwrap_err().strip_backtrace();
        assert_snapshot!(err, @"Resources exhausted: Failed to allocate additional 70.0 B for r3 with 0.0 B already allocated for this reservation - 40.0 B remain available for the total memory pool: fair(pool_size: 100.0 B)");

        //Shrinking r2 to zero doesn't allow a3 to allocate more than 45
        r2.free();
        let err = r3.try_grow(70).unwrap_err().strip_backtrace();
        assert_snapshot!(err, @"Resources exhausted: Failed to allocate additional 70.0 B for r3 with 0.0 B already allocated for this reservation - 40.0 B remain available for the total memory pool: fair(pool_size: 100.0 B)");

        // But dropping r2 does
        drop(r2);
        assert_eq!(pool.reserved(), 20);
        r3.try_grow(80).unwrap();

        assert_eq!(pool.reserved(), 100);
        r1.free();
        assert_eq!(pool.reserved(), 80);

        let r4 = MemoryConsumer::new("s4").register(&pool);
        let err = r4.try_grow(30).unwrap_err().strip_backtrace();
        assert_snapshot!(err, @"Resources exhausted: Failed to allocate additional 30.0 B for s4 with 0.0 B already allocated for this reservation - 20.0 B remain available for the total memory pool: fair(pool_size: 100.0 B)");
    }

    #[test]
    fn test_tracked_consumers_pool() {
        let setting = make_settings();
        let _bound = setting.bind_to_scope();
        let pool: Arc<dyn MemoryPool> = Arc::new(TrackConsumersPool::new(
            GreedyMemoryPool::new(100),
            NonZeroUsize::new(3).unwrap(),
        ));

        // Test: use all the different interfaces to change reservation size

        // set r1=50, using grow and shrink
        let r1 = MemoryConsumer::new("r1").register(&pool);
        r1.grow(50);
        r1.grow(20);
        r1.shrink(20);

        // set r2=15 using try_grow
        let r2 = MemoryConsumer::new("r2").register(&pool);
        r2.try_grow(15)
            .expect("should succeed in memory allotment for r2");

        // set r3=20 using try_resize
        let r3 = MemoryConsumer::new("r3").register(&pool);
        r3.try_resize(25)
            .expect("should succeed in memory allotment for r3");
        r3.try_resize(20)
            .expect("should succeed in memory allotment for r3");

        // set r4=10
        // this should not be reported in top 3
        let r4 = MemoryConsumer::new("r4").register(&pool);
        r4.grow(10);

        // Test: reports if new reservation causes error
        // using the previously set sizes for other consumers
        let r5 = MemoryConsumer::new("r5").register(&pool);
        let res = r5.try_grow(150);
        assert!(res.is_err());
        let error = res.unwrap_err().strip_backtrace();
        assert_snapshot!(error, @r"
        Resources exhausted: Additional allocation failed for r5 with top memory consumers (across reservations) as:
          r1#[ID](can spill: false) consumed 50.0 B, peak 70.0 B,
          r3#[ID](can spill: false) consumed 20.0 B, peak 25.0 B,
          r2#[ID](can spill: false) consumed 15.0 B, peak 15.0 B.
        Error: Failed to allocate additional 150.0 B for r5 with 0.0 B already allocated for this reservation - 5.0 B remain available for the total memory pool: greedy(used: 95.0 B, pool_size: 100.0 B)
        ");
    }

    #[test]
    fn test_tracked_consumers_pool_register() {
        let setting = make_settings();
        let _bound = setting.bind_to_scope();
        let pool: Arc<dyn MemoryPool> = Arc::new(TrackConsumersPool::new(
            GreedyMemoryPool::new(100),
            NonZeroUsize::new(3).unwrap(),
        ));

        let same_name = "foo";

        // Test: see error message when no consumers recorded yet
        let r0 = MemoryConsumer::new(same_name).register(&pool);
        let res = r0.try_grow(150);
        assert!(res.is_err());
        let error = res.unwrap_err().strip_backtrace();
        assert_snapshot!(error, @r"
        Resources exhausted: Additional allocation failed for foo with top memory consumers (across reservations) as:
          foo#[ID](can spill: false) consumed 0.0 B, peak 0.0 B.
        Error: Failed to allocate additional 150.0 B for foo with 0.0 B already allocated for this reservation - 100.0 B remain available for the total memory pool: greedy(used: 0.0 B, pool_size: 100.0 B)
        ");

        // API: multiple registrations using the same hashed consumer,
        // will be recognized *differently* in the TrackConsumersPool.

        r0.grow(10); // make r0=10, pool available=90
        let new_consumer_same_name = MemoryConsumer::new(same_name);
        let r1 = new_consumer_same_name.register(&pool);
        // TODO: the insufficient_capacity_err() message is per reservation, not per consumer.
        // a followup PR will clarify this message "0 bytes already allocated for this reservation"
        let res = r1.try_grow(150);
        assert!(res.is_err());
        let error = res.unwrap_err().strip_backtrace();
        assert_snapshot!(error, @r"
        Resources exhausted: Additional allocation failed for foo with top memory consumers (across reservations) as:
          foo#[ID](can spill: false) consumed 10.0 B, peak 10.0 B,
          foo#[ID](can spill: false) consumed 0.0 B, peak 0.0 B.
        Error: Failed to allocate additional 150.0 B for foo with 0.0 B already allocated for this reservation - 90.0 B remain available for the total memory pool: greedy(used: 10.0 B, pool_size: 100.0 B)
        ");

        // Test: will accumulate size changes per consumer, not per reservation
        r1.grow(20);

        let res = r1.try_grow(150);
        assert!(res.is_err());
        let error = res.unwrap_err().strip_backtrace();
        assert_snapshot!(error, @r"
        Resources exhausted: Additional allocation failed for foo with top memory consumers (across reservations) as:
          foo#[ID](can spill: false) consumed 20.0 B, peak 20.0 B,
          foo#[ID](can spill: false) consumed 10.0 B, peak 10.0 B.
        Error: Failed to allocate additional 150.0 B for foo with 20.0 B already allocated for this reservation - 70.0 B remain available for the total memory pool: greedy(used: 30.0 B, pool_size: 100.0 B)
        ");

        // Test: different hashed consumer, (even with the same name),
        // will be recognized as different in the TrackConsumersPool
        let consumer_with_same_name_but_different_hash =
            MemoryConsumer::new(same_name).with_can_spill(true);
        let r2 = consumer_with_same_name_but_different_hash.register(&pool);
        let res = r2.try_grow(150);
        assert!(res.is_err());
        let error = res.unwrap_err().strip_backtrace();
        assert_snapshot!(error, @r"
        Resources exhausted: Additional allocation failed for foo with top memory consumers (across reservations) as:
          foo#[ID](can spill: false) consumed 20.0 B, peak 20.0 B,
          foo#[ID](can spill: false) consumed 10.0 B, peak 10.0 B,
          foo#[ID](can spill: true) consumed 0.0 B, peak 0.0 B.
        Error: Failed to allocate additional 150.0 B for foo with 0.0 B already allocated for this reservation - 70.0 B remain available for the total memory pool: greedy(used: 30.0 B, pool_size: 100.0 B)
        ");
    }

    #[test]
    fn test_tracked_consumers_pool_deregister() {
        fn test_per_pool_type<P: MemoryPool + 'static>(pool: Arc<TrackConsumersPool<P>>) {
            // `snapshot_suffix` ties each insta snapshot to this pool's inner backend; filters
            // normalize inner pool `Display` so fair vs greedy share the same `@` reference text.
            with_settings!({
                snapshot_suffix => pool.inner().name().to_string(),
                filters => vec![
                    (
                        r"([^\s]+)\#\d+\(can spill: (true|false)\)",
                        "$1#[ID](can spill: $2)",
                    ),
                    (
                        r"for the total memory pool: [^\n]+",
                        "for the total memory pool: [INNER_POOL]",
                    ),
                ],
            }, {
                let memory_pool: Arc<dyn MemoryPool> = Arc::<TrackConsumersPool<P>>::clone(&pool);
                let r0 = MemoryConsumer::new("r0").register(&memory_pool);
                r0.grow(10);
                let r1 = MemoryConsumer::new("r1").register(&memory_pool);
                r1.grow(20);

                // Baseline: see the 2 memory consumers
                let error = r0.try_grow(150).unwrap_err().strip_backtrace();
                assert_snapshot!(error, @r"
                Resources exhausted: Additional allocation failed for r0 with top memory consumers (across reservations) as:
                  r1#[ID](can spill: false) consumed 20.0 B, peak 20.0 B,
                  r0#[ID](can spill: false) consumed 10.0 B, peak 10.0 B.
                Error: Failed to allocate additional 150.0 B for r0 with 10.0 B already allocated for this reservation - 70.0 B remain available for the total memory pool: [INNER_POOL]
                ");

                // Test: unregister one — only the remaining consumer should be listed
                drop(r1);
                let error = r0.try_grow(150).unwrap_err().strip_backtrace();
                assert_snapshot!(error, @r"
                Resources exhausted: Additional allocation failed for r0 with top memory consumers (across reservations) as:
                  r0#[ID](can spill: false) consumed 10.0 B, peak 10.0 B.
                Error: Failed to allocate additional 150.0 B for r0 with 10.0 B already allocated for this reservation - 90.0 B remain available for the total memory pool: [INNER_POOL]
                ");

                // Test: actual message we see is the `available is 70`. When it should be `available is 90`.
                // This is because the pool.shrink() does not automatically occur within the inner_pool.deregister().
                let error = r0.try_grow(150).unwrap_err().strip_backtrace();
                assert_snapshot!(error, @r"
                Resources exhausted: Additional allocation failed for r0 with top memory consumers (across reservations) as:
                  r0#[ID](can spill: false) consumed 10.0 B, peak 10.0 B.
                Error: Failed to allocate additional 150.0 B for r0 with 10.0 B already allocated for this reservation - 90.0 B remain available for the total memory pool: [INNER_POOL]
                ");

                // Test: the registration needs to free itself (or be dropped),
                // for the proper error message
                let error = r0.try_grow(150).unwrap_err().strip_backtrace();
                assert_snapshot!(error, @r"
                Resources exhausted: Additional allocation failed for r0 with top memory consumers (across reservations) as:
                  r0#[ID](can spill: false) consumed 10.0 B, peak 10.0 B.
                Error: Failed to allocate additional 150.0 B for r0 with 10.0 B already allocated for this reservation - 90.0 B remain available for the total memory pool: [INNER_POOL]
                ");
                }
            );
        }

        allow_duplicates! {
            let tracked_spill_pool = Arc::new(TrackConsumersPool::new(
                FairSpillPool::new(100),
                NonZeroUsize::new(3).unwrap(),
            ));
            test_per_pool_type(tracked_spill_pool);

            let tracked_greedy_pool = Arc::new(TrackConsumersPool::new(
                GreedyMemoryPool::new(100),
                NonZeroUsize::new(3).unwrap(),
            ));
            test_per_pool_type(tracked_greedy_pool);
        }
    }

    #[test]
    fn test_track_consumers_pool_metrics() {
        let track_consumers_pool = Arc::new(TrackConsumersPool::new(
            GreedyMemoryPool::new(1000),
            NonZeroUsize::new(3).unwrap(),
        ));
        let memory_pool: Arc<dyn MemoryPool> = Arc::clone(&track_consumers_pool) as _;

        // Empty pool has no metrics
        assert!(track_consumers_pool.metrics().is_empty());

        // Register consumers with different spill settings
        let r1 = MemoryConsumer::new("spilling")
            .with_can_spill(true)
            .register(&memory_pool);
        let r2 = MemoryConsumer::new("non-spilling").register(&memory_pool);

        // Grow r1 in two steps to verify peak tracking
        r1.grow(100);
        r1.grow(50);
        r1.shrink(50); // reserved=100, peak=150

        r2.grow(200); // reserved=200, peak=200

        let mut metrics = track_consumers_pool.metrics();
        metrics.sort_by_key(|m| m.name.clone());

        assert_eq!(metrics.len(), 2);

        let m_non = &metrics[0];
        assert_eq!(m_non.name, "non-spilling");
        assert!(!m_non.can_spill);
        assert_eq!(m_non.reserved, 200);
        assert_eq!(m_non.peak, 200);

        let m_spill = &metrics[1];
        assert_eq!(m_spill.name, "spilling");
        assert!(m_spill.can_spill);
        assert_eq!(m_spill.reserved, 100);
        assert_eq!(m_spill.peak, 150);

        // Unregistered consumers are removed from metrics
        drop(r2);
        let metrics = track_consumers_pool.metrics();
        assert_eq!(metrics.len(), 1);
        assert_eq!(metrics[0].name, "spilling");
    }

    #[test]
    fn test_tracked_consumers_pool_use_beyond_errors() {
        let setting = make_settings();
        let _bound = setting.bind_to_scope();
        let upcasted: Arc<dyn std::any::Any + Send + Sync> =
            Arc::new(TrackConsumersPool::new(
                GreedyMemoryPool::new(100),
                NonZeroUsize::new(3).unwrap(),
            ));
        let pool: Arc<dyn MemoryPool> = Arc::clone(&upcasted)
            .downcast::<TrackConsumersPool<GreedyMemoryPool>>()
            .unwrap();
        // set r1=20
        let r1 = MemoryConsumer::new("r1").register(&pool);
        r1.grow(20);
        // set r2=15
        let r2 = MemoryConsumer::new("r2").register(&pool);
        r2.grow(15);
        // set r3=45
        let r3 = MemoryConsumer::new("r3").register(&pool);
        r3.grow(45);

        let downcasted = upcasted
            .downcast::<TrackConsumersPool<GreedyMemoryPool>>()
            .unwrap();

        // Test: can get runtime metrics, even without an error thrown
        let res = downcasted.report_top(2);
        assert_snapshot!(res, @r"
        r3#[ID](can spill: false) consumed 45.0 B, peak 45.0 B,
        r1#[ID](can spill: false) consumed 20.0 B, peak 20.0 B.
        ");
    }

    #[test]
    fn test_memory_pool_display_fmt() {
        let top = NonZeroUsize::new(5).unwrap();

        // UnboundedMemoryPool Display with default allocation: 0.0B
        let unbounded = UnboundedMemoryPool::default();
        assert_eq!(
            unbounded.to_string(),
            "unbounded(used: 0.0 B)",
            "UnboundedMemoryPool Display"
        );

        // UnboundedMemoryPool Display with reservations
        let unbounded_arc: Arc<dyn MemoryPool> = Arc::new(UnboundedMemoryPool::default());
        let r = MemoryConsumer::new("u").register(&unbounded_arc);
        r.grow(2048);
        assert_eq!(
            unbounded_arc.as_ref().to_string(),
            "unbounded(used: 2.0 KB)",
            "UnboundedMemoryPool Display with reservations"
        );

        // GreedyMemoryPool Display with default allocation: 100.0B
        let greedy = GreedyMemoryPool::new(100);
        assert_eq!(
            greedy.to_string(),
            "greedy(used: 0.0 B, pool_size: 100.0 B)",
            "GreedyMemoryPool Display"
        );

        // GreedyMemoryPool Display with reservations
        let greedy_arc: Arc<dyn MemoryPool> = Arc::new(GreedyMemoryPool::new(100));
        let r = MemoryConsumer::new("g").register(&greedy_arc);
        r.grow(50);
        assert_eq!(
            greedy_arc.as_ref().to_string(),
            "greedy(used: 50.0 B, pool_size: 100.0 B)",
            "GreedyMemoryPool Display with reservations"
        );

        // FairSpillPool Display with default allocation: 4.0KB and without reservations
        let fair = FairSpillPool::new(4096);
        assert_eq!(
            fair.to_string(),
            "fair(pool_size: 4.0 KB)",
            "FairSpillPool Display"
        );

        // TrackConsumersPool<GreedyMemoryPool> Display with default allocation: 128.0B and without reservations
        let tracked_greedy = TrackConsumersPool::new(GreedyMemoryPool::new(128), top);
        assert_eq!(
            tracked_greedy.to_string(),
            "track_consumers(inner_pool: greedy(used: 0.0 B, pool_size: 128.0 B), num_of_top_consumers: 5)",
            "TrackConsumersPool<GreedyMemoryPool> Display"
        );

        // TrackConsumersPool<FairSpillPool> Display with default allocation: 256.0B and without reservations
        let tracked_fair = TrackConsumersPool::new(FairSpillPool::new(256), top);
        assert_eq!(
            tracked_fair.to_string(),
            "track_consumers(inner_pool: fair(pool_size: 256.0 B), num_of_top_consumers: 5)",
            "TrackConsumersPool<FairSpillPool> Display"
        );

        // TrackConsumersPool<UnboundedMemoryPool> Display without reservations
        let tracked_unbounded =
            TrackConsumersPool::new(UnboundedMemoryPool::default(), top);
        assert_eq!(
            tracked_unbounded.to_string(),
            "track_consumers(inner_pool: unbounded(used: 0.0 B), num_of_top_consumers: 5)",
            "TrackConsumersPool<UnboundedMemoryPool> Display"
        );
    }
}