Spark 4.1: New Async Spark Micro Batch Planner#15299
Spark 4.1: New Async Spark Micro Batch Planner#15299RjLi13 wants to merge 4 commits intoapache:mainfrom
Conversation
7815e12 to
64f07d6
Compare
|
Reposting this comment about benchmark here: #15059 (comment) |
|
@bryanck @singhpk234 any chance you were able to review this? Thanks in advance! |
| class AsyncSparkMicroBatchPlanner extends BaseSparkMicroBatchPlanner implements AutoCloseable { | ||
| private static final Logger LOG = LoggerFactory.getLogger(AsyncSparkMicroBatchPlanner.class); | ||
| private static final int PLAN_FILES_CACHE_MAX_SIZE = 10; | ||
| private static final long QUEUE_POLL_TIMEOUT_MS = 100L; // 100 ms |
There was a problem hiding this comment.
These could be configurable but it also is valuable not to add too many options.
There was a problem hiding this comment.
Yup I was thinking there were a few knobs already based on what I documented, and these were less critical for users to tune.
|
This LGTM! Thanks @RjLi13 for the contribution. @singhpk234 do you happen to have any feedback? |
This feature was originally built by Drew Goya <dgoya@netflix.com> for Spark 3.3 and Iceberg 1.4.
ea8c9ad to
98d0edf
Compare
|
my bad I accidentally pulled instead of force pushed after rebase and brought in the commits from main into this PR. Fixed now, but got a bunch of labels, this only touches Spark. And also Docs if that counts |
|
Also @bryanck thanks for the approval. Would you and @singhpk234 mind taking a look again as I updated two doc pages with the user facing config and also small blurb in the structured streaming section on this feature? |
|
The docs LGTM as well. |
This is part 2 after splitting PR #15059
Part 1 PR is here: #15298.
This PR focuses on only introducing the new async spark micro batch planner and all changes to enable it.
Full context is in #15059 but posted below again:
Implements a new feature for Spark Structured Streaming and Iceberg users known as Async Spark Micro Batch Planner
Currently Microbatch planning in Iceberg is synchronous. Streaming queries plan out what batches to read and how many rows / files in each batch. Then it processes the data and repeats. By introducing an async planner, it improves streaming performance by pre-fetching table metadata and file scan tasks in a background thread, reducing micro-batch planning latency. This way planning can overlap with data processing and speed up dealing with large volumes.
This PR adds the option for users to set spark.sql.iceberg.async-micro-batch-planning-enabled if they want to use async planning. The code in SparkMicroBatchStream.java is moved to SyncSparkMicroBatchPlanner.java and SparkMicroBatchStream configures which planner to use. This option is defaulted to false, so existing behavior is unchanged.
This feature was originally authored by Drew Goya in our Netflix fork for Spark 3.3 & Iceberg 1.4. I built upon Drew's work by porting this to Spark 3.5 4.1 and current Iceberg version.