Skip to content

Introduce selection vector repartitioning #15423

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

Open
wants to merge 37 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
37 commits
Select commit Hold shift + click to select a range
b97cd24
introduce selection vector repartitioning
goldmedal Mar 25, 2025
c26db85
finish the phsyical plan side
goldmedal Mar 26, 2025
65a3031
add config
goldmedal Mar 26, 2025
271658f
support for proto
goldmedal Mar 26, 2025
2e49946
add sqllogictests
goldmedal Mar 26, 2025
6cbe760
fix fmt and clippy
goldmedal Mar 26, 2025
25c3e2c
rename column and address comment
goldmedal Mar 26, 2025
cb2606f
fix config test
goldmedal Mar 27, 2025
7e147e1
remove hash join test
goldmedal Mar 27, 2025
29fa0e6
fix typo
goldmedal Mar 28, 2025
575af63
add back the join test
goldmedal Mar 28, 2025
a789e5c
add HashPartitionMode
goldmedal Mar 29, 2025
13d9bb8
update sqllogictests
goldmedal Mar 29, 2025
316998d
fix fmt
goldmedal Mar 29, 2025
adc00d8
add todo comment
goldmedal Mar 29, 2025
35e5402
fix typo
goldmedal Mar 29, 2025
b489378
address review comment
goldmedal Mar 29, 2025
0f9f398
fix compile and ehance doc
goldmedal Mar 29, 2025
2b93d80
rename config
goldmedal Mar 29, 2025
bad2e15
fix fmt
goldmedal Mar 29, 2025
bc70653
fix test
goldmedal Mar 29, 2025
764cce7
fix sqllogictests
goldmedal Mar 29, 2025
c01119c
fix fmt
goldmedal Apr 13, 2025
057660f
optimze repartition
goldmedal Apr 19, 2025
40caf0c
fix fmt
goldmedal Apr 20, 2025
84ebdc9
fix clippy and fmt
goldmedal Apr 20, 2025
ebdab1f
use scalar instead array
goldmedal Apr 21, 2025
401cb9c
fix clippy
goldmedal Apr 29, 2025
8d7d038
fix fmt
goldmedal Apr 29, 2025
e3c7021
rename selection_vector to selection_bitmap
goldmedal May 4, 2025
c7e4ac4
fix sqllogictest
goldmedal May 4, 2025
5c46c6b
add sanity check for filtered batch
goldmedal May 7, 2025
be34338
fix doc
goldmedal May 7, 2025
d11f0cc
fix sanity check and add test
goldmedal May 7, 2025
4c83644
fix clippy
goldmedal May 8, 2025
f76eba1
fix test and fmt
goldmedal May 10, 2025
0a0055d
update config
goldmedal May 10, 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
4 changes: 4 additions & 0 deletions datafusion/common/src/config.rs
Original file line number Diff line number Diff line change
Expand Up @@ -700,6 +700,10 @@ config_namespace! {
/// HashJoin can work more efficiently than SortMergeJoin but consumes more memory
pub prefer_hash_join: bool, default = true

/// When set to true, the physical plan optimizer will prefer HashSelectionBitmapPartitioning for HashAggregate
/// over HashPartitioning. HashSelectionBitmapPartitioning can work without data copying.
pub prefer_hash_selection_bitmap_partitioning_agg: bool, default = false

/// The maximum estimated size in bytes for one input side of a HashJoin
/// will be collected into a single partition
pub hash_join_single_partition_threshold: usize, default = 1024 * 1024
Expand Down
23 changes: 20 additions & 3 deletions datafusion/core/src/physical_planner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,7 @@ use datafusion_expr::{
};
use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr};
use datafusion_physical_expr::expressions::{Column, Literal};
use datafusion_physical_expr::LexOrdering;
use datafusion_physical_expr::{HashPartitionMode, LexOrdering};
use datafusion_physical_optimizer::PhysicalOptimizerRule;
use datafusion_physical_plan::execution_plan::InvariantLevel;
use datafusion_physical_plan::placeholder_row::PlaceholderRowExec;
Expand Down Expand Up @@ -741,8 +741,17 @@ impl DefaultPhysicalPlanner {
let updated_aggregates = initial_aggr.aggr_expr().to_vec();

let next_partition_mode = if can_repartition {
let mode = if session_state
.config_options()
.optimizer
.prefer_hash_selection_bitmap_partitioning_agg
{
HashPartitionMode::SelectionBitmap
} else {
HashPartitionMode::HashPartitioned
};
// construct a second aggregation with 'AggregateMode::FinalPartitioned'
AggregateMode::FinalPartitioned
AggregateMode::FinalPartitioned(mode)
} else {
// construct a second aggregation, keeping the final column name equal to the
// first aggregation and the expressions corresponding to the respective aggregate
Expand Down Expand Up @@ -804,7 +813,15 @@ impl DefaultPhysicalPlanner {
)
})
.collect::<Result<Vec<_>>>()?;
Partitioning::Hash(runtime_expr, *n)
if session_state
.config_options()
.optimizer
.prefer_hash_selection_bitmap_partitioning_agg
{
Partitioning::HashSelectionBitmap(runtime_expr, *n)
} else {
Partitioning::Hash(runtime_expr, *n)
}
}
LogicalPartitioning::DistributeBy(_) => {
return not_impl_err!(
Expand Down
46 changes: 23 additions & 23 deletions datafusion/core/tests/dataframe/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -2514,7 +2514,7 @@ async fn test_count_wildcard_on_sort() -> Result<()> {

assert_snapshot!(
pretty_format_batches(&sql_results).unwrap(),
@r###"
@r"
+---------------+------------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+------------------------------------------------------------------------------------------------------------+
Expand All @@ -2527,37 +2527,37 @@ async fn test_count_wildcard_on_sort() -> Result<()> {
| | SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] |
| | SortExec: expr=[count(Int64(1))@2 ASC NULLS LAST], preserve_partitioning=[true] |
| | ProjectionExec: expr=[b@0 as b, count(Int64(1))@1 as count(*), count(Int64(1))@1 as count(Int64(1))] |
| | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(Int64(1))] |
| | AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[b@0 as b], aggr=[count(Int64(1))] |
| | CoalesceBatchesExec: target_batch_size=8192 |
| | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 |
| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |
| | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(Int64(1))] |
| | DataSourceExec: partitions=1, partition_sizes=[1] |
| | |
+---------------+------------------------------------------------------------------------------------------------------------+
"###
"
);

assert_snapshot!(
pretty_format_batches(&df_results).unwrap(),
@r###"
+---------------+--------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+--------------------------------------------------------------------------------+
| logical_plan | Sort: count(*) ASC NULLS LAST |
| | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1)) AS count(*)]] |
| | TableScan: t1 projection=[b] |
| physical_plan | SortPreservingMergeExec: [count(*)@1 ASC NULLS LAST] |
| | SortExec: expr=[count(*)@1 ASC NULLS LAST], preserve_partitioning=[true] |
| | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(*)] |
| | CoalesceBatchesExec: target_batch_size=8192 |
| | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 |
| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |
| | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(*)] |
| | DataSourceExec: partitions=1, partition_sizes=[1] |
| | |
+---------------+--------------------------------------------------------------------------------+
"###
@r"
+---------------+--------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+--------------------------------------------------------------------------------------------+
| logical_plan | Sort: count(*) ASC NULLS LAST |
| | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1)) AS count(*)]] |
| | TableScan: t1 projection=[b] |
| physical_plan | SortPreservingMergeExec: [count(*)@1 ASC NULLS LAST] |
| | SortExec: expr=[count(*)@1 ASC NULLS LAST], preserve_partitioning=[true] |
| | AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[b@0 as b], aggr=[count(*)] |
| | CoalesceBatchesExec: target_batch_size=8192 |
| | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 |
| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |
| | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(*)] |
| | DataSourceExec: partitions=1, partition_sizes=[1] |
| | |
+---------------+--------------------------------------------------------------------------------------------+
"
);
Ok(())
}
Expand Down Expand Up @@ -2870,7 +2870,7 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> {
| | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] |
| | DataSourceExec: partitions=1, partition_sizes=[1] |
| | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] |
| | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] |
| | AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[count(Int64(1))] |
| | CoalesceBatchesExec: target_batch_size=8192 |
| | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 |
| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |
Expand Down Expand Up @@ -2927,7 +2927,7 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> {
| | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] |
| | DataSourceExec: partitions=1, partition_sizes=[1] |
| | ProjectionExec: expr=[count(*)@1 as count(*), a@0 as a, true as __always_true] |
| | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(*)] |
| | AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[count(*)] |
| | CoalesceBatchesExec: target_batch_size=8192 |
| | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 |
| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |
Expand Down
Loading