Skip to content

Commit

Permalink
feat(swordfish): Optimize grouped aggregations (#3534)
Browse files Browse the repository at this point in the history
Optimize swordfish grouped aggs for high cardinality groups

### Approach

There's 3 strategies for grouped aggs:
1. Partition each input morsel into `N` partitions, then do a partial
agg. (good for high cardinality).
2. Do a partial agg, then partition into `N` partitions. (good for low
cardinality). Can be optimized with
#3556
3. Partition only, no partial agg. (only for map_groups, which has no
partial agg).

### Notes on alternative approaches
- Distributing partitions across workers (i.e. having each worker being
responsible for accumulating only one partition) is much slower for low
cardinality aggs (TPCH Q1 would have been 1.5x slower). This is because
most of the work will end up being on only a few workers, reducing
parallelism.
- Simply partitioning the input and then only aggregating at the end
works well with higher cardinality, but low cardinality takes a hit.
(TPCH Q1 would have been 2.5x slower).
- Probe Table approach was much slower, due to many calls to the
multi-table dyn comparator. It was also much more complex to implement.

### Benchmarks
[MrPowers Benchmarks](https://github.com/MrPowers/mrpowers-benchmarks)
results (seconds, lower is better).

| Query | this PR | Pyrunner | Current swordfish |
|-------|---------|----------|-------------------|
| q1    | 0.285720| 0.768858 | 0.356499         |
| q2    | 4.780064| 6.122199 | 53.340565        |
| q3    | 2.201079| 3.922857 | 16.935125        |
| q4    | 0.313106| 0.545192 | 0.335541         |
| q5    | 1.618228| 2.889354 | 10.665339        |
| q7    | 2.087872| 3.856998 | 16.072660        |
| q10   | 6.306756| 8.173738 | 53.800501        |

---------

Co-authored-by: EC2 Default User <[email protected]>
Co-authored-by: Colin Ho <[email protected]>
Co-authored-by: EC2 Default User <[email protected]>
  • Loading branch information
4 people authored Dec 17, 2024
1 parent 6c21917 commit e148248
Show file tree
Hide file tree
Showing 13 changed files with 537 additions and 154 deletions.
8 changes: 7 additions & 1 deletion daft/context.py
Original file line number Diff line number Diff line change
Expand Up @@ -344,6 +344,8 @@ def set_execution_config(
csv_target_filesize: int | None = None,
csv_inflation_factor: float | None = None,
shuffle_aggregation_default_partitions: int | None = None,
partial_aggregation_threshold: int | None = None,
high_cardinality_aggregation_threshold: float | None = None,
read_sql_partition_size_bytes: int | None = None,
enable_aqe: bool | None = None,
enable_native_executor: bool | None = None,
Expand Down Expand Up @@ -384,7 +386,9 @@ def set_execution_config(
parquet_inflation_factor: Inflation Factor of parquet files (In-Memory-Size / File-Size) ratio. Defaults to 3.0
csv_target_filesize: Target File Size when writing out CSV Files. Defaults to 512MB
csv_inflation_factor: Inflation Factor of CSV files (In-Memory-Size / File-Size) ratio. Defaults to 0.5
shuffle_aggregation_default_partitions: Maximum number of partitions to create when performing aggregations. Defaults to 200, unless the number of input partitions is less than 200.
shuffle_aggregation_default_partitions: Maximum number of partitions to create when performing aggregations on the Ray Runner. Defaults to 200, unless the number of input partitions is less than 200.
partial_aggregation_threshold: Threshold for performing partial aggregations on the Native Runner. Defaults to 10000 rows.
high_cardinality_aggregation_threshold: Threshold selectivity for performing high cardinality aggregations on the Native Runner. Defaults to 0.8.
read_sql_partition_size_bytes: Target size of partition when reading from SQL databases. Defaults to 512MB
enable_aqe: Enables Adaptive Query Execution, Defaults to False
enable_native_executor: Enables the native executor, Defaults to False
Expand Down Expand Up @@ -413,6 +417,8 @@ def set_execution_config(
csv_target_filesize=csv_target_filesize,
csv_inflation_factor=csv_inflation_factor,
shuffle_aggregation_default_partitions=shuffle_aggregation_default_partitions,
partial_aggregation_threshold=partial_aggregation_threshold,
high_cardinality_aggregation_threshold=high_cardinality_aggregation_threshold,
read_sql_partition_size_bytes=read_sql_partition_size_bytes,
enable_aqe=enable_aqe,
enable_native_executor=enable_native_executor,
Expand Down
6 changes: 6 additions & 0 deletions daft/daft/__init__.pyi
Original file line number Diff line number Diff line change
Expand Up @@ -1730,6 +1730,8 @@ class PyDaftExecutionConfig:
csv_target_filesize: int | None = None,
csv_inflation_factor: float | None = None,
shuffle_aggregation_default_partitions: int | None = None,
partial_aggregation_threshold: int | None = None,
high_cardinality_aggregation_threshold: float | None = None,
read_sql_partition_size_bytes: int | None = None,
enable_aqe: bool | None = None,
enable_native_executor: bool | None = None,
Expand Down Expand Up @@ -1765,6 +1767,10 @@ class PyDaftExecutionConfig:
@property
def shuffle_aggregation_default_partitions(self) -> int: ...
@property
def partial_aggregation_threshold(self) -> int: ...
@property
def high_cardinality_aggregation_threshold(self) -> float: ...
@property
def read_sql_partition_size_bytes(self) -> int: ...
@property
def enable_aqe(self) -> bool: ...
Expand Down
4 changes: 4 additions & 0 deletions src/common/daft-config/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,8 @@ pub struct DaftExecutionConfig {
pub csv_target_filesize: usize,
pub csv_inflation_factor: f64,
pub shuffle_aggregation_default_partitions: usize,
pub partial_aggregation_threshold: usize,
pub high_cardinality_aggregation_threshold: f64,
pub read_sql_partition_size_bytes: usize,
pub enable_aqe: bool,
pub enable_native_executor: bool,
Expand All @@ -70,6 +72,8 @@ impl Default for DaftExecutionConfig {
csv_target_filesize: 512 * 1024 * 1024, // 512MB
csv_inflation_factor: 0.5,
shuffle_aggregation_default_partitions: 200,
partial_aggregation_threshold: 10000,
high_cardinality_aggregation_threshold: 0.8,
read_sql_partition_size_bytes: 512 * 1024 * 1024, // 512MB
enable_aqe: false,
enable_native_executor: false,
Expand Down
19 changes: 19 additions & 0 deletions src/common/daft-config/src/python.rs
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,8 @@ impl PyDaftExecutionConfig {
csv_target_filesize: Option<usize>,
csv_inflation_factor: Option<f64>,
shuffle_aggregation_default_partitions: Option<usize>,
partial_aggregation_threshold: Option<usize>,
high_cardinality_aggregation_threshold: Option<f64>,
read_sql_partition_size_bytes: Option<usize>,
enable_aqe: Option<bool>,
enable_native_executor: Option<bool>,
Expand Down Expand Up @@ -146,6 +148,13 @@ impl PyDaftExecutionConfig {
{
config.shuffle_aggregation_default_partitions = shuffle_aggregation_default_partitions;
}
if let Some(partial_aggregation_threshold) = partial_aggregation_threshold {
config.partial_aggregation_threshold = partial_aggregation_threshold;
}
if let Some(high_cardinality_aggregation_threshold) = high_cardinality_aggregation_threshold
{
config.high_cardinality_aggregation_threshold = high_cardinality_aggregation_threshold;
}
if let Some(read_sql_partition_size_bytes) = read_sql_partition_size_bytes {
config.read_sql_partition_size_bytes = read_sql_partition_size_bytes;
}
Expand Down Expand Up @@ -245,6 +254,16 @@ impl PyDaftExecutionConfig {
Ok(self.config.shuffle_aggregation_default_partitions)
}

#[getter]
fn get_partial_aggregation_threshold(&self) -> PyResult<usize> {
Ok(self.config.partial_aggregation_threshold)
}

#[getter]
fn get_high_cardinality_aggregation_threshold(&self) -> PyResult<f64> {
Ok(self.config.high_cardinality_aggregation_threshold)
}

#[getter]
fn get_read_sql_partition_size_bytes(&self) -> PyResult<usize> {
Ok(self.config.read_sql_partition_size_bytes)
Expand Down
56 changes: 0 additions & 56 deletions src/daft-local-execution/src/intermediate_ops/aggregate.rs

This file was deleted.

1 change: 0 additions & 1 deletion src/daft-local-execution/src/intermediate_ops/mod.rs
Original file line number Diff line number Diff line change
@@ -1,5 +1,4 @@
pub mod actor_pool_project;
pub mod aggregate;
pub mod anti_semi_hash_join_probe;
pub mod cross_join;
pub mod explode;
Expand Down
2 changes: 2 additions & 0 deletions src/daft-local-execution/src/lib.rs
Original file line number Diff line number Diff line change
@@ -1,4 +1,6 @@
#![feature(let_chains)]
#![feature(option_get_or_insert_default)]

mod buffer;
mod channel;
mod dispatcher;
Expand Down
89 changes: 12 additions & 77 deletions src/daft-local-execution/src/pipeline.rs
Original file line number Diff line number Diff line change
Expand Up @@ -10,15 +10,14 @@ use daft_core::{
prelude::{Schema, SchemaRef},
utils::supertype,
};
use daft_dsl::{col, join::get_common_join_keys, Expr};
use daft_dsl::{col, join::get_common_join_keys};
use daft_local_plan::{
ActorPoolProject, Concat, CrossJoin, EmptyScan, Explode, Filter, HashAggregate, HashJoin,
InMemoryScan, Limit, LocalPhysicalPlan, MonotonicallyIncreasingId, PhysicalWrite, Pivot,
Project, Sample, Sort, UnGroupedAggregate, Unpivot,
};
use daft_logical_plan::{stats::StatsState, JoinType};
use daft_micropartition::{partitioning::PartitionSet, MicroPartition};
use daft_physical_plan::{extract_agg_expr, populate_aggregation_stages};
use daft_scan::ScanTaskRef;
use daft_writers::make_physical_writer_factory;
use indexmap::IndexSet;
Expand All @@ -27,7 +26,7 @@ use snafu::ResultExt;
use crate::{
channel::Receiver,
intermediate_ops::{
actor_pool_project::ActorPoolProjectOperator, aggregate::AggregateOperator,
actor_pool_project::ActorPoolProjectOperator,
anti_semi_hash_join_probe::AntiSemiProbeOperator, cross_join::CrossJoinOperator,
explode::ExplodeOperator, filter::FilterOperator,
inner_hash_join_probe::InnerHashJoinProbeOperator, intermediate_op::IntermediateNode,
Expand All @@ -38,6 +37,7 @@ use crate::{
blocking_sink::BlockingSinkNode,
concat::ConcatSink,
cross_join_collect::CrossJoinCollectSink,
grouped_aggregate::GroupedAggregateSink,
hash_join_build::HashJoinBuildSink,
limit::LimitSink,
monotonically_increasing_id::MonotonicallyIncreasingIdSink,
Expand Down Expand Up @@ -170,42 +170,13 @@ pub fn physical_plan_to_pipeline(
schema,
..
}) => {
let aggregations = aggregations
.iter()
.map(extract_agg_expr)
.collect::<DaftResult<Vec<_>>>()
.with_context(|_| PipelineCreationSnafu {
plan_name: physical_plan.name(),
})?;

let (first_stage_aggs, second_stage_aggs, final_exprs) =
populate_aggregation_stages(&aggregations, schema, &[]);
let first_stage_agg_op = AggregateOperator::new(
first_stage_aggs
.values()
.cloned()
.map(|e| Arc::new(Expr::Agg(e)))
.collect(),
vec![],
);
let child_node = physical_plan_to_pipeline(input, psets, cfg)?;
let post_first_agg_node =
IntermediateNode::new(Arc::new(first_stage_agg_op), vec![child_node]).boxed();

let second_stage_agg_sink = AggregateSink::new(
second_stage_aggs
.values()
.cloned()
.map(|e| Arc::new(Expr::Agg(e)))
.collect(),
vec![],
);
let second_stage_node =
BlockingSinkNode::new(Arc::new(second_stage_agg_sink), post_first_agg_node).boxed();

let final_stage_project = ProjectOperator::new(final_exprs);

IntermediateNode::new(Arc::new(final_stage_project), vec![second_stage_node]).boxed()
let agg_sink = AggregateSink::new(aggregations, schema).with_context(|_| {
PipelineCreationSnafu {
plan_name: physical_plan.name(),
}
})?;
BlockingSinkNode::new(Arc::new(agg_sink), child_node).boxed()
}
LocalPhysicalPlan::HashAggregate(HashAggregate {
input,
Expand All @@ -214,48 +185,12 @@ pub fn physical_plan_to_pipeline(
schema,
..
}) => {
let aggregations = aggregations
.iter()
.map(extract_agg_expr)
.collect::<DaftResult<Vec<_>>>()
let child_node = physical_plan_to_pipeline(input, psets, cfg)?;
let agg_sink = GroupedAggregateSink::new(aggregations, group_by, schema, cfg)
.with_context(|_| PipelineCreationSnafu {
plan_name: physical_plan.name(),
})?;

let (first_stage_aggs, second_stage_aggs, final_exprs) =
populate_aggregation_stages(&aggregations, schema, group_by);
let child_node = physical_plan_to_pipeline(input, psets, cfg)?;
let (post_first_agg_node, group_by) = if !first_stage_aggs.is_empty() {
let agg_op = AggregateOperator::new(
first_stage_aggs
.values()
.cloned()
.map(|e| Arc::new(Expr::Agg(e)))
.collect(),
group_by.clone(),
);
(
IntermediateNode::new(Arc::new(agg_op), vec![child_node]).boxed(),
&group_by.iter().map(|e| col(e.name())).collect(),
)
} else {
(child_node, group_by)
};

let second_stage_agg_sink = AggregateSink::new(
second_stage_aggs
.values()
.cloned()
.map(|e| Arc::new(Expr::Agg(e)))
.collect(),
group_by.clone(),
);
let second_stage_node =
BlockingSinkNode::new(Arc::new(second_stage_agg_sink), post_first_agg_node).boxed();

let final_stage_project = ProjectOperator::new(final_exprs);

IntermediateNode::new(Arc::new(final_stage_project), vec![second_stage_node]).boxed()
BlockingSinkNode::new(Arc::new(agg_sink), child_node).boxed()
}
LocalPhysicalPlan::Unpivot(Unpivot {
input,
Expand Down
Loading

0 comments on commit e148248

Please sign in to comment.