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

chore: Reimplement ShuffleWriterExec using interleave_record_batch #1511

Open
wants to merge 6 commits into
base: main
Choose a base branch
from

Conversation

Kontinuation
Copy link
Member

@Kontinuation Kontinuation commented Mar 12, 2025

Closes #1235, #1449 and partially #1446, #1453.

Rationale for this change

Using interleave_record_batch instead of maintaining array builders for each partition improves memory utilization and reduces the possibility of excessive spilling. For instance, the test case test_large_number_of_partitions_spilling spills 3332 times before this change, now it spills only 3 times.

Switching from array builders to interleave_record_batch also reduces the amount of code we have to maintain. Now we can completely get rid of builder.rs, and the logic for handle partitioning and spilling is easier to to track than before.

What changes are included in this PR?

Reimplement ShuffleWriteExec using interleave_record_batch and get rid of the old builder-based approach.

How are these changes tested?

  • Passing existing tests
  • Tested on TPC-H SF=100

@Kontinuation Kontinuation changed the title Reimplement ShuffleWriterExec using interleave_record_batch chore: Reimplement ShuffleWriterExec using interleave_record_batch Mar 12, 2025
@codecov-commenter
Copy link

codecov-commenter commented Mar 12, 2025

Codecov Report

All modified and coverable lines are covered by tests ✅

Project coverage is 58.49%. Comparing base (f09f8af) to head (6ec846d).
Report is 104 commits behind head on main.

Additional details and impacted files
@@             Coverage Diff              @@
##               main    #1511      +/-   ##
============================================
+ Coverage     56.12%   58.49%   +2.36%     
- Complexity      976      977       +1     
============================================
  Files           119      122       +3     
  Lines         11743    12231     +488     
  Branches       2251     2278      +27     
============================================
+ Hits           6591     7154     +563     
+ Misses         4012     3945      -67     
+ Partials       1140     1132       -8     

☔ 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.

@Kontinuation
Copy link
Member Author

The failure in java-native_iceberg_compat test is caused by a bug in the fast shuffle writer. The null buffers written for sliced arrays are not correct. I'll submit a dedicated patch to fix it later.

@andygrove
Copy link
Member

Thanks @Kontinuation. I'll try running some benchmarks today on this.

@andygrove
Copy link
Member

I ran a quick TPC-H benchmark. There is a slight regression in performance, with this PR taking 323 seconds compared to 295 seconds in the main branch. However, we have the advantage of less memory overhead and support for complex types (in theory).

@andygrove
Copy link
Member

The failure in java-native_iceberg_compat test is caused by a bug in the fast shuffle writer. The null buffers written for sliced arrays are not correct. I'll submit a dedicated patch to fix it later.

There is now an option in the Arrow IPC reader to disable re-validating the data and we may be able to remove our custom encoding if that solves the performance issues we saw previously.

@andygrove
Copy link
Member

@EmilyMatt you may be interested in this PR

@EmilyMatt
Copy link
Contributor

EmilyMatt commented Mar 12, 2025

@andygrove Indeed! I think this is a much better implementation memory wise, but can still maintains a huge amount of memory if I understand correctly, as no RecordBatch can be dropped until all of the PartitionIterators are finished with it, meaning data skew can make us keep a lot of idle memory.
I do think on average this will probably balance itself out.
And generally there's a give and take to be done with copying the data(which degrades performance, but can make us use much less memory, e.g., by using take on each record batch to partition it then letting each partition have its own row count), vs having the fastest shuffle possible, which in turn may cause undesired memory spikes in real life scenarios

@andygrove
Copy link
Member

I plan on starting to review this next week since I am busy with the 0.7.0 release at the moment.

@Kontinuation
Copy link
Member Author

Kontinuation commented Mar 19, 2025

I have ran TPC-H SF=100 benchmarks with various off-heap size configurations. The results showed that

  • interleave_record_batch is slower than main branch when no spilling happens.
  • interleave_record_batch is faster running Q10 when off-heap memory is less than 8GB, while the main branch could be slower than Spark because of excessive spilling.
  • interleave_record_batch is much slower running Q18, this problem is still under investigation.

The following table shows detailed results.

On-heap size Off-heap size Spark 3.5.4 Comet main Comet PR Bar plot
3g 3g 1054 s 551 s 523 s tpch_queries_compare_3g
3g 5g 1050s 512s 522s tpch_queries_compare_5g
3g 8g 1032s 490s 492s tpch_queries_compare_8g

Comet main could be slower when running Q10 because it suffers from excessive spilling. Q10 shuffle writes batches containing string columns, the current shuffle writer implementation pre-allocates lots of space for string array builders so it consumes lots of memory even when only a few batches were ingested. We've already seen this in #887.

Here is the comparison of Spark metrics for CometExchange nodes:

Comet main Comet PR
comet-main-exchange comet-interleave-exchange

@mbutrovich
Copy link
Contributor

mbutrovich commented Mar 21, 2025

Dumb question: is take_record_batch an option here (it looks like interleave can actually coalesce a vector of RecordBatch. What would the performance implication be of take vs. interleave?

@Kontinuation
Copy link
Member Author

Dumb question: is take_record_batch an option here (it looks like interleave can actually coalesce a vector of RecordBatch. What would the performance implication be of take vs. interleave?

The take_record_batch function is likely to result in numerous small memory allocations for each input batch, which may lead to poorer performance compared to interleave_record_batch. However, it's essential to prototype both implementations to fully comprehend their performance characteristics.

@Kontinuation
Copy link
Member Author

Kontinuation commented Mar 22, 2025

interleave_record_batch is much slower running Q18, this problem is still under investigation.

This problem happens on macOS 15.3.1 with Apple M1 Pro chip. I suspect that this is an OS-specific problem. I'll rerun the benchmarks on EC2 instances and see if it still happens. I have some interesting findings when troubleshooting this problem. I'd like to share it here to get more understanding about it from other developers.

The slowness of Q18 is not caused by switching to a different shuffle write implementation, it is because one stage of this query sometimes exhibits large performance outlier, and we happen to hit the outlier when benchmarking interleave_record_batch.

The problematic stage is in the 5th Spark job of Q18. It has the following native query plan.

SortMergeJoin: join_type=LeftSemi, on=[(col_0@0, col_0@0)]
  SortExec: expr=[col_0@0 ASC], preserve_partitioning=[false]
    CopyExec [UnpackOrDeepCopy]
      ScanExec: source=[ShuffleQueryStage (unknown), Statistics(sizeInBytes=5.0 GiB, rowCount=1.50E+8)], schema=[col_0: Int64, col_1: Int64, col_2: Decimal128(12, 2), col_3: Date32]
  SortExec: expr=[col_0@0 ASC], preserve_partitioning=[false]
    CopyExec [UnpackOrClone]
      ProjectionExec: expr=[col_0@0 as col_0]
        CometFilterExec: col_1@1 IS NOT NULL AND col_1@1 > Some(31300),22,2
          ProjectionExec: expr=[col_0@0 as col_0, sum@1 as col_1]
            AggregateExec: mode=Final, gby=[col_0@0 as col_0], aggr=[sum]
              ScanExec: source=[Exchange (unknown)], schema=[col_0: Int64, col_1: Decimal128(22, 2), col_2: Boolean]

Sometimes we observe that the aggregation operator is obviously slower than usual, and the stage timeline showed that tasks are taking very long time to finish in a specific period of time:

Normal Slow
SQL Metrics comet-aggr-normal comet-aggr-slow
Stage Timeline comet-timeline-normal comet-timeline-slow

I fired instruments to profile the thread states and see if they are blocked on system calls or if there's some scheduler issues. I found that all the executors are blocked on a mach_msg2_trap system call for more than 4 seconds when the problem was reproduced. This system call is initiated by a Vec resize in SumDecimalGroupsAccumulator::merge_batch. Here is the backtrace:

mach_msg2_trap [0x19597e000 + 3925]	
mach_msg2_internal [0x19597e000 + 79364]	
vm_copy [0x19597e000 + 16392]	
szone_realloc [0x1957c7000 + 23316]	
_malloc_zone_realloc [0x1957c7000 + 199632]	
_realloc [0x1957c7000 + 201852]	
alloc::raw_vec::finish_grow::hd119b9e7c589b6bd [0x30d7ec000 + 37008760]	
alloc::raw_vec::RawVecInner$LT$A$GT$::reserve::do_reserve_and_handle::h6f1db5e8b36dc76d [0x30d7ec000 + 37009068]	
_$LT$datafusion_comet_spark_expr..agg_funcs..sum_decimal..SumDecimalGroupsAccumulator$u20$as$u20$datafusion_expr_common..groups_accumulator..GroupsAccumulator$GT$::merge_batch::haff18eaf0521d806 [0x30d7ec000 + 4524316]	
datafusion_physical_plan::aggregates::row_hash::GroupedHashAggregateStream::group_aggregate_batch::h5261364a61419b41 [0x30d7ec000 + 12827928]	
_$LT$datafusion_physical_plan..aggregates..row_hash..GroupedHashAggregateStream$u20$as$u20$futures_core..stream..Stream$GT$::poll_next::h505f0a302fafaa47 [0x30d7ec000 + 12820756]	
_$LT$datafusion_physical_plan..projection..ProjectionStream$u20$as$u20$futures_core..stream..Stream$GT$::poll_next::he04d32ceceda7b27 [0x30d7ec000 + 13434900]	
_$LT$comet..execution..operators..filter..FilterExecStream$u20$as$u20$futures_core..stream..Stream$GT$::poll_next::had23e7d114a415e4 [0x30d7ec000 + 1230048]	
_$LT$datafusion_physical_plan..projection..ProjectionStream$u20$as$u20$futures_core..stream..Stream$GT$::poll_next::he04d32ceceda7b27 [0x30d7ec000 + 13434900]	
_$LT$comet..execution..operators..copy..CopyStream$u20$as$u20$futures_core..stream..Stream$GT$::poll_next::h5344363c739c6ceb [0x30d7ec000 + 1202008]	
_$LT$datafusion_physical_plan..stream..RecordBatchStreamAdapter$LT$S$GT$$u20$as$u20$futures_core..stream..Stream$GT$::poll_next::h316229feb0b76bab [0x30d7ec000 + 13694588]	
datafusion_physical_plan::joins::sort_merge_join::SortMergeJoinStream::poll_buffered_batches::hf8abd48295b3827c [0x30d7ec000 + 13258056]	
_$LT$datafusion_physical_plan..joins..sort_merge_join..SortMergeJoinStream$u20$as$u20$futures_core..stream..Stream$GT$::poll_next::h942cbfae18ce5bbf [0x30d7ec000 + 13251856]	
Java_org_apache_comet_Native_executePlan [0x30d7ec000 + 1181408]	
0x11296e8d3 [0x0 + 4606847187]	
0x112bdd86f [0x0 + 4609398895]	
0x20000130a7 [0x0 + 137439031463]	

I have added some logs around this Vec resize to measure the size of memory allocated and elapsed time:

         // Make sure we have enough capacity for the additional groups
+        let groups_before = self.sum.len();
+        let allocated_size_before = self.sum.allocated_size();
+        let start_ns = Instant::now();
         self.sum.resize(total_num_groups, 0);
+        let end_ns = Instant::now();
+        let system_time = SystemTime::now();
+        let datetime: DateTime<Local> = DateTime::from(system_time);
+        let formatted_date = datetime.format("%Y-%m-%d %H:%M:%S");
+        let duration = end_ns.duration_since(start_ns);
+        let allocated_size_after = self.sum.allocated_size();
+        if duration.as_secs() >= 1 {
+            println!(
+                "[{}] resize sum, total_num_groups: from {} to {}, cost: {:?}, allocated_size: from {} to {}",
+                formatted_date,
+                groups_before,
+                total_num_groups,
+                duration,
+                allocated_size_before,
+                allocated_size_after
+            );
+        }
+

The results are quite interesting: all the Vec resizes that take long time are always resizing from 4MB to 8MB.

[2025-03-22 11:43:38] resize sum, total_num_groups: from 257716 to 265908, cost: 1.175402708s, allocated_size: from 4194304 to 8388608
[2025-03-22 11:43:42] resize sum, total_num_groups: from 257823 to 266015, cost: 3.334687417s, allocated_size: from 4194304 to 8388608
[2025-03-22 11:43:43] resize sum, total_num_groups: from 258481 to 266673, cost: 1.752088083s, allocated_size: from 4194304 to 8388608
[2025-03-22 11:43:43] resize sum, total_num_groups: from 258330 to 266522, cost: 2.100056958s, allocated_size: from 4194304 to 8388608
[2025-03-22 11:43:43] resize sum, total_num_groups: from 258318 to 266510, cost: 4.440020292s, allocated_size: from 4194304 to 8388608
[2025-03-22 11:43:43] resize sum, total_num_groups: from 257869 to 266061, cost: 1.192288583s, allocated_size: from 4194304 to 8388608
[2025-03-22 11:43:43] resize sum, total_num_groups: from 257632 to 265824, cost: 2.533776709s, allocated_size: from 4194304 to 8388608
...
[2025-03-22 11:45:38] resize sum, total_num_groups: from 258033 to 266225, cost: 2.53457375s, allocated_size: from 4194304 to 8388608
[2025-03-22 11:45:38] resize sum, total_num_groups: from 257077 to 265269, cost: 2.824218625s, allocated_size: from 4194304 to 8388608
[2025-03-22 11:45:39] resize sum, total_num_groups: from 257830 to 266022, cost: 3.062193208s, allocated_size: from 4194304 to 8388608
[2025-03-22 11:45:39] resize sum, total_num_groups: from 257463 to 265655, cost: 1.567723125s, allocated_size: from 4194304 to 8388608
[2025-03-22 11:45:39] resize sum, total_num_groups: from 257922 to 266114, cost: 3.057905042s, allocated_size: from 4194304 to 8388608
[2025-03-22 11:45:39] resize sum, total_num_groups: from 257271 to 265463, cost: 3.084830708s, allocated_size: from 4194304 to 8388608
[2025-03-22 11:45:39] resize sum, total_num_groups: from 257960 to 266152, cost: 3.058556334s, allocated_size: from 4194304 to 8388608

I have summarized the reallocation target size from the logs of running Q18 10 times, the allocation size of Vec when running SumDecimalGroupsAccumulator::merge_batch will usually grow from 128KB to 16MB for each partition/task. It is strange that the slowness only happens when growing from 4MB to 8MB, but not for other reallocation sizes.

Allocation Cost Summary (sorted by average cost)
----------------------------------------------------------------------------------------------------
     From →         To | Count |     Avg Cost |     Min Cost |     Max Cost |   Total Cost
----------------------------------------------------------------------------------------------------
4,194,304 →  8,388,608 |  4000 |     64.51 ms |      1.50 µs |      13.91 s |     258.04 s
8,388,608 → 16,777,216 |  4000 |      2.80 ms |      1.16 ms |     48.09 ms |      11.18 s
2,097,152 →  4,194,304 |  4000 |    152.83 µs |      1.33 µs |      5.33 ms |    611.31 ms
1,048,576 →  2,097,152 |  4000 |     60.74 µs |      1.29 µs |      2.05 ms |    242.97 ms
  524,288 →  1,048,576 |  4000 |     28.98 µs |    292.00 ns |      2.70 ms |    115.92 ms
  262,144 →    524,288 |  4000 |      9.66 µs |     83.00 ns |    775.25 µs |     38.66 ms
  131,072 →    262,144 |  4000 |      7.32 µs |    875.00 ns |      1.94 ms |     29.30 ms
        0 →    131,072 |  4000 |      3.11 µs |    792.00 ns |    761.17 µs |     12.44 ms
        0 →        192 |    10 |    262.50 ns |     41.00 ns |    667.00 ns |      2.62 µs
        0 →         80 |   290 |    187.34 ns |      0.00 ns |      1.25 µs |     54.33 µs
        0 →         96 |   240 |    175.21 ns |      0.00 ns |    875.00 ns |     42.05 µs
        0 →        128 |   100 |    147.50 ns |      0.00 ns |    708.00 ns |     14.75 µs
        0 →         64 |  1110 |    132.08 ns |      0.00 ns |      2.42 µs |    146.61 µs
        0 →        112 |   120 |    127.44 ns |     41.00 ns |    459.00 ns |     15.29 µs
        0 →        144 |    50 |    115.86 ns |     41.00 ns |    292.00 ns |      5.79 µs
        0 →        160 |    30 |    107.10 ns |     41.00 ns |    250.00 ns |      3.21 µs
        0 →        176 |    30 |    105.53 ns |     41.00 ns |    250.00 ns |      3.17 µs
----------------------------------------------------------------------------------------------------

kazuyukitanimura pushed a commit that referenced this pull request Mar 25, 2025
)

## Which issue does this PR close?

Closes #1520.

## Rationale for this change

This is a problem I found when working on #1511, the null bits were not correctly written and caused test failures. This patch is an attempt to fix it.

This patch is only aiming for fixing correctness problems. As #1190 (comment) pointed out, the fast BatchWriter may write full data buffer for sliced `Utf8` arrays, so there's still some performance implications when working with sliced arrays.

## What changes are included in this PR?

Correctly take slicing indices and length into account when writing BooleanBuffers. This applies to null bits of all arrays, and the values of boolean arrays.

## How are these changes tested?

Added a new round-trip test for sliced record batches.
@Kontinuation
Copy link
Member Author

Reran TPC-H SF=100 on an m7i.4xlarge instances with master = local[8], Most of the disk accesses hit the OS cache so the slow EBS didn't affect the query performance too much. The unstable reallocation performance problem we've seen on macOS does not show up and the performance of Q18 is quite stable.

On-heap size Off-heap size Spark 3.5.4 Comet main Comet PR Bar plot
3g 3g 1157 s 579 s 555 s tpch_queries_compare_3g
3g 5g 1150s 573s 556s tpch_queries_compare_5g
3g 8g 1158s 552s 549s tpch_queries_compare_8g
3g 16g 1153s 535s 535s tpch_queries_compare_16g

@Kontinuation
Copy link
Member Author

Kontinuation commented Mar 26, 2025

I have also refactored the handling for repartitioning to a single partition (#1453) in ef79ab5, this avoids saturating the off-heap memory and fixes OOM in the TPC-DS (hash-join) test. We can also separate this out as its own PR and reduce the off-heap size of TPC-DS test from 15g to 4g in this PR.

One difference from the approach described in #1453 is that we coalesce small input batches into larger batches, because we found that small batches hurts the performance of downstream Comet operators. TPC-DS (hash-join) Q72 in the CometTPCDSQuerySuite takes 5 minutes without coalescing, with batch coalescing applied it takes 2 minutes.

@Kontinuation Kontinuation marked this pull request as ready for review March 26, 2025 22:27
@rluvaton
Copy link
Contributor

interleave_record_batch is much slower running Q18, this problem is still under investigation.

This problem happens on macOS 15.3.1 with Apple M1 Pro chip. I suspect that this is an OS-specific problem. I'll rerun the benchmarks on EC2 instances and see if it still happens. I have some interesting findings when troubleshooting this problem. I'd like to share it here to get more understanding about it from other developers.

...

The results are quite interesting: all the Vec resizes that take long time are always resizing from 4MB to 8MB.

...

I have summarized the reallocation target size from the logs of running Q18 10 times, the allocation size of Vec when running SumDecimalGroupsAccumulator::merge_batch will usually grow from 128KB to 16MB for each partition/task. It is strange that the slowness only happens when growing from 4MB to 8MB, but not for other reallocation sizes.

I saw that too on my MacBook Pro M3 Max 15.3.2 but with different sizes, maybe it's related to L1/2/3 cache?

@Kontinuation
Copy link
Member Author

Kontinuation commented Mar 27, 2025

I saw that too on my MacBook Pro M3 Max 15.3.2 but with different sizes, maybe it's related to L1/2/3 cache?

I guess it is a kernel issue. 4 seconds is way too long for copying 4~8MB data even if every memory access is an L3 cache miss, the longest time I've observed for this realloc is 13 seconds.

The kernel could be doing page remapping for large reallocations, and this page remapping could be blocked by lock contention and spent extraordinary long time. I tried to capture kernel stack traces but failed to resolve symbols for the addresses so I'm still not sure what happened.

Copy link
Contributor

@mbutrovich mbutrovich left a comment

Choose a reason for hiding this comment

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

First round: no major changes. This is looking really good!

hashes_buf: Vec<u32>,
/// Partition ids for each row in the current batch.
partition_ids: Vec<u32>,
/// The row indices of the rows in each partition. This array is conceptually dividied into
Copy link
Contributor

Choose a reason for hiding this comment

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

Typo: dividied

Copy link
Member Author

Choose a reason for hiding this comment

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

Fixed typo.

hashes_buf.set_len(batch_size);
partition_ids.set_len(batch_size);
}
let num_output_partitions = partitioning.partition_count();
Copy link
Contributor

Choose a reason for hiding this comment

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

Is assert_ne(num_output_partitions, 1, "Use SinglePartitionShufflePartitioner for 1 output partition.") a valid assertion to add?

Copy link
Member Author

Choose a reason for hiding this comment

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

Added the assertion.

File::open(spill_data.temp_file.path()).map_err(Self::to_df_err)?,
);
std::io::copy(&mut spill_file, &mut output_data).map_err(Self::to_df_err)?;
for (partition_id, (&start, &end)) in partition_starts
Copy link
Contributor

Choose a reason for hiding this comment

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

Could you add high level documentation what this chunk of code is doing? The combination of for with the filter and a nested for make it non-obvious.

Copy link
Member Author

Choose a reason for hiding this comment

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

I have added comments to describe what we are doing here. I have also found that shuffle_partition_ids is not a good name, I have renamed it to partition_row_indices and the code become easier to understand.

let mut partition_counters = vec![0usize; *num_output_partitions];
let partition_counters = &mut scratch.partition_starts;
partition_counters.resize(num_output_partitions + 1, 0);
partition_counters.fill(0);
partition_ids
.iter()
.for_each(|partition_id| partition_counters[*partition_id as usize] += 1);

// accumulate partition counters into partition ends
// e.g. partition counter: [1, 3, 2, 1] => [1, 4, 6, 7]
Copy link
Contributor

Choose a reason for hiding this comment

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

Just to confirm my understanding: this is an inclusive prefix sum? Does scan help the readability here or is that more esoteric? Maybe another downside of scan would be that collect() would allocate a new vector rather than mutating in-place.

Copy link
Member Author

@Kontinuation Kontinuation Mar 28, 2025

Choose a reason for hiding this comment

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

Yes, this is an inclusive prefix sum. If we use scan and also avoids allocating a new vector we have to write the following code, which is not much readable than the current approach.

partition_ends.iter_mut().scan(0, |accum, v| {
    *accum += *v;
    Some(*accum)
}).enumerate().for_each(|(i, sum)| {
    partition_ends[i] = sum;
});

Copy link
Contributor

Choose a reason for hiding this comment

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

Agreed. I don't think that is any more readable. Thank you for the explanation!

@mbutrovich
Copy link
Contributor

mbutrovich commented Mar 27, 2025

While reading this I also wonder if we would be able to hook into DF's new SpillManager. That's a task for another PR, but just leaving the thought here.

@2010YOUY01
Copy link

While reading this I also wonder if we would be able to hook into DF's new SpillManager. That's a task for another PR, but just leaving the thought here.

Its implementation is inspired by ShuffleWriterExec's spilling utilities, and I think there are still some functionality gaps (For example: datafusion's SpillManager does not support general-purpose compression like lz4 yet)
We plan to include those features in the future to make it reusable.

Copy link
Contributor

@mbutrovich mbutrovich left a comment

Choose a reason for hiding this comment

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

LGTM! Thank you for the contribution @Kontinuation! @andygrove was in this code recently so hopefully he can take a look next week (and maybe run one more round of benchmarks on his cluster if he's concerned about performance).

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.

Optimize repartitioning logic in ShuffleWriterExec using interleave_record_batch
7 participants