forked from apache/datafusion
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathmemory_pool_execution_plan.rs
More file actions
303 lines (253 loc) · 10.3 KB
/
memory_pool_execution_plan.rs
File metadata and controls
303 lines (253 loc) · 10.3 KB
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
// 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.
//! See `main.rs` for how to run it.
//!
//! This example demonstrates how to implement custom ExecutionPlans that properly
//! use memory tracking through TrackConsumersPool.
//!
//! This shows the pattern for implementing memory-aware operators that:
//! - Register memory consumers with the pool
//! - Reserve memory before allocating
//! - Handle memory pressure by spilling to disk
//! - Release memory when done
use arrow::record_batch::RecordBatch;
use arrow_schema::SchemaRef;
use datafusion::common::record_batch;
use datafusion::common::{exec_datafusion_err, internal_err};
use datafusion::datasource::{DefaultTableSource, memory::MemTable};
use datafusion::error::Result;
use datafusion::execution::memory_pool::{MemoryConsumer, MemoryReservation};
use datafusion::execution::runtime_env::RuntimeEnvBuilder;
use datafusion::execution::{SendableRecordBatchStream, TaskContext};
use datafusion::logical_expr::LogicalPlanBuilder;
use datafusion::physical_plan::stream::RecordBatchStreamAdapter;
use datafusion::physical_plan::{
DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, Statistics,
};
use datafusion::prelude::*;
use futures::stream::{StreamExt, TryStreamExt};
use std::any::Any;
use std::fmt;
use std::sync::Arc;
/// Shows how to implement memory-aware ExecutionPlan with memory reservation and spilling
pub async fn memory_pool_execution_plan() -> Result<()> {
println!("=== DataFusion ExecutionPlan Memory Tracking Example ===\n");
// Set up a runtime with memory tracking
// Set a low memory limit to trigger spilling on the second batch
let runtime = RuntimeEnvBuilder::new()
.with_memory_limit(15000, 1.0) // Allow only enough for 1 batch at once
.build_arc()?;
let config = SessionConfig::new().with_coalesce_batches(false);
let ctx = SessionContext::new_with_config_rt(config, runtime.clone());
// Create smaller batches to ensure we get multiple RecordBatches from the scan
// Make each batch smaller than the memory limit to force multiple batches
let batch1 = record_batch!(
("id", Int32, vec![1; 800]),
("name", Utf8, vec!["Alice"; 800])
)?;
let batch2 = record_batch!(
("id", Int32, vec![2; 800]),
("name", Utf8, vec!["Bob"; 800])
)?;
let batch3 = record_batch!(
("id", Int32, vec![3; 800]),
("name", Utf8, vec!["Charlie"; 800])
)?;
let batch4 = record_batch!(
("id", Int32, vec![4; 800]),
("name", Utf8, vec!["David"; 800])
)?;
let schema = batch1.schema();
// Create a single MemTable with all batches in one partition to preserve order but ensure streaming
let mem_table = Arc::new(MemTable::try_new(
Arc::clone(&schema),
vec![vec![batch1, batch2, batch3, batch4]], // Single partition with multiple batches
)?);
// Build logical plan with a single scan that will yield multiple batches
let table_source = Arc::new(DefaultTableSource::new(mem_table));
let logical_plan =
LogicalPlanBuilder::scan("multi_batch_table", table_source, None)?.build()?;
// Convert to physical plan
let physical_plan = ctx.state().create_physical_plan(&logical_plan).await?;
println!("Example: Custom Memory-Aware BufferingExecutionPlan");
println!("---------------------------------------------------");
// Wrap our input plan with our custom BufferingExecutionPlan
let buffering_plan = Arc::new(BufferingExecutionPlan::new(schema, physical_plan));
// Create a task context from our runtime
let task_ctx = Arc::new(TaskContext::default().with_runtime(runtime));
// Execute the plan directly to demonstrate memory tracking
println!("Executing BufferingExecutionPlan with memory tracking...");
println!("Memory limit: 15000 bytes - should trigger spill on later batches\n");
let stream = buffering_plan.execute(0, task_ctx.clone())?;
let _results: Vec<RecordBatch> = stream.try_collect().await?;
println!("\nSuccessfully executed BufferingExecutionPlan!");
println!("\nThe BufferingExecutionPlan processed 4 input batches and");
println!("demonstrated memory tracking with spilling behavior when the");
println!("memory limit was exceeded by later batches.");
println!("Check the console output above to see the spill messages.");
Ok(())
}
/// Example of an external batch bufferer that uses memory reservation.
///
/// It's a simple example which spills all existing data to disk
/// whenever the memory limit is reached.
struct ExternalBatchBufferer {
buffer: Vec<u8>,
reservation: MemoryReservation,
spill_count: usize,
}
impl ExternalBatchBufferer {
fn new(reservation: MemoryReservation) -> Self {
Self {
buffer: Vec::new(),
reservation,
spill_count: 0,
}
}
#[expect(clippy::needless_pass_by_value)]
fn add_batch(&mut self, batch_data: Vec<u8>) -> Result<()> {
let additional_memory = batch_data.len();
// Try to reserve memory before allocating
if self.reservation.try_grow(additional_memory).is_err() {
// Memory limit reached - handle by spilling
println!(
"Memory limit reached, spilling {} bytes to disk",
self.buffer.len()
);
self.spill_to_disk()?;
// Try again after spilling
self.reservation.try_grow(additional_memory)?;
}
self.buffer.extend_from_slice(&batch_data);
println!(
"Added batch of {} bytes, total buffered: {} bytes",
additional_memory,
self.buffer.len()
);
Ok(())
}
fn spill_to_disk(&mut self) -> Result<()> {
// Simulate writing buffer to disk
self.spill_count += 1;
println!(
"Spill #{}: Writing {} bytes to disk",
self.spill_count,
self.buffer.len()
);
// Free the memory after spilling
let freed_bytes = self.buffer.len();
self.buffer.clear();
self.reservation.shrink(freed_bytes);
Ok(())
}
fn finish(&mut self) -> Vec<u8> {
let result = std::mem::take(&mut self.buffer);
// Free the memory when done
self.reservation.free();
println!("Finished processing, released {} bytes", result.len());
result
}
}
/// Example of an ExecutionPlan that uses the ExternalBatchBufferer.
#[derive(Debug)]
struct BufferingExecutionPlan {
schema: SchemaRef,
input: Arc<dyn ExecutionPlan>,
properties: Arc<PlanProperties>,
}
impl BufferingExecutionPlan {
fn new(schema: SchemaRef, input: Arc<dyn ExecutionPlan>) -> Self {
let properties = input.properties().clone();
Self {
schema,
input,
properties,
}
}
}
impl DisplayAs for BufferingExecutionPlan {
fn fmt_as(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result {
write!(f, "BufferingExecutionPlan")
}
}
impl ExecutionPlan for BufferingExecutionPlan {
fn name(&self) -> &'static str {
"BufferingExecutionPlan"
}
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
fn properties(&self) -> &Arc<PlanProperties> {
&self.properties
}
fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
vec![&self.input]
}
fn with_new_children(
self: Arc<Self>,
children: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>> {
if children.len() == 1 {
Ok(Arc::new(BufferingExecutionPlan::new(
self.schema.clone(),
children[0].clone(),
)))
} else {
internal_err!("BufferingExecutionPlan must have exactly one child")
}
}
fn execute(
&self,
partition: usize,
context: Arc<TaskContext>,
) -> Result<SendableRecordBatchStream> {
// Register memory consumer with the context's memory pool
let reservation = MemoryConsumer::new("MyExternalBatchBufferer")
.with_can_spill(true)
.register(context.memory_pool());
// Incoming stream of batches
let mut input_stream = self.input.execute(partition, context)?;
// Process the stream and collect all batches
Ok(Box::pin(RecordBatchStreamAdapter::new(
self.schema(),
futures::stream::once(async move {
let mut operator = ExternalBatchBufferer::new(reservation);
while let Some(batch) = input_stream.next().await {
let batch = batch?;
// Convert RecordBatch to bytes for this example
let batch_data = vec![1u8; batch.get_array_memory_size()];
operator.add_batch(batch_data)?;
}
// Finish processing and get results
let _final_result = operator.finish();
// In a real implementation, you would convert final_result back to RecordBatches
// Since this is a simplified example, return an empty batch
// In a real implementation, you would create a batch stream from the processed results
record_batch!(("id", Int32, vec![5]), ("name", Utf8, vec!["Eve"]))
.map_err(|e| {
exec_datafusion_err!("Failed to create final RecordBatch: {e}")
})
}),
)))
}
fn statistics(&self) -> Result<Statistics> {
Ok(Statistics::new_unknown(&self.schema))
}
}