Skip to content

fix: distributed RangePartitioning bounds calculation with native shuffle#2258

Merged
mbutrovich merged 47 commits intoapache:mainfrom
mbutrovich:fix_range_partitioning
Sep 24, 2025
Merged

fix: distributed RangePartitioning bounds calculation with native shuffle#2258
mbutrovich merged 47 commits intoapache:mainfrom
mbutrovich:fix_range_partitioning

Conversation

@mbutrovich
Copy link
Copy Markdown
Contributor

@mbutrovich mbutrovich commented Aug 28, 2025

Which issue does this PR close?

Closes #1906.

Rationale for this change

#1862 tried to implement RangePartitioning with native shuffle. The implementation for bounds calculation didn't work in a distributed setting because executors calculated their own partition boundaries.

What changes are included in this PR?

This modifies the flow for the driver to calculate the boundaries (like Spark). At a high level:

  • Hoist code from Spark's ShuffleExchangeExec for using Spark's RangePartitioner to calculate boundary rows.
  • Serialize boundary rows to native side.
  • Deserialize boundary rows and pass as part of the partitioning scheme. Each executor should have the boundary values now.
  • Remove range_partitioner.rs which performed reservoir sampling and bounds calculations in native code.

How are these changes tested?

@mbutrovich mbutrovich self-assigned this Aug 28, 2025
@codecov-commenter
Copy link
Copy Markdown

codecov-commenter commented Aug 28, 2025

Codecov Report

❌ Patch coverage is 87.35632% with 11 lines in your changes missing coverage. Please review.
✅ Project coverage is 58.46%. Comparing base (f09f8af) to head (c87aba7).
⚠️ Report is 539 commits behind head on main.

Files with missing lines Patch % Lines
...t/execution/shuffle/CometNativeShuffleWriter.scala 80.95% 3 Missing and 5 partials ⚠️
...t/execution/shuffle/CometShuffleExchangeExec.scala 90.00% 1 Missing and 2 partials ⚠️
Additional details and impacted files
@@             Coverage Diff              @@
##               main    #2258      +/-   ##
============================================
+ Coverage     56.12%   58.46%   +2.34%     
- Complexity      976     1440     +464     
============================================
  Files           119      146      +27     
  Lines         11743    13520    +1777     
  Branches       2251     2351     +100     
============================================
+ Hits           6591     7905    +1314     
- Misses         4012     4381     +369     
- Partials       1140     1234      +94     

☔ View full report in Codecov by Sentry.
📢 Have feedback on the report? Share it here.

🚀 New features to boost your workflow:
  • ❄️ Test Analytics: Detect flaky tests, report on failures, and find test suite problems.

@mbutrovich mbutrovich changed the title fix: RangePartitioning boundaries with native shuffle fix: RangePartitioning with native shuffle Aug 28, 2025
@mbutrovich mbutrovich added this to the 0.11.0 milestone Sep 11, 2025
@mbutrovich mbutrovich marked this pull request as ready for review September 16, 2025 21:25
…erate the partitioning scheme. This solves the issue where the input schema says it contains dictionaries that were later going to be unpacked by CopyExec. Will open an issue to understand why we even wrap the child in CopyExec in the first place.
# Conflicts:
#	native/core/src/execution/planner.rs
@mbutrovich
Copy link
Copy Markdown
Contributor Author

Depends on #2434

Updated this branch after merging and looks clean still.

@mbutrovich mbutrovich changed the title fix: RangePartitioning with native shuffle fix: distributed RangePartitioning bounds calculation with native shuffle Sep 22, 2025
@mbutrovich
Copy link
Copy Markdown
Contributor Author

For Spark it can be something like

That's effectively what the test "fix: range partitioning #1906" does, but I can abstract out the partitions bounds checking and add some more interesting data sets.

MapStatus.apply(SparkEnv.get.blockManager.shuffleServerId, partitionLengths, mapId)
}

private def isSinglePartitioning(p: Partitioning): Boolean = p match {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the entire method might be simplified just
p.numPartitions <= 1

?

Copy link
Copy Markdown
Contributor Author

@mbutrovich mbutrovich Sep 22, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The number of partition bounds can be less than the number of target partitions based on value cardinality, so we still need to check rangePartitionBounds.

Comment thread common/src/main/scala/org/apache/comet/CometConf.scala Outdated
Comment thread native/core/src/execution/planner.rs Outdated
Comment thread native/core/src/execution/planner.rs Outdated
val numParts = rdd.getNumPartitions

// The code block below is mostly brought over from
// ShuffleExchangeExec::prepareShuffleDependency
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It might be non trivial to do so but we could think about this being a plan we could execute on the native side. Essentially, your original range partitioner but distributed.

serializer: Serializer,
metrics: Map[String, SQLMetric]): ShuffleDependency[Int, ColumnarBatch, ColumnarBatch] = {
val numParts = rdd.getNumPartitions

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we report the time spent in this? It might be useful to decide if this is worthy of optimization.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'll open a followup issue.

@@ -26,15 +27,15 @@ pub enum CometPartitioning {
Hash(Vec<Arc<dyn PhysicalExpr>>, usize),
/// Allocate rows based on the lexical order of one of more expressions and the specified number of
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm thinking would be that intuitive for the user to have

Arc<RowConverter>, Vec<OwnedRow>

here? 🤔

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you expand on this please? I'm not sure I understand the requested change.

Copy link
Copy Markdown
Contributor

@comphead comphead Sep 24, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry for misleading comment.
I was thinking if to compare with others variants like Hash

 Hash(Vec<Arc<dyn PhysicalExpr>>, usize),

it is quite intuitive that hash depends on numPartitions and expression that supposed to be hashed.

for Range it is

RangePartitioning(LexOrdering, usize, Arc<RowConverter>, Vec<OwnedRow>),

which looks no so intuitive IMO, because cannot say when reading what is the meaning of last 2 params.
Anyway, this design question can be addressed in follow up if needed

Comment thread native/core/src/execution/planner.rs Outdated
// Create a RowConverter and use to create OwnedRows from the Arrays
let converter = RowConverter::new(sort_fields)?;
let rows = converter.convert_columns(&arrays)?;
let owned_rows: Vec<OwnedRow> = rows.iter().map(|row| row.owned()).collect();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

maybe we comment here what is owned_rows here?
Is it actual rows before shuffle?

For simplicity attaching a diagram with RR flow

                 [ Before Shuffle ]
 Executor 1: (5,A) (15,B) (30,C)    Executor 2: (40,D) (70,E) (90,F)
                          |                               |
                          v                               v

                 [ Shuffle Write: Buckets on Disk ]
 Executor 1: [P0:(5,15,30)] [P1: ] [P2: ]   Executor 2: [P0: ] [P1:(40)] [P2:(70,90)]

                          |                               |
                          v                               v

                 [ Shuffle Read: Reducers Fetch Buckets ]
 Reducer P0  <----  E1.Bucket0 + E2.Bucket0  ---->  (5,15,30)
 Reducer P1  <----  E1.Bucket1 + E2.Bucket1  ---->  (40)
 Reducer P2  <----  E1.Bucket2 + E2.Bucket2  ---->  (70,90)

                 [ After Shuffle = Range Partitions ]
 P0: (5,15,30)   P1: (40)   P2: (70,90)

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

owned_rows are the boundary values. I can make it more explicit.

.doc("Whether to enable range partitioning for Comet native shuffle.")
.booleanConf
.createWithDefault(false)
.createWithDefault(true)
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should we keep it as false

then run some benches and real tests with this param true and later enable it by default?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I discussed with @andygrove and we were comfortable merging with true back in June. I think if you're opting into native shuffle we should try to accelerate all partitioning schemes, and if we discover issues it can be toggled off.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Enabling it by default now gives us more opportunities to find bugs over the next few weeks before we release 0.11.0 and we can always disable if we find issues in that time.

Copy link
Copy Markdown
Contributor

@comphead comphead left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @mbutrovich I left some minors, but overall I think the PR would be good to go

mbutrovich and others added 2 commits September 24, 2025 12:51
…fle/CometNativeShuffleWriter.scala

Co-authored-by: Oleks V <comphead@users.noreply.github.com>
Copy link
Copy Markdown
Member

@andygrove andygrove left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is a significant improvement! Thanks @mbutrovich

@mbutrovich mbutrovich merged commit 25d5924 into apache:main Sep 24, 2025
102 checks passed
coderfender pushed a commit to coderfender/datafusion-comet that referenced this pull request Dec 13, 2025
@mbutrovich mbutrovich deleted the fix_range_partitioning branch March 13, 2026 18:58
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

RangePartitioning does not yield correct results with native shuffle

5 participants