Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat(swordfish): Optimize grouped aggregations #3534

Merged
merged 20 commits into from
Dec 17, 2024
Merged
Show file tree
Hide file tree
Changes from 6 commits
Commits
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
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
95 changes: 16 additions & 79 deletions src/daft-local-execution/src/pipeline.rs
Original file line number Diff line number Diff line change
Expand Up @@ -10,15 +10,14 @@
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 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 @@
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 @@
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(),
}

Check warning on line 177 in src/daft-local-execution/src/pipeline.rs

View check run for this annotation

Codecov / codecov/patch

src/daft-local-execution/src/pipeline.rs#L175-L177

Added lines #L175 - L177 were not covered by tests
})?;
BlockingSinkNode::new(Arc::new(agg_sink), child_node).boxed()
}
LocalPhysicalPlan::HashAggregate(HashAggregate {
input,
Expand All @@ -214,48 +185,14 @@
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, 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()
let agg_sink =
GroupedAggregateSink::new(aggregations, group_by, schema).with_context(|_| {
PipelineCreationSnafu {
plan_name: physical_plan.name(),
}

Check warning on line 193 in src/daft-local-execution/src/pipeline.rs

View check run for this annotation

Codecov / codecov/patch

src/daft-local-execution/src/pipeline.rs#L191-L193

Added lines #L191 - L193 were not covered by tests
})?;
BlockingSinkNode::new(Arc::new(agg_sink), child_node).boxed()
}
LocalPhysicalPlan::Unpivot(Unpivot {
input,
Expand Down
62 changes: 45 additions & 17 deletions src/daft-local-execution/src/sinks/aggregate.rs
Original file line number Diff line number Diff line change
Expand Up @@ -2,8 +2,10 @@ use std::sync::Arc;

use common_error::DaftResult;
use common_runtime::RuntimeRef;
use daft_dsl::ExprRef;
use daft_core::prelude::SchemaRef;
use daft_dsl::{Expr, ExprRef};
use daft_micropartition::MicroPartition;
use daft_physical_plan::extract_agg_expr;
use tracing::instrument;

use super::blocking_sink::{
Expand Down Expand Up @@ -44,22 +46,41 @@ impl BlockingSinkState for AggregateState {
}

struct AggParams {
agg_exprs: Vec<ExprRef>,
group_by: Vec<ExprRef>,
sink_agg_exprs: Vec<ExprRef>,
finalize_agg_exprs: Vec<ExprRef>,
final_projections: Vec<ExprRef>,
}

pub struct AggregateSink {
agg_sink_params: Arc<AggParams>,
}

impl AggregateSink {
pub fn new(agg_exprs: Vec<ExprRef>, group_by: Vec<ExprRef>) -> Self {
Self {
pub fn new(aggregations: &[ExprRef], schema: &SchemaRef) -> DaftResult<Self> {
let aggregations = aggregations
.iter()
.map(extract_agg_expr)
.collect::<DaftResult<Vec<_>>>()?;
let (sink_aggs, finalize_aggs, final_projections) =
daft_physical_plan::populate_aggregation_stages(&aggregations, schema, &[]);
let sink_agg_exprs = sink_aggs
.values()
.cloned()
.map(|e| Arc::new(Expr::Agg(e)))
.collect();
let finalize_agg_exprs = finalize_aggs
.values()
.cloned()
.map(|e| Arc::new(Expr::Agg(e)))
.collect();

Ok(Self {
agg_sink_params: Arc::new(AggParams {
agg_exprs,
group_by,
sink_agg_exprs,
finalize_agg_exprs,
final_projections,
}),
}
})
}
}

Expand All @@ -69,14 +90,20 @@ impl BlockingSink for AggregateSink {
&self,
input: Arc<MicroPartition>,
mut state: Box<dyn BlockingSinkState>,
_runtime: &RuntimeRef,
runtime: &RuntimeRef,
) -> BlockingSinkSinkResult {
state
.as_any_mut()
.downcast_mut::<AggregateState>()
.expect("AggregateSink should have AggregateState")
.push(input);
Ok(BlockingSinkStatus::NeedMoreInput(state)).into()
let params = self.agg_sink_params.clone();
runtime
.spawn(async move {
let agg_state = state
.as_any_mut()
.downcast_mut::<AggregateState>()
.expect("AggregateSink should have AggregateState");
let agged = Arc::new(input.agg(&params.sink_agg_exprs, &[])?);
agg_state.push(agged);
Ok(BlockingSinkStatus::NeedMoreInput(state))
})
.into()
}

#[instrument(skip_all, name = "AggregateSink::finalize")]
Expand All @@ -96,8 +123,9 @@ impl BlockingSink for AggregateSink {
.finalize()
});
let concated = MicroPartition::concat(all_parts)?;
let agged = Arc::new(concated.agg(&params.agg_exprs, &params.group_by)?);
Ok(Some(agged))
let agged = concated.agg(&params.finalize_agg_exprs, &[])?;
let projected = agged.eval_expression_list(&params.final_projections)?;
Ok(Some(Arc::new(projected)))
})
.into()
}
Expand Down
Loading
Loading