yaooqinn opened a new pull request, #54135:
URL: https://github.com/apache/spark/pull/54135
### What changes were proposed in this pull request?
This PR adds a new optimizer rule `CrossJoinArrayContainsToInnerJoin` that
converts cross joins with `array_contains` filter into inner joins using
explode/unnest.
**Transformation:**
```
-- Before (O(N×M) cross join):
Filter(array_contains(arr, elem))
CrossJoin(left, right)
-- After (O(N+M) inner join):
InnerJoin(unnested = elem)
Generate(Explode(ArrayDistinct(arr)), left)
right
```
This optimization is ported from Presto's
`CrossJoinWithArrayContainsToInnerJoin` optimizer rule.
### Why are the changes needed?
Queries with `array_contains` in join conditions currently execute as
expensive cross joins with filter. This optimization significantly improves
performance by converting them to efficient inner joins.
**Benchmark Results (10K orders × 1K items):**
| JDK | Unoptimized | Optimized | Speedup |
|-----|-------------|-----------|---------|
| 17 | 504 ms | 45 ms | **11.2X** |
| 21 | 582 ms | 36 ms | **16.1X** |
### Does this PR introduce _any_ user-facing change?
No. This is an internal optimizer improvement that automatically applies to
applicable queries.
### How was this patch tested?
1. **Unit tests:** `CrossJoinArrayContainsToInnerJoinSuite` with 6 test cases
2. **Microbenchmarks:** `CrossJoinArrayContainsToInnerJoinBenchmark` run on
JDK 17 and 21
3. All existing tests pass
### Was this patch authored or co-authored using generative AI tooling?
Yes, GitHub Copilot CLI was used to assist with porting the optimization
from Presto.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]