Skip to content

Commit 5a510cc

Browse files
committed
replace err with warn log
1 parent af0aac6 commit 5a510cc

File tree

4 files changed

+18
-21
lines changed

4 files changed

+18
-21
lines changed

datafusion/core/tests/sql/runtime_config.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -193,7 +193,7 @@ async fn test_max_temp_directory_size_enforcement() {
193193
.unwrap();
194194

195195
let result = ctx.sql(query).await.unwrap().collect().await;
196-
println!("result is {result:?}");
196+
197197
assert!(
198198
result.is_ok(),
199199
"Should not fail due to max temp directory size limit"

datafusion/execution/src/memory_pool/pool.rs

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -44,7 +44,6 @@ impl MemoryPool for UnboundedMemoryPool {
4444

4545
fn try_grow(&self, reservation: &MemoryReservation, additional: usize) -> Result<()> {
4646
self.grow(reservation, additional);
47-
println!("[mem pool] {} used", self.reserved());
4847
Ok(())
4948
}
5049

@@ -71,8 +70,7 @@ pub struct GreedyMemoryPool {
7170
impl GreedyMemoryPool {
7271
/// Create a new pool that can allocate up to `pool_size` bytes
7372
pub fn new(pool_size: usize) -> Self {
74-
// debug!("Created new GreedyMemoryPool(pool_size={pool_size})");
75-
println!("Created new GreedyMemoryPool(pool_size={pool_size})");
73+
debug!("Created new GreedyMemoryPool(pool_size={pool_size})");
7674
Self {
7775
pool_size,
7876
used: AtomicUsize::new(0),

datafusion/physical-plan/src/sorts/streaming_merge.rs

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -29,9 +29,8 @@ use arrow::array::*;
2929
use arrow::datatypes::{DataType, SchemaRef};
3030
use datafusion_common::{internal_err, Result};
3131
use datafusion_execution::disk_manager::RefCountedTempFile;
32-
#[allow(unused_imports)]
3332
use datafusion_execution::memory_pool::{
34-
human_readable_size, GreedyMemoryPool, MemoryConsumer, MemoryPool, MemoryReservation,
33+
human_readable_size, MemoryConsumer, MemoryPool, MemoryReservation,
3534
UnboundedMemoryPool,
3635
};
3736
use datafusion_physical_expr_common::sort_expr::LexOrdering;

datafusion/physical-plan/src/spill/mod.rs

Lines changed: 15 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -43,6 +43,7 @@ use datafusion_common_runtime::SpawnedTask;
4343
use datafusion_execution::disk_manager::RefCountedTempFile;
4444
use datafusion_execution::RecordBatchStream;
4545
use futures::{FutureExt as _, Stream};
46+
use log::warn;
4647

4748
/// Stream that reads spill files from disk where each batch is read in a spawned blocking task
4849
/// It will read one batch at a time and will not do any buffering, to buffer data use [`crate::common::spawn_buffered`]
@@ -54,12 +55,15 @@ use futures::{FutureExt as _, Stream};
5455
struct SpillReaderStream {
5556
schema: SchemaRef,
5657
state: SpillReaderStreamState,
57-
/// how much memory the largest memory batch is taking
58-
pub max_record_batch_memory: Option<usize>,
58+
/// Maximum memory size observed among spilling sorted record batches.
59+
/// This is used for validation purposes during reading each RecordBatch from spill.
60+
/// For context on why this value is recorded and validated,
61+
/// see `physical_plan/sort/multi_level_merge.rs`.
62+
max_record_batch_memory: Option<usize>,
5963
}
6064

6165
// Small margin allowed to accommodate slight memory accounting variation
62-
const MEMORY_MARGIN: usize = 4096;
66+
const SPILL_BATCH_MEMORY_MARGIN: usize = 4096;
6367

6468
/// When we poll for the next batch, we will get back both the batch and the reader,
6569
/// so we can call `next` again.
@@ -141,19 +145,15 @@ impl SpillReaderStream {
141145
let actual_size =
142146
get_record_batch_memory_size(&batch);
143147
if actual_size
144-
> max_record_batch_memory + MEMORY_MARGIN
148+
> max_record_batch_memory
149+
+ SPILL_BATCH_MEMORY_MARGIN
145150
{
146-
return Poll::Ready(Some(Err(
147-
DataFusionError::ResourcesExhausted(
148-
format!(
149-
"Record batch memory usage ({actual_size} bytes) exceeds the expected limit ({max_record_batch_memory} bytes)\n
150-
by more than the allowed tolerance ({MEMORY_MARGIN} bytes).\n
151-
This likely indicates a bug in memory accounting during spilling.\n
152-
Please report this issue",
153-
)
154-
.to_owned(),
155-
),
156-
)));
151+
warn!(
152+
"Record batch memory usage ({actual_size} bytes) exceeds the expected limit ({max_record_batch_memory} bytes) \n\
153+
by more than the allowed tolerance ({SPILL_BATCH_MEMORY_MARGIN} bytes).\n\
154+
This likely indicates a bug in memory accounting during spilling.\n\
155+
Please report this issue in https://github.com/apache/datafusion/issues/17340."
156+
);
157157
}
158158
}
159159
self.state = SpillReaderStreamState::Waiting(reader);

0 commit comments

Comments
 (0)