jerrypeng commented on code in PR #38517: URL: https://github.com/apache/spark/pull/38517#discussion_r1048946439
########## sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecution.scala: ########## @@ -0,0 +1,282 @@ +/* + * 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.spark.sql.execution.streaming + +import java.util.concurrent._ +import java.util.concurrent.atomic.AtomicLong + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.streaming.WriteToStream +import org.apache.spark.sql.streaming.Trigger +import org.apache.spark.util.{Clock, ThreadUtils} + +object AsyncProgressTrackingMicroBatchExecution { + val ASYNC_PROGRESS_TRACKING_ENABLED = "asyncProgressTrackingEnabled" + val ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS = + "asyncProgressTrackingCheckpointIntervalMs" + + // for testing purposes + val ASYNC_PROGRESS_TRACKING_OVERRIDE_SINK_SUPPORT_CHECK = + "_asyncProgressTrackingOverrideSinkSupportCheck" + + private def getAsyncProgressTrackingCheckpointingIntervalMs( + extraOptions: Map[String, String]): Long = { + extraOptions + .getOrElse( + AsyncProgressTrackingMicroBatchExecution.ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS, + "1000" + ) + .toLong + } +} + +/** + * Class to execute micro-batches when async progress tracking is enabled + */ +class AsyncProgressTrackingMicroBatchExecution( + sparkSession: SparkSession, + trigger: Trigger, + triggerClock: Clock, + extraOptions: Map[String, String], + plan: WriteToStream) + extends MicroBatchExecution(sparkSession, trigger, triggerClock, extraOptions, plan) { + + protected val asyncProgressTrackingCheckpointingIntervalMs: Long + = AsyncProgressTrackingMicroBatchExecution + .getAsyncProgressTrackingCheckpointingIntervalMs(extraOptions) + + // Offsets that are ready to be committed by the source. + // This is needed so that we can call source commit in the same thread as micro-batch execution + // to be thread safe + private val sourceCommitQueue = new ConcurrentLinkedQueue[OffsetSeq]() + + // to cache the batch id of the last batch written to storage + private val lastBatchPersistedToDurableStorage = new AtomicLong(-1) + + override val triggerExecutor: TriggerExecutor = validateAndGetTrigger() + + private var isFirstBatch: Boolean = true + + // thread pool is only one thread because we want offset + // writes to execute in order in a serialized fashion + protected val asyncWritesExecutorService + = ThreadUtils.newDaemonSingleThreadExecutorWithRejectedExecutionHandler( + "async-log-write", + 2, // one for offset commit and one for completion commit + new RejectedExecutionHandler() { + override def rejectedExecution(r: Runnable, executor: ThreadPoolExecutor): Unit = { + try { + if (!executor.isShutdown) { + val start = System.currentTimeMillis() + executor.getQueue.put(r) + logDebug( + s"Async write paused execution for " + + s"${System.currentTimeMillis() - start} due to task queue being full." + ) + } + } catch { + case e: InterruptedException => + Thread.currentThread.interrupt() + throw new RejectedExecutionException("Producer interrupted", e) + case e: Throwable => + logError("Encountered error in async write executor service", e) + errorNotifier.markError(e) + } + } + }) + + override val offsetLog = new AsyncOffsetSeqLog( + sparkSession, + checkpointFile("offsets"), + asyncWritesExecutorService, + asyncProgressTrackingCheckpointingIntervalMs, + clock = triggerClock + ) + + override val commitLog = + new AsyncCommitLog(sparkSession, checkpointFile("commits"), asyncWritesExecutorService) + + override def markMicroBatchExecutionStart(): Unit = { + // check if pipeline is stateful + checkNotStatefulPipeline + } + + override def cleanUpLastExecutedMicroBatch(): Unit = { + // this is a no op for async progress tracking since we only want to commit sources only + // after the offset WAL commit has be successfully written + } + + /** + * Should not call super method as we need to do something completely different + * in this method for async progress tracking + */ + override def markMicroBatchStart(): Unit = { + // Because we are using a thread pool with only one thread, async writes to the offset log + // are still written in a serial / in order fashion + offsetLog + .addAsync(currentBatchId, availableOffsets.toOffsetSeq(sources, offsetSeqMetadata)) + .thenAccept(tuple => { + val (batchId, persistedToDurableStorage) = tuple + if (persistedToDurableStorage) { + + // batch id cache not initialized + if (lastBatchPersistedToDurableStorage.get == -1) { + lastBatchPersistedToDurableStorage.set( + offsetLog.getPrevBatchFromStorage(batchId).getOrElse(-1)) + } + + if (batchId != 0 && lastBatchPersistedToDurableStorage.get != -1) { + // sanity check to make sure batch ids are monotonically increasing + assert(lastBatchPersistedToDurableStorage.get < batchId) + val prevBatchOff = offsetLog.get(lastBatchPersistedToDurableStorage.get()) + if (prevBatchOff.isDefined) { + // Offset is ready to be committed by the source. Add to queue + sourceCommitQueue.add(prevBatchOff.get) + } else { + throw new IllegalStateException( + s"batch ${lastBatchPersistedToDurableStorage.get()} doesn't exist" Review Comment: will improve error message -- 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]
