Skip to content

Spark: Async Spark Micro Batch Planner#15059

Closed
RjLi13 wants to merge 8 commits intoapache:mainfrom
RjLi13:async-micro-batch-planner-spark-3-5
Closed

Spark: Async Spark Micro Batch Planner#15059
RjLi13 wants to merge 8 commits intoapache:mainfrom
RjLi13:async-micro-batch-planner-spark-3-5

Conversation

@RjLi13
Copy link
Contributor

@RjLi13 RjLi13 commented Jan 15, 2026

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.

Changes

  • New AsyncSparkMicroBatchPlanner that queues file scan tasks asynchronously
  • Refactored existing sync logic into SyncSparkMicroBatchPlanner
  • Created SparkMicroBatchPlanner interface for both implementations
  • SparkMicroBatchStream now selects planner based on configuration
  • Created new BaseSparkMicroBatchPlanner to dedupe code between Sync and Async Planners

@github-actions github-actions bot added the spark label Jan 15, 2026
@RjLi13
Copy link
Contributor Author

RjLi13 commented Jan 15, 2026

cc @bryanck


class StreamingOffset extends Offset {
static final StreamingOffset START_OFFSET = new StreamingOffset(-1L, -1, false);
public class StreamingOffset extends Offset {
Copy link
Contributor

Choose a reason for hiding this comment

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

Does this need to be public?

* @param snapshotTotalRows Total rows in the snapshot
*/
StreamingOffset(long snapshotId, long position, boolean scanAllFiles) {
public StreamingOffset(
Copy link
Contributor

@bryanck bryanck Jan 15, 2026

Choose a reason for hiding this comment

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

Also here, curious why this needs to be public. Also below there are a few static methods that were made public.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good catch, I originally had it to match the public interface SparkMicroBatchPlanner, only to realize all of them are implementation details. Reverting and making the planners package private as well

this.minQueuedRows = readConf.maxRecordsPerMicroBatch();
this.readConf = readConf;
this.lastOffsetForTriggerAvailableNow = lastOffsetForTriggerAvailableNow;
this.planFilesCache = Caffeine.newBuilder().maximumSize(10).build();
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: make 10 a constant or configurable

this::refreshAndTrapException, pollingIntervalMs, pollingIntervalMs, TimeUnit.MILLISECONDS);
// Schedule queue fill to run frequently (use polling interval for tests, cap at 100ms for
// production)
long queueFillIntervalMs = Math.min(100L, pollingIntervalMs);
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: make 100L a constant or configurable

private final long position;
private final boolean scanAllFiles;
private final long snapshotTimestampMillis;
private final long snapshotTotalRows;
Copy link
Contributor

@bryanck bryanck Jan 15, 2026

Choose a reason for hiding this comment

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

I wasn't clear why these need to be added, given we already have the snapshot ID?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Reverted all changes to StreamingOffset

@bryanck
Copy link
Contributor

bryanck commented Jan 15, 2026

There is a fair amount of duplicated code between the existing microbatch planner and this, I'm wondering if there are opportunities for reuse.

@RjLi13 RjLi13 force-pushed the async-micro-batch-planner-spark-3-5 branch 2 times, most recently from 97cfdbf to 0cfb405 Compare January 16, 2026 21:42
@RjLi13
Copy link
Contributor Author

RjLi13 commented Jan 16, 2026

There is a fair amount of duplicated code between the existing microbatch planner and this, I'm wondering if there are opportunities for reuse.

Moved duplicated code out to BaseSparkMicroBatchPlanner abstract class that both Planners now extend.

@RjLi13 RjLi13 force-pushed the async-micro-batch-planner-spark-3-5 branch from d31b909 to d7f2883 Compare January 16, 2026 23:16
@bryanck
Copy link
Contributor

bryanck commented Jan 19, 2026

This should be added to the latest Spark version. Once that is merged, we'll backport the changes.


interface SparkMicroBatchPlanner {
List<FileScanTask> planFiles(StreamingOffset start, StreamingOffset end)
throws ExecutionException;
Copy link
Contributor

@bryanck bryanck Jan 19, 2026

Choose a reason for hiding this comment

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

I feel ExecutionException should not be part of the method signature, the implementation can throw an unchecked exception.

List<FileScanTask> fileScanTasks;
try {
fileScanTasks = microBatchPlanner.planFiles(startOffset, endOffset);
} catch (ExecutionException e) {
Copy link
Contributor

@bryanck bryanck Jan 19, 2026

Choose a reason for hiding this comment

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

We can move exception handling to implementation if we make the above change to the interface.

return ((ReadMaxFiles) limit).maxFiles();
}
@Override
public Map<String, String> metrics(Optional<Offset> latestConsumedOffset) {
Copy link
Contributor

Choose a reason for hiding this comment

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

It seems like this is only useful for the async planner, for the sync planner it will report the same thing.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Any recommendations on how to isolate it to Async Planner? Else I will remove it, and users can rely on logging instead, which will minimize implementing on ReportsSourceMetrics Interface

@bryanck
Copy link
Contributor

bryanck commented Jan 19, 2026

If you have any benchmarks or metrics that show the benefit of this, that would be helpful.

List<FileScanTask> fileScanTasks = Lists.newArrayList();
StreamingOffset batchStartOffset =
StreamingOffset.START_OFFSET.equals(start)
? SparkMicroBatchStream.determineStartingOffset(table(), fromTimestamp)
Copy link
Contributor

Choose a reason for hiding this comment

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

There is a circular dependency here, it would be better to have the planners not rely on SparkMicroBatchStream.

}

private static StreamingOffset determineStartingOffset(Table table, Long fromTimestamp) {
static StreamingOffset determineStartingOffset(Table table, long fromTimestamp) {
Copy link
Contributor

Choose a reason for hiding this comment

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

IMO we should move this somewhere else so the planners don't need to reference SparkMicroBatchStream.

@RjLi13
Copy link
Contributor Author

RjLi13 commented Jan 20, 2026

This should be added to the latest Spark version. Once that is merged, we'll backport the changes.

@bryanck Would it be ok to move these files to 4.1 in this PR? Or would you prefer having a separate PR there and moving conversation and review to that PR?

@RjLi13 RjLi13 force-pushed the async-micro-batch-planner-spark-3-5 branch from d7f2883 to f5e7c6b Compare January 21, 2026 22:20
@RjLi13
Copy link
Contributor Author

RjLi13 commented Jan 22, 2026

commit f5e7c6b Spark 3.5: Address circular dep, unneeded code: Addresses all the review comments about circular dependency, executionexception and possibly unnecessary metrics call

commit [e079fd3] Spark: Move feature to latest Spark: Addresses comment about making this feature for latest / current Spark, then separate PR for backporting

cc @bryanck

The only remaining comment I did not address for code review was #15059 (comment), to discuss.

@RjLi13
Copy link
Contributor Author

RjLi13 commented Jan 22, 2026

Sorry the branch name has 3-5 in it. But change PR to now target 4.1.

@RjLi13 RjLi13 changed the title Spark 3.5: Async Spark Micro Batch Planner Spark: Async Spark Micro Batch Planner Jan 22, 2026
// Data appended after the timestamp should appear
appendData(data);
// Allow async background thread to refresh, else test sometimes fails
Thread.sleep(50);
Copy link
Contributor Author

Choose a reason for hiding this comment

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

I'm open to other suggestions. This test sometimes fails, because background thread runs and isn't able to refresh before test finishes executing. Hence I added sleep here to wait before test finishes. In real use case, user should not have issues with this.

It failed for me locally once before this, but ran multiple times without it failing.

@RjLi13
Copy link
Contributor Author

RjLi13 commented Jan 27, 2026

If you have any benchmarks or metrics that show the benefit of this, that would be helpful.

I don't have the production metrics or benchmarks, but I took a stab at trying to create a JMH benchmark on latestOffset() calls with different snapshot numbers. Hopefully, this showcases sync planner scales linearly to number of snapshots whereas async planner is constant, but trading off detection latency and memory usage. It is great for tables with large number of snapshots and where processing time takes long enough for background thread to poll. Here is the link to it: RjLi13#2

@bryanck
Copy link
Contributor

bryanck commented Feb 2, 2026

nit: I feel the refactor could have reused the existing code a little bit better, i.e. renamed the existing class to be the base class first, that would help reduce the size of the PR

@RjLi13
Copy link
Contributor Author

RjLi13 commented Feb 11, 2026

Spoke with @bryanck offline. Decided that this PR should be split into two, one refactoring SparkMicroBatchStream logic into SyncSparkMicroBatchPlanner, and two actually introducing async. This PR will be closed in favor for those two new ones:
#15298
#15299

@RjLi13 RjLi13 closed this Feb 11, 2026
@RjLi13 RjLi13 deleted the async-micro-batch-planner-spark-3-5 branch February 11, 2026 19:53
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants