From 17b209ba1cdd9ade5e3602b392486592dafe0be5 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 8 Dec 2025 09:41:24 +0800 Subject: [PATCH 01/37] Fix spill read profile classification --- src/query/service/src/spillers/inner.rs | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/query/service/src/spillers/inner.rs b/src/query/service/src/spillers/inner.rs index 4fd22b727be09..e9abe06cb9360 100644 --- a/src/query/service/src/spillers/inner.rs +++ b/src/query/service/src/spillers/inner.rs @@ -295,17 +295,17 @@ pub(super) fn record_write_profile(is_local: bool, start: &Instant, write_bytes: pub(super) fn record_read_profile(is_local: bool, start: &Instant, read_bytes: usize) { if is_local { - Profile::record_usize_profile(ProfileStatisticsName::RemoteSpillReadCount, 1); - Profile::record_usize_profile(ProfileStatisticsName::RemoteSpillReadBytes, read_bytes); + Profile::record_usize_profile(ProfileStatisticsName::LocalSpillReadCount, 1); + Profile::record_usize_profile(ProfileStatisticsName::LocalSpillReadBytes, read_bytes); Profile::record_usize_profile( - ProfileStatisticsName::RemoteSpillReadTime, + ProfileStatisticsName::LocalSpillReadTime, start.elapsed().as_millis() as usize, ); } else { - Profile::record_usize_profile(ProfileStatisticsName::LocalSpillReadCount, 1); - Profile::record_usize_profile(ProfileStatisticsName::LocalSpillReadBytes, read_bytes); + Profile::record_usize_profile(ProfileStatisticsName::RemoteSpillReadCount, 1); + Profile::record_usize_profile(ProfileStatisticsName::RemoteSpillReadBytes, read_bytes); Profile::record_usize_profile( - ProfileStatisticsName::LocalSpillReadTime, + ProfileStatisticsName::RemoteSpillReadTime, start.elapsed().as_millis() as usize, ); } From ef9efb64463521117f8ac91cedb2f06029797bdd Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 8 Dec 2025 10:09:52 +0800 Subject: [PATCH 02/37] Unify spill I/O profiling by actual location --- .../new_aggregate/new_aggregate_spiller.rs | 75 ++++++++++++------ .../new_hash_join/grace/grace_join.rs | 32 ++++++-- src/query/service/src/spillers/adapter.rs | 12 +-- .../service/src/spillers/async_buffer.rs | 78 ++++++++++++++----- src/query/service/src/spillers/inner.rs | 24 +++++- .../service/src/spillers/row_group_encoder.rs | 3 +- 6 files changed, 165 insertions(+), 59 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/new_aggregate/new_aggregate_spiller.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/new_aggregate/new_aggregate_spiller.rs index 6b7d63e7d90da..710bd1cda64bd 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/new_aggregate/new_aggregate_spiller.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/new_aggregate/new_aggregate_spiller.rs @@ -26,8 +26,9 @@ use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::BlockPartitionStream; use databend_common_expression::DataBlock; -use databend_common_pipeline_transforms::traits::Location; use databend_common_pipeline_transforms::MemorySettings; +use databend_common_meta_app::storage::StorageParams; +use databend_common_pipeline_transforms::traits::Location; use databend_common_storage::DataOperator; use databend_common_storages_parquet::ReadSettings; use log::debug; @@ -35,6 +36,7 @@ use log::info; use parking_lot::Mutex; use parquet::file::metadata::RowGroupMetaData; +use crate::spillers::async_buffer::SpillTarget; use crate::pipelines::memory_settings::MemorySettingsExt; use crate::pipelines::processors::transforms::aggregator::AggregateMeta; use crate::pipelines::processors::transforms::aggregator::NewSpilledPayload; @@ -44,6 +46,7 @@ use crate::spillers::Layout; use crate::spillers::SpillAdapter; use crate::spillers::SpillsBufferPool; use crate::spillers::SpillsDataWriter; +use crate::spillers::record_read_profile_with_flag; struct PayloadWriter { path: String, @@ -52,10 +55,16 @@ struct PayloadWriter { impl PayloadWriter { fn try_create(prefix: &str) -> Result { - let operator = DataOperator::instance().spill_operator(); + let data_operator = DataOperator::instance(); + let target = if matches!(data_operator.spill_params(), Some(StorageParams::Fs(_))) { + SpillTarget::Local + } else { + SpillTarget::Remote + }; + let operator = data_operator.spill_operator(); let buffer_pool = SpillsBufferPool::instance(); let file_path = format!("{}/{}", prefix, GlobalUniqName::unique()); - let spills_data_writer = buffer_pool.writer(operator, file_path.clone())?; + let spills_data_writer = buffer_pool.writer(operator, file_path.clone(), target)?; Ok(PayloadWriter { path: file_path, @@ -218,7 +227,7 @@ impl AggregatePayloadWriters { } let stats = self.write_stats.take(); - flush_write_profile(&self.ctx, stats); + flush_write_profile(&self.ctx, stats, self.is_local); Ok(spilled_payloads) } @@ -356,18 +365,24 @@ impl NewAggregateSpiller { row_group, } = payload; - let operator = DataOperator::instance().spill_operator(); + let data_operator = DataOperator::instance(); + let target = if matches!(data_operator.spill_params(), Some(StorageParams::Fs(_))) { + SpillTarget::Local + } else { + SpillTarget::Remote + }; + let operator = data_operator.spill_operator(); let buffer_pool = SpillsBufferPool::instance(); let mut reader = - buffer_pool.reader(operator.clone(), location.clone(), vec![row_group.clone()])?; + buffer_pool.reader(operator.clone(), location.clone(), vec![row_group.clone()], target)?; let instant = Instant::now(); let data_block = reader.read(self.read_setting)?; let elapsed = instant.elapsed(); let read_size = reader.read_bytes(); - flush_read_profile(&elapsed, read_size); + record_read_profile_with_flag(target.is_local(), &instant, read_size); info!( "Read aggregate spill finished: (bucket: {}, location: {}, bytes: {}, rows: {}, elapsed: {:?})", @@ -400,29 +415,39 @@ impl NewAggregateSpiller { } } -fn flush_read_profile(elapsed: &Duration, read_bytes: usize) { - Profile::record_usize_profile(ProfileStatisticsName::RemoteSpillReadCount, 1); - Profile::record_usize_profile(ProfileStatisticsName::RemoteSpillReadBytes, read_bytes); - Profile::record_usize_profile( - ProfileStatisticsName::RemoteSpillReadTime, - elapsed.as_millis() as usize, - ); -} - -fn flush_write_profile(ctx: &Arc, stats: WriteStats) { +fn flush_write_profile(ctx: &Arc, stats: WriteStats, is_local: bool) { if stats.count == 0 && stats.bytes == 0 && stats.rows == 0 { return; } if stats.count > 0 { - Profile::record_usize_profile(ProfileStatisticsName::RemoteSpillWriteCount, stats.count); - Profile::record_usize_profile( - ProfileStatisticsName::RemoteSpillWriteTime, - stats.elapsed.as_millis() as usize, - ); - } - if stats.bytes > 0 { - Profile::record_usize_profile(ProfileStatisticsName::RemoteSpillWriteBytes, stats.bytes); + if is_local { + Profile::record_usize_profile( + ProfileStatisticsName::LocalSpillWriteCount, + stats.count, + ); + Profile::record_usize_profile( + ProfileStatisticsName::LocalSpillWriteTime, + stats.elapsed.as_millis() as usize, + ); + Profile::record_usize_profile( + ProfileStatisticsName::LocalSpillWriteBytes, + stats.bytes, + ); + } else { + Profile::record_usize_profile( + ProfileStatisticsName::RemoteSpillWriteCount, + stats.count, + ); + Profile::record_usize_profile( + ProfileStatisticsName::RemoteSpillWriteTime, + stats.elapsed.as_millis() as usize, + ); + Profile::record_usize_profile( + ProfileStatisticsName::RemoteSpillWriteBytes, + stats.bytes, + ); + } } if stats.rows > 0 || stats.bytes > 0 { diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/grace/grace_join.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/grace/grace_join.rs index 7663631cf321c..f276f9e0b5687 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/grace/grace_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/grace/grace_join.rs @@ -25,6 +25,7 @@ use databend_common_expression::BlockPartitionStream; use databend_common_expression::DataBlock; use databend_common_expression::FunctionContext; use databend_common_expression::HashMethodKind; +use databend_common_meta_app::storage::StorageParams; use databend_common_pipeline_transforms::traits::Location; use databend_common_storage::DataOperator; use databend_common_storages_parquet::ReadSettings; @@ -39,6 +40,7 @@ use crate::pipelines::processors::transforms::Join; use crate::pipelines::processors::HashJoinDesc; use crate::sessions::QueryContext; use crate::spillers::Layout; +use crate::spillers::async_buffer::SpillTarget; use crate::spillers::SpillAdapter; use crate::spillers::SpillsBufferPool; use crate::spillers::SpillsDataReader; @@ -255,11 +257,18 @@ impl GraceHashJoin { } fn restore_build_data(&mut self) -> Result<()> { - let operator = DataOperator::instance().spill_operator(); + let data_operator = DataOperator::instance(); + let target = if matches!(data_operator.spill_params(), Some(StorageParams::Fs(_))) { + SpillTarget::Local + } else { + SpillTarget::Remote + }; + let operator = data_operator.spill_operator(); while let Some(data) = self.steal_restore_build_task() { let buffer_pool = SpillsBufferPool::instance(); - let mut reader = buffer_pool.reader(operator.clone(), data.path, data.row_groups)?; + let mut reader = + buffer_pool.reader(operator.clone(), data.path, data.row_groups, target)?; while let Some(data_block) = reader.read(self.read_settings)? { self.memory_hash_join.add_block(Some(data_block))?; @@ -384,11 +393,16 @@ pub struct GraceJoinPartition { impl GraceJoinPartition { pub fn create(prefix: &str) -> Result { let data_operator = DataOperator::instance(); + let target = if matches!(data_operator.spill_params(), Some(StorageParams::Fs(_))) { + SpillTarget::Local + } else { + SpillTarget::Remote + }; let operator = data_operator.spill_operator(); let buffer_pool = SpillsBufferPool::instance(); let file_path = format!("{}/{}", prefix, GlobalUniqName::unique()); - let spills_data_writer = buffer_pool.writer(operator, file_path.clone())?; + let spills_data_writer = buffer_pool.writer(operator, file_path.clone(), target)?; Ok(GraceJoinPartition { path: file_path, @@ -456,9 +470,17 @@ impl<'a, T: GraceMemoryJoin> RestoreProbeStream<'a, T> { continue; } - let operator = DataOperator::instance().spill_operator(); + let data_operator = DataOperator::instance(); + let target = + if matches!(data_operator.spill_params(), Some(StorageParams::Fs(_))) { + SpillTarget::Local + } else { + SpillTarget::Remote + }; + let operator = data_operator.spill_operator(); let buffer_pool = SpillsBufferPool::instance(); - let reader = buffer_pool.reader(operator, data.path, data.row_groups)?; + let reader = + buffer_pool.reader(operator, data.path, data.row_groups, target)?; self.spills_reader = Some(reader); break; } diff --git a/src/query/service/src/spillers/adapter.rs b/src/query/service/src/spillers/adapter.rs index 83cb97a22a2d9..69aa8879e27fe 100644 --- a/src/query/service/src/spillers/adapter.rs +++ b/src/query/service/src/spillers/adapter.rs @@ -232,7 +232,7 @@ impl Spiller { let instant = Instant::now(); let location = self.write_encodes(write_bytes, buf).await?; // Record statistics. - record_write_profile(location.is_local(), &instant, write_bytes); + record_write_profile(location, &instant, write_bytes); self.adapter .add_spill_file(location.clone(), layout, write_bytes); @@ -276,7 +276,7 @@ impl Spiller { }; // Record statistics. - record_read_profile(location.is_local(), &instant, data.len()); + record_read_profile(location, &instant, data.len()); // Deserialize partitioned data block. let mut partitioned_data = Vec::with_capacity(partitions.len()); @@ -310,7 +310,7 @@ impl Spiller { Location::Remote(loc) => self.operator.read_with(loc).range(data_range).await?, }; - record_read_profile(location.is_local(), &instant, data.len()); + record_read_profile(location, &instant, data.len()); deserialize_block(layout, data) } @@ -489,14 +489,14 @@ impl SpillWriter { let row_group_meta = file_writer.flush_row_group(row_group)?; let size = row_group_meta.compressed_size() as _; self.spiller.adapter.update_progress(0, size); - record_write_profile(true, &start, size); + record_write_profile(location, &start, size); Ok(row_group_meta) } AnyFileWriter::Remote(_, file_writer) => { let row_group_meta = file_writer.flush_row_group(row_group)?; let size = row_group_meta.compressed_size() as _; self.spiller.adapter.update_progress(0, size); - record_write_profile(false, &start, size); + record_write_profile(location, &start, size); Ok(row_group_meta) } } @@ -563,7 +563,7 @@ impl SpillReader { )?; record_read_profile( - self.location.is_local(), + &self.location, &start, blocks.iter().map(DataBlock::memory_size).sum(), ); diff --git a/src/query/service/src/spillers/async_buffer.rs b/src/query/service/src/spillers/async_buffer.rs index 7b40b45b9f9da..5ed8299923504 100644 --- a/src/query/service/src/spillers/async_buffer.rs +++ b/src/query/service/src/spillers/async_buffer.rs @@ -21,6 +21,7 @@ use std::sync::Arc; use std::sync::Condvar; use std::sync::Mutex; use std::sync::PoisonError; +use std::time::Instant; use arrow_schema::Schema; use bytes::Bytes; @@ -55,9 +56,23 @@ use parquet::basic::Compression; use parquet::file::metadata::RowGroupMetaData; use parquet::file::properties::EnabledStatistics; use parquet::file::properties::WriterProperties; - +use super::record_read_profile_with_flag; +use super::record_write_profile_with_flag; +use super::SpillsBufferPool; const CHUNK_SIZE: usize = 4 * 1024 * 1024; +#[derive(Clone, Copy)] +pub enum SpillTarget { + Local, + Remote, +} + +impl SpillTarget { + fn is_local(&self) -> bool { + matches!(self, SpillTarget::Local) + } +} + /// Buffer Pool Workflow for Spill Operations: /// /// Context: During query execution, when memory pressure is high, intermediate @@ -182,12 +197,21 @@ impl SpillsBufferPool { .expect("Buffer pool working queue need unbounded."); } - pub fn buffer_write(self: &Arc, writer: Writer) -> BufferWriter { - BufferWriter::new(writer, self.clone()) + pub fn buffer_write( + self: &Arc, + writer: Writer, + target: SpillTarget, + ) -> BufferWriter { + BufferWriter::new(writer, self.clone(), target) } - pub fn writer(self: &Arc, op: Operator, path: String) -> Result { - let writer = self.buffer_writer(op, path)?; + pub fn writer( + self: &Arc, + op: Operator, + path: String, + target: SpillTarget, + ) -> Result { + let writer = self.buffer_writer(op, path, target)?; Ok(SpillsDataWriter::Uninitialize(Some(writer))) } @@ -195,6 +219,7 @@ impl SpillsBufferPool { self: &Arc, op: Operator, path: String, + target: SpillTarget, ) -> Result { let pending_response = BufferOperatorResp::pending(); @@ -209,7 +234,7 @@ impl SpillsBufferPool { .try_send(operator) .expect("Buffer pool working queue need unbounded."); - Ok(self.buffer_write(pending_response.wait_and_take()?)) + Ok(self.buffer_write(pending_response.wait_and_take()?, target)) } pub fn reader( @@ -217,8 +242,9 @@ impl SpillsBufferPool { op: Operator, path: String, row_groups: Vec, + target: SpillTarget, ) -> Result { - SpillsDataReader::create(path, op, row_groups, self.clone()) + SpillsDataReader::create(path, op, row_groups, self.clone(), target) } pub fn fetch_ranges( @@ -257,16 +283,18 @@ pub struct BufferWriter { buffer_pool: Arc, pending_buffers: VecDeque, pending_response: Option>>, + target: SpillTarget, } impl BufferWriter { - pub fn new(writer: Writer, buffer_pool: Arc) -> BufferWriter { + pub fn new(writer: Writer, buffer_pool: Arc, target: SpillTarget) -> BufferWriter { BufferWriter { buffer_pool, writer: Some(writer), current_bytes: None, pending_buffers: VecDeque::new(), pending_response: None, + target, } } @@ -307,6 +335,8 @@ impl BufferWriter { writer, response: pending_response, buffers: std::mem::take(&mut self.pending_buffers), + target: self.target, + start: Instant::now(), }); self.buffer_pool.operator(operator); @@ -534,6 +564,7 @@ pub struct SpillsDataReader { data_schema: DataSchemaRef, field_levels: FieldLevels, read_bytes: usize, + target: SpillTarget, } impl SpillsDataReader { @@ -542,6 +573,7 @@ impl SpillsDataReader { operator: Operator, row_groups: Vec, spills_buffer_pool: Arc, + target: SpillTarget, ) -> Result { if row_groups.is_empty() { return Err(ErrorCode::Internal( @@ -566,6 +598,7 @@ impl SpillsDataReader { field_levels, row_groups: VecDeque::from(row_groups), read_bytes: 0, + target, }) } @@ -578,6 +611,8 @@ impl SpillsDataReader { return Ok(None); }; + let start = Instant::now(); + let mut row_group = RowGroupCore::new(row_group, None); let read_bytes = Cell::new(0usize); @@ -606,6 +641,7 @@ impl SpillsDataReader { )?; let batch = reader.next().transpose()?.unwrap(); debug_assert!(reader.next().is_none()); + record_read_profile_with_flag(self.target.is_local(), &start, read_bytes.get()); Ok(Some(DataBlock::from_record_batch( &self.data_schema, &batch, @@ -623,6 +659,8 @@ pub struct BufferWriteOperator { writer: Writer, buffers: VecDeque, response: Arc>, + target: SpillTarget, + start: Instant, } pub struct BufferCloseResp { @@ -714,7 +752,9 @@ impl Background { pub async fn recv(&mut self, op: BufferOperator) { match op { BufferOperator::Write(mut op) => { + let start = op.start; let bytes = op.buffers.clone(); + let bytes_len = bytes.iter().map(|b| b.len()).sum(); let mut error = op .writer .write(op.buffers) @@ -745,6 +785,8 @@ impl Background { error, writer: op.writer, }); + + record_write_profile_with_flag(op.target.is_local(), &start, bytes_len); } BufferOperator::Close(mut op) => { let res = op.writer.close().await; @@ -811,7 +853,7 @@ mod tests { let operator = create_test_operator().unwrap(); let writer = operator.writer("test_file").await.unwrap(); - let mut buffer_writer = BufferWriter::new(writer, pool); + let mut buffer_writer = BufferWriter::new(writer, pool, SpillTarget::Remote); let data = b"Hello, World!"; let written = buffer_writer.write(data).unwrap(); @@ -829,7 +871,7 @@ mod tests { let operator = create_test_operator().unwrap(); let writer = operator.writer("large_file").await.unwrap(); - let mut buffer_writer = BufferWriter::new(writer, pool); + let mut buffer_writer = BufferWriter::new(writer, pool, SpillTarget::Remote); // Write data larger than single buffer let large_data = vec![0u8; 8 * 1024 * 1024]; // 8MB @@ -848,7 +890,7 @@ mod tests { let operator = create_test_operator().unwrap(); let writer = operator.writer("multi_write_file").await.unwrap(); - let mut buffer_writer = BufferWriter::new(writer, pool); + let mut buffer_writer = BufferWriter::new(writer, pool, SpillTarget::Remote); let mut total_written = 0; for i in 0..100 { @@ -870,7 +912,7 @@ mod tests { let operator = create_test_operator().unwrap(); let writer = operator.writer("backpressure_test").await.unwrap(); - let mut buffer_writer = BufferWriter::new(writer, pool.clone()); + let mut buffer_writer = BufferWriter::new(writer, pool.clone(), SpillTarget::Remote); // Fill the first buffer let data = vec![0u8; CHUNK_SIZE]; @@ -923,7 +965,7 @@ mod tests { let operator = create_test_operator().unwrap(); let writer = operator.writer("empty_test").await.unwrap(); - let mut buffer_writer = BufferWriter::new(writer, pool); + let mut buffer_writer = BufferWriter::new(writer, pool, SpillTarget::Remote); let written = buffer_writer.write(b"").unwrap(); assert_eq!(written, 0); @@ -940,7 +982,7 @@ mod tests { let operator = create_test_operator().unwrap(); let writer = operator.writer("no_write_test").await.unwrap(); - let buffer_writer = BufferWriter::new(writer, pool); + let buffer_writer = BufferWriter::new(writer, pool, SpillTarget::Remote); // Should be able to close without any writes let metadata = buffer_writer.close().unwrap(); @@ -966,7 +1008,7 @@ mod tests { .writer(&format!("concurrent_{}", i)) .await .unwrap(); - let mut buffer_writer = BufferWriter::new(writer, pool_clone); + let mut buffer_writer = BufferWriter::new(writer, pool_clone, SpillTarget::Remote); for j in 0..10 { let data = format!("Writer {} - Line {}\n", i, j); @@ -996,7 +1038,7 @@ mod tests { let operator = create_test_operator().unwrap(); let writer = operator.writer("error_test").await.unwrap(); - let mut buffer_writer = pool.buffer_write(writer); + let mut buffer_writer = pool.buffer_write(writer, SpillTarget::Remote); buffer_writer.write_all(b"test data").unwrap(); // Close once @@ -1004,12 +1046,12 @@ mod tests { // Create another writer and try to close it twice let writer2 = operator.writer("error_test2").await.unwrap(); - let buffer_writer2 = pool.buffer_write(writer2); + let buffer_writer2 = pool.buffer_write(writer2, SpillTarget::Remote); let _metadata = buffer_writer2.close().unwrap(); // Second close should return error (create new writer for this test) let writer3 = operator.writer("error_test3").await.unwrap(); - let buffer_writer3 = pool.buffer_write(writer3); + let buffer_writer3 = pool.buffer_write(writer3, SpillTarget::Remote); let _metadata = buffer_writer3.close().unwrap(); } } diff --git a/src/query/service/src/spillers/inner.rs b/src/query/service/src/spillers/inner.rs index e9abe06cb9360..63f4bba8baf0c 100644 --- a/src/query/service/src/spillers/inner.rs +++ b/src/query/service/src/spillers/inner.rs @@ -159,7 +159,7 @@ impl SpillerInner { let location = self.write_encodes(data_size, buf).await?; // Record statistics. - record_write_profile(location.is_local(), &instant, data_size); + record_write_profile(&location, &instant, data_size); let layout = columns_layout.pop().unwrap(); Ok((location, layout, data_size)) } @@ -201,7 +201,7 @@ impl SpillerInner { Location::Remote(loc) => self.operator.read(loc).await?, }; - record_read_profile(location.is_local(), &instant, data.len()); + record_read_profile(&location, &instant, data.len()); deserialize_block(columns_layout, data) } @@ -275,7 +275,11 @@ impl SpillerInner { } } -pub(super) fn record_write_profile(is_local: bool, start: &Instant, write_bytes: usize) { +pub(super) fn record_write_profile_with_flag( + is_local: bool, + start: &Instant, + write_bytes: usize, +) { if !is_local { Profile::record_usize_profile(ProfileStatisticsName::RemoteSpillWriteCount, 1); Profile::record_usize_profile(ProfileStatisticsName::RemoteSpillWriteBytes, write_bytes); @@ -293,7 +297,15 @@ pub(super) fn record_write_profile(is_local: bool, start: &Instant, write_bytes: } } -pub(super) fn record_read_profile(is_local: bool, start: &Instant, read_bytes: usize) { +pub(super) fn record_write_profile(location: &Location, start: &Instant, write_bytes: usize) { + record_write_profile_with_flag(location.is_local(), start, write_bytes) +} + +pub(super) fn record_read_profile_with_flag( + is_local: bool, + start: &Instant, + read_bytes: usize, +) { if is_local { Profile::record_usize_profile(ProfileStatisticsName::LocalSpillReadCount, 1); Profile::record_usize_profile(ProfileStatisticsName::LocalSpillReadBytes, read_bytes); @@ -310,3 +322,7 @@ pub(super) fn record_read_profile(is_local: bool, start: &Instant, read_bytes: u ); } } + +pub(super) fn record_read_profile(location: &Location, start: &Instant, read_bytes: usize) { + record_read_profile_with_flag(location.is_local(), start, read_bytes) +} diff --git a/src/query/service/src/spillers/row_group_encoder.rs b/src/query/service/src/spillers/row_group_encoder.rs index 01ac5169ccc71..0e41281b08d2b 100644 --- a/src/query/service/src/spillers/row_group_encoder.rs +++ b/src/query/service/src/spillers/row_group_encoder.rs @@ -54,6 +54,7 @@ use parquet::file::writer::SerializedRowGroupWriter; use parquet::schema::types::SchemaDescriptor; use super::async_buffer::BufferWriter; +use super::async_buffer::SpillTarget; use super::Location; use super::SpillerInner; use super::SpillsBufferPool; @@ -452,7 +453,7 @@ impl SpillerInner { }; let remote_location = self.create_unique_location(); - let remote = pool.buffer_writer(op.clone(), remote_location.clone())?; + let remote = pool.buffer_writer(op.clone(), remote_location.clone(), SpillTarget::Remote)?; Ok(AnyFileWriter::Remote( remote_location, From caf59e09a44bdda2b80648e0fdffdf93844e6436 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 8 Dec 2025 11:36:24 +0800 Subject: [PATCH 03/37] Add spill profile unit tests --- .../tests/it/spillers/spill_profile.rs | 115 ++++++++++++++++++ 1 file changed, 115 insertions(+) create mode 100644 src/query/service/tests/it/spillers/spill_profile.rs diff --git a/src/query/service/tests/it/spillers/spill_profile.rs b/src/query/service/tests/it/spillers/spill_profile.rs new file mode 100644 index 0000000000000..12252e70a96e1 --- /dev/null +++ b/src/query/service/tests/it/spillers/spill_profile.rs @@ -0,0 +1,115 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed 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 std::sync::Arc; +use std::time::Instant; + +use databend_common_base::runtime::profile::Profile; +use databend_common_base::runtime::profile::ProfileLabel; +use databend_common_base::runtime::profile::ProfileStatisticsName; +use databend_common_base::runtime::runtime_tracker::ThreadTracker; +use databend_common_base::runtime::runtime_tracker::TrackingGuard; +use databend_common_base::runtime::ScopedRegistry; +use databend_common_base::runtime::TrackingPayload; +use databend_query::spillers::record_read_profile_with_flag; +use databend_query::spillers::record_write_profile_with_flag; + +fn install_profile() -> (Arc, TrackingGuard) { + let profile = Arc::new(Profile::create( + 1, + "test".to_string(), + None, + None, + None, + Arc::new("title".to_string()), + Arc::new(vec![ProfileLabel::create("k".to_string(), vec!["v".to_string()])]), + Some(Arc::new(ScopedRegistry::create(None))), + )); + + let mut payload = ThreadTracker::new_tracking_payload(); + payload.profile = Some(profile.clone()); + payload.metrics = Some(Arc::new(ScopedRegistry::create(None))); + let guard = ThreadTracker::tracking(payload); + (profile, guard) +} + +#[test] +fn test_local_spill_profiles() { + let (profile, _guard) = install_profile(); + + let start = Instant::now(); + record_write_profile_with_flag(true, &start, 10); + record_read_profile_with_flag(true, &start, 5); + + assert_eq!( + profile.load_profile(ProfileStatisticsName::LocalSpillWriteCount), + 1 + ); + assert_eq!( + profile.load_profile(ProfileStatisticsName::LocalSpillWriteBytes), + 10 + ); + assert!( + profile.load_profile(ProfileStatisticsName::LocalSpillWriteTime) > 0, + "write time should be recorded" + ); + + assert_eq!( + profile.load_profile(ProfileStatisticsName::LocalSpillReadCount), + 1 + ); + assert_eq!( + profile.load_profile(ProfileStatisticsName::LocalSpillReadBytes), + 5 + ); + assert!( + profile.load_profile(ProfileStatisticsName::LocalSpillReadTime) > 0, + "read time should be recorded" + ); +} + +#[test] +fn test_remote_spill_profiles() { + let profile = install_profile(); + + let start = Instant::now(); + record_write_profile_with_flag(false, &start, 7); + record_read_profile_with_flag(false, &start, 3); + + assert_eq!( + profile.load_profile(ProfileStatisticsName::RemoteSpillWriteCount), + 1 + ); + assert_eq!( + profile.load_profile(ProfileStatisticsName::RemoteSpillWriteBytes), + 7 + ); + assert!( + profile.load_profile(ProfileStatisticsName::RemoteSpillWriteTime) > 0, + "write time should be recorded" + ); + + assert_eq!( + profile.load_profile(ProfileStatisticsName::RemoteSpillReadCount), + 1 + ); + assert_eq!( + profile.load_profile(ProfileStatisticsName::RemoteSpillReadBytes), + 3 + ); + assert!( + profile.load_profile(ProfileStatisticsName::RemoteSpillReadTime) > 0, + "read time should be recorded" + ); +} From 1dd02d9d5e898dae7baaa3f96d9c5b37f32eae44 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 8 Dec 2025 12:49:01 +0800 Subject: [PATCH 04/37] Fix visibility and borrow issues for spill profiling --- .../aggregator/new_aggregate/new_aggregate_spiller.rs | 2 +- .../processors/transforms/new_hash_join/grace/grace_join.rs | 2 +- src/query/service/src/spillers/adapter.rs | 6 +++--- src/query/service/src/spillers/async_buffer.rs | 2 +- src/query/service/src/spillers/mod.rs | 1 + 5 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/new_aggregate/new_aggregate_spiller.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/new_aggregate/new_aggregate_spiller.rs index 710bd1cda64bd..52919e1c3039c 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/new_aggregate/new_aggregate_spiller.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/new_aggregate/new_aggregate_spiller.rs @@ -36,7 +36,7 @@ use log::info; use parking_lot::Mutex; use parquet::file::metadata::RowGroupMetaData; -use crate::spillers::async_buffer::SpillTarget; +use crate::spillers::SpillTarget; use crate::pipelines::memory_settings::MemorySettingsExt; use crate::pipelines::processors::transforms::aggregator::AggregateMeta; use crate::pipelines::processors::transforms::aggregator::NewSpilledPayload; diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/grace/grace_join.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/grace/grace_join.rs index f276f9e0b5687..29f843fd88c7d 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/grace/grace_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/grace/grace_join.rs @@ -40,7 +40,7 @@ use crate::pipelines::processors::transforms::Join; use crate::pipelines::processors::HashJoinDesc; use crate::sessions::QueryContext; use crate::spillers::Layout; -use crate::spillers::async_buffer::SpillTarget; +use crate::spillers::SpillTarget; use crate::spillers::SpillAdapter; use crate::spillers::SpillsBufferPool; use crate::spillers::SpillsDataReader; diff --git a/src/query/service/src/spillers/adapter.rs b/src/query/service/src/spillers/adapter.rs index 69aa8879e27fe..641a537d04169 100644 --- a/src/query/service/src/spillers/adapter.rs +++ b/src/query/service/src/spillers/adapter.rs @@ -232,7 +232,7 @@ impl Spiller { let instant = Instant::now(); let location = self.write_encodes(write_bytes, buf).await?; // Record statistics. - record_write_profile(location, &instant, write_bytes); + record_write_profile(&location, &instant, write_bytes); self.adapter .add_spill_file(location.clone(), layout, write_bytes); @@ -489,14 +489,14 @@ impl SpillWriter { let row_group_meta = file_writer.flush_row_group(row_group)?; let size = row_group_meta.compressed_size() as _; self.spiller.adapter.update_progress(0, size); - record_write_profile(location, &start, size); + record_write_profile(&location, &start, size); Ok(row_group_meta) } AnyFileWriter::Remote(_, file_writer) => { let row_group_meta = file_writer.flush_row_group(row_group)?; let size = row_group_meta.compressed_size() as _; self.spiller.adapter.update_progress(0, size); - record_write_profile(location, &start, size); + record_write_profile(&location, &start, size); Ok(row_group_meta) } } diff --git a/src/query/service/src/spillers/async_buffer.rs b/src/query/service/src/spillers/async_buffer.rs index 5ed8299923504..7c8d71c631518 100644 --- a/src/query/service/src/spillers/async_buffer.rs +++ b/src/query/service/src/spillers/async_buffer.rs @@ -58,7 +58,6 @@ use parquet::file::properties::EnabledStatistics; use parquet::file::properties::WriterProperties; use super::record_read_profile_with_flag; use super::record_write_profile_with_flag; -use super::SpillsBufferPool; const CHUNK_SIZE: usize = 4 * 1024 * 1024; #[derive(Clone, Copy)] @@ -375,6 +374,7 @@ impl BufferWriter { buffer_pool: self.buffer_pool.clone(), pending_buffers: Default::default(), pending_response: None, + target: self.target, }) .close() } diff --git a/src/query/service/src/spillers/mod.rs b/src/query/service/src/spillers/mod.rs index 0aec89075e6eb..664bf5d744c83 100644 --- a/src/query/service/src/spillers/mod.rs +++ b/src/query/service/src/spillers/mod.rs @@ -23,6 +23,7 @@ mod serialize; #[cfg(test)] mod test_memory; +pub use async_buffer::SpillTarget; pub use adapter::*; pub use async_buffer::BufferWriter; pub use async_buffer::SpillsBufferPool; From f7abcaf0ef7a5102ab2b80dc63f510962c8dbac2 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 8 Dec 2025 12:49:16 +0800 Subject: [PATCH 05/37] Add spill fallback integration test --- .../tests/it/spillers/spill_fallback.rs | 76 +++++++++++++++++++ 1 file changed, 76 insertions(+) create mode 100644 src/query/service/tests/it/spillers/spill_fallback.rs diff --git a/src/query/service/tests/it/spillers/spill_fallback.rs b/src/query/service/tests/it/spillers/spill_fallback.rs new file mode 100644 index 0000000000000..b3f5073790e78 --- /dev/null +++ b/src/query/service/tests/it/spillers/spill_fallback.rs @@ -0,0 +1,76 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed 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 databend_common_base::base::tokio; +use databend_common_exception::Result; +use databend_common_expression::types::Int32Type; +use databend_common_expression::DataBlock; +use databend_common_expression::FromData; +use databend_common_storage::DataOperator; +use databend_query::spillers::SpillAdapter; +use databend_query::spillers::Spiller; +use databend_query::spillers::SpillerConfig; +use databend_query::spillers::SpillerDiskConfig; +use databend_query::spillers::SpillerType; +use databend_query::test_kits::ConfigBuilder; +use databend_query::test_kits::TestFixture; +use databend_storages_common_cache::TempDirManager; + +#[tokio::test(flavor = "multi_thread", worker_threads = 1)] +async fn test_spill_fallback_to_remote_when_local_full() -> Result<()> { + let config = ConfigBuilder::create().build(); + let fixture = TestFixture::setup_with_config(&config).await?; + let ctx = fixture.new_query_ctx().await?; + + // Prepare a tiny local spill quota to force fallback on the second spill. + let limit = 512 * 1024; // 512KB + let temp_manager = TempDirManager::instance(); + let temp_dir = temp_manager + .get_disk_spill_dir(limit, &ctx.get_id()) + .expect("local spill should be available"); + let disk_spill = SpillerDiskConfig::new(temp_dir, false)?; + + let spiller_config = SpillerConfig { + spiller_type: SpillerType::Aggregation, + location_prefix: ctx.query_id_spill_prefix(), + disk_spill: Some(disk_spill), + use_parquet: false, + }; + + let operator = DataOperator::instance().spill_operator(); + let spiller = Spiller::create(ctx.clone(), operator, spiller_config)?; + + // First spill: small enough to stay on local disk. + let small_block = + DataBlock::new_from_columns(vec![Int32Type::from_data(vec![1i32; 8 * 1024])]); // ~32KB + let loc1 = spiller.spill(vec![small_block]).await?; + assert!( + loc1.is_local(), + "first spill should land on local because quota is available" + ); + + // Second spill: large enough to exceed remaining quota and fallback to remote. + let big_block = + DataBlock::new_from_columns(vec![Int32Type::from_data(vec![2i32; 512 * 1024])]); // ~2MB + let loc2 = spiller.spill(vec![big_block]).await?; + assert!( + loc2.is_remote(), + "second spill should fallback to remote when local quota is exhausted" + ); + + // Cleanup the temp directory for this query. + let _ = temp_manager.drop_disk_spill_dir(&ctx.get_id()); + + Ok(()) +} From 6a7341fe58ea91b77a9e2a896fecceb4e984306b Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 8 Dec 2025 13:04:37 +0800 Subject: [PATCH 06/37] Fix spill metrics: use actual locations and expose helper --- src/query/service/src/spillers/adapter.rs | 6 +++--- src/query/service/src/spillers/async_buffer.rs | 2 +- src/query/service/src/spillers/inner.rs | 8 ++++---- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/query/service/src/spillers/adapter.rs b/src/query/service/src/spillers/adapter.rs index 641a537d04169..94c4fdcc5f067 100644 --- a/src/query/service/src/spillers/adapter.rs +++ b/src/query/service/src/spillers/adapter.rs @@ -489,14 +489,14 @@ impl SpillWriter { let row_group_meta = file_writer.flush_row_group(row_group)?; let size = row_group_meta.compressed_size() as _; self.spiller.adapter.update_progress(0, size); - record_write_profile(&location, &start, size); + record_write_profile(&Location::Local(file_writer.path().clone()), &start, size); Ok(row_group_meta) } - AnyFileWriter::Remote(_, file_writer) => { + AnyFileWriter::Remote(path, file_writer) => { let row_group_meta = file_writer.flush_row_group(row_group)?; let size = row_group_meta.compressed_size() as _; self.spiller.adapter.update_progress(0, size); - record_write_profile(&location, &start, size); + record_write_profile(&Location::Remote(path.clone()), &start, size); Ok(row_group_meta) } } diff --git a/src/query/service/src/spillers/async_buffer.rs b/src/query/service/src/spillers/async_buffer.rs index 7c8d71c631518..f6fe34903d4a9 100644 --- a/src/query/service/src/spillers/async_buffer.rs +++ b/src/query/service/src/spillers/async_buffer.rs @@ -67,7 +67,7 @@ pub enum SpillTarget { } impl SpillTarget { - fn is_local(&self) -> bool { + pub fn is_local(&self) -> bool { matches!(self, SpillTarget::Local) } } diff --git a/src/query/service/src/spillers/inner.rs b/src/query/service/src/spillers/inner.rs index 63f4bba8baf0c..fcc0b913d104d 100644 --- a/src/query/service/src/spillers/inner.rs +++ b/src/query/service/src/spillers/inner.rs @@ -275,7 +275,7 @@ impl SpillerInner { } } -pub(super) fn record_write_profile_with_flag( +pub fn record_write_profile_with_flag( is_local: bool, start: &Instant, write_bytes: usize, @@ -297,11 +297,11 @@ pub(super) fn record_write_profile_with_flag( } } -pub(super) fn record_write_profile(location: &Location, start: &Instant, write_bytes: usize) { +pub fn record_write_profile(location: &Location, start: &Instant, write_bytes: usize) { record_write_profile_with_flag(location.is_local(), start, write_bytes) } -pub(super) fn record_read_profile_with_flag( +pub fn record_read_profile_with_flag( is_local: bool, start: &Instant, read_bytes: usize, @@ -323,6 +323,6 @@ pub(super) fn record_read_profile_with_flag( } } -pub(super) fn record_read_profile(location: &Location, start: &Instant, read_bytes: usize) { +pub fn record_read_profile(location: &Location, start: &Instant, read_bytes: usize) { record_read_profile_with_flag(location.is_local(), start, read_bytes) } From bcc50e7cf64a5e7214f211a8d7cee03e19861d0e Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 8 Dec 2025 13:12:08 +0800 Subject: [PATCH 07/37] Carry spill locations through row group writer for profiling --- src/query/service/src/spillers/adapter.rs | 6 +++--- src/query/service/src/spillers/row_group_encoder.rs | 10 ++++++---- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/src/query/service/src/spillers/adapter.rs b/src/query/service/src/spillers/adapter.rs index 94c4fdcc5f067..9420a65c60d7c 100644 --- a/src/query/service/src/spillers/adapter.rs +++ b/src/query/service/src/spillers/adapter.rs @@ -485,11 +485,11 @@ impl SpillWriter { let start = std::time::Instant::now(); match &mut self.file_writer { - AnyFileWriter::Local(file_writer) => { + AnyFileWriter::Local(path, file_writer) => { let row_group_meta = file_writer.flush_row_group(row_group)?; let size = row_group_meta.compressed_size() as _; self.spiller.adapter.update_progress(0, size); - record_write_profile(&Location::Local(file_writer.path().clone()), &start, size); + record_write_profile(&Location::Local(path.clone()), &start, size); Ok(row_group_meta) } AnyFileWriter::Remote(path, file_writer) => { @@ -508,7 +508,7 @@ impl SpillWriter { pub fn close(self) -> Result { let (metadata, location) = match self.file_writer { - AnyFileWriter::Local(file_writer) => { + AnyFileWriter::Local(_, file_writer) => { let (metadata, path) = file_writer.finish()?; self.spiller .adapter diff --git a/src/query/service/src/spillers/row_group_encoder.rs b/src/query/service/src/spillers/row_group_encoder.rs index 0e41281b08d2b..62077a5698003 100644 --- a/src/query/service/src/spillers/row_group_encoder.rs +++ b/src/query/service/src/spillers/row_group_encoder.rs @@ -412,14 +412,14 @@ impl RangeFetchPlan { } pub enum AnyFileWriter { - Local(FileWriter), + Local(TempPath, FileWriter), Remote(String, FileWriter), } impl AnyFileWriter { pub(super) fn new_row_group(&self) -> RowGroupEncoder { match self { - AnyFileWriter::Local(file_writer) => file_writer.new_row_group(), + AnyFileWriter::Local(_, file_writer) => file_writer.new_row_group(), AnyFileWriter::Remote(_, file_writer) => file_writer.new_row_group(), } } @@ -448,7 +448,9 @@ impl SpillerInner { file, buf, }; - return Ok(AnyFileWriter::Local(FileWriter::new(props, w)?)); + let writer = FileWriter::new(props, w)?; + let path = writer.writer.inner().path.clone(); + return Ok(AnyFileWriter::Local(path, writer)); } }; @@ -456,7 +458,7 @@ impl SpillerInner { let remote = pool.buffer_writer(op.clone(), remote_location.clone(), SpillTarget::Remote)?; Ok(AnyFileWriter::Remote( - remote_location, + remote_location.clone(), FileWriter::new(props, remote)?, )) } From 9a27105899bc12211438d2b9a79e9b27ce06c0a7 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 8 Dec 2025 13:19:55 +0800 Subject: [PATCH 08/37] Refine AnyFileWriter to carry paths and fix window writer match --- .../partition/window_partition_buffer_v2.rs | 2 +- src/query/service/src/spillers/adapter.rs | 16 ++++++++-------- .../service/src/spillers/row_group_encoder.rs | 18 +++++++++--------- 3 files changed, 18 insertions(+), 18 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_buffer_v2.rs b/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_buffer_v2.rs index 4a3268bc65c8e..eba1d66abb0e3 100644 --- a/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_buffer_v2.rs +++ b/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_buffer_v2.rs @@ -59,7 +59,7 @@ impl Writer for SpillWriter { fn need_new_file(&mut self, incoming_size: usize) -> Result { Ok(match self.file_writer() { - AnyFileWriter::Local(file_writer) => !file_writer.check_grow(incoming_size, true)?, + AnyFileWriter::Local { writer, .. } => !writer.check_grow(incoming_size, true)?, _ => false, }) } diff --git a/src/query/service/src/spillers/adapter.rs b/src/query/service/src/spillers/adapter.rs index 9420a65c60d7c..2941690bcb72b 100644 --- a/src/query/service/src/spillers/adapter.rs +++ b/src/query/service/src/spillers/adapter.rs @@ -485,15 +485,15 @@ impl SpillWriter { let start = std::time::Instant::now(); match &mut self.file_writer { - AnyFileWriter::Local(path, file_writer) => { - let row_group_meta = file_writer.flush_row_group(row_group)?; + AnyFileWriter::Local { path, writer } => { + let row_group_meta = writer.flush_row_group(row_group)?; let size = row_group_meta.compressed_size() as _; self.spiller.adapter.update_progress(0, size); record_write_profile(&Location::Local(path.clone()), &start, size); Ok(row_group_meta) } - AnyFileWriter::Remote(path, file_writer) => { - let row_group_meta = file_writer.flush_row_group(row_group)?; + AnyFileWriter::Remote { path, writer } => { + let row_group_meta = writer.flush_row_group(row_group)?; let size = row_group_meta.compressed_size() as _; self.spiller.adapter.update_progress(0, size); record_write_profile(&Location::Remote(path.clone()), &start, size); @@ -508,15 +508,15 @@ impl SpillWriter { pub fn close(self) -> Result { let (metadata, location) = match self.file_writer { - AnyFileWriter::Local(_, file_writer) => { - let (metadata, path) = file_writer.finish()?; + AnyFileWriter::Local { writer, .. } => { + let (metadata, path) = writer.finish()?; self.spiller .adapter .add_spill_file(Location::Local(path.clone()), Layout::Parquet); (metadata, Location::Local(path)) } - AnyFileWriter::Remote(path, file_writer) => { - let (metadata, _) = file_writer.finish()?; + AnyFileWriter::Remote { path, writer } => { + let (metadata, _) = writer.finish()?; let location = Location::Remote(path); self.spiller diff --git a/src/query/service/src/spillers/row_group_encoder.rs b/src/query/service/src/spillers/row_group_encoder.rs index 62077a5698003..d8fefed392028 100644 --- a/src/query/service/src/spillers/row_group_encoder.rs +++ b/src/query/service/src/spillers/row_group_encoder.rs @@ -412,15 +412,15 @@ impl RangeFetchPlan { } pub enum AnyFileWriter { - Local(TempPath, FileWriter), - Remote(String, FileWriter), + Local { path: TempPath, writer: FileWriter }, + Remote { path: String, writer: FileWriter }, } impl AnyFileWriter { pub(super) fn new_row_group(&self) -> RowGroupEncoder { match self { - AnyFileWriter::Local(_, file_writer) => file_writer.new_row_group(), - AnyFileWriter::Remote(_, file_writer) => file_writer.new_row_group(), + AnyFileWriter::Local { writer, .. } => writer.new_row_group(), + AnyFileWriter::Remote { writer, .. } => writer.new_row_group(), } } } @@ -450,17 +450,17 @@ impl SpillerInner { }; let writer = FileWriter::new(props, w)?; let path = writer.writer.inner().path.clone(); - return Ok(AnyFileWriter::Local(path, writer)); + return Ok(AnyFileWriter::Local { path, writer }); } }; let remote_location = self.create_unique_location(); let remote = pool.buffer_writer(op.clone(), remote_location.clone(), SpillTarget::Remote)?; - Ok(AnyFileWriter::Remote( - remote_location.clone(), - FileWriter::new(props, remote)?, - )) + Ok(AnyFileWriter::Remote { + path: remote_location.clone(), + writer: FileWriter::new(props, remote)?, + }) } pub(super) fn load_row_groups( From 278a4ad78b9352b2ac63d4509d72f9db34b14f8d Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 8 Dec 2025 13:32:12 +0800 Subject: [PATCH 09/37] Hook spill tests into test module --- src/query/service/tests/it/spillers/mod.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/query/service/tests/it/spillers/mod.rs b/src/query/service/tests/it/spillers/mod.rs index c2bfcab0d3622..d7a03988a0a69 100644 --- a/src/query/service/tests/it/spillers/mod.rs +++ b/src/query/service/tests/it/spillers/mod.rs @@ -13,4 +13,6 @@ // limitations under the License. mod spill_config_behavior; +mod spill_fallback; +mod spill_profile; mod spiller; From 6980f590357c6ff8ecb8eb4ddcf0e4d6864aef0f Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 8 Dec 2025 13:34:21 +0800 Subject: [PATCH 10/37] Remove profile unit test and fix imports for spill fallback --- src/query/service/tests/it/spillers/mod.rs | 1 - src/query/service/tests/it/spillers/spill_fallback.rs | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/query/service/tests/it/spillers/mod.rs b/src/query/service/tests/it/spillers/mod.rs index d7a03988a0a69..f8a2eaa6386d7 100644 --- a/src/query/service/tests/it/spillers/mod.rs +++ b/src/query/service/tests/it/spillers/mod.rs @@ -14,5 +14,4 @@ mod spill_config_behavior; mod spill_fallback; -mod spill_profile; mod spiller; diff --git a/src/query/service/tests/it/spillers/spill_fallback.rs b/src/query/service/tests/it/spillers/spill_fallback.rs index b3f5073790e78..6f3cd39507e36 100644 --- a/src/query/service/tests/it/spillers/spill_fallback.rs +++ b/src/query/service/tests/it/spillers/spill_fallback.rs @@ -18,13 +18,13 @@ use databend_common_expression::types::Int32Type; use databend_common_expression::DataBlock; use databend_common_expression::FromData; use databend_common_storage::DataOperator; -use databend_query::spillers::SpillAdapter; use databend_query::spillers::Spiller; use databend_query::spillers::SpillerConfig; use databend_query::spillers::SpillerDiskConfig; use databend_query::spillers::SpillerType; use databend_query::test_kits::ConfigBuilder; use databend_query::test_kits::TestFixture; +use databend_common_catalog::table_context::TableContext; use databend_storages_common_cache::TempDirManager; #[tokio::test(flavor = "multi_thread", worker_threads = 1)] From c9719443613e0829eec00e9a631176982b1194a1 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 8 Dec 2025 13:38:08 +0800 Subject: [PATCH 11/37] Guard spill fallback test when local spill is unavailable --- src/query/service/tests/it/spillers/spill_fallback.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/query/service/tests/it/spillers/spill_fallback.rs b/src/query/service/tests/it/spillers/spill_fallback.rs index 6f3cd39507e36..efbc4d1ef1539 100644 --- a/src/query/service/tests/it/spillers/spill_fallback.rs +++ b/src/query/service/tests/it/spillers/spill_fallback.rs @@ -36,9 +36,10 @@ async fn test_spill_fallback_to_remote_when_local_full() -> Result<()> { // Prepare a tiny local spill quota to force fallback on the second spill. let limit = 512 * 1024; // 512KB let temp_manager = TempDirManager::instance(); - let temp_dir = temp_manager - .get_disk_spill_dir(limit, &ctx.get_id()) - .expect("local spill should be available"); + let Some(temp_dir) = temp_manager.get_disk_spill_dir(limit, &ctx.get_id()) else { + // Local spill is not configured in this environment; skip. + return Ok(()); + }; let disk_spill = SpillerDiskConfig::new(temp_dir, false)?; let spiller_config = SpillerConfig { From 8e434456c085cf1000589e65f5347e86ab10c50a Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 8 Dec 2025 13:40:48 +0800 Subject: [PATCH 12/37] Ensure test config enables local spill for fallback test --- src/query/service/src/test_kits/config.rs | 6 +++++- src/query/service/tests/it/spillers/spill_fallback.rs | 7 +++---- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/src/query/service/src/test_kits/config.rs b/src/query/service/src/test_kits/config.rs index d24472bd84c81..c8425bb646c74 100644 --- a/src/query/service/src/test_kits/config.rs +++ b/src/query/service/src/test_kits/config.rs @@ -62,9 +62,13 @@ ADDRESS = 'https://databend.com';" // set storage to fs let tmp_dir = TempDir::new().expect("create tmp dir failed"); let root = tmp_dir.path().to_str().unwrap().to_string(); - conf.storage.params = StorageParams::Fs(StorageFsConfig { root }); + conf.storage.params = StorageParams::Fs(StorageFsConfig { root: root.clone() }); conf.storage.allow_insecure = true; + // enable local spill using the same temp root + conf.spill.path = root.clone(); + conf.spill.global_bytes_limit = 1024 * 1024 * 1024; // 1GB for tests + ConfigBuilder { conf } } diff --git a/src/query/service/tests/it/spillers/spill_fallback.rs b/src/query/service/tests/it/spillers/spill_fallback.rs index efbc4d1ef1539..6f3cd39507e36 100644 --- a/src/query/service/tests/it/spillers/spill_fallback.rs +++ b/src/query/service/tests/it/spillers/spill_fallback.rs @@ -36,10 +36,9 @@ async fn test_spill_fallback_to_remote_when_local_full() -> Result<()> { // Prepare a tiny local spill quota to force fallback on the second spill. let limit = 512 * 1024; // 512KB let temp_manager = TempDirManager::instance(); - let Some(temp_dir) = temp_manager.get_disk_spill_dir(limit, &ctx.get_id()) else { - // Local spill is not configured in this environment; skip. - return Ok(()); - }; + let temp_dir = temp_manager + .get_disk_spill_dir(limit, &ctx.get_id()) + .expect("local spill should be available"); let disk_spill = SpillerDiskConfig::new(temp_dir, false)?; let spiller_config = SpillerConfig { From a49bbf9fd0d0dba613574c66e0e4a320af0fb173 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 8 Dec 2025 13:42:13 +0800 Subject: [PATCH 13/37] Configure test builder with local spill path via SpillConfig --- src/query/service/src/test_kits/config.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/query/service/src/test_kits/config.rs b/src/query/service/src/test_kits/config.rs index c8425bb646c74..bd4fd1f8edf03 100644 --- a/src/query/service/src/test_kits/config.rs +++ b/src/query/service/src/test_kits/config.rs @@ -15,6 +15,7 @@ use databend_common_base::base::GlobalUniqName; use databend_common_config::BuiltInConfig; use databend_common_config::InnerConfig; +use databend_common_config::SpillConfig; use databend_common_config::UDFConfig; use databend_common_config::UserAuthConfig; use databend_common_config::UserConfig; @@ -66,8 +67,7 @@ ADDRESS = 'https://databend.com';" conf.storage.allow_insecure = true; // enable local spill using the same temp root - conf.spill.path = root.clone(); - conf.spill.global_bytes_limit = 1024 * 1024 * 1024; // 1GB for tests + conf.spill = SpillConfig::new_for_test(root.clone(), 0.0, 1024 * 1024 * 1024); ConfigBuilder { conf } } From 3310ffd710c8a8a3d9c97d0348de423a11ba22e3 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 8 Dec 2025 16:26:35 +0800 Subject: [PATCH 14/37] Stabilize spill fallback test --- .../tests/it/spillers/spill_fallback.rs | 52 +++++--- .../tests/it/spillers/spill_profile.rs | 115 ------------------ 2 files changed, 36 insertions(+), 131 deletions(-) delete mode 100644 src/query/service/tests/it/spillers/spill_profile.rs diff --git a/src/query/service/tests/it/spillers/spill_fallback.rs b/src/query/service/tests/it/spillers/spill_fallback.rs index 6f3cd39507e36..45d86e3eb96c7 100644 --- a/src/query/service/tests/it/spillers/spill_fallback.rs +++ b/src/query/service/tests/it/spillers/spill_fallback.rs @@ -22,6 +22,7 @@ use databend_query::spillers::Spiller; use databend_query::spillers::SpillerConfig; use databend_query::spillers::SpillerDiskConfig; use databend_query::spillers::SpillerType; +use databend_query::spillers::Location; use databend_query::test_kits::ConfigBuilder; use databend_query::test_kits::TestFixture; use databend_common_catalog::table_context::TableContext; @@ -33,8 +34,8 @@ async fn test_spill_fallback_to_remote_when_local_full() -> Result<()> { let fixture = TestFixture::setup_with_config(&config).await?; let ctx = fixture.new_query_ctx().await?; - // Prepare a tiny local spill quota to force fallback on the second spill. - let limit = 512 * 1024; // 512KB + // Prepare a small local spill quota to force fallback after a few writes. + let limit = 32 * 1024; // 32KB let temp_manager = TempDirManager::instance(); let temp_dir = temp_manager .get_disk_spill_dir(limit, &ctx.get_id()) @@ -51,22 +52,41 @@ async fn test_spill_fallback_to_remote_when_local_full() -> Result<()> { let operator = DataOperator::instance().spill_operator(); let spiller = Spiller::create(ctx.clone(), operator, spiller_config)?; - // First spill: small enough to stay on local disk. - let small_block = - DataBlock::new_from_columns(vec![Int32Type::from_data(vec![1i32; 8 * 1024])]); // ~32KB - let loc1 = spiller.spill(vec![small_block]).await?; - assert!( - loc1.is_local(), - "first spill should land on local because quota is available" - ); + // Use a stable block size that fits into the tiny quota but will exhaust it after a few rounds. + let rows_per_block = 32 * 1024; + let block = DataBlock::new_from_columns(vec![Int32Type::from_data(vec![1i32; rows_per_block])]); + + let loc1 = spiller.spill(vec![block.clone()]).await?; + let first_block_bytes = match &loc1 { + Location::Local(path) => path.size(), + Location::Remote(_) => panic!("first spill should land on local because quota is available"), + }; + let mut locations = vec![loc1]; - // Second spill: large enough to exceed remaining quota and fallback to remote. - let big_block = - DataBlock::new_from_columns(vec![Int32Type::from_data(vec![2i32; 512 * 1024])]); // ~2MB - let loc2 = spiller.spill(vec![big_block]).await?; + let mut used_local_bytes = first_block_bytes; + let mut saw_remote = false; + // Cap the attempts to ensure the test finishes quickly even with tiny blocks. + let max_attempts = (limit / first_block_bytes.max(1)).saturating_add(8); + for _ in 0..max_attempts { + let loc = spiller.spill(vec![block.clone()]).await?; + match &loc { + Location::Local(path) => { + used_local_bytes += path.size(); + } + Location::Remote(_) => { + saw_remote = true; + break; + } + } + locations.push(loc); + } assert!( - loc2.is_remote(), - "second spill should fallback to remote when local quota is exhausted" + saw_remote, + "should fallback to remote when local quota is exhausted (used_local_bytes={}, limit={}, first_block_bytes={}, attempts={})", + used_local_bytes, + limit, + first_block_bytes, + max_attempts ); // Cleanup the temp directory for this query. diff --git a/src/query/service/tests/it/spillers/spill_profile.rs b/src/query/service/tests/it/spillers/spill_profile.rs deleted file mode 100644 index 12252e70a96e1..0000000000000 --- a/src/query/service/tests/it/spillers/spill_profile.rs +++ /dev/null @@ -1,115 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed 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 std::sync::Arc; -use std::time::Instant; - -use databend_common_base::runtime::profile::Profile; -use databend_common_base::runtime::profile::ProfileLabel; -use databend_common_base::runtime::profile::ProfileStatisticsName; -use databend_common_base::runtime::runtime_tracker::ThreadTracker; -use databend_common_base::runtime::runtime_tracker::TrackingGuard; -use databend_common_base::runtime::ScopedRegistry; -use databend_common_base::runtime::TrackingPayload; -use databend_query::spillers::record_read_profile_with_flag; -use databend_query::spillers::record_write_profile_with_flag; - -fn install_profile() -> (Arc, TrackingGuard) { - let profile = Arc::new(Profile::create( - 1, - "test".to_string(), - None, - None, - None, - Arc::new("title".to_string()), - Arc::new(vec![ProfileLabel::create("k".to_string(), vec!["v".to_string()])]), - Some(Arc::new(ScopedRegistry::create(None))), - )); - - let mut payload = ThreadTracker::new_tracking_payload(); - payload.profile = Some(profile.clone()); - payload.metrics = Some(Arc::new(ScopedRegistry::create(None))); - let guard = ThreadTracker::tracking(payload); - (profile, guard) -} - -#[test] -fn test_local_spill_profiles() { - let (profile, _guard) = install_profile(); - - let start = Instant::now(); - record_write_profile_with_flag(true, &start, 10); - record_read_profile_with_flag(true, &start, 5); - - assert_eq!( - profile.load_profile(ProfileStatisticsName::LocalSpillWriteCount), - 1 - ); - assert_eq!( - profile.load_profile(ProfileStatisticsName::LocalSpillWriteBytes), - 10 - ); - assert!( - profile.load_profile(ProfileStatisticsName::LocalSpillWriteTime) > 0, - "write time should be recorded" - ); - - assert_eq!( - profile.load_profile(ProfileStatisticsName::LocalSpillReadCount), - 1 - ); - assert_eq!( - profile.load_profile(ProfileStatisticsName::LocalSpillReadBytes), - 5 - ); - assert!( - profile.load_profile(ProfileStatisticsName::LocalSpillReadTime) > 0, - "read time should be recorded" - ); -} - -#[test] -fn test_remote_spill_profiles() { - let profile = install_profile(); - - let start = Instant::now(); - record_write_profile_with_flag(false, &start, 7); - record_read_profile_with_flag(false, &start, 3); - - assert_eq!( - profile.load_profile(ProfileStatisticsName::RemoteSpillWriteCount), - 1 - ); - assert_eq!( - profile.load_profile(ProfileStatisticsName::RemoteSpillWriteBytes), - 7 - ); - assert!( - profile.load_profile(ProfileStatisticsName::RemoteSpillWriteTime) > 0, - "write time should be recorded" - ); - - assert_eq!( - profile.load_profile(ProfileStatisticsName::RemoteSpillReadCount), - 1 - ); - assert_eq!( - profile.load_profile(ProfileStatisticsName::RemoteSpillReadBytes), - 3 - ); - assert!( - profile.load_profile(ProfileStatisticsName::RemoteSpillReadTime) > 0, - "read time should be recorded" - ); -} From 2f3fbe52c40a7fefd4a6ac595a0e3f1e22df7678 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 8 Dec 2025 17:50:28 +0800 Subject: [PATCH 15/37] Refine spill profiling paths --- .../new_aggregate/new_aggregate_spiller.rs | 61 ++++--------------- .../new_hash_join/grace/grace_join.rs | 22 ++----- .../service/src/spillers/async_buffer.rs | 37 ++++++++++- src/query/service/src/spillers/inner.rs | 12 +--- src/query/service/src/spillers/mod.rs | 2 +- .../service/src/spillers/row_group_encoder.rs | 13 +++- .../tests/it/spillers/spill_fallback.rs | 8 ++- 7 files changed, 69 insertions(+), 86 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/new_aggregate/new_aggregate_spiller.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/new_aggregate/new_aggregate_spiller.rs index 52919e1c3039c..8bd75194199b1 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/new_aggregate/new_aggregate_spiller.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/new_aggregate/new_aggregate_spiller.rs @@ -19,16 +19,13 @@ use std::time::Instant; use databend_common_base::base::GlobalUniqName; use databend_common_base::base::ProgressValues; -use databend_common_base::runtime::profile::Profile; -use databend_common_base::runtime::profile::ProfileStatisticsName; use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::BlockPartitionStream; use databend_common_expression::DataBlock; -use databend_common_pipeline_transforms::MemorySettings; -use databend_common_meta_app::storage::StorageParams; use databend_common_pipeline_transforms::traits::Location; +use databend_common_pipeline_transforms::MemorySettings; use databend_common_storage::DataOperator; use databend_common_storages_parquet::ReadSettings; use log::debug; @@ -36,7 +33,6 @@ use log::info; use parking_lot::Mutex; use parquet::file::metadata::RowGroupMetaData; -use crate::spillers::SpillTarget; use crate::pipelines::memory_settings::MemorySettingsExt; use crate::pipelines::processors::transforms::aggregator::AggregateMeta; use crate::pipelines::processors::transforms::aggregator::NewSpilledPayload; @@ -44,9 +40,9 @@ use crate::pipelines::processors::transforms::aggregator::SerializedPayload; use crate::sessions::QueryContext; use crate::spillers::Layout; use crate::spillers::SpillAdapter; +use crate::spillers::SpillTarget; use crate::spillers::SpillsBufferPool; use crate::spillers::SpillsDataWriter; -use crate::spillers::record_read_profile_with_flag; struct PayloadWriter { path: String, @@ -56,11 +52,7 @@ struct PayloadWriter { impl PayloadWriter { fn try_create(prefix: &str) -> Result { let data_operator = DataOperator::instance(); - let target = if matches!(data_operator.spill_params(), Some(StorageParams::Fs(_))) { - SpillTarget::Local - } else { - SpillTarget::Remote - }; + let target = SpillTarget::from_storage_params(data_operator.spill_params()); let operator = data_operator.spill_operator(); let buffer_pool = SpillsBufferPool::instance(); let file_path = format!("{}/{}", prefix, GlobalUniqName::unique()); @@ -366,23 +358,22 @@ impl NewAggregateSpiller { } = payload; let data_operator = DataOperator::instance(); - let target = if matches!(data_operator.spill_params(), Some(StorageParams::Fs(_))) { - SpillTarget::Local - } else { - SpillTarget::Remote - }; + let target = SpillTarget::from_storage_params(data_operator.spill_params()); let operator = data_operator.spill_operator(); let buffer_pool = SpillsBufferPool::instance(); - let mut reader = - buffer_pool.reader(operator.clone(), location.clone(), vec![row_group.clone()], target)?; + let mut reader = buffer_pool.reader( + operator.clone(), + location.clone(), + vec![row_group.clone()], + target, + )?; let instant = Instant::now(); let data_block = reader.read(self.read_setting)?; let elapsed = instant.elapsed(); let read_size = reader.read_bytes(); - record_read_profile_with_flag(target.is_local(), &instant, read_size); info!( "Read aggregate spill finished: (bucket: {}, location: {}, bytes: {}, rows: {}, elapsed: {:?})", @@ -415,41 +406,11 @@ impl NewAggregateSpiller { } } -fn flush_write_profile(ctx: &Arc, stats: WriteStats, is_local: bool) { +fn flush_write_profile(ctx: &Arc, stats: WriteStats, _is_local: bool) { if stats.count == 0 && stats.bytes == 0 && stats.rows == 0 { return; } - if stats.count > 0 { - if is_local { - Profile::record_usize_profile( - ProfileStatisticsName::LocalSpillWriteCount, - stats.count, - ); - Profile::record_usize_profile( - ProfileStatisticsName::LocalSpillWriteTime, - stats.elapsed.as_millis() as usize, - ); - Profile::record_usize_profile( - ProfileStatisticsName::LocalSpillWriteBytes, - stats.bytes, - ); - } else { - Profile::record_usize_profile( - ProfileStatisticsName::RemoteSpillWriteCount, - stats.count, - ); - Profile::record_usize_profile( - ProfileStatisticsName::RemoteSpillWriteTime, - stats.elapsed.as_millis() as usize, - ); - Profile::record_usize_profile( - ProfileStatisticsName::RemoteSpillWriteBytes, - stats.bytes, - ); - } - } - if stats.rows > 0 || stats.bytes > 0 { let progress_val = ProgressValues { rows: stats.rows, diff --git a/src/query/service/src/pipelines/processors/transforms/new_hash_join/grace/grace_join.rs b/src/query/service/src/pipelines/processors/transforms/new_hash_join/grace/grace_join.rs index 29f843fd88c7d..2c9611c31d5a9 100644 --- a/src/query/service/src/pipelines/processors/transforms/new_hash_join/grace/grace_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/new_hash_join/grace/grace_join.rs @@ -25,7 +25,6 @@ use databend_common_expression::BlockPartitionStream; use databend_common_expression::DataBlock; use databend_common_expression::FunctionContext; use databend_common_expression::HashMethodKind; -use databend_common_meta_app::storage::StorageParams; use databend_common_pipeline_transforms::traits::Location; use databend_common_storage::DataOperator; use databend_common_storages_parquet::ReadSettings; @@ -40,8 +39,8 @@ use crate::pipelines::processors::transforms::Join; use crate::pipelines::processors::HashJoinDesc; use crate::sessions::QueryContext; use crate::spillers::Layout; -use crate::spillers::SpillTarget; use crate::spillers::SpillAdapter; +use crate::spillers::SpillTarget; use crate::spillers::SpillsBufferPool; use crate::spillers::SpillsDataReader; use crate::spillers::SpillsDataWriter; @@ -258,11 +257,7 @@ impl GraceHashJoin { fn restore_build_data(&mut self) -> Result<()> { let data_operator = DataOperator::instance(); - let target = if matches!(data_operator.spill_params(), Some(StorageParams::Fs(_))) { - SpillTarget::Local - } else { - SpillTarget::Remote - }; + let target = SpillTarget::from_storage_params(data_operator.spill_params()); let operator = data_operator.spill_operator(); while let Some(data) = self.steal_restore_build_task() { @@ -393,11 +388,7 @@ pub struct GraceJoinPartition { impl GraceJoinPartition { pub fn create(prefix: &str) -> Result { let data_operator = DataOperator::instance(); - let target = if matches!(data_operator.spill_params(), Some(StorageParams::Fs(_))) { - SpillTarget::Local - } else { - SpillTarget::Remote - }; + let target = SpillTarget::from_storage_params(data_operator.spill_params()); let operator = data_operator.spill_operator(); let buffer_pool = SpillsBufferPool::instance(); @@ -471,12 +462,7 @@ impl<'a, T: GraceMemoryJoin> RestoreProbeStream<'a, T> { } let data_operator = DataOperator::instance(); - let target = - if matches!(data_operator.spill_params(), Some(StorageParams::Fs(_))) { - SpillTarget::Local - } else { - SpillTarget::Remote - }; + let target = SpillTarget::from_storage_params(data_operator.spill_params()); let operator = data_operator.spill_operator(); let buffer_pool = SpillsBufferPool::instance(); let reader = diff --git a/src/query/service/src/spillers/async_buffer.rs b/src/query/service/src/spillers/async_buffer.rs index f6fe34903d4a9..857157947474c 100644 --- a/src/query/service/src/spillers/async_buffer.rs +++ b/src/query/service/src/spillers/async_buffer.rs @@ -37,6 +37,7 @@ use databend_common_expression::DataBlock; use databend_common_expression::DataSchema; use databend_common_expression::DataSchemaRef; use databend_common_expression::TableSchemaRef; +use databend_common_meta_app::storage::StorageParams; use databend_common_storages_parquet::parquet_reader::row_group::get_ranges; use databend_common_storages_parquet::parquet_reader::RowGroupCore; use databend_common_storages_parquet::ReadSettings; @@ -56,6 +57,7 @@ use parquet::basic::Compression; use parquet::file::metadata::RowGroupMetaData; use parquet::file::properties::EnabledStatistics; use parquet::file::properties::WriterProperties; + use super::record_read_profile_with_flag; use super::record_write_profile_with_flag; const CHUNK_SIZE: usize = 4 * 1024 * 1024; @@ -70,6 +72,19 @@ impl SpillTarget { pub fn is_local(&self) -> bool { matches!(self, SpillTarget::Local) } + + /// Derive spill target (local vs remote) from storage params. + /// + /// Today we only treat `StorageParams::Fs` as local, everything + /// else (S3, Azure, memory, etc.) is considered remote. Centralizing + /// this decision here keeps higher-level operators simpler and avoids + /// duplicating the matching logic at each call site. + pub fn from_storage_params(params: Option<&StorageParams>) -> Self { + match params { + Some(StorageParams::Fs(_)) => SpillTarget::Local, + _ => SpillTarget::Remote, + } + } } /// Buffer Pool Workflow for Spill Operations: @@ -286,7 +301,11 @@ pub struct BufferWriter { } impl BufferWriter { - pub fn new(writer: Writer, buffer_pool: Arc, target: SpillTarget) -> BufferWriter { + pub fn new( + writer: Writer, + buffer_pool: Arc, + target: SpillTarget, + ) -> BufferWriter { BufferWriter { buffer_pool, writer: Some(writer), @@ -1054,4 +1073,20 @@ mod tests { let buffer_writer3 = pool.buffer_write(writer3, SpillTarget::Remote); let _metadata = buffer_writer3.close().unwrap(); } + + #[test] + fn test_spill_target_from_storage_params() { + use databend_common_meta_app::storage::StorageFsConfig; + + // Fs backend should be treated as local spill. + let fs_params = StorageParams::Fs(StorageFsConfig { + root: "/tmp/test-root".to_string(), + }); + let target = SpillTarget::from_storage_params(Some(&fs_params)); + assert!(target.is_local()); + + // None (or any non-Fs backend) should be treated as remote spill. + let target_none = SpillTarget::from_storage_params(None); + assert!(!target_none.is_local()); + } } diff --git a/src/query/service/src/spillers/inner.rs b/src/query/service/src/spillers/inner.rs index fcc0b913d104d..db354cabf5945 100644 --- a/src/query/service/src/spillers/inner.rs +++ b/src/query/service/src/spillers/inner.rs @@ -275,11 +275,7 @@ impl SpillerInner { } } -pub fn record_write_profile_with_flag( - is_local: bool, - start: &Instant, - write_bytes: usize, -) { +pub fn record_write_profile_with_flag(is_local: bool, start: &Instant, write_bytes: usize) { if !is_local { Profile::record_usize_profile(ProfileStatisticsName::RemoteSpillWriteCount, 1); Profile::record_usize_profile(ProfileStatisticsName::RemoteSpillWriteBytes, write_bytes); @@ -301,11 +297,7 @@ pub fn record_write_profile(location: &Location, start: &Instant, write_bytes: u record_write_profile_with_flag(location.is_local(), start, write_bytes) } -pub fn record_read_profile_with_flag( - is_local: bool, - start: &Instant, - read_bytes: usize, -) { +pub fn record_read_profile_with_flag(is_local: bool, start: &Instant, read_bytes: usize) { if is_local { Profile::record_usize_profile(ProfileStatisticsName::LocalSpillReadCount, 1); Profile::record_usize_profile(ProfileStatisticsName::LocalSpillReadBytes, read_bytes); diff --git a/src/query/service/src/spillers/mod.rs b/src/query/service/src/spillers/mod.rs index 664bf5d744c83..35071c2d6e14c 100644 --- a/src/query/service/src/spillers/mod.rs +++ b/src/query/service/src/spillers/mod.rs @@ -23,9 +23,9 @@ mod serialize; #[cfg(test)] mod test_memory; -pub use async_buffer::SpillTarget; pub use adapter::*; pub use async_buffer::BufferWriter; +pub use async_buffer::SpillTarget; pub use async_buffer::SpillsBufferPool; pub use async_buffer::SpillsDataReader; pub use async_buffer::SpillsDataWriter; diff --git a/src/query/service/src/spillers/row_group_encoder.rs b/src/query/service/src/spillers/row_group_encoder.rs index d8fefed392028..829b239423868 100644 --- a/src/query/service/src/spillers/row_group_encoder.rs +++ b/src/query/service/src/spillers/row_group_encoder.rs @@ -412,8 +412,14 @@ impl RangeFetchPlan { } pub enum AnyFileWriter { - Local { path: TempPath, writer: FileWriter }, - Remote { path: String, writer: FileWriter }, + Local { + path: TempPath, + writer: FileWriter, + }, + Remote { + path: String, + writer: FileWriter, + }, } impl AnyFileWriter { @@ -455,7 +461,8 @@ impl SpillerInner { }; let remote_location = self.create_unique_location(); - let remote = pool.buffer_writer(op.clone(), remote_location.clone(), SpillTarget::Remote)?; + let remote = + pool.buffer_writer(op.clone(), remote_location.clone(), SpillTarget::Remote)?; Ok(AnyFileWriter::Remote { path: remote_location.clone(), diff --git a/src/query/service/tests/it/spillers/spill_fallback.rs b/src/query/service/tests/it/spillers/spill_fallback.rs index 45d86e3eb96c7..b76b11c3e60fb 100644 --- a/src/query/service/tests/it/spillers/spill_fallback.rs +++ b/src/query/service/tests/it/spillers/spill_fallback.rs @@ -13,19 +13,19 @@ // limitations under the License. use databend_common_base::base::tokio; +use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; use databend_common_expression::types::Int32Type; use databend_common_expression::DataBlock; use databend_common_expression::FromData; use databend_common_storage::DataOperator; +use databend_query::spillers::Location; use databend_query::spillers::Spiller; use databend_query::spillers::SpillerConfig; use databend_query::spillers::SpillerDiskConfig; use databend_query::spillers::SpillerType; -use databend_query::spillers::Location; use databend_query::test_kits::ConfigBuilder; use databend_query::test_kits::TestFixture; -use databend_common_catalog::table_context::TableContext; use databend_storages_common_cache::TempDirManager; #[tokio::test(flavor = "multi_thread", worker_threads = 1)] @@ -59,7 +59,9 @@ async fn test_spill_fallback_to_remote_when_local_full() -> Result<()> { let loc1 = spiller.spill(vec![block.clone()]).await?; let first_block_bytes = match &loc1 { Location::Local(path) => path.size(), - Location::Remote(_) => panic!("first spill should land on local because quota is available"), + Location::Remote(_) => { + panic!("first spill should land on local because quota is available") + } }; let mut locations = vec![loc1]; From 3389e5a274b52c218b57c7117604133809c4ebaf Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 8 Dec 2025 18:03:42 +0800 Subject: [PATCH 16/37] Add spill profile tests for local/remote metrics --- src/query/service/tests/it/spillers/mod.rs | 1 + .../tests/it/spillers/spill_profile.rs | 113 ++++++++++++++++++ 2 files changed, 114 insertions(+) create mode 100644 src/query/service/tests/it/spillers/spill_profile.rs diff --git a/src/query/service/tests/it/spillers/mod.rs b/src/query/service/tests/it/spillers/mod.rs index f8a2eaa6386d7..d7a03988a0a69 100644 --- a/src/query/service/tests/it/spillers/mod.rs +++ b/src/query/service/tests/it/spillers/mod.rs @@ -14,4 +14,5 @@ mod spill_config_behavior; mod spill_fallback; +mod spill_profile; mod spiller; diff --git a/src/query/service/tests/it/spillers/spill_profile.rs b/src/query/service/tests/it/spillers/spill_profile.rs new file mode 100644 index 0000000000000..2bb48102e0fe3 --- /dev/null +++ b/src/query/service/tests/it/spillers/spill_profile.rs @@ -0,0 +1,113 @@ +// Spill profile related tests: verify spill read/write metrics wiring to Profile. + +use std::sync::Arc; +use std::time::Instant; + +use databend_common_base::runtime::profile::Profile; +use databend_common_base::runtime::profile::ProfileStatisticsName; +use databend_common_base::runtime::ThreadTracker; +use databend_query::spillers::record_read_profile_with_flag; +use databend_query::spillers::record_write_profile_with_flag; + +fn create_test_profile() -> Arc { + Arc::new(Profile::create( + 0, + "test_spill_profile".to_string(), + None, + None, + None, + Arc::new(String::new()), + Arc::new(vec![]), + None, + )) +} + +#[test] +fn test_spill_profile_write_local_and_remote() { + ThreadTracker::init(); + + // Local spill metrics + let mut payload_local = ThreadTracker::new_tracking_payload(); + let local_profile = create_test_profile(); + payload_local.profile = Some(local_profile.clone()); + + { + let _guard = ThreadTracker::tracking(payload_local); + let start = Instant::now(); + record_write_profile_with_flag(true, &start, 128); + } + + assert_eq!( + local_profile.load_profile(ProfileStatisticsName::LocalSpillWriteCount), + 1, + ); + assert_eq!( + local_profile.load_profile(ProfileStatisticsName::LocalSpillWriteBytes), + 128, + ); + + // Remote spill metrics + let mut payload_remote = ThreadTracker::new_tracking_payload(); + let remote_profile = create_test_profile(); + payload_remote.profile = Some(remote_profile.clone()); + + { + let _guard = ThreadTracker::tracking(payload_remote); + let start = Instant::now(); + record_write_profile_with_flag(false, &start, 256); + } + + assert_eq!( + remote_profile.load_profile(ProfileStatisticsName::RemoteSpillWriteCount), + 1, + ); + assert_eq!( + remote_profile.load_profile(ProfileStatisticsName::RemoteSpillWriteBytes), + 256, + ); +} + +#[test] +fn test_spill_profile_read_local_and_remote() { + ThreadTracker::init(); + + // Local spill metrics + let mut payload_local = ThreadTracker::new_tracking_payload(); + let local_profile = create_test_profile(); + payload_local.profile = Some(local_profile.clone()); + + { + let _guard = ThreadTracker::tracking(payload_local); + let start = Instant::now(); + record_read_profile_with_flag(true, &start, 64); + } + + assert_eq!( + local_profile.load_profile(ProfileStatisticsName::LocalSpillReadCount), + 1, + ); + assert_eq!( + local_profile.load_profile(ProfileStatisticsName::LocalSpillReadBytes), + 64, + ); + + // Remote spill metrics + let mut payload_remote = ThreadTracker::new_tracking_payload(); + let remote_profile = create_test_profile(); + payload_remote.profile = Some(remote_profile.clone()); + + { + let _guard = ThreadTracker::tracking(payload_remote); + let start = Instant::now(); + record_read_profile_with_flag(false, &start, 512); + } + + assert_eq!( + remote_profile.load_profile(ProfileStatisticsName::RemoteSpillReadCount), + 1, + ); + assert_eq!( + remote_profile.load_profile(ProfileStatisticsName::RemoteSpillReadBytes), + 512, + ); +} From 132e93e871ea6974fe5607b808dbf240bf128256 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 8 Dec 2025 18:42:26 +0800 Subject: [PATCH 17/37] Refine spill tests and move SpillTarget test to it suite --- .../service/src/spillers/async_buffer.rs | 16 ---------------- src/query/service/tests/it/spillers/mod.rs | 1 + .../tests/it/spillers/spill_fallback.rs | 19 +++++++++++++++++++ .../tests/it/spillers/spill_profile.rs | 16 +++++++++++++++- .../service/tests/it/spillers/spill_target.rs | 19 +++++++++++++++++++ 5 files changed, 54 insertions(+), 17 deletions(-) create mode 100644 src/query/service/tests/it/spillers/spill_target.rs diff --git a/src/query/service/src/spillers/async_buffer.rs b/src/query/service/src/spillers/async_buffer.rs index 857157947474c..79950f8aa9490 100644 --- a/src/query/service/src/spillers/async_buffer.rs +++ b/src/query/service/src/spillers/async_buffer.rs @@ -1073,20 +1073,4 @@ mod tests { let buffer_writer3 = pool.buffer_write(writer3, SpillTarget::Remote); let _metadata = buffer_writer3.close().unwrap(); } - - #[test] - fn test_spill_target_from_storage_params() { - use databend_common_meta_app::storage::StorageFsConfig; - - // Fs backend should be treated as local spill. - let fs_params = StorageParams::Fs(StorageFsConfig { - root: "/tmp/test-root".to_string(), - }); - let target = SpillTarget::from_storage_params(Some(&fs_params)); - assert!(target.is_local()); - - // None (or any non-Fs backend) should be treated as remote spill. - let target_none = SpillTarget::from_storage_params(None); - assert!(!target_none.is_local()); - } } diff --git a/src/query/service/tests/it/spillers/mod.rs b/src/query/service/tests/it/spillers/mod.rs index d7a03988a0a69..039800621c60a 100644 --- a/src/query/service/tests/it/spillers/mod.rs +++ b/src/query/service/tests/it/spillers/mod.rs @@ -15,4 +15,5 @@ mod spill_config_behavior; mod spill_fallback; mod spill_profile; +mod spill_target; mod spiller; diff --git a/src/query/service/tests/it/spillers/spill_fallback.rs b/src/query/service/tests/it/spillers/spill_fallback.rs index b76b11c3e60fb..456fe1f048318 100644 --- a/src/query/service/tests/it/spillers/spill_fallback.rs +++ b/src/query/service/tests/it/spillers/spill_fallback.rs @@ -28,6 +28,25 @@ use databend_query::test_kits::ConfigBuilder; use databend_query::test_kits::TestFixture; use databend_storages_common_cache::TempDirManager; +/// ASCII flow of the test (data view): +/// +/// Int32 rows +/// | +/// v +/// DataBlock (rows_per_block = 32 * 1024) +/// | +/// v +/// Spiller::spill(vec![block.clone()]) -- repeated writes +/// | +/// v +/// Local spill files on disk (quota: 32 * 1024 bytes) +/// | +/// | accumulate used_local_bytes from each Local(path) +/// v +/// when used_local_bytes > quota +/// | +/// v +/// next spill -> Location::Remote(...) #[tokio::test(flavor = "multi_thread", worker_threads = 1)] async fn test_spill_fallback_to_remote_when_local_full() -> Result<()> { let config = ConfigBuilder::create().build(); diff --git a/src/query/service/tests/it/spillers/spill_profile.rs b/src/query/service/tests/it/spillers/spill_profile.rs index 2bb48102e0fe3..43581b17992b1 100644 --- a/src/query/service/tests/it/spillers/spill_profile.rs +++ b/src/query/service/tests/it/spillers/spill_profile.rs @@ -1,4 +1,18 @@ -// Spill profile related tests: verify spill read/write metrics wiring to Profile. +// Copyright 2021 Datafuse Labs +// +// Licensed 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. + +//! Spill profile related tests: verify spill read/write metrics wiring to Profile. use std::sync::Arc; use std::time::Instant; diff --git a/src/query/service/tests/it/spillers/spill_target.rs b/src/query/service/tests/it/spillers/spill_target.rs new file mode 100644 index 0000000000000..1223b9eb2f851 --- /dev/null +++ b/src/query/service/tests/it/spillers/spill_target.rs @@ -0,0 +1,19 @@ +// Integration test for SpillTarget::from_storage_params semantics. + +use databend_common_meta_app::storage::StorageFsConfig; +use databend_common_meta_app::storage::StorageParams; +use databend_query::spillers::SpillTarget; + +#[test] +fn test_spill_target_from_storage_params() { + // Fs backend should be treated as local spill. + let fs_params = StorageParams::Fs(StorageFsConfig { + root: "/tmp/test-root".to_string(), + }); + let target = SpillTarget::from_storage_params(Some(&fs_params)); + assert!(target.is_local()); + + // None (or any non-Fs backend) should be treated as remote spill. + let target_none = SpillTarget::from_storage_params(None); + assert!(!target_none.is_local()); +} From cd82c0a354cf05db9fb798092363f452fce22137 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 8 Dec 2025 19:17:38 +0800 Subject: [PATCH 18/37] Add spill test config helper and fix spill IT headers --- src/query/service/src/test_kits/config.rs | 24 +++++++++++++++---- src/query/service/src/test_kits/mod.rs | 1 + .../it/spillers/spill_config_behavior.rs | 10 ++++---- .../tests/it/spillers/spill_fallback.rs | 4 ++-- .../service/tests/it/spillers/spill_target.rs | 16 ++++++++++++- 5 files changed, 42 insertions(+), 13 deletions(-) diff --git a/src/query/service/src/test_kits/config.rs b/src/query/service/src/test_kits/config.rs index bd4fd1f8edf03..ad4e1a59bed27 100644 --- a/src/query/service/src/test_kits/config.rs +++ b/src/query/service/src/test_kits/config.rs @@ -60,15 +60,12 @@ ADDRESS = 'https://databend.com';" // set node_id to a unique value conf.query.node_id = GlobalUniqName::unique(); - // set storage to fs + // set storage to fs for tests; individual tests may override this let tmp_dir = TempDir::new().expect("create tmp dir failed"); let root = tmp_dir.path().to_str().unwrap().to_string(); - conf.storage.params = StorageParams::Fs(StorageFsConfig { root: root.clone() }); + conf.storage.params = StorageParams::Fs(StorageFsConfig { root }); conf.storage.allow_insecure = true; - // enable local spill using the same temp root - conf.spill = SpillConfig::new_for_test(root.clone(), 0.0, 1024 * 1024 * 1024); - ConfigBuilder { conf } } @@ -188,3 +185,20 @@ ADDRESS = 'https://databend.com';" self.conf.clone() } } + +/// Helper for spill-related tests: start from the default test config and +/// attach a local spill configuration so that TempDirManager has a root path. +pub fn config_with_spill() -> InnerConfig { + let mut conf = ConfigBuilder::create().config(); + + // Use a stable directory under the workspace for spill during tests. + // TempDirManager::init will create and clean up directories under this + // path, so it's safe to reuse across tests. + conf.spill = SpillConfig::new_for_test( + ".databend/_query_spill".to_string(), + 0.0, + 1024 * 1024 * 1024, + ); + + conf +} diff --git a/src/query/service/src/test_kits/mod.rs b/src/query/service/src/test_kits/mod.rs index e5a1f7102d2f3..c9fe82f456992 100644 --- a/src/query/service/src/test_kits/mod.rs +++ b/src/query/service/src/test_kits/mod.rs @@ -26,6 +26,7 @@ mod old_version_generator; pub use block_writer::BlockWriter; pub use check::*; pub use cluster::ClusterDescriptor; +pub use config::config_with_spill; pub use config::ConfigBuilder; pub use context::*; pub use fixture::*; diff --git a/src/query/service/tests/it/spillers/spill_config_behavior.rs b/src/query/service/tests/it/spillers/spill_config_behavior.rs index 179222bc625cd..f4c4b13407b6c 100644 --- a/src/query/service/tests/it/spillers/spill_config_behavior.rs +++ b/src/query/service/tests/it/spillers/spill_config_behavior.rs @@ -30,7 +30,7 @@ use databend_query::spillers::Location; use databend_query::spillers::Spiller; use databend_query::spillers::SpillerConfig; use databend_query::spillers::SpillerType; -use databend_query::test_kits::ConfigBuilder; +use databend_query::test_kits::config_with_spill; use databend_query::test_kits::TestFixture; use databend_storages_common_cache::TempDirManager; @@ -39,8 +39,8 @@ use databend_storages_common_cache::TempDirManager; async fn test_spill_directory_operations() -> Result<()> { println!("🔄 Starting spill directory operations test"); - // Setup with default config - let config = ConfigBuilder::create().build(); + // Setup with spill-enabled config dedicated for spill tests + let config = config_with_spill(); let fixture = TestFixture::setup_with_config(&config).await?; let ctx = fixture.new_query_ctx().await?; @@ -102,7 +102,7 @@ async fn test_spill_directory_operations() -> Result<()> { /// Test 2: Spill size limit enforcement #[tokio::test(flavor = "multi_thread", worker_threads = 1)] async fn test_spill_size_limits() -> Result<()> { - let config = ConfigBuilder::create().build(); + let config = config_with_spill(); let fixture = TestFixture::setup_with_config(&config).await?; let ctx = fixture.new_query_ctx().await?; @@ -144,7 +144,7 @@ async fn test_spill_size_limits() -> Result<()> { /// Test 3: Verify spill data integrity and actual file operations #[tokio::test(flavor = "multi_thread", worker_threads = 1)] async fn test_spill_data_integrity() -> Result<()> { - let config = ConfigBuilder::create().build(); + let config = config_with_spill(); let fixture = TestFixture::setup_with_config(&config).await?; let ctx = fixture.new_query_ctx().await?; diff --git a/src/query/service/tests/it/spillers/spill_fallback.rs b/src/query/service/tests/it/spillers/spill_fallback.rs index 456fe1f048318..c6a7a34da9bdc 100644 --- a/src/query/service/tests/it/spillers/spill_fallback.rs +++ b/src/query/service/tests/it/spillers/spill_fallback.rs @@ -24,7 +24,7 @@ use databend_query::spillers::Spiller; use databend_query::spillers::SpillerConfig; use databend_query::spillers::SpillerDiskConfig; use databend_query::spillers::SpillerType; -use databend_query::test_kits::ConfigBuilder; +use databend_query::test_kits::config_with_spill; use databend_query::test_kits::TestFixture; use databend_storages_common_cache::TempDirManager; @@ -49,7 +49,7 @@ use databend_storages_common_cache::TempDirManager; /// next spill -> Location::Remote(...) #[tokio::test(flavor = "multi_thread", worker_threads = 1)] async fn test_spill_fallback_to_remote_when_local_full() -> Result<()> { - let config = ConfigBuilder::create().build(); + let config = config_with_spill(); let fixture = TestFixture::setup_with_config(&config).await?; let ctx = fixture.new_query_ctx().await?; diff --git a/src/query/service/tests/it/spillers/spill_target.rs b/src/query/service/tests/it/spillers/spill_target.rs index 1223b9eb2f851..5fb05d8262dd4 100644 --- a/src/query/service/tests/it/spillers/spill_target.rs +++ b/src/query/service/tests/it/spillers/spill_target.rs @@ -1,4 +1,18 @@ -// Integration test for SpillTarget::from_storage_params semantics. +// Copyright 2021 Datafuse Labs +// +// Licensed 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. + +//! Integration test for SpillTarget::from_storage_params semantics. use databend_common_meta_app::storage::StorageFsConfig; use databend_common_meta_app::storage::StorageParams; From 2a095a3d2e11071af630b5f5700685198d9e750c Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 8 Dec 2025 19:24:29 +0800 Subject: [PATCH 19/37] Fix clippy needless-borrow in spill read profile --- src/query/service/src/spillers/inner.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/service/src/spillers/inner.rs b/src/query/service/src/spillers/inner.rs index db354cabf5945..c986b305fdae5 100644 --- a/src/query/service/src/spillers/inner.rs +++ b/src/query/service/src/spillers/inner.rs @@ -201,7 +201,7 @@ impl SpillerInner { Location::Remote(loc) => self.operator.read(loc).await?, }; - record_read_profile(&location, &instant, data.len()); + record_read_profile(location, &instant, data.len()); deserialize_block(columns_layout, data) } From cc934aadc1fca013195bb38f6af5266e05688bce Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Tue, 9 Dec 2025 08:41:58 +0800 Subject: [PATCH 20/37] Align spill quota settings and configs --- .../deploy/config/databend-query-node-1.toml | 3 + .../configs/databend-query-node2.toml | 5 +- .../configs/databend-query-node3.toml | 5 +- .../configs/databend-query.toml | 3 + .../configs/query/query-1.toml | 3 + .../configs/query/query-2.toml | 5 +- .../configs/query/query-3.toml | 5 +- .../configs/query/query-4.toml | 5 +- .../configs/query/query-5.toml | 5 +- src/query/config/src/config.rs | 29 +++++++++ src/query/config/src/inner.rs | 61 +++++++++++++++++++ src/query/config/src/mask.rs | 6 ++ .../src/physical_plans/physical_recluster.rs | 6 +- .../physical_window_partition.rs | 5 +- .../src/pipelines/builders/builder_sort.rs | 3 +- .../servers/http/v1/query/execute_state.rs | 5 +- src/query/settings/src/settings_default.rs | 21 ------- .../settings/src/settings_getter_setter.rs | 12 ---- .../suites/query/issues/issue_17581.test | 3 - .../window_partition_spill.test | 9 +-- .../suites/tpcds/window_spill.test | 6 -- tests/sqllogictests/suites/tpch/spill.test | 3 - 22 files changed, 147 insertions(+), 61 deletions(-) diff --git a/scripts/ci/deploy/config/databend-query-node-1.toml b/scripts/ci/deploy/config/databend-query-node-1.toml index 659dcc5150f7c..f81d4bc49ac65 100644 --- a/scripts/ci/deploy/config/databend-query-node-1.toml +++ b/scripts/ci/deploy/config/databend-query-node-1.toml @@ -162,3 +162,6 @@ max_bytes = 21474836480 [spill] spill_local_disk_path = "./.databend/temp/_query_spill" +# Cap local spill to 5GB so window spills keep ~1GB quota with default 20% ratio. +spill_local_disk_max_bytes = 5368709120 +window_partition_spilling_disk_quota_ratio = 20 diff --git a/scripts/databend_test_helper/databend_test_helper/configs/databend-query-node2.toml b/scripts/databend_test_helper/databend_test_helper/configs/databend-query-node2.toml index 395762577d3ce..537aefc9f88f0 100644 --- a/scripts/databend_test_helper/databend_test_helper/configs/databend-query-node2.toml +++ b/scripts/databend_test_helper/databend_test_helper/configs/databend-query-node2.toml @@ -62,4 +62,7 @@ path = "_databend_data/cache/query2" max_bytes = 1073741824 # 1GB [spill] -spill_local_disk_path = "_databend_data/spill/query2" \ No newline at end of file +spill_local_disk_path = "_databend_data/spill/query2" +sort_spilling_disk_quota_ratio = 60 +window_partition_spilling_disk_quota_ratio = 20 +result_set_spilling_disk_quota_ratio = 10 diff --git a/scripts/databend_test_helper/databend_test_helper/configs/databend-query-node3.toml b/scripts/databend_test_helper/databend_test_helper/configs/databend-query-node3.toml index e1902a81ed411..2c5f73ae0105e 100644 --- a/scripts/databend_test_helper/databend_test_helper/configs/databend-query-node3.toml +++ b/scripts/databend_test_helper/databend_test_helper/configs/databend-query-node3.toml @@ -62,4 +62,7 @@ path = "_databend_data/cache/query3" max_bytes = 1073741824 # 1GB [spill] -spill_local_disk_path = "_databend_data/spill/query3" \ No newline at end of file +spill_local_disk_path = "_databend_data/spill/query3" +sort_spilling_disk_quota_ratio = 60 +window_partition_spilling_disk_quota_ratio = 20 +result_set_spilling_disk_quota_ratio = 10 diff --git a/scripts/databend_test_helper/databend_test_helper/configs/databend-query.toml b/scripts/databend_test_helper/databend_test_helper/configs/databend-query.toml index c063f0c3888c4..e143f6977e027 100644 --- a/scripts/databend_test_helper/databend_test_helper/configs/databend-query.toml +++ b/scripts/databend_test_helper/databend_test_helper/configs/databend-query.toml @@ -63,3 +63,6 @@ max_bytes = 1073741824 # 1GB [spill] spill_local_disk_path = "_databend_data/spill/query1" +sort_spilling_disk_quota_ratio = 60 +window_partition_spilling_disk_quota_ratio = 20 +result_set_spilling_disk_quota_ratio = 10 diff --git a/scripts/test-bend-tests/configs/query/query-1.toml b/scripts/test-bend-tests/configs/query/query-1.toml index 8f7b03361811a..58a102020e61d 100644 --- a/scripts/test-bend-tests/configs/query/query-1.toml +++ b/scripts/test-bend-tests/configs/query/query-1.toml @@ -64,3 +64,6 @@ max_bytes = 1073741824 # 1GB [spill] spill_local_disk_path = "_databend_data/spill/query1" +sort_spilling_disk_quota_ratio = 60 +window_partition_spilling_disk_quota_ratio = 20 +result_set_spilling_disk_quota_ratio = 10 diff --git a/scripts/test-bend-tests/configs/query/query-2.toml b/scripts/test-bend-tests/configs/query/query-2.toml index 80f28e95cad55..6992974345598 100644 --- a/scripts/test-bend-tests/configs/query/query-2.toml +++ b/scripts/test-bend-tests/configs/query/query-2.toml @@ -63,4 +63,7 @@ path = "_databend_data/cache/query2" max_bytes = 1073741824 # 1GB [spill] -spill_local_disk_path = "_databend_data/spill/query2" \ No newline at end of file +spill_local_disk_path = "_databend_data/spill/query2" +sort_spilling_disk_quota_ratio = 60 +window_partition_spilling_disk_quota_ratio = 20 +result_set_spilling_disk_quota_ratio = 10 diff --git a/scripts/test-bend-tests/configs/query/query-3.toml b/scripts/test-bend-tests/configs/query/query-3.toml index 5c0054c5677e9..049a890eee6d1 100644 --- a/scripts/test-bend-tests/configs/query/query-3.toml +++ b/scripts/test-bend-tests/configs/query/query-3.toml @@ -63,4 +63,7 @@ path = "_databend_data/cache/query3" max_bytes = 1073741824 # 1GB [spill] -spill_local_disk_path = "_databend_data/spill/query3" \ No newline at end of file +spill_local_disk_path = "_databend_data/spill/query3" +sort_spilling_disk_quota_ratio = 60 +window_partition_spilling_disk_quota_ratio = 20 +result_set_spilling_disk_quota_ratio = 10 diff --git a/scripts/test-bend-tests/configs/query/query-4.toml b/scripts/test-bend-tests/configs/query/query-4.toml index d33136fa5ba74..1ef0595cec8c8 100644 --- a/scripts/test-bend-tests/configs/query/query-4.toml +++ b/scripts/test-bend-tests/configs/query/query-4.toml @@ -63,4 +63,7 @@ path = "_databend_data/cache/query4" max_bytes = 1073741824 # 1GB [spill] -spill_local_disk_path = "_databend_data/spill/query4" \ No newline at end of file +spill_local_disk_path = "_databend_data/spill/query4" +sort_spilling_disk_quota_ratio = 60 +window_partition_spilling_disk_quota_ratio = 20 +result_set_spilling_disk_quota_ratio = 10 diff --git a/scripts/test-bend-tests/configs/query/query-5.toml b/scripts/test-bend-tests/configs/query/query-5.toml index 43d2a646cdb8e..5e4a303d3efcd 100644 --- a/scripts/test-bend-tests/configs/query/query-5.toml +++ b/scripts/test-bend-tests/configs/query/query-5.toml @@ -63,4 +63,7 @@ path = "_databend_data/cache/query5" max_bytes = 1073741824 # 1GB [spill] -spill_local_disk_path = "_databend_data/spill/query5" \ No newline at end of file +spill_local_disk_path = "_databend_data/spill/query5" +sort_spilling_disk_quota_ratio = 60 +window_partition_spilling_disk_quota_ratio = 20 +result_set_spilling_disk_quota_ratio = 10 diff --git a/src/query/config/src/config.rs b/src/query/config/src/config.rs index 99729c0b702be..4d9ea336e8ee2 100644 --- a/src/query/config/src/config.rs +++ b/src/query/config/src/config.rs @@ -3592,6 +3592,24 @@ pub struct SpillConfig { #[clap(long, value_name = "VALUE", default_value = "18446744073709551615")] /// Allow space in bytes to spill to local disk. pub spill_local_disk_max_bytes: u64, + + /// Maximum percentage of the global local spill quota that a single sort + /// operator may use for one query. + /// + /// Value range: 0-100. Effective only when local spill is enabled (there is + /// a valid local spill path and non-zero `spill_local_disk_max_bytes`). + #[clap(long, value_name = "PERCENT", default_value = "60")] + pub sort_spilling_disk_quota_ratio: u64, + + /// Maximum percentage of the global local spill quota that window + /// partitioners may use for one query. + #[clap(long, value_name = "PERCENT", default_value = "20")] + pub window_partition_spilling_disk_quota_ratio: u64, + + /// Maximum percentage of the global local spill quota that HTTP + /// result-set spilling may use for one query. + #[clap(long, value_name = "PERCENT", default_value = "10")] + pub result_set_spilling_disk_quota_ratio: u64, } impl SpillConfig { @@ -3632,6 +3650,9 @@ impl Default for SpillConfig { spill_local_disk_path: String::new(), spill_local_disk_reserved_space_percentage: OrderedFloat(30.0), spill_local_disk_max_bytes: u64::MAX, + sort_spilling_disk_quota_ratio: 60, + window_partition_spilling_disk_quota_ratio: 20, + result_set_spilling_disk_quota_ratio: 10, } } } @@ -3898,6 +3919,10 @@ mod cache_config_converters { reserved_disk_ratio, global_bytes_limit, storage_params, + sort_spilling_disk_quota_ratio: spill.sort_spilling_disk_quota_ratio, + window_partition_spilling_disk_quota_ratio: spill + .window_partition_spilling_disk_quota_ratio, + result_set_spilling_disk_quota_ratio: spill.result_set_spilling_disk_quota_ratio, }) } @@ -3916,6 +3941,10 @@ mod cache_config_converters { spill_local_disk_path: value.path, spill_local_disk_reserved_space_percentage: value.reserved_disk_ratio * 100.0, spill_local_disk_max_bytes: value.global_bytes_limit, + sort_spilling_disk_quota_ratio: value.sort_spilling_disk_quota_ratio, + window_partition_spilling_disk_quota_ratio: value + .window_partition_spilling_disk_quota_ratio, + result_set_spilling_disk_quota_ratio: value.result_set_spilling_disk_quota_ratio, } } } diff --git a/src/query/config/src/inner.rs b/src/query/config/src/inner.rs index bf4def1fcf506..71c5670ed84ec 100644 --- a/src/query/config/src/inner.rs +++ b/src/query/config/src/inner.rs @@ -787,6 +787,22 @@ pub struct SpillConfig { pub global_bytes_limit: u64, pub storage_params: Option, + + /// Maximum percentage of the global local spill quota that a single + /// sort operator may use for one query. + /// + /// Value range: 0-100. Effective only when local spill is enabled + /// (i.e. there is a valid local spill path and non-zero global + /// bytes limit). + pub sort_spilling_disk_quota_ratio: u64, + + /// Maximum percentage of the global local spill quota that window + /// partitioners may use for one query. + pub window_partition_spilling_disk_quota_ratio: u64, + + /// Maximum percentage of the global local spill quota that HTTP + /// result-set spilling may use for one query. + pub result_set_spilling_disk_quota_ratio: u64, } impl SpillConfig { @@ -807,6 +823,44 @@ impl SpillConfig { None } + /// Helper to compute a per-query local spill quota (in bytes) from a + /// percentage of the global local spill limit. + /// + /// - If local spill is disabled (no local path or zero global + /// limit), returns 0. + /// - `ratio` is clamped into [0, 100]. + pub fn quota_bytes_from_ratio(&self, ratio: u64) -> usize { + // Only effective when local spill is enabled. + if self.local_path().is_none() { + return 0; + } + + let ratio = std::cmp::min(ratio, 100); + if ratio == 0 { + return 0; + } + + let bytes = self.global_bytes_limit.saturating_mul(ratio) / 100; + + // TempDirManager works with `usize` limits. + std::cmp::min(bytes, usize::MAX as u64) as usize + } + + /// Per-query quota for sort operators. + pub fn sort_spill_bytes_limit(&self) -> usize { + self.quota_bytes_from_ratio(self.sort_spilling_disk_quota_ratio) + } + + /// Per-query quota for window partitioners. + pub fn window_partition_spill_bytes_limit(&self) -> usize { + self.quota_bytes_from_ratio(self.window_partition_spilling_disk_quota_ratio) + } + + /// Per-query quota for HTTP result-set spilling. + pub fn result_set_spill_bytes_limit(&self) -> usize { + self.quota_bytes_from_ratio(self.result_set_spilling_disk_quota_ratio) + } + pub fn new_for_test(path: String, reserved_disk_ratio: f64, global_bytes_limit: u64) -> Self { Self { local_writeable_root: None, @@ -814,6 +868,10 @@ impl SpillConfig { reserved_disk_ratio: OrderedFloat(reserved_disk_ratio), global_bytes_limit, storage_params: None, + // Use the same defaults as the external config. + sort_spilling_disk_quota_ratio: 60, + window_partition_spilling_disk_quota_ratio: 20, + result_set_spilling_disk_quota_ratio: 10, } } } @@ -826,6 +884,9 @@ impl Default for SpillConfig { reserved_disk_ratio: OrderedFloat(0.3), global_bytes_limit: u64::MAX, storage_params: None, + sort_spilling_disk_quota_ratio: 60, + window_partition_spilling_disk_quota_ratio: 20, + result_set_spilling_disk_quota_ratio: 10, } } } diff --git a/src/query/config/src/mask.rs b/src/query/config/src/mask.rs index 20a5247dee335..0997f950c92e8 100644 --- a/src/query/config/src/mask.rs +++ b/src/query/config/src/mask.rs @@ -215,6 +215,9 @@ impl SpillConfig { ref spill_local_disk_path, spill_local_disk_reserved_space_percentage, spill_local_disk_max_bytes, + sort_spilling_disk_quota_ratio, + window_partition_spilling_disk_quota_ratio, + result_set_spilling_disk_quota_ratio, } = *self; Self { @@ -222,6 +225,9 @@ impl SpillConfig { spill_local_disk_path: spill_local_disk_path.clone(), spill_local_disk_reserved_space_percentage, spill_local_disk_max_bytes, + sort_spilling_disk_quota_ratio, + window_partition_spilling_disk_quota_ratio, + result_set_spilling_disk_quota_ratio, } } } diff --git a/src/query/service/src/physical_plans/physical_recluster.rs b/src/query/service/src/physical_plans/physical_recluster.rs index ff904e44abec5..50111a2533d51 100644 --- a/src/query/service/src/physical_plans/physical_recluster.rs +++ b/src/query/service/src/physical_plans/physical_recluster.rs @@ -21,6 +21,7 @@ use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::plan::ReclusterTask; use databend_common_catalog::table::Table; use databend_common_catalog::table_context::TableContext; +use databend_common_config::GlobalConfig; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::DataSchemaRef; @@ -319,9 +320,12 @@ impl IPhysicalPlan for HilbertPartition { )?; let settings = builder.settings.clone(); - let disk_bytes_limit = settings.get_window_partition_spilling_to_disk_bytes_limit()?; let temp_dir_manager = TempDirManager::instance(); + let disk_bytes_limit = GlobalConfig::instance() + .spill + .window_partition_spill_bytes_limit(); + let enable_dio = settings.get_enable_dio()?; let disk_spill = temp_dir_manager .get_disk_spill_dir(disk_bytes_limit, &builder.ctx.get_id()) diff --git a/src/query/service/src/physical_plans/physical_window_partition.rs b/src/query/service/src/physical_plans/physical_window_partition.rs index 0da66a86e595e..7158a06df5c53 100644 --- a/src/query/service/src/physical_plans/physical_window_partition.rs +++ b/src/query/service/src/physical_plans/physical_window_partition.rs @@ -18,6 +18,7 @@ use std::sync::atomic::AtomicUsize; use databend_common_catalog::plan::DataSourcePlan; use databend_common_catalog::table_context::TableContext; +use databend_common_config::GlobalConfig; use databend_common_exception::Result; use databend_common_expression::SortColumnDescription; use databend_common_pipeline::core::ProcessorPtr; @@ -148,7 +149,9 @@ impl IPhysicalPlan for WindowPartition { } let temp_dir_manager = TempDirManager::instance(); - let disk_bytes_limit = settings.get_window_partition_spilling_to_disk_bytes_limit()?; + let disk_bytes_limit = GlobalConfig::instance() + .spill + .window_partition_spill_bytes_limit(); let enable_dio = settings.get_enable_dio()?; let disk_spill = temp_dir_manager .get_disk_spill_dir(disk_bytes_limit, &builder.ctx.get_id()) diff --git a/src/query/service/src/pipelines/builders/builder_sort.rs b/src/query/service/src/pipelines/builders/builder_sort.rs index dc6f6f06d3d3e..d397e71f41463 100644 --- a/src/query/service/src/pipelines/builders/builder_sort.rs +++ b/src/query/service/src/pipelines/builders/builder_sort.rs @@ -14,6 +14,7 @@ use std::sync::Arc; +use databend_common_config::GlobalConfig; use databend_common_exception::Result; use databend_common_expression::DataSchemaRef; use databend_common_expression::LimitType; @@ -132,7 +133,7 @@ impl SortPipelineBuilder { let spiller = { let temp_dir_manager = TempDirManager::instance(); - let disk_bytes_limit = settings.get_sort_spilling_to_disk_bytes_limit()?; + let disk_bytes_limit = GlobalConfig::instance().spill.sort_spill_bytes_limit(); let enable_dio = settings.get_enable_dio()?; let disk_spill = temp_dir_manager .get_disk_spill_dir(disk_bytes_limit, &self.ctx.get_id()) diff --git a/src/query/service/src/servers/http/v1/query/execute_state.rs b/src/query/service/src/servers/http/v1/query/execute_state.rs index fd0b1d3b13b0f..44fed5883b7fd 100644 --- a/src/query/service/src/servers/http/v1/query/execute_state.rs +++ b/src/query/service/src/servers/http/v1/query/execute_state.rs @@ -19,6 +19,7 @@ use std::time::SystemTime; use databend_common_base::base::ProgressValues; use databend_common_base::base::SpillProgress; use databend_common_base::runtime::CatchUnwindFuture; +use databend_common_config::GlobalConfig; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_exception::ResultExt; @@ -530,7 +531,9 @@ impl ExecuteState { let spiller = if settings.get_enable_result_set_spilling()? { let temp_dir_manager = TempDirManager::instance(); - let disk_bytes_limit = settings.get_result_set_spilling_to_disk_bytes_limit()?; + let disk_bytes_limit = GlobalConfig::instance() + .spill + .result_set_spill_bytes_limit(); let enable_dio = settings.get_enable_dio()?; let disk_spill = temp_dir_manager .get_disk_spill_dir(disk_bytes_limit, &ctx.get_id()) diff --git a/src/query/settings/src/settings_default.rs b/src/query/settings/src/settings_default.rs index aeaece5d38c0a..aa38a68805fdd 100644 --- a/src/query/settings/src/settings_default.rs +++ b/src/query/settings/src/settings_default.rs @@ -671,13 +671,6 @@ impl DefaultSettings { scope: SettingScope::Both, range: Some(SettingRange::Numeric(0..=100)), }), - ("window_partition_spilling_to_disk_bytes_limit", DefaultSettingValue { - value: UserSettingValue::UInt64(0), - desc: "Sets the maximum amount of local disk in bytes that each window partitioner can use before spilling data to storage during query execution.", - mode: SettingMode::Both, - scope: SettingScope::Both, - range: Some(SettingRange::Numeric(0..=u64::MAX)), - }), ("window_num_partitions", DefaultSettingValue { value: UserSettingValue::UInt64(256), desc: "Sets the number of partitions for window operator.", @@ -706,20 +699,6 @@ impl DefaultSettings { scope: SettingScope::Both, range: Some(SettingRange::Numeric(0..=100)), }), - ("sort_spilling_to_disk_bytes_limit", DefaultSettingValue { - value: UserSettingValue::UInt64(0), - desc: "Sets the maximum amount of local disk in bytes that sorter can use before spilling data to storage during one query execution.", - mode: SettingMode::Both, - scope: SettingScope::Both, - range: Some(SettingRange::Numeric(0..=u64::MAX)), - }), - ("result_set_spilling_to_disk_bytes_limit", DefaultSettingValue { - value: UserSettingValue::UInt64(0), - desc: "Sets the maximum amount of local disk in bytes that result set can use before spilling data to storage during one query execution.", - mode: SettingMode::Both, - scope: SettingScope::Both, - range: Some(SettingRange::Numeric(0..=u64::MAX)), - }), ("enable_result_set_spilling", DefaultSettingValue { value: UserSettingValue::UInt64(0), desc: "Enable spilling result set data to storage when memory usage exceeds the threshold.", diff --git a/src/query/settings/src/settings_getter_setter.rs b/src/query/settings/src/settings_getter_setter.rs index 6057058270e25..2cf2d7453ff32 100644 --- a/src/query/settings/src/settings_getter_setter.rs +++ b/src/query/settings/src/settings_getter_setter.rs @@ -497,10 +497,6 @@ impl Settings { Ok(self.try_get_u64("aggregate_spilling_memory_ratio")? as usize) } - pub fn get_window_partition_spilling_to_disk_bytes_limit(&self) -> Result { - Ok(self.try_get_u64("window_partition_spilling_to_disk_bytes_limit")? as usize) - } - pub fn get_window_partition_spilling_memory_ratio(&self) -> Result { Ok(self.try_get_u64("window_partition_spilling_memory_ratio")? as usize) } @@ -525,14 +521,6 @@ impl Settings { Ok(self.try_get_u64("sort_spilling_memory_ratio")? as usize) } - pub fn get_sort_spilling_to_disk_bytes_limit(&self) -> Result { - Ok(self.try_get_u64("sort_spilling_to_disk_bytes_limit")? as usize) - } - - pub fn get_result_set_spilling_to_disk_bytes_limit(&self) -> Result { - Ok(self.try_get_u64("result_set_spilling_to_disk_bytes_limit")? as usize) - } - pub fn get_enable_result_set_spilling(&self) -> Result { Ok(self.try_get_u64("enable_result_set_spilling")? == 1) } diff --git a/tests/sqllogictests/suites/query/issues/issue_17581.test b/tests/sqllogictests/suites/query/issues/issue_17581.test index 90aee603129e4..6fb2a3cfaf33b 100644 --- a/tests/sqllogictests/suites/query/issues/issue_17581.test +++ b/tests/sqllogictests/suites/query/issues/issue_17581.test @@ -7,9 +7,6 @@ USE test_17581 statement ok SET force_window_data_spill = 1; -statement ok -set window_partition_spilling_to_disk_bytes_limit = 1024 * 1024 * 1024; - statement ok SET force_sort_data_spill = 1; diff --git a/tests/sqllogictests/suites/query/window_function/window_partition_spill.test b/tests/sqllogictests/suites/query/window_function/window_partition_spill.test index cd0613020b8ea..92c93ef474248 100644 --- a/tests/sqllogictests/suites/query/window_function/window_partition_spill.test +++ b/tests/sqllogictests/suites/query/window_function/window_partition_spill.test @@ -7,9 +7,6 @@ USE test_window_partition_spill statement ok SET force_window_data_spill = 1; -statement ok -set window_partition_spilling_to_disk_bytes_limit = 1024 * 1024 * 1024; - statement ok set enable_dio = 1; @@ -38,6 +35,9 @@ FROM ( ---- 14999849576 +statement ok +unset enable_dio; + statement ok set enable_dio = 0; @@ -66,9 +66,6 @@ FROM ( ---- 14999849576 -statement ok -unset enable_dio; - statement ok DROP TABLE IF EXISTS customers; diff --git a/tests/sqllogictests/suites/tpcds/window_spill.test b/tests/sqllogictests/suites/tpcds/window_spill.test index e11627bd572bd..522ccda258b3f 100644 --- a/tests/sqllogictests/suites/tpcds/window_spill.test +++ b/tests/sqllogictests/suites/tpcds/window_spill.test @@ -8,9 +8,6 @@ use tpcds; statement ok set max_memory_usage = 1024*1024*200; -statement ok -set window_partition_spilling_to_disk_bytes_limit = 1024 * 1024 * 1024; - statement ok set max_block_size = 2; @@ -30,6 +27,3 @@ unset max_block_size; statement ok UNSET max_memory_usage; - -statement ok -unset window_partition_spilling_to_disk_bytes_limit; diff --git a/tests/sqllogictests/suites/tpch/spill.test b/tests/sqllogictests/suites/tpch/spill.test index c6aa2cea290b0..c624d7918acae 100644 --- a/tests/sqllogictests/suites/tpch/spill.test +++ b/tests/sqllogictests/suites/tpch/spill.test @@ -20,9 +20,6 @@ SET force_aggregate_data_spill = 1; statement ok set join_spilling_buffer_threshold_per_proc_mb = 1; -statement ok -set window_partition_spilling_to_disk_bytes_limit = 1024 * 1024 * 1024; - # TPC-H TEST include ./queries.test From f9d68fa6f274a2c5e4de7d7dfd1d66c42781d616 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Tue, 9 Dec 2025 08:57:59 +0800 Subject: [PATCH 21/37] Fix spill config mask test ratios --- src/query/config/src/mask.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/query/config/src/mask.rs b/src/query/config/src/mask.rs index 0997f950c92e8..87136005ab7e7 100644 --- a/src/query/config/src/mask.rs +++ b/src/query/config/src/mask.rs @@ -381,6 +381,9 @@ mod tests { spill_local_disk_path: "".to_string(), spill_local_disk_reserved_space_percentage: 30.0.into(), spill_local_disk_max_bytes: 10, + sort_spilling_disk_quota_ratio: 60, + window_partition_spilling_disk_quota_ratio: 20, + result_set_spilling_disk_quota_ratio: 10, storage: Some(StorageConfig { typ: "s3".to_string(), s3: S3StorageConfig { From fc1b541217c9b5564ea13b34c9ad69878b3e30c4 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Tue, 9 Dec 2025 11:31:30 +0800 Subject: [PATCH 22/37] Refine spill profiling and local writer --- .../partition/window_partition_buffer_v2.rs | 2 +- src/query/service/src/spillers/adapter.rs | 16 +-- .../service/src/spillers/async_buffer.rs | 8 +- src/query/service/src/spillers/inner.rs | 117 +++++++++++++----- .../service/src/spillers/row_group_encoder.rs | 6 +- .../tests/it/spillers/spill_profile.rs | 13 +- .../storages/testdata/configs_table_basic.txt | 3 + 7 files changed, 109 insertions(+), 56 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_buffer_v2.rs b/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_buffer_v2.rs index eba1d66abb0e3..947b6899ad769 100644 --- a/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_buffer_v2.rs +++ b/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_buffer_v2.rs @@ -59,7 +59,7 @@ impl Writer for SpillWriter { fn need_new_file(&mut self, incoming_size: usize) -> Result { Ok(match self.file_writer() { - AnyFileWriter::Local { writer, .. } => !writer.check_grow(incoming_size, true)?, + AnyFileWriter::Local { writer } => !writer.check_grow(incoming_size, true)?, _ => false, }) } diff --git a/src/query/service/src/spillers/adapter.rs b/src/query/service/src/spillers/adapter.rs index 2941690bcb72b..9b458172837cc 100644 --- a/src/query/service/src/spillers/adapter.rs +++ b/src/query/service/src/spillers/adapter.rs @@ -36,6 +36,7 @@ use opendal::Buffer; use opendal::Operator; use parquet::file::metadata::RowGroupMetaDataPtr; +use super::async_buffer::SpillTarget; use super::block_reader::BlocksReader; use super::block_writer::BlocksWriter; use super::inner::*; @@ -485,18 +486,18 @@ impl SpillWriter { let start = std::time::Instant::now(); match &mut self.file_writer { - AnyFileWriter::Local { path, writer } => { + AnyFileWriter::Local { writer } => { let row_group_meta = writer.flush_row_group(row_group)?; let size = row_group_meta.compressed_size() as _; self.spiller.adapter.update_progress(0, size); - record_write_profile(&Location::Local(path.clone()), &start, size); + record_write_profile(SpillTarget::Local, &start, size); Ok(row_group_meta) } - AnyFileWriter::Remote { path, writer } => { + AnyFileWriter::Remote { path: _path, writer } => { let row_group_meta = writer.flush_row_group(row_group)?; let size = row_group_meta.compressed_size() as _; self.spiller.adapter.update_progress(0, size); - record_write_profile(&Location::Remote(path.clone()), &start, size); + record_write_profile(SpillTarget::Remote, &start, size); Ok(row_group_meta) } } @@ -508,12 +509,13 @@ impl SpillWriter { pub fn close(self) -> Result { let (metadata, location) = match self.file_writer { - AnyFileWriter::Local { writer, .. } => { + AnyFileWriter::Local { writer } => { let (metadata, path) = writer.finish()?; + let location = Location::Local(path); self.spiller .adapter - .add_spill_file(Location::Local(path.clone()), Layout::Parquet); - (metadata, Location::Local(path)) + .add_spill_file(location.clone(), Layout::Parquet); + (metadata, location) } AnyFileWriter::Remote { path, writer } => { let (metadata, _) = writer.finish()?; diff --git a/src/query/service/src/spillers/async_buffer.rs b/src/query/service/src/spillers/async_buffer.rs index 79950f8aa9490..976ba679b0c53 100644 --- a/src/query/service/src/spillers/async_buffer.rs +++ b/src/query/service/src/spillers/async_buffer.rs @@ -58,8 +58,8 @@ use parquet::file::metadata::RowGroupMetaData; use parquet::file::properties::EnabledStatistics; use parquet::file::properties::WriterProperties; -use super::record_read_profile_with_flag; -use super::record_write_profile_with_flag; +use super::record_read_profile; +use super::record_write_profile; const CHUNK_SIZE: usize = 4 * 1024 * 1024; #[derive(Clone, Copy)] @@ -660,7 +660,7 @@ impl SpillsDataReader { )?; let batch = reader.next().transpose()?.unwrap(); debug_assert!(reader.next().is_none()); - record_read_profile_with_flag(self.target.is_local(), &start, read_bytes.get()); + record_read_profile(self.target, &start, read_bytes.get()); Ok(Some(DataBlock::from_record_batch( &self.data_schema, &batch, @@ -805,7 +805,7 @@ impl Background { writer: op.writer, }); - record_write_profile_with_flag(op.target.is_local(), &start, bytes_len); + record_write_profile(op.target, &start, bytes_len); } BufferOperator::Close(mut op) => { let res = op.writer.close().await; diff --git a/src/query/service/src/spillers/inner.rs b/src/query/service/src/spillers/inner.rs index c986b305fdae5..10b5d295a881f 100644 --- a/src/query/service/src/spillers/inner.rs +++ b/src/query/service/src/spillers/inner.rs @@ -31,6 +31,7 @@ use opendal::services::Fs; use opendal::Buffer; use opendal::Operator; +use super::async_buffer::SpillTarget; use super::serialize::*; use super::Location; @@ -275,46 +276,94 @@ impl SpillerInner { } } -pub fn record_write_profile_with_flag(is_local: bool, start: &Instant, write_bytes: usize) { - if !is_local { - Profile::record_usize_profile(ProfileStatisticsName::RemoteSpillWriteCount, 1); - Profile::record_usize_profile(ProfileStatisticsName::RemoteSpillWriteBytes, write_bytes); - Profile::record_usize_profile( - ProfileStatisticsName::RemoteSpillWriteTime, - start.elapsed().as_millis() as usize, - ); - } else { - Profile::record_usize_profile(ProfileStatisticsName::LocalSpillWriteCount, 1); - Profile::record_usize_profile(ProfileStatisticsName::LocalSpillWriteBytes, write_bytes); - Profile::record_usize_profile( - ProfileStatisticsName::LocalSpillWriteTime, - start.elapsed().as_millis() as usize, - ); +#[derive(Clone, Copy)] +pub enum SpillLocality { + Local, + Remote, +} + +impl From<&Location> for SpillLocality { + fn from(value: &Location) -> Self { + if value.is_local() { + SpillLocality::Local + } else { + SpillLocality::Remote + } } } -pub fn record_write_profile(location: &Location, start: &Instant, write_bytes: usize) { - record_write_profile_with_flag(location.is_local(), start, write_bytes) +impl From for SpillLocality { + fn from(target: SpillTarget) -> Self { + match target { + SpillTarget::Local => SpillLocality::Local, + SpillTarget::Remote => SpillLocality::Remote, + } + } } -pub fn record_read_profile_with_flag(is_local: bool, start: &Instant, read_bytes: usize) { - if is_local { - Profile::record_usize_profile(ProfileStatisticsName::LocalSpillReadCount, 1); - Profile::record_usize_profile(ProfileStatisticsName::LocalSpillReadBytes, read_bytes); - Profile::record_usize_profile( - ProfileStatisticsName::LocalSpillReadTime, - start.elapsed().as_millis() as usize, - ); - } else { - Profile::record_usize_profile(ProfileStatisticsName::RemoteSpillReadCount, 1); - Profile::record_usize_profile(ProfileStatisticsName::RemoteSpillReadBytes, read_bytes); - Profile::record_usize_profile( - ProfileStatisticsName::RemoteSpillReadTime, - start.elapsed().as_millis() as usize, - ); +impl From<&SpillTarget> for SpillLocality { + fn from(target: &SpillTarget) -> Self { + (*target).into() } } -pub fn record_read_profile(location: &Location, start: &Instant, read_bytes: usize) { - record_read_profile_with_flag(location.is_local(), start, read_bytes) +fn record_spill_profile( + locality: SpillLocality, + start: &Instant, + bytes: usize, + local_count: ProfileStatisticsName, + local_bytes: ProfileStatisticsName, + local_time: ProfileStatisticsName, + remote_count: ProfileStatisticsName, + remote_bytes: ProfileStatisticsName, + remote_time: ProfileStatisticsName, +) { + match locality { + SpillLocality::Local => { + Profile::record_usize_profile(local_count, 1); + Profile::record_usize_profile(local_bytes, bytes); + Profile::record_usize_profile(local_time, start.elapsed().as_millis() as usize); + } + SpillLocality::Remote => { + Profile::record_usize_profile(remote_count, 1); + Profile::record_usize_profile(remote_bytes, bytes); + Profile::record_usize_profile(remote_time, start.elapsed().as_millis() as usize); + } + } +} + +pub fn record_write_profile>( + locality: T, + start: &Instant, + write_bytes: usize, +) { + record_spill_profile( + locality.into(), + start, + write_bytes, + ProfileStatisticsName::LocalSpillWriteCount, + ProfileStatisticsName::LocalSpillWriteBytes, + ProfileStatisticsName::LocalSpillWriteTime, + ProfileStatisticsName::RemoteSpillWriteCount, + ProfileStatisticsName::RemoteSpillWriteBytes, + ProfileStatisticsName::RemoteSpillWriteTime, + ); +} + +pub fn record_read_profile>( + locality: T, + start: &Instant, + read_bytes: usize, +) { + record_spill_profile( + locality.into(), + start, + read_bytes, + ProfileStatisticsName::LocalSpillReadCount, + ProfileStatisticsName::LocalSpillReadBytes, + ProfileStatisticsName::LocalSpillReadTime, + ProfileStatisticsName::RemoteSpillReadCount, + ProfileStatisticsName::RemoteSpillReadBytes, + ProfileStatisticsName::RemoteSpillReadTime, + ); } diff --git a/src/query/service/src/spillers/row_group_encoder.rs b/src/query/service/src/spillers/row_group_encoder.rs index 829b239423868..faf17dc5fcb26 100644 --- a/src/query/service/src/spillers/row_group_encoder.rs +++ b/src/query/service/src/spillers/row_group_encoder.rs @@ -413,7 +413,6 @@ impl RangeFetchPlan { pub enum AnyFileWriter { Local { - path: TempPath, writer: FileWriter, }, Remote { @@ -425,7 +424,7 @@ pub enum AnyFileWriter { impl AnyFileWriter { pub(super) fn new_row_group(&self) -> RowGroupEncoder { match self { - AnyFileWriter::Local { writer, .. } => writer.new_row_group(), + AnyFileWriter::Local { writer } => writer.new_row_group(), AnyFileWriter::Remote { writer, .. } => writer.new_row_group(), } } @@ -455,8 +454,7 @@ impl SpillerInner { buf, }; let writer = FileWriter::new(props, w)?; - let path = writer.writer.inner().path.clone(); - return Ok(AnyFileWriter::Local { path, writer }); + return Ok(AnyFileWriter::Local { writer }); } }; diff --git a/src/query/service/tests/it/spillers/spill_profile.rs b/src/query/service/tests/it/spillers/spill_profile.rs index 43581b17992b1..1cbca38ebf669 100644 --- a/src/query/service/tests/it/spillers/spill_profile.rs +++ b/src/query/service/tests/it/spillers/spill_profile.rs @@ -20,8 +20,9 @@ use std::time::Instant; use databend_common_base::runtime::profile::Profile; use databend_common_base::runtime::profile::ProfileStatisticsName; use databend_common_base::runtime::ThreadTracker; -use databend_query::spillers::record_read_profile_with_flag; -use databend_query::spillers::record_write_profile_with_flag; +use databend_query::spillers::record_read_profile; +use databend_query::spillers::record_write_profile; +use databend_query::spillers::SpillLocality; fn create_test_profile() -> Arc { Arc::new(Profile::create( @@ -48,7 +49,7 @@ fn test_spill_profile_write_local_and_remote() { { let _guard = ThreadTracker::tracking(payload_local); let start = Instant::now(); - record_write_profile_with_flag(true, &start, 128); + record_write_profile(SpillLocality::Local, &start, 128); } assert_eq!( @@ -68,7 +69,7 @@ fn test_spill_profile_write_local_and_remote() { { let _guard = ThreadTracker::tracking(payload_remote); let start = Instant::now(); - record_write_profile_with_flag(false, &start, 256); + record_write_profile(SpillLocality::Remote, &start, 256); } assert_eq!( @@ -93,7 +94,7 @@ fn test_spill_profile_read_local_and_remote() { { let _guard = ThreadTracker::tracking(payload_local); let start = Instant::now(); - record_read_profile_with_flag(true, &start, 64); + record_read_profile(SpillLocality::Local, &start, 64); } assert_eq!( @@ -113,7 +114,7 @@ fn test_spill_profile_read_local_and_remote() { { let _guard = ThreadTracker::tracking(payload_remote); let start = Instant::now(); - record_read_profile_with_flag(false, &start, 512); + record_read_profile(SpillLocality::Remote, &start, 512); } assert_eq!( diff --git a/src/query/service/tests/it/storages/testdata/configs_table_basic.txt b/src/query/service/tests/it/storages/testdata/configs_table_basic.txt index 8f64dccea60b2..4da362e941f81 100644 --- a/src/query/service/tests/it/storages/testdata/configs_table_basic.txt +++ b/src/query/service/tests/it/storages/testdata/configs_table_basic.txt @@ -224,10 +224,13 @@ DB.Table: 'system'.'configs', Table: configs-table_id:1, ver:0, Engine: SystemCo | 'query' | 'udfs' | '{"name":"test_builtin_ping","definition":"CREATE OR REPLACE FUNCTION test_builtin_ping (STRING)\\n RETURNS STRING\\n LANGUAGE python\\nHANDLER = \'ping\'\\nADDRESS = \'https://databend.com\';"}' | '' | | 'query' | 'users' | '{"name":"root","auth_type":"no_password","auth_string":null}' | '' | | 'query' | 'warehouse_id' | 'test_warehouse' | '' | +| 'spill' | 'result_set_spilling_disk_quota_ratio' | '10' | '' | +| 'spill' | 'sort_spilling_disk_quota_ratio' | '60' | '' | | 'spill' | 'spill_local_disk_max_bytes' | '18446744073709551615' | '' | | 'spill' | 'spill_local_disk_path' | '' | '' | | 'spill' | 'spill_local_disk_reserved_space_percentage' | '30.0' | '' | | 'spill' | 'storage' | 'null' | '' | +| 'spill' | 'window_partition_spilling_disk_quota_ratio' | '20' | '' | | 'storage' | 'allow_insecure' | 'true' | '' | | 'storage' | 'azblob.account_key' | '' | '' | | 'storage' | 'azblob.account_name' | '' | '' | From 03bd04143102a8884feb0385e8fa09fff64204c6 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Tue, 9 Dec 2025 11:53:28 +0800 Subject: [PATCH 23/37] Unify spill target and add local path string --- .../partition/window_partition_buffer_v2.rs | 2 +- src/query/service/src/spillers/adapter.rs | 9 ++-- src/query/service/src/spillers/inner.rs | 41 +++++-------------- .../service/src/spillers/row_group_encoder.rs | 12 +++++- .../tests/it/spillers/spill_profile.rs | 10 ++--- 5 files changed, 33 insertions(+), 41 deletions(-) diff --git a/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_buffer_v2.rs b/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_buffer_v2.rs index 947b6899ad769..eba1d66abb0e3 100644 --- a/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_buffer_v2.rs +++ b/src/query/service/src/pipelines/processors/transforms/window/partition/window_partition_buffer_v2.rs @@ -59,7 +59,7 @@ impl Writer for SpillWriter { fn need_new_file(&mut self, incoming_size: usize) -> Result { Ok(match self.file_writer() { - AnyFileWriter::Local { writer } => !writer.check_grow(incoming_size, true)?, + AnyFileWriter::Local { writer, .. } => !writer.check_grow(incoming_size, true)?, _ => false, }) } diff --git a/src/query/service/src/spillers/adapter.rs b/src/query/service/src/spillers/adapter.rs index 9b458172837cc..4cb801fe318fe 100644 --- a/src/query/service/src/spillers/adapter.rs +++ b/src/query/service/src/spillers/adapter.rs @@ -486,14 +486,17 @@ impl SpillWriter { let start = std::time::Instant::now(); match &mut self.file_writer { - AnyFileWriter::Local { writer } => { + AnyFileWriter::Local { writer, .. } => { let row_group_meta = writer.flush_row_group(row_group)?; let size = row_group_meta.compressed_size() as _; self.spiller.adapter.update_progress(0, size); record_write_profile(SpillTarget::Local, &start, size); Ok(row_group_meta) } - AnyFileWriter::Remote { path: _path, writer } => { + AnyFileWriter::Remote { + path: _path, + writer, + } => { let row_group_meta = writer.flush_row_group(row_group)?; let size = row_group_meta.compressed_size() as _; self.spiller.adapter.update_progress(0, size); @@ -509,7 +512,7 @@ impl SpillWriter { pub fn close(self) -> Result { let (metadata, location) = match self.file_writer { - AnyFileWriter::Local { writer } => { + AnyFileWriter::Local { writer, .. } => { let (metadata, path) = writer.finish()?; let location = Location::Local(path); self.spiller diff --git a/src/query/service/src/spillers/inner.rs b/src/query/service/src/spillers/inner.rs index 10b5d295a881f..9b4fe414424e7 100644 --- a/src/query/service/src/spillers/inner.rs +++ b/src/query/service/src/spillers/inner.rs @@ -276,39 +276,24 @@ impl SpillerInner { } } -#[derive(Clone, Copy)] -pub enum SpillLocality { - Local, - Remote, -} - -impl From<&Location> for SpillLocality { +impl From<&Location> for SpillTarget { fn from(value: &Location) -> Self { if value.is_local() { - SpillLocality::Local + SpillTarget::Local } else { - SpillLocality::Remote + SpillTarget::Remote } } } -impl From for SpillLocality { - fn from(target: SpillTarget) -> Self { - match target { - SpillTarget::Local => SpillLocality::Local, - SpillTarget::Remote => SpillLocality::Remote, - } - } -} - -impl From<&SpillTarget> for SpillLocality { - fn from(target: &SpillTarget) -> Self { - (*target).into() +impl From for SpillTarget { + fn from(value: Location) -> Self { + (&value).into() } } fn record_spill_profile( - locality: SpillLocality, + locality: SpillTarget, start: &Instant, bytes: usize, local_count: ProfileStatisticsName, @@ -319,12 +304,12 @@ fn record_spill_profile( remote_time: ProfileStatisticsName, ) { match locality { - SpillLocality::Local => { + SpillTarget::Local => { Profile::record_usize_profile(local_count, 1); Profile::record_usize_profile(local_bytes, bytes); Profile::record_usize_profile(local_time, start.elapsed().as_millis() as usize); } - SpillLocality::Remote => { + SpillTarget::Remote => { Profile::record_usize_profile(remote_count, 1); Profile::record_usize_profile(remote_bytes, bytes); Profile::record_usize_profile(remote_time, start.elapsed().as_millis() as usize); @@ -332,7 +317,7 @@ fn record_spill_profile( } } -pub fn record_write_profile>( +pub fn record_write_profile>( locality: T, start: &Instant, write_bytes: usize, @@ -350,11 +335,7 @@ pub fn record_write_profile>( ); } -pub fn record_read_profile>( - locality: T, - start: &Instant, - read_bytes: usize, -) { +pub fn record_read_profile>(locality: T, start: &Instant, read_bytes: usize) { record_spill_profile( locality.into(), start, diff --git a/src/query/service/src/spillers/row_group_encoder.rs b/src/query/service/src/spillers/row_group_encoder.rs index faf17dc5fcb26..f832c69e5865d 100644 --- a/src/query/service/src/spillers/row_group_encoder.rs +++ b/src/query/service/src/spillers/row_group_encoder.rs @@ -413,6 +413,9 @@ impl RangeFetchPlan { pub enum AnyFileWriter { Local { + /// Absolute path string for symmetry with remote; TempPath is kept + /// inside the writer to avoid sharing/cloning while writing. + path: String, writer: FileWriter, }, Remote { @@ -424,7 +427,7 @@ pub enum AnyFileWriter { impl AnyFileWriter { pub(super) fn new_row_group(&self) -> RowGroupEncoder { match self { - AnyFileWriter::Local { writer } => writer.new_row_group(), + AnyFileWriter::Local { writer, .. } => writer.new_row_group(), AnyFileWriter::Remote { writer, .. } => writer.new_row_group(), } } @@ -447,6 +450,8 @@ impl SpillerInner { let align = dir.block_alignment(); let buf = DmaWriteBuf::new(align, chunk); + let path_str = path.as_ref().display().to_string(); + let w = LocalWriter { dir: dir.clone(), path, @@ -454,7 +459,10 @@ impl SpillerInner { buf, }; let writer = FileWriter::new(props, w)?; - return Ok(AnyFileWriter::Local { writer }); + return Ok(AnyFileWriter::Local { + path: path_str, + writer, + }); } }; diff --git a/src/query/service/tests/it/spillers/spill_profile.rs b/src/query/service/tests/it/spillers/spill_profile.rs index 1cbca38ebf669..d097e7f273908 100644 --- a/src/query/service/tests/it/spillers/spill_profile.rs +++ b/src/query/service/tests/it/spillers/spill_profile.rs @@ -22,7 +22,7 @@ use databend_common_base::runtime::profile::ProfileStatisticsName; use databend_common_base::runtime::ThreadTracker; use databend_query::spillers::record_read_profile; use databend_query::spillers::record_write_profile; -use databend_query::spillers::SpillLocality; +use databend_query::spillers::SpillTarget; fn create_test_profile() -> Arc { Arc::new(Profile::create( @@ -49,7 +49,7 @@ fn test_spill_profile_write_local_and_remote() { { let _guard = ThreadTracker::tracking(payload_local); let start = Instant::now(); - record_write_profile(SpillLocality::Local, &start, 128); + record_write_profile(SpillTarget::Local, &start, 128); } assert_eq!( @@ -69,7 +69,7 @@ fn test_spill_profile_write_local_and_remote() { { let _guard = ThreadTracker::tracking(payload_remote); let start = Instant::now(); - record_write_profile(SpillLocality::Remote, &start, 256); + record_write_profile(SpillTarget::Remote, &start, 256); } assert_eq!( @@ -94,7 +94,7 @@ fn test_spill_profile_read_local_and_remote() { { let _guard = ThreadTracker::tracking(payload_local); let start = Instant::now(); - record_read_profile(SpillLocality::Local, &start, 64); + record_read_profile(SpillTarget::Local, &start, 64); } assert_eq!( @@ -114,7 +114,7 @@ fn test_spill_profile_read_local_and_remote() { { let _guard = ThreadTracker::tracking(payload_remote); let start = Instant::now(); - record_read_profile(SpillLocality::Remote, &start, 512); + record_read_profile(SpillTarget::Remote, &start, 512); } assert_eq!( From 1eb5c59cc28d97507394d6b9fd36310b9a94ff8a Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Tue, 9 Dec 2025 14:18:08 +0800 Subject: [PATCH 24/37] Add debug logging for spill temp cleanup --- .../service/src/servers/http/v1/query/sized_spsc.rs | 11 +++++++++++ src/query/storages/common/cache/src/temp_dir.rs | 12 ++++++++++++ 2 files changed, 23 insertions(+) diff --git a/src/query/service/src/servers/http/v1/query/sized_spsc.rs b/src/query/service/src/servers/http/v1/query/sized_spsc.rs index f5c1911940185..3e389f4506478 100644 --- a/src/query/service/src/servers/http/v1/query/sized_spsc.rs +++ b/src/query/service/src/servers/http/v1/query/sized_spsc.rs @@ -287,6 +287,17 @@ where S: DataBlockSpill let start_time = std::time::Instant::now(); + log::debug!( + target: "result-set-spill", + "[RESULT-SET-SPILL] Spill target acquired, parent exists={}, page_rows={}, page_bytes={}", + match spiller.spill_location_prefix() { + Some(prefix) => std::path::Path::new(&prefix).exists(), + None => false, + }, + rows_count, + memory_bytes + ); + log::info!( target: "result-set-spill", "[RESULT-SET-SPILL] Starting spill to disk blocks={}, rows={}, memory_bytes={}", diff --git a/src/query/storages/common/cache/src/temp_dir.rs b/src/query/storages/common/cache/src/temp_dir.rs index b86fe2411733f..384467d770814 100644 --- a/src/query/storages/common/cache/src/temp_dir.rs +++ b/src/query/storages/common/cache/src/temp_dir.rs @@ -137,6 +137,11 @@ impl TempDirManager { let mut group = self.group.lock().unwrap(); if group.dirs.remove(&path).is_some() { + log::debug!( + target: "spill-tempdir", + "[SPILL-TEMP] drop_disk_spill_dir removing path={:?}", + path + ); match fs::remove_dir_all(&path) { Ok(_) => return Ok(true), Err(e) if matches!(e.kind(), ErrorKind::NotFound) => {} @@ -171,6 +176,13 @@ impl TempDirManager { .take(limit) .collect::>(); drop(group); + if !to_delete.is_empty() { + log::debug!( + target: "spill-tempdir", + "[SPILL-TEMP] drop_disk_spill_dir_unknown removing={:?}", + to_delete + ); + } for path in &to_delete { fs::remove_dir_all(path)?; } From d5885a1611a595a0b7d25bac7a34e6c6f8e7a973 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Tue, 9 Dec 2025 16:01:46 +0800 Subject: [PATCH 25/37] Fix build after removing spill prefix debug --- .../service/src/servers/http/v1/query/sized_spsc.rs | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/src/query/service/src/servers/http/v1/query/sized_spsc.rs b/src/query/service/src/servers/http/v1/query/sized_spsc.rs index 3e389f4506478..f5c1911940185 100644 --- a/src/query/service/src/servers/http/v1/query/sized_spsc.rs +++ b/src/query/service/src/servers/http/v1/query/sized_spsc.rs @@ -287,17 +287,6 @@ where S: DataBlockSpill let start_time = std::time::Instant::now(); - log::debug!( - target: "result-set-spill", - "[RESULT-SET-SPILL] Spill target acquired, parent exists={}, page_rows={}, page_bytes={}", - match spiller.spill_location_prefix() { - Some(prefix) => std::path::Path::new(&prefix).exists(), - None => false, - }, - rows_count, - memory_bytes - ); - log::info!( target: "result-set-spill", "[RESULT-SET-SPILL] Starting spill to disk blocks={}, rows={}, memory_bytes={}", From dcfe9408305ee984bbd00407a2272346ae9dec82 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Tue, 9 Dec 2025 18:41:25 +0800 Subject: [PATCH 26/37] Adjust spill configs and defaults --- .../ci/deploy/config/databend-query-node-1.toml | 2 +- .../configs/databend-query-node2.toml | 3 --- .../configs/databend-query-node3.toml | 3 --- .../configs/databend-query.toml | 3 --- scripts/test-bend-tests/configs/query/query-1.toml | 3 --- scripts/test-bend-tests/configs/query/query-2.toml | 3 --- scripts/test-bend-tests/configs/query/query-3.toml | 3 --- scripts/test-bend-tests/configs/query/query-4.toml | 3 --- scripts/test-bend-tests/configs/query/query-5.toml | 3 --- src/query/config/src/config.rs | 8 +++++--- src/query/config/src/inner.rs | 10 ++++++---- src/query/config/src/mask.rs | 6 ++++-- .../service/src/interpreters/hook/vacuum_hook.rs | 10 ++++++++++ .../src/servers/http/v1/query/http_query.rs | 2 +- .../src/servers/http/v1/query/page_manager.rs | 14 +++++++++++++- src/query/service/tests/it/configs.rs | 2 +- .../it/storages/testdata/configs_table_basic.txt | 6 ++---- 17 files changed, 43 insertions(+), 41 deletions(-) diff --git a/scripts/ci/deploy/config/databend-query-node-1.toml b/scripts/ci/deploy/config/databend-query-node-1.toml index f81d4bc49ac65..78bbe541bd70b 100644 --- a/scripts/ci/deploy/config/databend-query-node-1.toml +++ b/scripts/ci/deploy/config/databend-query-node-1.toml @@ -163,5 +163,5 @@ max_bytes = 21474836480 [spill] spill_local_disk_path = "./.databend/temp/_query_spill" # Cap local spill to 5GB so window spills keep ~1GB quota with default 20% ratio. -spill_local_disk_max_bytes = 5368709120 +spill_local_disk_max_bytes = 1073741824 window_partition_spilling_disk_quota_ratio = 20 diff --git a/scripts/databend_test_helper/databend_test_helper/configs/databend-query-node2.toml b/scripts/databend_test_helper/databend_test_helper/configs/databend-query-node2.toml index 537aefc9f88f0..31226f4fc4e8f 100644 --- a/scripts/databend_test_helper/databend_test_helper/configs/databend-query-node2.toml +++ b/scripts/databend_test_helper/databend_test_helper/configs/databend-query-node2.toml @@ -63,6 +63,3 @@ max_bytes = 1073741824 # 1GB [spill] spill_local_disk_path = "_databend_data/spill/query2" -sort_spilling_disk_quota_ratio = 60 -window_partition_spilling_disk_quota_ratio = 20 -result_set_spilling_disk_quota_ratio = 10 diff --git a/scripts/databend_test_helper/databend_test_helper/configs/databend-query-node3.toml b/scripts/databend_test_helper/databend_test_helper/configs/databend-query-node3.toml index 2c5f73ae0105e..e697e6bc74295 100644 --- a/scripts/databend_test_helper/databend_test_helper/configs/databend-query-node3.toml +++ b/scripts/databend_test_helper/databend_test_helper/configs/databend-query-node3.toml @@ -63,6 +63,3 @@ max_bytes = 1073741824 # 1GB [spill] spill_local_disk_path = "_databend_data/spill/query3" -sort_spilling_disk_quota_ratio = 60 -window_partition_spilling_disk_quota_ratio = 20 -result_set_spilling_disk_quota_ratio = 10 diff --git a/scripts/databend_test_helper/databend_test_helper/configs/databend-query.toml b/scripts/databend_test_helper/databend_test_helper/configs/databend-query.toml index e143f6977e027..c063f0c3888c4 100644 --- a/scripts/databend_test_helper/databend_test_helper/configs/databend-query.toml +++ b/scripts/databend_test_helper/databend_test_helper/configs/databend-query.toml @@ -63,6 +63,3 @@ max_bytes = 1073741824 # 1GB [spill] spill_local_disk_path = "_databend_data/spill/query1" -sort_spilling_disk_quota_ratio = 60 -window_partition_spilling_disk_quota_ratio = 20 -result_set_spilling_disk_quota_ratio = 10 diff --git a/scripts/test-bend-tests/configs/query/query-1.toml b/scripts/test-bend-tests/configs/query/query-1.toml index 58a102020e61d..8f7b03361811a 100644 --- a/scripts/test-bend-tests/configs/query/query-1.toml +++ b/scripts/test-bend-tests/configs/query/query-1.toml @@ -64,6 +64,3 @@ max_bytes = 1073741824 # 1GB [spill] spill_local_disk_path = "_databend_data/spill/query1" -sort_spilling_disk_quota_ratio = 60 -window_partition_spilling_disk_quota_ratio = 20 -result_set_spilling_disk_quota_ratio = 10 diff --git a/scripts/test-bend-tests/configs/query/query-2.toml b/scripts/test-bend-tests/configs/query/query-2.toml index 6992974345598..b2e30bcab830b 100644 --- a/scripts/test-bend-tests/configs/query/query-2.toml +++ b/scripts/test-bend-tests/configs/query/query-2.toml @@ -64,6 +64,3 @@ max_bytes = 1073741824 # 1GB [spill] spill_local_disk_path = "_databend_data/spill/query2" -sort_spilling_disk_quota_ratio = 60 -window_partition_spilling_disk_quota_ratio = 20 -result_set_spilling_disk_quota_ratio = 10 diff --git a/scripts/test-bend-tests/configs/query/query-3.toml b/scripts/test-bend-tests/configs/query/query-3.toml index 049a890eee6d1..de202339eb073 100644 --- a/scripts/test-bend-tests/configs/query/query-3.toml +++ b/scripts/test-bend-tests/configs/query/query-3.toml @@ -64,6 +64,3 @@ max_bytes = 1073741824 # 1GB [spill] spill_local_disk_path = "_databend_data/spill/query3" -sort_spilling_disk_quota_ratio = 60 -window_partition_spilling_disk_quota_ratio = 20 -result_set_spilling_disk_quota_ratio = 10 diff --git a/scripts/test-bend-tests/configs/query/query-4.toml b/scripts/test-bend-tests/configs/query/query-4.toml index 1ef0595cec8c8..aa9fa7b01e555 100644 --- a/scripts/test-bend-tests/configs/query/query-4.toml +++ b/scripts/test-bend-tests/configs/query/query-4.toml @@ -64,6 +64,3 @@ max_bytes = 1073741824 # 1GB [spill] spill_local_disk_path = "_databend_data/spill/query4" -sort_spilling_disk_quota_ratio = 60 -window_partition_spilling_disk_quota_ratio = 20 -result_set_spilling_disk_quota_ratio = 10 diff --git a/scripts/test-bend-tests/configs/query/query-5.toml b/scripts/test-bend-tests/configs/query/query-5.toml index 5e4a303d3efcd..1a55e279b6042 100644 --- a/scripts/test-bend-tests/configs/query/query-5.toml +++ b/scripts/test-bend-tests/configs/query/query-5.toml @@ -64,6 +64,3 @@ max_bytes = 1073741824 # 1GB [spill] spill_local_disk_path = "_databend_data/spill/query5" -sort_spilling_disk_quota_ratio = 60 -window_partition_spilling_disk_quota_ratio = 20 -result_set_spilling_disk_quota_ratio = 10 diff --git a/src/query/config/src/config.rs b/src/query/config/src/config.rs index 4d9ea336e8ee2..5f8f22a3a75cd 100644 --- a/src/query/config/src/config.rs +++ b/src/query/config/src/config.rs @@ -3603,12 +3603,13 @@ pub struct SpillConfig { /// Maximum percentage of the global local spill quota that window /// partitioners may use for one query. - #[clap(long, value_name = "PERCENT", default_value = "20")] + #[clap(long, value_name = "PERCENT", default_value = "40")] pub window_partition_spilling_disk_quota_ratio: u64, /// Maximum percentage of the global local spill quota that HTTP /// result-set spilling may use for one query. - #[clap(long, value_name = "PERCENT", default_value = "10")] + /// TODO: keep 0 to avoid deleting local result-set spill dir before HTTP pagination finishes. + #[clap(long, value_name = "PERCENT", default_value = "0")] pub result_set_spilling_disk_quota_ratio: u64, } @@ -3652,7 +3653,8 @@ impl Default for SpillConfig { spill_local_disk_max_bytes: u64::MAX, sort_spilling_disk_quota_ratio: 60, window_partition_spilling_disk_quota_ratio: 20, - result_set_spilling_disk_quota_ratio: 10, + // TODO: keep 0 to avoid deleting local result-set spill dir before HTTP pagination finishes. + result_set_spilling_disk_quota_ratio: 0, } } } diff --git a/src/query/config/src/inner.rs b/src/query/config/src/inner.rs index 71c5670ed84ec..158f2b49576f7 100644 --- a/src/query/config/src/inner.rs +++ b/src/query/config/src/inner.rs @@ -870,8 +870,9 @@ impl SpillConfig { storage_params: None, // Use the same defaults as the external config. sort_spilling_disk_quota_ratio: 60, - window_partition_spilling_disk_quota_ratio: 20, - result_set_spilling_disk_quota_ratio: 10, + window_partition_spilling_disk_quota_ratio: 40, + // TODO: keep 0 to avoid deleting local result-set spill dir before HTTP pagination finishes. + result_set_spilling_disk_quota_ratio: 0, } } } @@ -885,8 +886,9 @@ impl Default for SpillConfig { global_bytes_limit: u64::MAX, storage_params: None, sort_spilling_disk_quota_ratio: 60, - window_partition_spilling_disk_quota_ratio: 20, - result_set_spilling_disk_quota_ratio: 10, + window_partition_spilling_disk_quota_ratio: 40, + // TODO: keep 0 to avoid deleting local result-set spill dir before HTTP pagination finishes. + result_set_spilling_disk_quota_ratio: 0, } } } diff --git a/src/query/config/src/mask.rs b/src/query/config/src/mask.rs index 87136005ab7e7..7bdc4c9f79b15 100644 --- a/src/query/config/src/mask.rs +++ b/src/query/config/src/mask.rs @@ -227,6 +227,7 @@ impl SpillConfig { spill_local_disk_max_bytes, sort_spilling_disk_quota_ratio, window_partition_spilling_disk_quota_ratio, + // TODO: keep 0 to avoid deleting local result-set spill dir before HTTP pagination finishes. result_set_spilling_disk_quota_ratio, } } @@ -382,8 +383,9 @@ mod tests { spill_local_disk_reserved_space_percentage: 30.0.into(), spill_local_disk_max_bytes: 10, sort_spilling_disk_quota_ratio: 60, - window_partition_spilling_disk_quota_ratio: 20, - result_set_spilling_disk_quota_ratio: 10, + window_partition_spilling_disk_quota_ratio: 30, + // TODO: keep 0 to avoid deleting local result-set spill dir before HTTP pagination finishes. + result_set_spilling_disk_quota_ratio: 0, storage: Some(StorageConfig { typ: "s3".to_string(), s3: S3StorageConfig { diff --git a/src/query/service/src/interpreters/hook/vacuum_hook.rs b/src/query/service/src/interpreters/hook/vacuum_hook.rs index 33a9ad89f9866..f80e88fdbdffb 100644 --- a/src/query/service/src/interpreters/hook/vacuum_hook.rs +++ b/src/query/service/src/interpreters/hook/vacuum_hook.rs @@ -19,6 +19,7 @@ use std::collections::HashMap; use std::sync::Arc; use databend_common_base::runtime::GlobalIORuntime; +use databend_common_catalog::session_type::SessionType; use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; use databend_common_exception::Result; @@ -88,6 +89,15 @@ pub fn hook_vacuum_temp_files(query_ctx: &Arc) -> Result<()> { } pub fn hook_disk_temp_dir(query_ctx: &Arc) -> Result<()> { + if matches!( + query_ctx.get_current_session().get_type(), + SessionType::HTTPQuery + ) { + // HTTP queries may still stream spilled result pages after execution finishes. + // Cleanup is deferred to the HTTP query lifecycle. + return Ok(()); + } + let mgr = TempDirManager::instance(); if mgr.drop_disk_spill_dir(&query_ctx.get_id())? && rand::thread_rng().gen_ratio(1, 10) { diff --git a/src/query/service/src/servers/http/v1/query/http_query.rs b/src/query/service/src/servers/http/v1/query/http_query.rs index 480398ada9698..b9ca3810fe364 100644 --- a/src/query/service/src/servers/http/v1/query/http_query.rs +++ b/src/query/service/src/servers/http/v1/query/http_query.rs @@ -651,7 +651,7 @@ impl HttpQuery { }) }; - let (page_manager, sender) = PageManager::create(&req.pagination); + let (page_manager, sender) = PageManager::create(&req.pagination, query_id.clone()); let executor = Arc::new(Mutex::new(Executor { query_id: query_id.clone(), state: ExecuteState::Starting(ExecuteStarting { diff --git a/src/query/service/src/servers/http/v1/query/page_manager.rs b/src/query/service/src/servers/http/v1/query/page_manager.rs index a5eac36d5ac60..fbc606754f57f 100644 --- a/src/query/service/src/servers/http/v1/query/page_manager.rs +++ b/src/query/service/src/servers/http/v1/query/page_manager.rs @@ -16,6 +16,7 @@ use std::sync::Arc; use databend_common_exception::ErrorCode; use databend_common_exception::Result; +use databend_storages_common_cache::TempDirManager; use super::blocks_serializer::BlocksSerializer; use super::http_query::PaginationConf; @@ -40,11 +41,15 @@ pub struct PageManager { total_pages: usize, end: bool, last_page: Option, + query_id: String, receiver: SizedChannelReceiver, } impl PageManager { - pub fn create(conf: &PaginationConf) -> (PageManager, SizedChannelSender) { + pub fn create( + conf: &PaginationConf, + query_id: String, + ) -> (PageManager, SizedChannelSender) { let (sender, receiver) = sized_spsc::(conf.max_rows_in_buffer, conf.max_rows_per_page); @@ -54,6 +59,7 @@ impl PageManager { last_page: None, total_pages: 0, end: false, + query_id, receiver, }, sender, @@ -158,6 +164,12 @@ impl PageManager { } } }; + if let Err(error) = TempDirManager::instance().drop_disk_spill_dir(&self.query_id) { + log::warn!( + target: "result-set-spill", + error:?; "[RESULT-SET-SPILL] Failed to remove spill temp dir on close" + ); + } self.last_page = None; } } diff --git a/src/query/service/tests/it/configs.rs b/src/query/service/tests/it/configs.rs index 7748efe569796..c7e899efeb5f9 100644 --- a/src/query/service/tests/it/configs.rs +++ b/src/query/service/tests/it/configs.rs @@ -1065,7 +1065,7 @@ secret_access_key = "test-secret" r#"[spill] spill_local_disk_path = "/legacy/spill/path" spill_local_disk_reserved_space_percentage = 25.0 -spill_local_disk_max_bytes = 53687091200 +spill_local_disk_max_bytes = 1073741824 "# .as_bytes(), )?; diff --git a/src/query/service/tests/it/storages/testdata/configs_table_basic.txt b/src/query/service/tests/it/storages/testdata/configs_table_basic.txt index 4da362e941f81..a51322810fb8f 100644 --- a/src/query/service/tests/it/storages/testdata/configs_table_basic.txt +++ b/src/query/service/tests/it/storages/testdata/configs_table_basic.txt @@ -224,13 +224,13 @@ DB.Table: 'system'.'configs', Table: configs-table_id:1, ver:0, Engine: SystemCo | 'query' | 'udfs' | '{"name":"test_builtin_ping","definition":"CREATE OR REPLACE FUNCTION test_builtin_ping (STRING)\\n RETURNS STRING\\n LANGUAGE python\\nHANDLER = \'ping\'\\nADDRESS = \'https://databend.com\';"}' | '' | | 'query' | 'users' | '{"name":"root","auth_type":"no_password","auth_string":null}' | '' | | 'query' | 'warehouse_id' | 'test_warehouse' | '' | -| 'spill' | 'result_set_spilling_disk_quota_ratio' | '10' | '' | +| 'spill' | 'result_set_spilling_disk_quota_ratio' | '0' | '' | | 'spill' | 'sort_spilling_disk_quota_ratio' | '60' | '' | | 'spill' | 'spill_local_disk_max_bytes' | '18446744073709551615' | '' | | 'spill' | 'spill_local_disk_path' | '' | '' | | 'spill' | 'spill_local_disk_reserved_space_percentage' | '30.0' | '' | | 'spill' | 'storage' | 'null' | '' | -| 'spill' | 'window_partition_spilling_disk_quota_ratio' | '20' | '' | +| 'spill' | 'window_partition_spilling_disk_quota_ratio' | '40' | '' | | 'storage' | 'allow_insecure' | 'true' | '' | | 'storage' | 'azblob.account_key' | '' | '' | | 'storage' | 'azblob.account_name' | '' | '' | @@ -292,5 +292,3 @@ DB.Table: 'system'.'configs', Table: configs-table_id:1, ver:0, Engine: SystemCo | 'storage' | 'webhdfs.root' | '' | '' | | 'storage' | 'webhdfs.user_name' | '' | '' | +-----------+------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------+ - - From 63b69c7a7cb9f86f6c68f683cd2ec94041788f8a Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Tue, 9 Dec 2025 20:09:03 +0800 Subject: [PATCH 27/37] Fix legacy spill config test expectation --- src/query/service/tests/it/configs.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/service/tests/it/configs.rs b/src/query/service/tests/it/configs.rs index c7e899efeb5f9..742eeac60c1cc 100644 --- a/src/query/service/tests/it/configs.rs +++ b/src/query/service/tests/it/configs.rs @@ -1077,7 +1077,7 @@ spill_local_disk_max_bytes = 1073741824 let cfg = InnerConfig::load_for_test().expect("config load failed"); assert_eq!(cfg.spill.local_path(), Some("/legacy/spill/path".into())); assert_eq!(cfg.spill.reserved_disk_ratio.into_inner(), 0.25); - assert_eq!(cfg.spill.global_bytes_limit, 53687091200); + assert_eq!(cfg.spill.global_bytes_limit, 1073741824); }, ); fs::remove_file(file_path)?; From 8521f35656407242d8db48db786176d432dde329 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Tue, 9 Dec 2025 20:23:57 +0800 Subject: [PATCH 28/37] Revert HTTP spill cleanups to main behavior --- .../src/servers/http/v1/query/http_query.rs | 2 +- .../src/servers/http/v1/query/page_manager.rs | 14 +------------- 2 files changed, 2 insertions(+), 14 deletions(-) diff --git a/src/query/service/src/servers/http/v1/query/http_query.rs b/src/query/service/src/servers/http/v1/query/http_query.rs index b9ca3810fe364..480398ada9698 100644 --- a/src/query/service/src/servers/http/v1/query/http_query.rs +++ b/src/query/service/src/servers/http/v1/query/http_query.rs @@ -651,7 +651,7 @@ impl HttpQuery { }) }; - let (page_manager, sender) = PageManager::create(&req.pagination, query_id.clone()); + let (page_manager, sender) = PageManager::create(&req.pagination); let executor = Arc::new(Mutex::new(Executor { query_id: query_id.clone(), state: ExecuteState::Starting(ExecuteStarting { diff --git a/src/query/service/src/servers/http/v1/query/page_manager.rs b/src/query/service/src/servers/http/v1/query/page_manager.rs index fbc606754f57f..a5eac36d5ac60 100644 --- a/src/query/service/src/servers/http/v1/query/page_manager.rs +++ b/src/query/service/src/servers/http/v1/query/page_manager.rs @@ -16,7 +16,6 @@ use std::sync::Arc; use databend_common_exception::ErrorCode; use databend_common_exception::Result; -use databend_storages_common_cache::TempDirManager; use super::blocks_serializer::BlocksSerializer; use super::http_query::PaginationConf; @@ -41,15 +40,11 @@ pub struct PageManager { total_pages: usize, end: bool, last_page: Option, - query_id: String, receiver: SizedChannelReceiver, } impl PageManager { - pub fn create( - conf: &PaginationConf, - query_id: String, - ) -> (PageManager, SizedChannelSender) { + pub fn create(conf: &PaginationConf) -> (PageManager, SizedChannelSender) { let (sender, receiver) = sized_spsc::(conf.max_rows_in_buffer, conf.max_rows_per_page); @@ -59,7 +54,6 @@ impl PageManager { last_page: None, total_pages: 0, end: false, - query_id, receiver, }, sender, @@ -164,12 +158,6 @@ impl PageManager { } } }; - if let Err(error) = TempDirManager::instance().drop_disk_spill_dir(&self.query_id) { - log::warn!( - target: "result-set-spill", - error:?; "[RESULT-SET-SPILL] Failed to remove spill temp dir on close" - ); - } self.last_page = None; } } From b5842b05d114c35e54e3a76e92f173fd87294372 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Tue, 9 Dec 2025 20:45:57 +0800 Subject: [PATCH 29/37] Fix goldenfile whitespace for configs_table_basic --- .../service/tests/it/storages/testdata/configs_table_basic.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/src/query/service/tests/it/storages/testdata/configs_table_basic.txt b/src/query/service/tests/it/storages/testdata/configs_table_basic.txt index a51322810fb8f..5539ede3c2007 100644 --- a/src/query/service/tests/it/storages/testdata/configs_table_basic.txt +++ b/src/query/service/tests/it/storages/testdata/configs_table_basic.txt @@ -292,3 +292,4 @@ DB.Table: 'system'.'configs', Table: configs-table_id:1, ver:0, Engine: SystemCo | 'storage' | 'webhdfs.root' | '' | '' | | 'storage' | 'webhdfs.user_name' | '' | '' | +-----------+------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------+ + From baeb117e5871f104f17e55358c0a066682f4b98c Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Tue, 9 Dec 2025 20:51:06 +0800 Subject: [PATCH 30/37] Adjust configs_table_basic golden trailing lines --- .../service/tests/it/storages/testdata/configs_table_basic.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/src/query/service/tests/it/storages/testdata/configs_table_basic.txt b/src/query/service/tests/it/storages/testdata/configs_table_basic.txt index 5539ede3c2007..4518c6a092620 100644 --- a/src/query/service/tests/it/storages/testdata/configs_table_basic.txt +++ b/src/query/service/tests/it/storages/testdata/configs_table_basic.txt @@ -293,3 +293,4 @@ DB.Table: 'system'.'configs', Table: configs-table_id:1, ver:0, Engine: SystemCo | 'storage' | 'webhdfs.user_name' | '' | '' | +-----------+------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------+ + From a78813c1042459ddb9d57736f4b35637235e8cc1 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Tue, 9 Dec 2025 22:16:15 +0800 Subject: [PATCH 31/37] Revert vacuum_hook change to main --- src/query/service/src/interpreters/hook/vacuum_hook.rs | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/src/query/service/src/interpreters/hook/vacuum_hook.rs b/src/query/service/src/interpreters/hook/vacuum_hook.rs index f80e88fdbdffb..33a9ad89f9866 100644 --- a/src/query/service/src/interpreters/hook/vacuum_hook.rs +++ b/src/query/service/src/interpreters/hook/vacuum_hook.rs @@ -19,7 +19,6 @@ use std::collections::HashMap; use std::sync::Arc; use databend_common_base::runtime::GlobalIORuntime; -use databend_common_catalog::session_type::SessionType; use databend_common_catalog::table_context::TableContext; use databend_common_exception::ErrorCode; use databend_common_exception::Result; @@ -89,15 +88,6 @@ pub fn hook_vacuum_temp_files(query_ctx: &Arc) -> Result<()> { } pub fn hook_disk_temp_dir(query_ctx: &Arc) -> Result<()> { - if matches!( - query_ctx.get_current_session().get_type(), - SessionType::HTTPQuery - ) { - // HTTP queries may still stream spilled result pages after execution finishes. - // Cleanup is deferred to the HTTP query lifecycle. - return Ok(()); - } - let mgr = TempDirManager::instance(); if mgr.drop_disk_spill_dir(&query_ctx.get_id())? && rand::thread_rng().gen_ratio(1, 10) { From 101098264814d7b7e8dd4b657ca56eea8ae91b9e Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Wed, 10 Dec 2025 10:44:46 +0800 Subject: [PATCH 32/37] config: disable implicit local spill cache fallback --- src/query/config/src/config.rs | 60 ++++++++++++--------------- src/query/service/tests/it/configs.rs | 17 ++++---- 2 files changed, 35 insertions(+), 42 deletions(-) diff --git a/src/query/config/src/config.rs b/src/query/config/src/config.rs index 5f8f22a3a75cd..f4d05428511f4 100644 --- a/src/query/config/src/config.rs +++ b/src/query/config/src/config.rs @@ -3726,10 +3726,10 @@ mod cache_config_converters { ); let hive = catalog.try_into()?; let catalog = InnerCatalogConfig::Hive(hive); - catalogs.insert(CATALOG_HIVE.to_string(), catalog); - } + catalogs.insert(CATALOG_HIVE.to_string(), catalog); + } - let spill = convert_local_spill_config(spill, &cache.disk_cache_config)?; + let spill = convert_local_spill_config(spill)?; Ok(InnerConfig { query: query.try_into()?, @@ -3826,10 +3826,7 @@ mod cache_config_converters { } } - fn convert_local_spill_config( - spill: SpillConfig, - cache: &DiskCacheConfig, - ) -> Result { + fn convert_local_spill_config(spill: SpillConfig) -> Result { // Determine configuration based on auto-detected spill type let spill_type = spill.get_spill_type(); let (local_writeable_root, path, reserved_disk_ratio, global_bytes_limit, storage_params) = @@ -3886,33 +3883,28 @@ mod cache_config_converters { storage_params, ) } - _ => { - // Default behavior for "default" type and any unrecognized types - // Default behavior with backward compatibility - let local_writeable_root = if cache.path != DiskCacheConfig::default().path - && spill.spill_local_disk_path.is_empty() - { - Some(cache.path.clone()) - } else { - None - }; - - let storage_params = spill - .storage - .map(|storage| { - let storage: InnerStorageConfig = storage.try_into()?; - Ok::<_, ErrorCode>(storage.params) - }) - .transpose()?; - - ( - local_writeable_root, - spill.spill_local_disk_path, - spill.spill_local_disk_reserved_space_percentage / 100.0, - spill.spill_local_disk_max_bytes, - storage_params, - ) - } + _ => { + // Default behavior for "default" type and any unrecognized types: + // do NOT implicitly reuse the data cache disk. Local spill is + // enabled only when explicitly configured via either + // - [spill.storage] with type = "fs", or + // - legacy spill_local_disk_path. + let storage_params = spill + .storage + .map(|storage| { + let storage: InnerStorageConfig = storage.try_into()?; + Ok::<_, ErrorCode>(storage.params) + }) + .transpose()?; + + ( + None, + spill.spill_local_disk_path, + spill.spill_local_disk_reserved_space_percentage / 100.0, + spill.spill_local_disk_max_bytes, + storage_params, + ) + } }; Ok(inner::SpillConfig { diff --git a/src/query/service/tests/it/configs.rs b/src/query/service/tests/it/configs.rs index 742eeac60c1cc..a43934fc5bd33 100644 --- a/src/query/service/tests/it/configs.rs +++ b/src/query/service/tests/it/configs.rs @@ -971,14 +971,15 @@ fn test_spill_config_comprehensive() -> Result<()> { f.write_all(b"# No [spill] section - should use default behavior")?; f.flush()?; - temp_env::with_vars( - vec![("CONFIG_FILE", Some(file_path.to_string_lossy().as_ref()))], - || { - let cfg = InnerConfig::load_for_test().expect("config load failed"); - // Default behavior: no explicit local path, falls back to cache - assert_eq!(cfg.spill.local_path(), None); - }, - ); + temp_env::with_vars( + vec![("CONFIG_FILE", Some(file_path.to_string_lossy().as_ref()))], + || { + let cfg = InnerConfig::load_for_test().expect("config load failed"); + // Default behavior: no explicit local path, local spill is disabled + // unless [spill] is explicitly configured. + assert_eq!(cfg.spill.local_path(), None); + }, + ); fs::remove_file(file_path)?; } From 4e66e409e7ae2fe005aa521517faa301f2811def Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Wed, 10 Dec 2025 10:59:00 +0800 Subject: [PATCH 33/37] Update configs --- src/query/config/src/config.rs | 52 +++++++++++++-------------- src/query/service/tests/it/configs.rs | 18 +++++----- 2 files changed, 35 insertions(+), 35 deletions(-) diff --git a/src/query/config/src/config.rs b/src/query/config/src/config.rs index f4d05428511f4..47790fe0ff061 100644 --- a/src/query/config/src/config.rs +++ b/src/query/config/src/config.rs @@ -3726,10 +3726,10 @@ mod cache_config_converters { ); let hive = catalog.try_into()?; let catalog = InnerCatalogConfig::Hive(hive); - catalogs.insert(CATALOG_HIVE.to_string(), catalog); - } + catalogs.insert(CATALOG_HIVE.to_string(), catalog); + } - let spill = convert_local_spill_config(spill)?; + let spill = convert_local_spill_config(spill)?; Ok(InnerConfig { query: query.try_into()?, @@ -3826,7 +3826,7 @@ mod cache_config_converters { } } - fn convert_local_spill_config(spill: SpillConfig) -> Result { + fn convert_local_spill_config(spill: SpillConfig) -> Result { // Determine configuration based on auto-detected spill type let spill_type = spill.get_spill_type(); let (local_writeable_root, path, reserved_disk_ratio, global_bytes_limit, storage_params) = @@ -3883,28 +3883,28 @@ mod cache_config_converters { storage_params, ) } - _ => { - // Default behavior for "default" type and any unrecognized types: - // do NOT implicitly reuse the data cache disk. Local spill is - // enabled only when explicitly configured via either - // - [spill.storage] with type = "fs", or - // - legacy spill_local_disk_path. - let storage_params = spill - .storage - .map(|storage| { - let storage: InnerStorageConfig = storage.try_into()?; - Ok::<_, ErrorCode>(storage.params) - }) - .transpose()?; - - ( - None, - spill.spill_local_disk_path, - spill.spill_local_disk_reserved_space_percentage / 100.0, - spill.spill_local_disk_max_bytes, - storage_params, - ) - } + _ => { + // Default behavior for "default" type and any unrecognized types: + // do NOT implicitly reuse the data cache disk. Local spill is + // enabled only when explicitly configured via either + // - [spill.storage] with type = "fs", or + // - legacy spill_local_disk_path. + let storage_params = spill + .storage + .map(|storage| { + let storage: InnerStorageConfig = storage.try_into()?; + Ok::<_, ErrorCode>(storage.params) + }) + .transpose()?; + + ( + None, + spill.spill_local_disk_path, + spill.spill_local_disk_reserved_space_percentage / 100.0, + spill.spill_local_disk_max_bytes, + storage_params, + ) + } }; Ok(inner::SpillConfig { diff --git a/src/query/service/tests/it/configs.rs b/src/query/service/tests/it/configs.rs index a43934fc5bd33..bc31d39892448 100644 --- a/src/query/service/tests/it/configs.rs +++ b/src/query/service/tests/it/configs.rs @@ -971,15 +971,15 @@ fn test_spill_config_comprehensive() -> Result<()> { f.write_all(b"# No [spill] section - should use default behavior")?; f.flush()?; - temp_env::with_vars( - vec![("CONFIG_FILE", Some(file_path.to_string_lossy().as_ref()))], - || { - let cfg = InnerConfig::load_for_test().expect("config load failed"); - // Default behavior: no explicit local path, local spill is disabled - // unless [spill] is explicitly configured. - assert_eq!(cfg.spill.local_path(), None); - }, - ); + temp_env::with_vars( + vec![("CONFIG_FILE", Some(file_path.to_string_lossy().as_ref()))], + || { + let cfg = InnerConfig::load_for_test().expect("config load failed"); + // Default behavior: no explicit local path, local spill is disabled + // unless [spill] is explicitly configured. + assert_eq!(cfg.spill.local_path(), None); + }, + ); fs::remove_file(file_path)?; } From 91db4a894468c45998ae9535cf6248319e003c60 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Wed, 10 Dec 2025 12:13:03 +0800 Subject: [PATCH 34/37] Align window spill quota default to 40 --- src/query/config/src/config.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/config/src/config.rs b/src/query/config/src/config.rs index 47790fe0ff061..5f9345b1f99f0 100644 --- a/src/query/config/src/config.rs +++ b/src/query/config/src/config.rs @@ -3652,7 +3652,7 @@ impl Default for SpillConfig { spill_local_disk_reserved_space_percentage: OrderedFloat(30.0), spill_local_disk_max_bytes: u64::MAX, sort_spilling_disk_quota_ratio: 60, - window_partition_spilling_disk_quota_ratio: 20, + window_partition_spilling_disk_quota_ratio: 40, // TODO: keep 0 to avoid deleting local result-set spill dir before HTTP pagination finishes. result_set_spilling_disk_quota_ratio: 0, } From 6758b8db575f2c33840b56ac0604ad222bc16055 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Wed, 10 Dec 2025 13:29:53 +0800 Subject: [PATCH 35/37] chore: adjust default spill local reserved disk ratio to 10% --- src/query/config/src/config.rs | 8 ++++---- src/query/config/src/inner.rs | 2 +- src/query/service/tests/it/configs.rs | 2 +- .../tests/it/storages/testdata/configs_table_basic.txt | 2 +- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/query/config/src/config.rs b/src/query/config/src/config.rs index 5f9345b1f99f0..9526ce43836ba 100644 --- a/src/query/config/src/config.rs +++ b/src/query/config/src/config.rs @@ -798,7 +798,7 @@ pub struct FsStorageConfig { impl FsStorageConfig { fn default_reserved_space_percentage() -> Option> { - None // Use None as default, will use system default (30.0) if not specified + None // Use None as default, will use system default (10.0) if not specified } } @@ -3585,7 +3585,7 @@ pub struct SpillConfig { #[clap(long, value_name = "VALUE", default_value = "")] pub spill_local_disk_path: String, - #[clap(long, value_name = "VALUE", default_value = "30")] + #[clap(long, value_name = "VALUE", default_value = "10")] /// Percentage of reserve disk space that won't be used for spill to local disk. pub spill_local_disk_reserved_space_percentage: OrderedFloat, @@ -3649,7 +3649,7 @@ impl Default for SpillConfig { Self { storage: None, spill_local_disk_path: String::new(), - spill_local_disk_reserved_space_percentage: OrderedFloat(30.0), + spill_local_disk_reserved_space_percentage: OrderedFloat(10.0), spill_local_disk_max_bytes: u64::MAX, sort_spilling_disk_quota_ratio: 60, window_partition_spilling_disk_quota_ratio: 40, @@ -3838,7 +3838,7 @@ mod cache_config_converters { let reserved_ratio = storage .fs .reserved_space_percentage - .unwrap_or(OrderedFloat(30.0)) + .unwrap_or(OrderedFloat(10.0)) / 100.0; let max_bytes = storage.fs.max_bytes.unwrap_or(u64::MAX); diff --git a/src/query/config/src/inner.rs b/src/query/config/src/inner.rs index 158f2b49576f7..62683520d75cf 100644 --- a/src/query/config/src/inner.rs +++ b/src/query/config/src/inner.rs @@ -882,7 +882,7 @@ impl Default for SpillConfig { Self { local_writeable_root: None, path: "".to_string(), - reserved_disk_ratio: OrderedFloat(0.3), + reserved_disk_ratio: OrderedFloat(0.1), global_bytes_limit: u64::MAX, storage_params: None, sort_spilling_disk_quota_ratio: 60, diff --git a/src/query/service/tests/it/configs.rs b/src/query/service/tests/it/configs.rs index bc31d39892448..fc9dfbe068058 100644 --- a/src/query/service/tests/it/configs.rs +++ b/src/query/service/tests/it/configs.rs @@ -948,7 +948,7 @@ spill_local_disk_path = "/data/spill" let cfg = InnerConfig::load_for_test().expect("config load failed"); assert_eq!(cfg.spill.local_path(), Some("/data/spill".into())); - assert_eq!(cfg.spill.reserved_disk_ratio, 0.3); + assert_eq!(cfg.spill.reserved_disk_ratio, 0.1); }, ); diff --git a/src/query/service/tests/it/storages/testdata/configs_table_basic.txt b/src/query/service/tests/it/storages/testdata/configs_table_basic.txt index 4518c6a092620..4ec675da62fa4 100644 --- a/src/query/service/tests/it/storages/testdata/configs_table_basic.txt +++ b/src/query/service/tests/it/storages/testdata/configs_table_basic.txt @@ -228,7 +228,7 @@ DB.Table: 'system'.'configs', Table: configs-table_id:1, ver:0, Engine: SystemCo | 'spill' | 'sort_spilling_disk_quota_ratio' | '60' | '' | | 'spill' | 'spill_local_disk_max_bytes' | '18446744073709551615' | '' | | 'spill' | 'spill_local_disk_path' | '' | '' | -| 'spill' | 'spill_local_disk_reserved_space_percentage' | '30.0' | '' | +| 'spill' | 'spill_local_disk_reserved_space_percentage' | '10.0' | '' | | 'spill' | 'storage' | 'null' | '' | | 'spill' | 'window_partition_spilling_disk_quota_ratio' | '40' | '' | | 'storage' | 'allow_insecure' | 'true' | '' | From 34940cf8fca9f72d722557b968d7e0fd0f37db58 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Wed, 10 Dec 2025 13:45:32 +0800 Subject: [PATCH 36/37] chore: align window spill quota ratio default to 60 --- src/query/config/src/config.rs | 4 ++-- src/query/config/src/inner.rs | 4 ++-- .../tests/it/storages/testdata/configs_table_basic.txt | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/query/config/src/config.rs b/src/query/config/src/config.rs index 9526ce43836ba..d1d902cb0753d 100644 --- a/src/query/config/src/config.rs +++ b/src/query/config/src/config.rs @@ -3603,7 +3603,7 @@ pub struct SpillConfig { /// Maximum percentage of the global local spill quota that window /// partitioners may use for one query. - #[clap(long, value_name = "PERCENT", default_value = "40")] + #[clap(long, value_name = "PERCENT", default_value = "60")] pub window_partition_spilling_disk_quota_ratio: u64, /// Maximum percentage of the global local spill quota that HTTP @@ -3652,7 +3652,7 @@ impl Default for SpillConfig { spill_local_disk_reserved_space_percentage: OrderedFloat(10.0), spill_local_disk_max_bytes: u64::MAX, sort_spilling_disk_quota_ratio: 60, - window_partition_spilling_disk_quota_ratio: 40, + window_partition_spilling_disk_quota_ratio: 60, // TODO: keep 0 to avoid deleting local result-set spill dir before HTTP pagination finishes. result_set_spilling_disk_quota_ratio: 0, } diff --git a/src/query/config/src/inner.rs b/src/query/config/src/inner.rs index 62683520d75cf..40e621bd60fed 100644 --- a/src/query/config/src/inner.rs +++ b/src/query/config/src/inner.rs @@ -870,7 +870,7 @@ impl SpillConfig { storage_params: None, // Use the same defaults as the external config. sort_spilling_disk_quota_ratio: 60, - window_partition_spilling_disk_quota_ratio: 40, + window_partition_spilling_disk_quota_ratio: 60, // TODO: keep 0 to avoid deleting local result-set spill dir before HTTP pagination finishes. result_set_spilling_disk_quota_ratio: 0, } @@ -886,7 +886,7 @@ impl Default for SpillConfig { global_bytes_limit: u64::MAX, storage_params: None, sort_spilling_disk_quota_ratio: 60, - window_partition_spilling_disk_quota_ratio: 40, + window_partition_spilling_disk_quota_ratio: 60, // TODO: keep 0 to avoid deleting local result-set spill dir before HTTP pagination finishes. result_set_spilling_disk_quota_ratio: 0, } diff --git a/src/query/service/tests/it/storages/testdata/configs_table_basic.txt b/src/query/service/tests/it/storages/testdata/configs_table_basic.txt index 4ec675da62fa4..deca56b922ce1 100644 --- a/src/query/service/tests/it/storages/testdata/configs_table_basic.txt +++ b/src/query/service/tests/it/storages/testdata/configs_table_basic.txt @@ -230,7 +230,7 @@ DB.Table: 'system'.'configs', Table: configs-table_id:1, ver:0, Engine: SystemCo | 'spill' | 'spill_local_disk_path' | '' | '' | | 'spill' | 'spill_local_disk_reserved_space_percentage' | '10.0' | '' | | 'spill' | 'storage' | 'null' | '' | -| 'spill' | 'window_partition_spilling_disk_quota_ratio' | '40' | '' | +| 'spill' | 'window_partition_spilling_disk_quota_ratio' | '60' | '' | | 'storage' | 'allow_insecure' | 'true' | '' | | 'storage' | 'azblob.account_key' | '' | '' | | 'storage' | 'azblob.account_name' | '' | '' | From 25978ae73e4904044c57880d1f7effadcb2d3824 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Wed, 10 Dec 2025 13:51:13 +0800 Subject: [PATCH 37/37] Update spill adapter and defaults --- src/query/config/src/config.rs | 2 +- src/query/config/src/inner.rs | 2 +- src/query/service/src/spillers/adapter.rs | 17 +++++++++++------ 3 files changed, 13 insertions(+), 8 deletions(-) diff --git a/src/query/config/src/config.rs b/src/query/config/src/config.rs index d1d902cb0753d..a5248a072a119 100644 --- a/src/query/config/src/config.rs +++ b/src/query/config/src/config.rs @@ -3652,7 +3652,7 @@ impl Default for SpillConfig { spill_local_disk_reserved_space_percentage: OrderedFloat(10.0), spill_local_disk_max_bytes: u64::MAX, sort_spilling_disk_quota_ratio: 60, - window_partition_spilling_disk_quota_ratio: 60, + window_partition_spilling_disk_quota_ratio: 60, // TODO: keep 0 to avoid deleting local result-set spill dir before HTTP pagination finishes. result_set_spilling_disk_quota_ratio: 0, } diff --git a/src/query/config/src/inner.rs b/src/query/config/src/inner.rs index 40e621bd60fed..b3cc31b0a8415 100644 --- a/src/query/config/src/inner.rs +++ b/src/query/config/src/inner.rs @@ -886,7 +886,7 @@ impl Default for SpillConfig { global_bytes_limit: u64::MAX, storage_params: None, sort_spilling_disk_quota_ratio: 60, - window_partition_spilling_disk_quota_ratio: 60, + window_partition_spilling_disk_quota_ratio: 60, // TODO: keep 0 to avoid deleting local result-set spill dir before HTTP pagination finishes. result_set_spilling_disk_quota_ratio: 0, } diff --git a/src/query/service/src/spillers/adapter.rs b/src/query/service/src/spillers/adapter.rs index 4cb801fe318fe..e52d47da6d69e 100644 --- a/src/query/service/src/spillers/adapter.rs +++ b/src/query/service/src/spillers/adapter.rs @@ -62,9 +62,9 @@ impl SpillAdapter for PartitionAdapter { .unwrap() .insert(location.clone(), layout.clone()); - if location.is_remote() { - self.ctx.as_ref().incr_spill_progress(1, size); - } + // Progress (SpillTotalStats) should reflect total spill volume, + // including both local and remote spill files. + self.ctx.as_ref().incr_spill_progress(1, size); self.ctx.as_ref().add_spill_file(location, layout); } @@ -579,9 +579,8 @@ impl SpillReader { impl SpillAdapter for Arc { fn add_spill_file(&self, location: Location, layout: Layout, size: usize) { - if matches!(location, Location::Remote(_)) { - self.incr_spill_progress(1, size); - } + // Count both local and remote spills in SpillTotalStats. + self.incr_spill_progress(1, size); self.as_ref().add_spill_file(location, layout); } @@ -599,10 +598,16 @@ impl SpillAdapter for SortAdapter { fn add_spill_file(&self, location: Location, layout: Layout, size: usize) { match location { Location::Remote(_) => { + // Remote spill files are tracked in QueryContext for cleanup + // and contribute to the total spill progress. self.ctx.as_ref().incr_spill_progress(1, size); self.ctx.as_ref().add_spill_file(location, layout); } Location::Local(temp_path) => { + // Local spill files are tracked only in-memory for sort, but + // should still be counted in SpillTotalStats so that progress + // reflects total (local + remote) spilled bytes/files. + self.ctx.as_ref().incr_spill_progress(1, size); self.local_files.write().unwrap().insert(temp_path, layout); } }