bryanck commented on code in PR #15059: URL: https://github.com/apache/iceberg/pull/15059#discussion_r2695671104
########## spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/AsyncSparkMicroBatchPlanner.java: ########## @@ -0,0 +1,636 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import java.util.LinkedList; +import java.util.List; +import java.util.Locale; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.iceberg.DataOperations; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.MicroBatches; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.SparkReadConf; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.spark.sql.connector.read.streaming.Offset; +import org.apache.spark.sql.connector.read.streaming.ReadAllAvailable; +import org.apache.spark.sql.connector.read.streaming.ReadLimit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class AsyncSparkMicroBatchPlanner implements SparkMicroBatchPlanner, AutoCloseable { + private static final Logger LOG = LoggerFactory.getLogger(AsyncSparkMicroBatchPlanner.class); + + private final Table table; + private final long minQueuedFiles; + private final long minQueuedRows; + private final SparkReadConf readConf; + + // Cache for planFiles results to handle duplicate calls + private final Cache<Pair<StreamingOffset, StreamingOffset>, List<FileScanTask>> planFilesCache; + + // Queue to buffer pre-fetched file scan tasks + private final LinkedBlockingQueue<Pair<StreamingOffset, FileScanTask>> queue; + + // Background executor for async operations + private final ScheduledExecutorService executor; + + // Error tracking + private volatile Throwable refreshFailedThrowable; + private volatile Throwable fillQueueFailedThrowable; + + // Tracking queue state + private final AtomicLong queuedFileCount = new AtomicLong(0); + private final AtomicLong queuedRowCount = new AtomicLong(0); + private volatile Pair<StreamingOffset, FileScanTask> tail; + private Snapshot lastQueuedSnapshot; + private boolean stopped; + + // Cap for Trigger.AvailableNow - don't process beyond this offset + private final StreamingOffset lastOffsetForTriggerAvailableNow; + + /** + * This class manages a queue of FileScanTask + StreamingOffset. On creation, it starts up an + * asynchronous polling process which populates the queue when a new snapshot arrives or the + * minimum amount of queued data is too low. + * + * <p>Note: this will capture the state of the table when snapshots are added to the queue. If a + * snapshot is expired after being added to the queue, the job will still process it. + */ + public AsyncSparkMicroBatchPlanner( + Table table, + SparkReadConf readConf, + StreamingOffset initialOffset, + StreamingOffset maybeEndOffset, + StreamingOffset lastOffsetForTriggerAvailableNow) { + this.table = table; + this.minQueuedFiles = readConf.maxFilesPerMicroBatch(); + this.minQueuedRows = readConf.maxRecordsPerMicroBatch(); + this.readConf = readConf; + this.lastOffsetForTriggerAvailableNow = lastOffsetForTriggerAvailableNow; + this.planFilesCache = Caffeine.newBuilder().maximumSize(10).build(); + this.queue = new LinkedBlockingQueue<>(); + + table.refresh(); + // Synchronously add data to the queue to meet our initial constraints + fillQueue(initialOffset, maybeEndOffset); + + this.executor = + Executors.newSingleThreadScheduledExecutor( + r -> { + Thread thread = new Thread(r, "iceberg-async-planner-" + table.name()); + thread.setDaemon(true); + return thread; + }); + // Schedule table refresh at configured interval + long pollingIntervalMs = readConf.streamingSnapshotPollingIntervalMs(); + this.executor.scheduleWithFixedDelay( + 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); Review Comment: nit: make `100L` a constant or configurable -- 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]
