-
Notifications
You must be signed in to change notification settings - Fork 1.2k
/
pool.rs
689 lines (604 loc) · 27.2 KB
/
pool.rs
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
// 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, MemoryPool, MemoryReservation};
use datafusion_common::HashMap;
use datafusion_common::{resources_datafusion_err, DataFusionError, Result};
use log::debug;
use parking_lot::Mutex;
use std::{
num::NonZeroUsize,
sync::atomic::{AtomicU64, AtomicUsize, Ordering},
};
/// A [`MemoryPool`] that enforces no limit
#[derive(Debug, Default)]
pub struct UnboundedMemoryPool {
used: AtomicUsize,
}
impl MemoryPool for UnboundedMemoryPool {
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)
}
}
/// 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 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),
)
})?;
Ok(())
}
fn reserved(&self) -> usize {
self.used.load(Ordering::Relaxed)
}
}
/// 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 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,
));
}
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,
));
}
state.unspillable += additional;
}
}
Ok(())
}
fn reserved(&self) -> usize {
let state = self.state.lock();
state.spillable + state.unspillable
}
}
/// 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,
) -> DataFusionError {
resources_datafusion_err!("Failed to allocate additional {} bytes for {} with {} bytes already allocated for this reservation - {} bytes remain available for the total pool", additional, reservation.registration.consumer.name, reservation.size, available)
}
/// 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
///
/// Tracking is per hashed [`MemoryConsumer`], not per [`MemoryReservation`].
/// The same consumer can have multiple reservations.
#[derive(Debug)]
pub struct TrackConsumersPool<I> {
inner: I,
top: NonZeroUsize,
tracked_consumers: Mutex<HashMap<MemoryConsumer, AtomicU64>>,
}
impl<I: MemoryPool> TrackConsumersPool<I> {
/// Creates a new [`TrackConsumersPool`].
///
/// 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(),
}
}
/// Determine if there are multiple [`MemoryConsumer`]s registered
/// which have the same name.
///
/// This is very tied to the implementation of the memory consumer.
fn has_multiple_consumers(&self, name: &String) -> bool {
let consumer = MemoryConsumer::new(name);
let consumer_with_spill = consumer.clone().with_can_spill(true);
let guard = self.tracked_consumers.lock();
guard.contains_key(&consumer) && guard.contains_key(&consumer_with_spill)
}
/// The top consumers in a report string.
pub fn report_top(&self, top: usize) -> String {
let mut consumers = self
.tracked_consumers
.lock()
.iter()
.map(|(consumer, reserved)| {
(
(consumer.name().to_owned(), consumer.can_spill()),
reserved.load(Ordering::Acquire),
)
})
.collect::<Vec<_>>();
consumers.sort_by(|a, b| b.1.cmp(&a.1)); // inverse ordering
consumers[0..std::cmp::min(top, consumers.len())]
.iter()
.map(|((name, can_spill), size)| {
if self.has_multiple_consumers(name) {
format!("{name}(can_spill={}) consumed {:?} bytes", can_spill, size)
} else {
format!("{name} consumed {:?} bytes", size)
}
})
.collect::<Vec<_>>()
.join(", ")
}
}
impl<I: MemoryPool> MemoryPool for TrackConsumersPool<I> {
fn register(&self, consumer: &MemoryConsumer) {
self.inner.register(consumer);
let mut guard = self.tracked_consumers.lock();
if let Some(already_reserved) = guard.insert(consumer.clone(), Default::default())
{
guard.entry_ref(consumer).and_modify(|bytes| {
bytes.fetch_add(
already_reserved.load(Ordering::Acquire),
Ordering::AcqRel,
);
});
}
}
fn unregister(&self, consumer: &MemoryConsumer) {
self.inner.unregister(consumer);
self.tracked_consumers.lock().remove(consumer);
}
fn grow(&self, reservation: &MemoryReservation, additional: usize) {
self.inner.grow(reservation, additional);
self.tracked_consumers
.lock()
.entry_ref(reservation.consumer())
.and_modify(|bytes| {
bytes.fetch_add(additional as u64, Ordering::AcqRel);
});
}
fn shrink(&self, reservation: &MemoryReservation, shrink: usize) {
self.inner.shrink(reservation, shrink);
self.tracked_consumers
.lock()
.entry_ref(reservation.consumer())
.and_modify(|bytes| {
bytes.fetch_sub(shrink as u64, Ordering::AcqRel);
});
}
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(
e,
self.report_top(self.top.into()),
),
)
}
_ => e,
})?;
self.tracked_consumers
.lock()
.entry_ref(reservation.consumer())
.and_modify(|bytes| {
bytes.fetch_add(additional as u64, Ordering::AcqRel);
});
Ok(())
}
fn reserved(&self) -> usize {
self.inner.reserved()
}
}
fn provide_top_memory_consumers_to_error_msg(
error_msg: String,
top_consumers: String,
) -> String {
format!("Additional allocation failed with top memory consumers (across reservations) as: {}. Error: {}", top_consumers, error_msg)
}
#[cfg(test)]
mod tests {
use super::*;
use std::sync::Arc;
#[test]
fn test_fair() {
let pool = Arc::new(FairSpillPool::new(100)) as _;
let mut r1 = MemoryConsumer::new("unspillable").register(&pool);
// Can grow beyond capacity of pool
r1.grow(2000);
assert_eq!(pool.reserved(), 2000);
let mut 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_eq!(err, "Resources exhausted: Failed to allocate additional 1 bytes for r2 with 2000 bytes already allocated for this reservation - 0 bytes remain available for the total pool");
let err = r2.try_grow(1).unwrap_err().strip_backtrace();
assert_eq!(err, "Resources exhausted: Failed to allocate additional 1 bytes for r2 with 2000 bytes already allocated for this reservation - 0 bytes remain available for the total pool");
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 mut r3 = MemoryConsumer::new("r3")
.with_can_spill(true)
.register(&pool);
let err = r3.try_grow(70).unwrap_err().strip_backtrace();
assert_eq!(err, "Resources exhausted: Failed to allocate additional 70 bytes for r3 with 0 bytes already allocated for this reservation - 40 bytes remain available for the total pool");
//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_eq!(err, "Resources exhausted: Failed to allocate additional 70 bytes for r3 with 0 bytes already allocated for this reservation - 40 bytes remain available for the total pool");
// 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 mut r4 = MemoryConsumer::new("s4").register(&pool);
let err = r4.try_grow(30).unwrap_err().strip_backtrace();
assert_eq!(err, "Resources exhausted: Failed to allocate additional 30 bytes for s4 with 0 bytes already allocated for this reservation - 20 bytes remain available for the total pool");
}
#[test]
fn test_tracked_consumers_pool() {
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 mut r1 = MemoryConsumer::new("r1").register(&pool);
r1.grow(70);
r1.shrink(20);
// set r2=15 using try_grow
let mut 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 mut 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 mut 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 mut r5 = MemoryConsumer::new("r5").register(&pool);
let expected = "Additional allocation failed with top memory consumers (across reservations) as: r1 consumed 50 bytes, r3 consumed 20 bytes, r2 consumed 15 bytes. Error: Failed to allocate additional 150 bytes for r5 with 0 bytes already allocated for this reservation - 5 bytes remain available for the total pool";
let res = r5.try_grow(150);
assert!(
matches!(
&res,
Err(DataFusionError::ResourcesExhausted(ref e)) if e.to_string().contains(expected)
),
"should provide list of top memory consumers, instead found {:?}",
res
);
}
#[test]
fn test_tracked_consumers_pool_register() {
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 mut r0 = MemoryConsumer::new(same_name).register(&pool);
let expected = "Additional allocation failed with top memory consumers (across reservations) as: foo consumed 0 bytes. Error: Failed to allocate additional 150 bytes for foo with 0 bytes already allocated for this reservation - 100 bytes remain available for the total pool";
let res = r0.try_grow(150);
assert!(
matches!(
&res,
Err(DataFusionError::ResourcesExhausted(ref e)) if e.to_string().contains(expected)
),
"should provide proper error when no reservations have been made yet, instead found {:?}", res
);
// API: multiple registrations using the same hashed consumer,
// will be recognized as the same in the TrackConsumersPool.
// Test: will be the same per Top Consumers reported.
r0.grow(10); // make r0=10, pool available=90
let new_consumer_same_name = MemoryConsumer::new(same_name);
let mut 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 expected = "Additional allocation failed with top memory consumers (across reservations) as: foo consumed 10 bytes. Error: Failed to allocate additional 150 bytes for foo with 0 bytes already allocated for this reservation - 90 bytes remain available for the total pool";
let res = r1.try_grow(150);
assert!(
matches!(
&res,
Err(DataFusionError::ResourcesExhausted(ref e)) if e.to_string().contains(expected)
),
"should provide proper error with same hashed consumer (a single foo=10 bytes, available=90), instead found {:?}", res
);
// Test: will accumulate size changes per consumer, not per reservation
r1.grow(20);
let expected = "Additional allocation failed with top memory consumers (across reservations) as: foo consumed 30 bytes. Error: Failed to allocate additional 150 bytes for foo with 20 bytes already allocated for this reservation - 70 bytes remain available for the total pool";
let res = r1.try_grow(150);
assert!(
matches!(
&res,
Err(DataFusionError::ResourcesExhausted(ref e)) if e.to_string().contains(expected)
),
"should provide proper error with same hashed consumer (a single foo=30 bytes, available=70), instead found {:?}", res
);
// 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 mut r2 = consumer_with_same_name_but_different_hash.register(&pool);
let expected = "Additional allocation failed with top memory consumers (across reservations) as: foo(can_spill=false) consumed 30 bytes, foo(can_spill=true) consumed 0 bytes. Error: Failed to allocate additional 150 bytes for foo with 0 bytes already allocated for this reservation - 70 bytes remain available for the total pool";
let res = r2.try_grow(150);
assert!(
matches!(
&res,
Err(DataFusionError::ResourcesExhausted(ref e)) if e.to_string().contains(expected)
),
"should provide proper error with different hashed consumer (foo(can_spill=false)=30 bytes and foo(can_spill=true)=0 bytes, available=70), instead found {:?}", res
);
}
#[test]
fn test_tracked_consumers_pool_deregister() {
fn test_per_pool_type(pool: Arc<dyn MemoryPool>) {
// Baseline: see the 2 memory consumers
let mut r0 = MemoryConsumer::new("r0").register(&pool);
r0.grow(10);
let r1_consumer = MemoryConsumer::new("r1");
let mut r1 = r1_consumer.clone().register(&pool);
r1.grow(20);
let expected = "Additional allocation failed with top memory consumers (across reservations) as: r1 consumed 20 bytes, r0 consumed 10 bytes. Error: Failed to allocate additional 150 bytes for r0 with 10 bytes already allocated for this reservation - 70 bytes remain available for the total pool";
let res = r0.try_grow(150);
assert!(
matches!(
&res,
Err(DataFusionError::ResourcesExhausted(ref e)) if e.to_string().contains(expected)
),
"should provide proper error with both consumers, instead found {:?}",
res
);
// Test: unregister one
// only the remaining one should be listed
pool.unregister(&r1_consumer);
let expected_consumers = "Additional allocation failed with top memory consumers (across reservations) as: r0 consumed 10 bytes";
let res = r0.try_grow(150);
assert!(
matches!(
&res,
Err(DataFusionError::ResourcesExhausted(ref e)) if e.to_string().contains(expected_consumers)
),
"should provide proper error with only 1 consumer left registered, instead found {:?}", res
);
// 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 expected_70_available = "Failed to allocate additional 150 bytes for r0 with 10 bytes already allocated for this reservation - 70 bytes remain available for the total pool";
let res = r0.try_grow(150);
assert!(
matches!(
&res,
Err(DataFusionError::ResourcesExhausted(ref e)) if e.to_string().contains(expected_70_available)
),
"should find that the inner pool will still count all bytes for the deregistered consumer until the reservation is dropped, instead found {:?}", res
);
// Test: the registration needs to free itself (or be dropped),
// for the proper error message
r1.free();
let expected_90_available = "Failed to allocate additional 150 bytes for r0 with 10 bytes already allocated for this reservation - 90 bytes remain available for the total pool";
let res = r0.try_grow(150);
assert!(
matches!(
&res,
Err(DataFusionError::ResourcesExhausted(ref e)) if e.to_string().contains(expected_90_available)
),
"should correctly account the total bytes after reservation is free, instead found {:?}", res
);
}
let tracked_spill_pool: Arc<dyn MemoryPool> = Arc::new(TrackConsumersPool::new(
FairSpillPool::new(100),
NonZeroUsize::new(3).unwrap(),
));
test_per_pool_type(tracked_spill_pool);
let tracked_greedy_pool: Arc<dyn MemoryPool> = Arc::new(TrackConsumersPool::new(
GreedyMemoryPool::new(100),
NonZeroUsize::new(3).unwrap(),
));
test_per_pool_type(tracked_greedy_pool);
}
#[test]
fn test_tracked_consumers_pool_use_beyond_errors() {
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 mut r1 = MemoryConsumer::new("r1").register(&pool);
r1.grow(20);
// set r2=15
let mut r2 = MemoryConsumer::new("r2").register(&pool);
r2.grow(15);
// set r3=45
let mut 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 expected = "r3 consumed 45 bytes, r1 consumed 20 bytes";
let res = downcasted.report_top(2);
assert_eq!(
res, expected,
"should provide list of top memory consumers, instead found {:?}",
res
);
}
}