/Users/andrewlamb/Software/datafusion/datafusion/execution/src/memory_pool/mod.rs
Line | Count | Source (jump to first uncovered line) |
1 | | // Licensed to the Apache Software Foundation (ASF) under one |
2 | | // or more contributor license agreements. See the NOTICE file |
3 | | // distributed with this work for additional information |
4 | | // regarding copyright ownership. The ASF licenses this file |
5 | | // to you under the Apache License, Version 2.0 (the |
6 | | // "License"); you may not use this file except in compliance |
7 | | // with the License. You may obtain a copy of the License at |
8 | | // |
9 | | // http://www.apache.org/licenses/LICENSE-2.0 |
10 | | // |
11 | | // Unless required by applicable law or agreed to in writing, |
12 | | // software distributed under the License is distributed on an |
13 | | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
14 | | // KIND, either express or implied. See the License for the |
15 | | // specific language governing permissions and limitations |
16 | | // under the License. |
17 | | |
18 | | //! [`MemoryPool`] for memory management during query execution, [`proxy]` for |
19 | | //! help with allocation accounting. |
20 | | |
21 | | use datafusion_common::{internal_err, Result}; |
22 | | use std::{cmp::Ordering, sync::Arc}; |
23 | | |
24 | | mod pool; |
25 | | pub mod proxy { |
26 | | pub use datafusion_common::utils::proxy::{RawTableAllocExt, VecAllocExt}; |
27 | | } |
28 | | |
29 | | pub use pool::*; |
30 | | |
31 | | /// Tracks and potentially limits memory use across operators during execution. |
32 | | /// |
33 | | /// # Memory Management Overview |
34 | | /// |
35 | | /// DataFusion is a streaming query engine, processing most queries without |
36 | | /// buffering the entire input. Most operators require a fixed amount of memory |
37 | | /// based on the schema and target batch size. However, certain operations such |
38 | | /// as sorting and grouping/joining, require buffering intermediate results, |
39 | | /// which can require memory proportional to the number of input rows. |
40 | | /// |
41 | | /// Rather than tracking all allocations, DataFusion takes a pragmatic approach: |
42 | | /// Intermediate memory used as data streams through the system is not accounted |
43 | | /// (it assumed to be "small") but the large consumers of memory must register |
44 | | /// and constrain their use. This design trades off the additional code |
45 | | /// complexity of memory tracking with limiting resource usage. |
46 | | /// |
47 | | /// When limiting memory with a `MemoryPool` you should typically reserve some |
48 | | /// overhead (e.g. 10%) for the "small" memory allocations that are not tracked. |
49 | | /// |
50 | | /// # Memory Management Design |
51 | | /// |
52 | | /// As explained above, DataFusion's design ONLY limits operators that require |
53 | | /// "large" amounts of memory (proportional to number of input rows), such as |
54 | | /// `GroupByHashExec`. It does NOT track and limit memory used internally by |
55 | | /// other operators such as `ParquetExec` or the `RecordBatch`es that flow |
56 | | /// between operators. |
57 | | /// |
58 | | /// In order to avoid allocating memory until the OS or the container system |
59 | | /// kills the process, DataFusion `ExecutionPlan`s (operators) that consume |
60 | | /// large amounts of memory must first request their desired allocation from a |
61 | | /// [`MemoryPool`] before allocating more. The request is typically managed via |
62 | | /// a [`MemoryReservation`] and [`MemoryConsumer`]. |
63 | | /// |
64 | | /// If the allocation is successful, the operator should proceed and allocate |
65 | | /// the desired memory. If the allocation fails, the operator must either first |
66 | | /// free memory (e.g. by spilling to local disk) and try again, or error. |
67 | | /// |
68 | | /// Note that a `MemoryPool` can be shared by concurrently executing plans, |
69 | | /// which can be used to control memory usage in a multi-tenant system. |
70 | | /// |
71 | | /// # Implementing `MemoryPool` |
72 | | /// |
73 | | /// You can implement a custom allocation policy by implementing the |
74 | | /// [`MemoryPool`] trait and configuring a `SessionContext` appropriately. |
75 | | /// However, mDataFusion comes with the following simple memory pool implementations that |
76 | | /// handle many common cases: |
77 | | /// |
78 | | /// * [`UnboundedMemoryPool`]: no memory limits (the default) |
79 | | /// |
80 | | /// * [`GreedyMemoryPool`]: Limits memory usage to a fixed size using a "first |
81 | | /// come first served" policy |
82 | | /// |
83 | | /// * [`FairSpillPool`]: Limits memory usage to a fixed size, allocating memory |
84 | | /// to all spilling operators fairly |
85 | | pub trait MemoryPool: Send + Sync + std::fmt::Debug { |
86 | | /// Registers a new [`MemoryConsumer`] |
87 | | /// |
88 | | /// Note: Subsequent calls to [`Self::grow`] must be made to reserve memory |
89 | 8.86k | fn register(&self, _consumer: &MemoryConsumer) {} |
90 | | |
91 | | /// Records the destruction of a [`MemoryReservation`] with [`MemoryConsumer`] |
92 | | /// |
93 | | /// Note: Prior calls to [`Self::shrink`] must be made to free any reserved memory |
94 | 8.86k | fn unregister(&self, _consumer: &MemoryConsumer) {} |
95 | | |
96 | | /// Infallibly grow the provided `reservation` by `additional` bytes |
97 | | /// |
98 | | /// This must always succeed |
99 | | fn grow(&self, reservation: &MemoryReservation, additional: usize); |
100 | | |
101 | | /// Infallibly shrink the provided `reservation` by `shrink` bytes |
102 | | fn shrink(&self, reservation: &MemoryReservation, shrink: usize); |
103 | | |
104 | | /// Attempt to grow the provided `reservation` by `additional` bytes |
105 | | /// |
106 | | /// On error the `allocation` will not be increased in size |
107 | | fn try_grow(&self, reservation: &MemoryReservation, additional: usize) -> Result<()>; |
108 | | |
109 | | /// Return the total amount of memory reserved |
110 | | fn reserved(&self) -> usize; |
111 | | } |
112 | | |
113 | | /// A memory consumer is a named allocation traced by a particular |
114 | | /// [`MemoryReservation`] in a [`MemoryPool`]. All allocations are registered to |
115 | | /// a particular `MemoryConsumer`; |
116 | | /// |
117 | | /// For help with allocation accounting, see the [proxy] module. |
118 | | /// |
119 | | /// [proxy]: crate::memory_pool::proxy |
120 | | #[derive(Debug, PartialEq, Eq, Hash, Clone)] |
121 | | pub struct MemoryConsumer { |
122 | | name: String, |
123 | | can_spill: bool, |
124 | | } |
125 | | |
126 | | impl MemoryConsumer { |
127 | | /// Create a new empty [`MemoryConsumer`] that can be grown using [`MemoryReservation`] |
128 | 8.98k | pub fn new(name: impl Into<String>) -> Self { |
129 | 8.98k | Self { |
130 | 8.98k | name: name.into(), |
131 | 8.98k | can_spill: false, |
132 | 8.98k | } |
133 | 8.98k | } |
134 | | |
135 | | /// Set whether this allocation can be spilled to disk |
136 | 178 | pub fn with_can_spill(self, can_spill: bool) -> Self { |
137 | 178 | Self { can_spill, ..self } |
138 | 178 | } |
139 | | |
140 | | /// Returns true if this allocation can spill to disk |
141 | 89 | pub fn can_spill(&self) -> bool { |
142 | 89 | self.can_spill |
143 | 89 | } |
144 | | |
145 | | /// Returns the name associated with this allocation |
146 | 89 | pub fn name(&self) -> &str { |
147 | 89 | &self.name |
148 | 89 | } |
149 | | |
150 | | /// Registers this [`MemoryConsumer`] with the provided [`MemoryPool`] returning |
151 | | /// a [`MemoryReservation`] that can be used to grow or shrink the memory reservation |
152 | 8.90k | pub fn register(self, pool: &Arc<dyn MemoryPool>) -> MemoryReservation { |
153 | 8.90k | pool.register(&self); |
154 | 8.90k | MemoryReservation { |
155 | 8.90k | registration: Arc::new(SharedRegistration { |
156 | 8.90k | pool: Arc::clone(pool), |
157 | 8.90k | consumer: self, |
158 | 8.90k | }), |
159 | 8.90k | size: 0, |
160 | 8.90k | } |
161 | 8.90k | } |
162 | | } |
163 | | |
164 | | /// A registration of a [`MemoryConsumer`] with a [`MemoryPool`]. |
165 | | /// |
166 | | /// Calls [`MemoryPool::unregister`] on drop to return any memory to |
167 | | /// the underlying pool. |
168 | | #[derive(Debug)] |
169 | | struct SharedRegistration { |
170 | | pool: Arc<dyn MemoryPool>, |
171 | | consumer: MemoryConsumer, |
172 | | } |
173 | | |
174 | | impl Drop for SharedRegistration { |
175 | 8.90k | fn drop(&mut self) { |
176 | 8.90k | self.pool.unregister(&self.consumer); |
177 | 8.90k | } |
178 | | } |
179 | | |
180 | | /// A [`MemoryReservation`] tracks an individual reservation of a |
181 | | /// number of bytes of memory in a [`MemoryPool`] that is freed back |
182 | | /// to the pool on drop. |
183 | | /// |
184 | | /// The reservation can be grown or shrunk over time. |
185 | | #[derive(Debug)] |
186 | | pub struct MemoryReservation { |
187 | | registration: Arc<SharedRegistration>, |
188 | | size: usize, |
189 | | } |
190 | | |
191 | | impl MemoryReservation { |
192 | | /// Returns the size of this reservation in bytes |
193 | 916 | pub fn size(&self) -> usize { |
194 | 916 | self.size |
195 | 916 | } |
196 | | |
197 | | /// Returns [MemoryConsumer] for this [MemoryReservation] |
198 | 177 | pub fn consumer(&self) -> &MemoryConsumer { |
199 | 177 | &self.registration.consumer |
200 | 177 | } |
201 | | |
202 | | /// Frees all bytes from this reservation back to the underlying |
203 | | /// pool, returning the number of bytes freed. |
204 | 8.95k | pub fn free(&mut self) -> usize { |
205 | 8.95k | let size = self.size; |
206 | 8.95k | if size != 0 { |
207 | 3.20k | self.shrink(size) |
208 | 5.75k | } |
209 | 8.95k | size |
210 | 8.95k | } |
211 | | |
212 | | /// Frees `capacity` bytes from this reservation |
213 | | /// |
214 | | /// # Panics |
215 | | /// |
216 | | /// Panics if `capacity` exceeds [`Self::size`] |
217 | 22.2k | pub fn shrink(&mut self, capacity: usize) { |
218 | 22.2k | let new_size = self.size.checked_sub(capacity).unwrap(); |
219 | 22.2k | self.registration.pool.shrink(self, capacity); |
220 | 22.2k | self.size = new_size |
221 | 22.2k | } |
222 | | |
223 | | /// Tries to free `capacity` bytes from this reservation |
224 | | /// if `capacity` does not exceed [`Self::size`] |
225 | | /// Returns new reservation size |
226 | | /// or error if shrinking capacity is more than allocated size |
227 | 82 | pub fn try_shrink(&mut self, capacity: usize) -> Result<usize> { |
228 | 82 | if let Some(new_size) = self.size.checked_sub(capacity) { |
229 | 82 | self.registration.pool.shrink(self, capacity); |
230 | 82 | self.size = new_size; |
231 | 82 | Ok(new_size) |
232 | | } else { |
233 | 0 | internal_err!( |
234 | 0 | "Cannot free the capacity {capacity} out of allocated size {}", |
235 | 0 | self.size |
236 | 0 | ) |
237 | | } |
238 | 82 | } |
239 | | |
240 | | /// Sets the size of this reservation to `capacity` |
241 | 0 | pub fn resize(&mut self, capacity: usize) { |
242 | 0 | match capacity.cmp(&self.size) { |
243 | 0 | Ordering::Greater => self.grow(capacity - self.size), |
244 | 0 | Ordering::Less => self.shrink(self.size - capacity), |
245 | 0 | _ => {} |
246 | | } |
247 | 0 | } |
248 | | |
249 | | /// Try to set the size of this reservation to `capacity` |
250 | 8.56k | pub fn try_resize(&mut self, capacity: usize) -> Result<()> { |
251 | 8.56k | match capacity.cmp(&self.size) { |
252 | 4.79k | Ordering::Greater => self.try_grow(capacity - self.size)?105 , |
253 | 998 | Ordering::Less => self.shrink(self.size - capacity), |
254 | 2.77k | _ => {} |
255 | | }; |
256 | 8.45k | Ok(()) |
257 | 8.56k | } |
258 | | |
259 | | /// Increase the size of this reservation by `capacity` bytes |
260 | 0 | pub fn grow(&mut self, capacity: usize) { |
261 | 0 | self.registration.pool.grow(self, capacity); |
262 | 0 | self.size += capacity; |
263 | 0 | } |
264 | | |
265 | | /// Try to increase the size of this reservation by `capacity` |
266 | | /// bytes, returning error if there is insufficient capacity left |
267 | | /// in the pool. |
268 | 43.9k | pub fn try_grow(&mut self, capacity: usize) -> Result<()> { |
269 | 43.9k | self.registration.pool.try_grow(self, capacity)?185 ; |
270 | 43.7k | self.size += capacity; |
271 | 43.7k | Ok(()) |
272 | 43.9k | } |
273 | | |
274 | | /// Splits off `capacity` bytes from this [`MemoryReservation`] |
275 | | /// into a new [`MemoryReservation`] with the same |
276 | | /// [`MemoryConsumer`]. |
277 | | /// |
278 | | /// This can be useful to free part of this reservation with RAAI |
279 | | /// style dropping |
280 | | /// |
281 | | /// # Panics |
282 | | /// |
283 | | /// Panics if `capacity` exceeds [`Self::size`] |
284 | 23 | pub fn split(&mut self, capacity: usize) -> MemoryReservation { |
285 | 23 | self.size = self.size.checked_sub(capacity).unwrap(); |
286 | 23 | Self { |
287 | 23 | size: capacity, |
288 | 23 | registration: Arc::clone(&self.registration), |
289 | 23 | } |
290 | 23 | } |
291 | | |
292 | | /// Returns a new empty [`MemoryReservation`] with the same [`MemoryConsumer`] |
293 | 22 | pub fn new_empty(&self) -> Self { |
294 | 22 | Self { |
295 | 22 | size: 0, |
296 | 22 | registration: Arc::clone(&self.registration), |
297 | 22 | } |
298 | 22 | } |
299 | | |
300 | | /// Splits off all the bytes from this [`MemoryReservation`] into |
301 | | /// a new [`MemoryReservation`] with the same [`MemoryConsumer`] |
302 | 23 | pub fn take(&mut self) -> MemoryReservation { |
303 | 23 | self.split(self.size) |
304 | 23 | } |
305 | | } |
306 | | |
307 | | impl Drop for MemoryReservation { |
308 | 8.94k | fn drop(&mut self) { |
309 | 8.94k | self.free(); |
310 | 8.94k | } |
311 | | } |
312 | | |
313 | | const TB: u64 = 1 << 40; |
314 | | const GB: u64 = 1 << 30; |
315 | | const MB: u64 = 1 << 20; |
316 | | const KB: u64 = 1 << 10; |
317 | | |
318 | | /// Present size in human readable form |
319 | 0 | pub fn human_readable_size(size: usize) -> String { |
320 | 0 | let size = size as u64; |
321 | 0 | let (value, unit) = { |
322 | 0 | if size >= 2 * TB { |
323 | 0 | (size as f64 / TB as f64, "TB") |
324 | 0 | } else if size >= 2 * GB { |
325 | 0 | (size as f64 / GB as f64, "GB") |
326 | 0 | } else if size >= 2 * MB { |
327 | 0 | (size as f64 / MB as f64, "MB") |
328 | 0 | } else if size >= 2 * KB { |
329 | 0 | (size as f64 / KB as f64, "KB") |
330 | | } else { |
331 | 0 | (size as f64, "B") |
332 | | } |
333 | | }; |
334 | 0 | format!("{value:.1} {unit}") |
335 | 0 | } |
336 | | |
337 | | #[cfg(test)] |
338 | | mod tests { |
339 | | use super::*; |
340 | | |
341 | | #[test] |
342 | | fn test_memory_pool_underflow() { |
343 | | let pool = Arc::new(GreedyMemoryPool::new(50)) as _; |
344 | | let mut a1 = MemoryConsumer::new("a1").register(&pool); |
345 | | assert_eq!(pool.reserved(), 0); |
346 | | |
347 | | a1.grow(100); |
348 | | assert_eq!(pool.reserved(), 100); |
349 | | |
350 | | assert_eq!(a1.free(), 100); |
351 | | assert_eq!(pool.reserved(), 0); |
352 | | |
353 | | a1.try_grow(100).unwrap_err(); |
354 | | assert_eq!(pool.reserved(), 0); |
355 | | |
356 | | a1.try_grow(30).unwrap(); |
357 | | assert_eq!(pool.reserved(), 30); |
358 | | |
359 | | let mut a2 = MemoryConsumer::new("a2").register(&pool); |
360 | | a2.try_grow(25).unwrap_err(); |
361 | | assert_eq!(pool.reserved(), 30); |
362 | | |
363 | | drop(a1); |
364 | | assert_eq!(pool.reserved(), 0); |
365 | | |
366 | | a2.try_grow(25).unwrap(); |
367 | | assert_eq!(pool.reserved(), 25); |
368 | | } |
369 | | |
370 | | #[test] |
371 | | fn test_split() { |
372 | | let pool = Arc::new(GreedyMemoryPool::new(50)) as _; |
373 | | let mut r1 = MemoryConsumer::new("r1").register(&pool); |
374 | | |
375 | | r1.try_grow(20).unwrap(); |
376 | | assert_eq!(r1.size(), 20); |
377 | | assert_eq!(pool.reserved(), 20); |
378 | | |
379 | | // take 5 from r1, should still have same reservation split |
380 | | let r2 = r1.split(5); |
381 | | assert_eq!(r1.size(), 15); |
382 | | assert_eq!(r2.size(), 5); |
383 | | assert_eq!(pool.reserved(), 20); |
384 | | |
385 | | // dropping r1 frees 15 but retains 5 as they have the same consumer |
386 | | drop(r1); |
387 | | assert_eq!(r2.size(), 5); |
388 | | assert_eq!(pool.reserved(), 5); |
389 | | } |
390 | | |
391 | | #[test] |
392 | | fn test_new_empty() { |
393 | | let pool = Arc::new(GreedyMemoryPool::new(50)) as _; |
394 | | let mut r1 = MemoryConsumer::new("r1").register(&pool); |
395 | | |
396 | | r1.try_grow(20).unwrap(); |
397 | | let mut r2 = r1.new_empty(); |
398 | | r2.try_grow(5).unwrap(); |
399 | | |
400 | | assert_eq!(r1.size(), 20); |
401 | | assert_eq!(r2.size(), 5); |
402 | | assert_eq!(pool.reserved(), 25); |
403 | | } |
404 | | |
405 | | #[test] |
406 | | fn test_take() { |
407 | | let pool = Arc::new(GreedyMemoryPool::new(50)) as _; |
408 | | let mut r1 = MemoryConsumer::new("r1").register(&pool); |
409 | | |
410 | | r1.try_grow(20).unwrap(); |
411 | | let mut r2 = r1.take(); |
412 | | r2.try_grow(5).unwrap(); |
413 | | |
414 | | assert_eq!(r1.size(), 0); |
415 | | assert_eq!(r2.size(), 25); |
416 | | assert_eq!(pool.reserved(), 25); |
417 | | |
418 | | // r1 can still grow again |
419 | | r1.try_grow(3).unwrap(); |
420 | | assert_eq!(r1.size(), 3); |
421 | | assert_eq!(r2.size(), 25); |
422 | | assert_eq!(pool.reserved(), 28); |
423 | | } |
424 | | } |