Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
24 commits
Select commit Hold shift + click to select a range
e9beec8
feat: add multi level merge sort that will always fit in memory
rluvaton Apr 13, 2025
58b299d
test: add fuzz test for aggregate
rluvaton Apr 13, 2025
a8ac81e
update
rluvaton Apr 15, 2025
082a009
add more tests
rluvaton Apr 15, 2025
6cdf5dc
fix test
rluvaton Apr 15, 2025
04bb5ed
update tests
rluvaton Apr 15, 2025
d630cf1
added more aggregate fuzz
rluvaton Apr 15, 2025
5a55135
align with add fuzz tests
rluvaton Apr 15, 2025
a818582
add sort fuzz
rluvaton Apr 15, 2025
9334cf4
Merge branch 'main' into add-multi-level-merge-sort
rluvaton Apr 15, 2025
ba17329
fix lints and formatting
rluvaton Apr 15, 2025
14487dd
Merge branch 'main' into add-multi-level-merge-sort
rluvaton Jul 7, 2025
efde4d8
moved spill in memory constrained envs to separate test
rluvaton Jul 17, 2025
ec0f4c7
rename `StreamExec` to `OnceExec`
rluvaton Jul 17, 2025
00fa9f0
added comment on the usize in the `in_progress_spill_file` inside Ext…
rluvaton Jul 17, 2025
6198d90
rename buffer_size to buffer_len
rluvaton Jul 17, 2025
af6b5c5
reuse code in spill fuzz
rluvaton Jul 17, 2025
ae1ed6d
double the amount of memory needed to sort
rluvaton Jul 17, 2025
0403296
add diagram for explaining the overview
rluvaton Jul 21, 2025
0010222
update based on code review
rluvaton Jul 21, 2025
8e72e79
fix test based on new memory calculation
rluvaton Jul 21, 2025
473d88b
Merge branch 'main' into add-multi-level-merge-sort
rluvaton Jul 21, 2025
bc76825
remove get_size in favor of get_sliced_size
rluvaton Jul 23, 2025
00dcf58
change to result
rluvaton Jul 24, 2025
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
23 changes: 14 additions & 9 deletions datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs
Original file line number Diff line number Diff line change
Expand Up @@ -38,21 +38,21 @@ use datafusion::prelude::{DataFrame, SessionConfig, SessionContext};
use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion, TreeNodeVisitor};
use datafusion_common::{HashMap, Result};
use datafusion_common_runtime::JoinSet;
use datafusion_functions_aggregate::sum::sum_udaf;
use datafusion_physical_expr::expressions::{col, lit, Column};
use datafusion_physical_expr::PhysicalSortExpr;
use datafusion_physical_plan::InputOrderMode;
use test_utils::{add_empty_batches, StringBatchGenerator};

use datafusion_execution::memory_pool::FairSpillPool;
use datafusion_execution::runtime_env::RuntimeEnvBuilder;
use datafusion_execution::TaskContext;
use datafusion_functions_aggregate::sum::sum_udaf;
use datafusion_physical_expr::aggregate::AggregateExprBuilder;
use datafusion_physical_expr::expressions::{col, lit, Column};
use datafusion_physical_expr::PhysicalSortExpr;
use datafusion_physical_plan::aggregates::{
AggregateExec, AggregateMode, PhysicalGroupBy,
};
use datafusion_physical_plan::metrics::MetricValue;
use datafusion_physical_plan::InputOrderMode;
use datafusion_physical_plan::{collect, displayable, ExecutionPlan};
use test_utils::{add_empty_batches, StringBatchGenerator};

use rand::rngs::StdRng;
use rand::{random, rng, Rng, SeedableRng};

Expand Down Expand Up @@ -632,8 +632,11 @@ fn extract_result_counts(results: Vec<RecordBatch>) -> HashMap<Option<String>, i
output
}

fn assert_spill_count_metric(expect_spill: bool, single_aggregate: Arc<AggregateExec>) {
if let Some(metrics_set) = single_aggregate.metrics() {
pub(crate) fn assert_spill_count_metric(
expect_spill: bool,
plan_that_spills: Arc<dyn ExecutionPlan>,
) -> usize {
if let Some(metrics_set) = plan_that_spills.metrics() {
let mut spill_count = 0;

// Inspect metrics for SpillCount
Expand All @@ -649,14 +652,16 @@ fn assert_spill_count_metric(expect_spill: bool, single_aggregate: Arc<Aggregate
} else if !expect_spill && spill_count > 0 {
panic!("Expected no spill but found SpillCount metric with value greater than 0.");
}

spill_count
} else {
panic!("No metrics returned from the operator; cannot verify spilling.");
}
}

// Fix for https://github.com/apache/datafusion/issues/15530
#[tokio::test]
async fn test_single_mode_aggregate_with_spill() -> Result<()> {
async fn test_single_mode_aggregate_single_mode_aggregate_with_spill() -> Result<()> {
let scan_schema = Arc::new(Schema::new(vec![
Field::new("col_0", DataType::Int64, true),
Field::new("col_1", DataType::Utf8, true),
Expand Down
2 changes: 2 additions & 0 deletions datafusion/core/tests/fuzz_cases/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -33,4 +33,6 @@ mod sort_preserving_repartition_fuzz;
mod window_fuzz;

// Utility modules
mod once_exec;
mod record_batch_generator;
mod spilling_fuzz_in_memory_constrained_env;
115 changes: 115 additions & 0 deletions datafusion/core/tests/fuzz_cases/once_exec.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,115 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.

use arrow_schema::SchemaRef;
use datafusion_common::DataFusionError;
use datafusion_execution::{SendableRecordBatchStream, TaskContext};
use datafusion_physical_expr::{EquivalenceProperties, Partitioning};
use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType};
use datafusion_physical_plan::{
DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties,
};
use std::any::Any;
use std::fmt::{Debug, Formatter};
use std::sync::{Arc, Mutex};

/// Execution plan that return the stream on the call to `execute`. further calls to `execute` will
/// return an error
pub struct OnceExec {
/// the results to send back
stream: Mutex<Option<SendableRecordBatchStream>>,
cache: PlanProperties,
}

impl Debug for OnceExec {
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
write!(f, "OnceExec")
}
}

impl OnceExec {
pub fn new(stream: SendableRecordBatchStream) -> Self {
let cache = Self::compute_properties(stream.schema());
Self {
stream: Mutex::new(Some(stream)),
cache,
}
}

/// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc.
fn compute_properties(schema: SchemaRef) -> PlanProperties {
PlanProperties::new(
EquivalenceProperties::new(schema),
Partitioning::UnknownPartitioning(1),
EmissionType::Incremental,
Boundedness::Bounded,
)
}
}

impl DisplayAs for OnceExec {
fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
write!(f, "OnceExec:")
}
DisplayFormatType::TreeRender => {
write!(f, "")
}
}
}
}

impl ExecutionPlan for OnceExec {
fn name(&self) -> &'static str {
Self::static_name()
}

fn as_any(&self) -> &dyn Any {
self
}

fn properties(&self) -> &PlanProperties {
&self.cache
}

fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
vec![]
}

fn with_new_children(
self: Arc<Self>,
_: Vec<Arc<dyn ExecutionPlan>>,
) -> datafusion_common::Result<Arc<dyn ExecutionPlan>> {
unimplemented!()
}

/// Returns a stream which yields data
fn execute(
&self,
partition: usize,
_context: Arc<TaskContext>,
) -> datafusion_common::Result<SendableRecordBatchStream> {
assert_eq!(partition, 0);

let stream = self.stream.lock().unwrap().take();

stream.ok_or(DataFusionError::Internal(
"Stream already consumed".to_string(),
))
}
}
Loading