From e33cdfb317498b04e077c4d6356fc3848cd78da0 Mon Sep 17 00:00:00 2001 From: Bo Zhang Date: Wed, 1 Sep 2021 15:02:21 +0900 Subject: [PATCH] [SPARK-36533][SS] Trigger.AvailableNow for running streaming queries like Trigger.Once in multiple batches ### What changes were proposed in this pull request? This change creates a new type of Trigger: Trigger.AvailableNow for streaming queries. It is like Trigger.Once, which process all available data then stop the query, but with better scalability since data can be processed in multiple batches instead of one. To achieve this, this change proposes a new interface `SupportsTriggerAvailableNow`, which is an extension of `SupportsAdmissionControl`. It has one method, `prepareForTriggerAvailableNow`, which will be called at the beginning of streaming queries with Trigger.AvailableNow, to let the source record the offset for the current latest data at the time (a.k.a. the target offset for the query). The source should then behave as if there is no new data coming in after the beginning of the query, i.e., the source will not return an offset higher than the target offset when `latestOffset` is called. This change also updates `FileStreamSource` to be an implementation of `SupportsTriggerAvailableNow`. For other sources that does not implement `SupportsTriggerAvailableNow`, this change also provides a new class `FakeLatestOffsetSupportsTriggerAvailableNow`, which wraps the sources and makes them support Trigger.AvailableNow, by overriding their `latestOffset` method to always return the latest offset at the beginning of the query. ### Why are the changes needed? Currently streaming queries with Trigger.Once will always load all of the available data in a single batch. Because of this, the amount of data a query can process is limited, or Spark driver will run out of memory. ### Does this PR introduce _any_ user-facing change? Users will be able to use Trigger.AvailableNow (to process all available data then stop the streaming query) with this change. ### How was this patch tested? Added unit tests. Closes #33763 from bozhang2820/new-trigger. Authored-by: Bo Zhang Signed-off-by: Jungtaek Lim --- .../structured-streaming-programming-guide.md | 21 ++ .../SupportsTriggerAvailableNow.java | 41 +++ .../apache/spark/sql/streaming/Trigger.java | 21 +- .../AvailableNowDataStreamWrapper.scala | 88 +++++++ .../AvailableNowMicroBatchStreamWrapper.scala | 39 +++ .../streaming/AvailableNowSourceWrapper.scala | 38 +++ .../streaming/FileStreamSource.scala | 22 +- .../streaming/MicroBatchExecution.scala | 84 +++++-- .../execution/streaming/TriggerExecutor.scala | 12 +- .../sql/execution/streaming/Triggers.scala | 8 +- .../sql/streaming/FileStreamSourceSuite.scala | 82 ++++++ .../streaming/TriggerAvailableNowSuite.scala | 233 ++++++++++++++++++ 12 files changed, 656 insertions(+), 33 deletions(-) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SupportsTriggerAvailableNow.java create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AvailableNowDataStreamWrapper.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AvailableNowMicroBatchStreamWrapper.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AvailableNowSourceWrapper.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/streaming/TriggerAvailableNowSuite.scala diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 18dfbec794bd8..6e98d5a1ca8b6 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -2748,6 +2748,15 @@ Here are the different kinds of triggers that are supported. cluster. In some case, this may lead to significant cost savings. + + Available-now micro-batch + + Similar to queries one-time micro-batch trigger, the query will process all the available data and then + stop on its own. The difference is that, it will process the data in (possibly) multiple micro-batches + based on the source options (e.g. maxFilesPerTrigger for file source), which will result + in better query scalability. + + Continuous with fixed checkpoint interval
(experimental) @@ -2782,6 +2791,12 @@ df.writeStream .trigger(Trigger.Once()) .start() +// Available-now trigger +df.writeStream + .format("console") + .trigger(Trigger.AvailableNow()) + .start() + // Continuous trigger with one-second checkpointing interval df.writeStream .format("console") @@ -2814,6 +2829,12 @@ df.writeStream .trigger(Trigger.Once()) .start(); +// Available-now trigger +df.writeStream + .format("console") + .trigger(Trigger.AvailableNow()) + .start(); + // Continuous trigger with one-second checkpointing interval df.writeStream .format("console") diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SupportsTriggerAvailableNow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SupportsTriggerAvailableNow.java new file mode 100644 index 0000000000000..47662dc97cc12 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SupportsTriggerAvailableNow.java @@ -0,0 +1,41 @@ +/* + * 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.connector.read.streaming; + +import org.apache.spark.annotation.Evolving; + +/** + * An interface for streaming sources that supports running in Trigger.AvailableNow mode, which + * will process all the available data at the beginning of the query in (possibly) multiple batches. + * + * This mode will have better scalability comparing to Trigger.Once mode. + * + * @since 3.3.0 + */ +@Evolving +public interface SupportsTriggerAvailableNow extends SupportsAdmissionControl { + + /** + * This will be called at the beginning of streaming queries with Trigger.AvailableNow, to let the + * source record the offset for the current latest data at the time (a.k.a the target offset for + * the query). The source will behave as if there is no new data coming in after the target + * offset, i.e., the source will not return an offset higher than the target offset when + * {@link #latestOffset(Offset, ReadLimit) latestOffset} is called. + */ + void prepareForTriggerAvailableNow(); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/streaming/Trigger.java b/sql/core/src/main/java/org/apache/spark/sql/streaming/Trigger.java index 1bd7b825328db..b6e105cfe9184 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/streaming/Trigger.java +++ b/sql/core/src/main/java/org/apache/spark/sql/streaming/Trigger.java @@ -19,12 +19,13 @@ import java.util.concurrent.TimeUnit; -import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.execution.streaming.ProcessingTimeTrigger; import scala.concurrent.duration.Duration; +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.execution.streaming.AvailableNowTrigger$; import org.apache.spark.sql.execution.streaming.ContinuousTrigger; import org.apache.spark.sql.execution.streaming.OneTimeTrigger$; +import org.apache.spark.sql.execution.streaming.ProcessingTimeTrigger; /** * Policy used to indicate how often results should be produced by a [[StreamingQuery]]. @@ -89,8 +90,10 @@ public static Trigger ProcessingTime(String interval) { } /** - * A trigger that process only one batch of data in a streaming query then terminates - * the query. + * A trigger that processes all available data in a single batch then terminates the query. + * + * For better scalability, AvailableNow can be used alternatively to process the data in + * multiple batches. * * @since 2.2.0 */ @@ -98,6 +101,16 @@ public static Trigger Once() { return OneTimeTrigger$.MODULE$; } + /** + * A trigger that processes all available data at the start of the query in one or multiple + * batches, then terminates the query. + * + * @since 3.3.0 + */ + public static Trigger AvailableNow() { + return AvailableNowTrigger$.MODULE$; + } + /** * A trigger that continuously processes streaming data, asynchronously checkpointing at * the specified interval. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AvailableNowDataStreamWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AvailableNowDataStreamWrapper.scala new file mode 100644 index 0000000000000..0dc5104762794 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AvailableNowDataStreamWrapper.scala @@ -0,0 +1,88 @@ +/* + * 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 org.apache.spark.internal.Logging +import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, ReadLimit, SparkDataStream, SupportsAdmissionControl, SupportsTriggerAvailableNow} +import org.apache.spark.sql.connector.read.streaming + +/** + * This class wraps a [[SparkDataStream]] and makes it support Trigger.AvailableNow, by overriding + * its [[latestOffset]] method to always return the latest offset at the beginning of the query. + */ +class AvailableNowDataStreamWrapper(val delegate: SparkDataStream) + extends SparkDataStream with SupportsTriggerAvailableNow with Logging { + + private var fetchedOffset: streaming.Offset = _ + + override def initialOffset(): streaming.Offset = delegate.initialOffset() + + override def deserializeOffset(json: String): streaming.Offset = delegate.deserializeOffset(json) + + override def commit(end: streaming.Offset): Unit = delegate.commit(end) + + override def stop(): Unit = delegate.stop() + + private def getInitialOffset: streaming.Offset = { + delegate match { + case _: Source => null + case m: MicroBatchStream => m.initialOffset + } + } + + /** + * Fetch and store the latest offset for all available data at the beginning of the query. + */ + override def prepareForTriggerAvailableNow(): Unit = { + fetchedOffset = delegate match { + case s: SupportsAdmissionControl => + s.latestOffset(getInitialOffset, ReadLimit.allAvailable()) + case s: Source => s.getOffset.orNull + case m: MicroBatchStream => m.latestOffset() + case s => throw new IllegalStateException(s"Unexpected source: $s") + } + } + + /** + * Always return [[ReadLimit.allAvailable]] + */ + override def getDefaultReadLimit: ReadLimit = delegate match { + case s: SupportsAdmissionControl => + val limit = s.getDefaultReadLimit + if (limit != ReadLimit.allAvailable()) { + logWarning(s"The read limit $limit is ignored because source $delegate does not " + + "support running Trigger.AvailableNow queries.") + } + ReadLimit.allAvailable() + + case _ => ReadLimit.allAvailable() + } + + /** + * Return the latest offset pre-fetched in [[prepareForTriggerAvailableNow]]. + */ + override def latestOffset(startOffset: streaming.Offset, limit: ReadLimit): streaming.Offset = + fetchedOffset + + override def reportLatestOffset: streaming.Offset = delegate match { + // Return the real latest offset here since this is only used for metrics + case s: SupportsAdmissionControl => s.reportLatestOffset() + case s: Source => s.getOffset.orNull + case s: MicroBatchStream => s.latestOffset() + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AvailableNowMicroBatchStreamWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AvailableNowMicroBatchStreamWrapper.scala new file mode 100644 index 0000000000000..f60468c85e6e3 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AvailableNowMicroBatchStreamWrapper.scala @@ -0,0 +1,39 @@ +/* + * 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 org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory} +import org.apache.spark.sql.connector.read.streaming +import org.apache.spark.sql.connector.read.streaming.MicroBatchStream + +/** + * This class wraps a [[MicroBatchStream]] and makes it supports Trigger.AvailableNow. + * + * See [[AvailableNowDataStreamWrapper]] for more details. + */ +class AvailableNowMicroBatchStreamWrapper(delegate: MicroBatchStream) + extends AvailableNowDataStreamWrapper(delegate) with MicroBatchStream { + + override def latestOffset(): streaming.Offset = throw new UnsupportedOperationException( + "latestOffset(Offset, ReadLimit) should be called instead of this method") + + override def planInputPartitions(start: streaming.Offset, end: streaming.Offset): + Array[InputPartition] = delegate.planInputPartitions(start, end) + + override def createReaderFactory(): PartitionReaderFactory = delegate.createReaderFactory() +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AvailableNowSourceWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AvailableNowSourceWrapper.scala new file mode 100644 index 0000000000000..e971b1737ccc4 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AvailableNowSourceWrapper.scala @@ -0,0 +1,38 @@ +/* + * 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 org.apache.spark.sql.DataFrame +import org.apache.spark.sql.types.StructType + +/** + * This class wraps a [[Source]] and makes it supports Trigger.AvailableNow. + * + * See [[AvailableNowDataStreamWrapper]] for more details. + */ +class AvailableNowSourceWrapper(delegate: Source) + extends AvailableNowDataStreamWrapper(delegate) with Source { + + override def schema: StructType = delegate.schema + + override def getOffset: Option[Offset] = throw new UnsupportedOperationException( + "latestOffset(Offset, ReadLimit) should be called instead of this method") + + override def getBatch(start: Option[Offset], end: Offset): DataFrame = + delegate.getBatch(start, end) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index 2682f7dcf00b0..5baf3d29a499e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -31,7 +31,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.connector.read.streaming -import org.apache.spark.sql.connector.read.streaming.{ReadAllAvailable, ReadLimit, ReadMaxFiles, SupportsAdmissionControl} +import org.apache.spark.sql.connector.read.streaming.{ReadAllAvailable, ReadLimit, ReadMaxFiles, SupportsAdmissionControl, SupportsTriggerAvailableNow} import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.datasources.{DataSource, InMemoryFileIndex, LogicalRelation} import org.apache.spark.sql.internal.SQLConf @@ -48,7 +48,11 @@ class FileStreamSource( override val schema: StructType, partitionColumns: Seq[String], metadataPath: String, - options: Map[String, String]) extends SupportsAdmissionControl with Source with Logging { + options: Map[String, String]) + extends SupportsAdmissionControl + with SupportsTriggerAvailableNow + with Source + with Logging { import FileStreamSource._ @@ -105,6 +109,8 @@ class FileStreamSource( // Visible for testing and debugging in production. val seenFiles = new SeenFilesMap(maxFileAgeMs, fileNameOnly) + private var allFilesForTriggerAvailableNow: Seq[(String, Long)] = _ + metadataLog.restore().foreach { entry => seenFiles.add(entry.path, entry.timestamp) } @@ -126,7 +132,13 @@ class FileStreamSource( unreadFiles } else { // All the new files found - ignore aged files and files that we have seen. - fetchAllFiles().filter { + // Use the pre-fetched list of files when Trigger.AvailableNow is enabled. + val allFiles = if (allFilesForTriggerAvailableNow != null) { + allFilesForTriggerAvailableNow + } else { + fetchAllFiles() + } + allFiles.filter { case (path, timestamp) => seenFiles.isNewFile(path, timestamp) } } @@ -194,6 +206,10 @@ class FileStreamSource( FileStreamSourceOffset(metadataLogCurrentOffset) } + override def prepareForTriggerAvailableNow(): Unit = { + allFilesForTriggerAvailableNow = fetchAllFiles() + } + override def getDefaultReadLimit: ReadLimit = { maxFilesPerBatch.map(ReadLimit.maxFiles).getOrElse(super.getDefaultReadLimit) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala index 47888b70ac3ee..e9e4be90a0449 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.streaming.{StreamingRelationV2, WriteToStre import org.apache.spark.sql.catalyst.trees.TreePattern.CURRENT_LIKE import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsWrite, TableCapability} -import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset => OffsetV2, ReadLimit, SparkDataStream, SupportsAdmissionControl} +import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset => OffsetV2, ReadLimit, SparkDataStream, SupportsAdmissionControl, SupportsTriggerAvailableNow} import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, StreamingDataSourceV2Relation, StreamWriterCommitProgress, WriteToDataSourceV2Exec} @@ -50,7 +50,8 @@ class MicroBatchExecution( private val triggerExecutor = trigger match { case t: ProcessingTimeTrigger => ProcessingTimeExecutor(t, triggerClock) - case OneTimeTrigger => OneTimeExecutor() + case OneTimeTrigger => SingleBatchExecutor() + case AvailableNowTrigger => MultiBatchExecutor() case _ => throw new IllegalStateException(s"Unknown type of trigger: $trigger") } @@ -121,18 +122,36 @@ class MicroBatchExecution( // v2 source case r: StreamingDataSourceV2Relation => r.stream } - uniqueSources = sources.distinct.map { - case source: SupportsAdmissionControl => - val limit = source.getDefaultReadLimit - if (trigger == OneTimeTrigger && limit != ReadLimit.allAvailable()) { - logWarning(s"The read limit $limit for $source is ignored when Trigger.Once() is used.") - source -> ReadLimit.allAvailable() - } else { - source -> limit - } - case other => - other -> ReadLimit.allAvailable() - }.toMap + uniqueSources = triggerExecutor match { + case _: SingleBatchExecutor => + sources.distinct.map { + case s: SupportsAdmissionControl => + val limit = s.getDefaultReadLimit + if (limit != ReadLimit.allAvailable()) { + logWarning( + s"The read limit $limit for $s is ignored when Trigger.Once is used.") + } + s -> ReadLimit.allAvailable() + case s => + s -> ReadLimit.allAvailable() + }.toMap + + case _: MultiBatchExecutor => + sources.distinct.map { + case s: SupportsTriggerAvailableNow => s + case s: Source => new AvailableNowSourceWrapper(s) + case s: MicroBatchStream => new AvailableNowMicroBatchStreamWrapper(s) + }.map { s => + s.prepareForTriggerAvailableNow() + s -> s.getDefaultReadLimit + }.toMap + + case _ => + sources.distinct.map { + case s: SupportsAdmissionControl => s -> s.getDefaultReadLimit + case s => s -> ReadLimit.allAvailable() + }.toMap + } // TODO (SPARK-27484): we should add the writing node before the plan is analyzed. sink match { @@ -243,6 +262,10 @@ class MicroBatchExecution( if (isCurrentBatchConstructed) { currentBatchId += 1 isCurrentBatchConstructed = false + } else if (triggerExecutor.isInstanceOf[MultiBatchExecutor]) { + logInfo("Finished processing all available data for the trigger, terminating this " + + "Trigger.AvailableNow query") + state.set(TERMINATED) } else Thread.sleep(pollingDelayMs) } updateStatusMessage("Waiting for next trigger") @@ -363,6 +386,21 @@ class MicroBatchExecution( } } + /** + * Get the startOffset from availableOffsets. This is to be used in + * latestOffset(startOffset, readLimit) + */ + private def getStartOffset(dataStream: SparkDataStream): OffsetV2 = { + val startOffsetOpt = availableOffsets.get(dataStream) + dataStream match { + case _: Source => + startOffsetOpt.orNull + case v2: MicroBatchStream => + startOffsetOpt.map(offset => v2.deserializeOffset(offset.json)) + .getOrElse(v2.initialOffset()) + } + } + /** * Attempts to construct a batch according to: * - Availability of new data @@ -382,18 +420,18 @@ class MicroBatchExecution( // Generate a map from each unique source to the next available offset. val (nextOffsets, recentOffsets) = uniqueSources.toSeq.map { + case (s: AvailableNowDataStreamWrapper, limit) => + updateStatusMessage(s"Getting offsets from $s") + val originalSource = s.delegate + reportTimeTaken("latestOffset") { + val next = s.latestOffset(getStartOffset(originalSource), limit) + val latest = s.reportLatestOffset() + ((originalSource, Option(next)), (originalSource, Option(latest))) + } case (s: SupportsAdmissionControl, limit) => updateStatusMessage(s"Getting offsets from $s") reportTimeTaken("latestOffset") { - val startOffsetOpt = availableOffsets.get(s) - val startOffset = s match { - case _: Source => - startOffsetOpt.orNull - case v2: MicroBatchStream => - startOffsetOpt.map(offset => v2.deserializeOffset(offset.json)) - .getOrElse(v2.initialOffset()) - } - val next = s.latestOffset(startOffset, limit) + val next = s.latestOffset(getStartOffset(s), limit) val latest = s.reportLatestOffset() ((s, Option(next)), (s, Option(latest))) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala index 088471053b6f8..e15fc8b3303d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala @@ -31,7 +31,7 @@ trait TriggerExecutor { /** * A trigger executor that runs a single batch only, then terminates. */ -case class OneTimeExecutor() extends TriggerExecutor { +case class SingleBatchExecutor() extends TriggerExecutor { /** * Execute a single batch using `batchRunner`. @@ -39,6 +39,16 @@ case class OneTimeExecutor() extends TriggerExecutor { override def execute(batchRunner: () => Boolean): Unit = batchRunner() } +/** + * A trigger executor that runs multiple batches then terminates. + */ +case class MultiBatchExecutor() extends TriggerExecutor { + /** + * Execute multiple batches using `batchRunner` + */ + override def execute(batchRunner: () => Boolean): Unit = while (batchRunner()) {} +} + /** * A trigger executor that runs a batch every `intervalMs` milliseconds. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala index ebd237bfc0c19..e6d1381b2b620 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala @@ -47,11 +47,15 @@ private object Triggers { } /** - * A [[Trigger]] that processes only one batch of data in a streaming query then terminates - * the query. + * A [[Trigger]] that processes all available data in one batch then terminates the query. */ case object OneTimeTrigger extends Trigger +/** + * A [[Trigger]] that processes all available data in multiple batches then terminates the query. + */ +case object AvailableNowTrigger extends Trigger + /** * A [[Trigger]] that runs a query periodically based on the processing time. If `interval` is 0, * the query will run as fast as possible. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 868946f4007e0..b907323702299 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -1304,6 +1304,88 @@ class FileStreamSourceSuite extends FileStreamSourceTest { } } + test("SPARK-36533: Trigger.AvailableNow - multiple queries with checkpoint") { + withTempDirs { (src, target) => + val checkpoint = new File(target, "chk").getCanonicalPath + var lastFileModTime: Option[Long] = None + + /** Create a text file with a single data item */ + def createFile(data: Int): File = { + val file = stringToFile(new File(src, s"$data.txt"), data.toString) + if (lastFileModTime.nonEmpty) file.setLastModified(lastFileModTime.get + 1000) + lastFileModTime = Some(file.lastModified) + file + } + + createFile(1) + createFile(2) + createFile(3) + + // Set up a query to read text files one at a time + val df = spark + .readStream + .option("maxFilesPerTrigger", 1) + .text(src.getCanonicalPath) + + def startTriggerOnceQuery(): StreamingQuery = { + df.writeStream + .foreachBatch((_: Dataset[Row], _: Long) => {}) + .trigger(Trigger.Once) + .option("checkpointLocation", checkpoint) + .start() + } + + // run a query with Trigger.Once first + val q = startTriggerOnceQuery() + + try { + assert(q.awaitTermination(streamingTimeout.toMillis)) + } finally { + q.stop() + } + + // For queries with Trigger.AvailableNow, maxFilesPerTrigger option will be honored, so we + // will have a one-to-one mapping between rows and micro-batches. + // This variable tracks the number of rows / micro-batches starting from here. + // It starts from 3 since we have processed the first 3 rows in the first query. + var index = 3 + def startTriggerAvailableNowQuery(): StreamingQuery = { + df.writeStream + .foreachBatch((df: Dataset[Row], _: Long) => { + index += 1 + checkAnswer(df, Row(index.toString)) + }) + .trigger(Trigger.AvailableNow) + .option("checkpointLocation", checkpoint) + .start() + } + + createFile(4) + createFile(5) + + // run a second query with Trigger.AvailableNow + val q2 = startTriggerAvailableNowQuery() + try { + assert(q2.awaitTermination(streamingTimeout.toMillis)) + assert(index == 5) + } finally { + q2.stop() + } + + createFile(6) + createFile(7) + + // run a third query with Trigger.AvailableNow + val q3 = startTriggerAvailableNowQuery() + try { + assert(q3.awaitTermination(streamingTimeout.toMillis)) + assert(index == 7) + } finally { + q3.stop() + } + } + } + test("explain") { withTempDirs { case (src, tmp) => src.mkdirs() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TriggerAvailableNowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TriggerAvailableNowSuite.scala new file mode 100644 index 0000000000000..0c7348b91c0a7 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TriggerAvailableNowSuite.scala @@ -0,0 +1,233 @@ +/* + * 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.streaming + +import java.io.File + +import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.catalyst.plans.logical.Range +import org.apache.spark.sql.catalyst.util.stringToFile +import org.apache.spark.sql.connector.read.streaming +import org.apache.spark.sql.connector.read.streaming.{ReadLimit, SupportsAdmissionControl} +import org.apache.spark.sql.execution.streaming.{LongOffset, MemoryStream, Offset, SerializedOffset, Source, StreamingExecutionRelation} +import org.apache.spark.sql.types.{LongType, StructType} + +class TriggerAvailableNowSuite extends FileStreamSourceTest { + + import testImplicits._ + + abstract class TestDataFrameProvider { + @volatile var currentOffset = 0L + + def toDF: DataFrame + + def incrementAvailableOffset(numNewRows: Int): Unit + + def sourceName: String + } + + class TestSource extends TestDataFrameProvider with Source { + override def getOffset: Option[Offset] = { + if (currentOffset <= 0) None else Some(LongOffset(currentOffset)) + } + + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + if (currentOffset == 0) currentOffset = getOffsetValue(end) + val plan = Range( + start.map(getOffsetValue).getOrElse(0L) + 1L, getOffsetValue(end) + 1L, 1, None, + isStreaming = true) + Dataset.ofRows(spark, plan) + } + + override def incrementAvailableOffset(numNewRows: Int): Unit = { + currentOffset += numNewRows + } + + override def toDF: DataFrame = + Dataset.ofRows(spark, StreamingExecutionRelation(this, spark)) + + override def schema: StructType = new StructType().add("value", LongType) + + override def stop(): Unit = {} + + private def getOffsetValue(offset: Offset): Long = { + offset match { + case s: SerializedOffset => LongOffset(s).offset + case l: LongOffset => l.offset + case _ => throw new IllegalArgumentException("incorrect offset type: " + offset) + } + } + + override def sourceName: String = this.getClass.getName + } + + class TestSourceWithAdmissionControl extends TestSource with SupportsAdmissionControl { + override def getDefaultReadLimit: ReadLimit = ReadLimit.maxRows(1) // this will be overridden + + override def latestOffset(startOffset: streaming.Offset, limit: ReadLimit): streaming.Offset = { + val currentOffset = getOffset + assert(currentOffset.nonEmpty, + "the latestOffset should be called after incrementAvailableOffset") + currentOffset.get + } + } + + class TestMicroBatchStream extends TestDataFrameProvider { + private lazy val memoryStream = MemoryStream[Long](0, spark.sqlContext) + + override def toDF: DataFrame = memoryStream.toDF() + + override def incrementAvailableOffset(numNewRows: Int): Unit = { + for (_ <- 1 to numNewRows) { + currentOffset += 1 + memoryStream.addData(currentOffset) + } + } + + // remove the trailing `$` in the class name + override def sourceName: String = MemoryStream.getClass.getSimpleName.dropRight(1) + } + + Seq( + new TestSource, + new TestSourceWithAdmissionControl, + new TestMicroBatchStream + ).foreach { testSource => + test(s"TriggerAvailableNow for multiple sources with ${testSource.getClass}") { + withTempDirs { (src, target) => + val checkpoint = new File(target, "chk").getCanonicalPath + val targetDir = new File(target, "data").getCanonicalPath + var lastFileModTime: Option[Long] = None + + /** Create a text file with a single data item */ + def createFile(data: Int): File = { + val file = stringToFile(new File(src, s"$data.txt"), data.toString) + if (lastFileModTime.nonEmpty) file.setLastModified(lastFileModTime.get + 1000) + lastFileModTime = Some(file.lastModified) + file + } + + // Set up a query to read text files one at a time + val df1 = spark + .readStream + .option("maxFilesPerTrigger", 1) + .text(src.getCanonicalPath) + + val df2 = testSource.toDF + + def startQuery(): StreamingQuery = { + df1.union(df2).writeStream + .format("parquet") + .trigger(Trigger.AvailableNow) + .option("checkpointLocation", checkpoint) + .start(targetDir) + } + + testSource.incrementAvailableOffset(3) + createFile(7) + createFile(8) + createFile(9) + + val q = startQuery() + + try { + assert(q.awaitTermination(streamingTimeout.toMillis)) + // only one batch has data in both sources, thus counted, see SPARK-24050 + assert(q.recentProgress.count(_.numInputRows != 0) == 1) + q.recentProgress.foreach { p => + assert(p.sources.exists(_.description.startsWith(testSource.sourceName))) + } + checkAnswer(sql(s"SELECT * from parquet.`$targetDir`"), + Seq(1, 2, 3, 7, 8, 9).map(_.toString).toDF()) + } finally { + q.stop() + } + + testSource.incrementAvailableOffset(3) + createFile(10) + createFile(11) + createFile(12) + + // run a second query + val q2 = startQuery() + try { + assert(q2.awaitTermination(streamingTimeout.toMillis)) + // only one batch has data in both sources, thus counted, see SPARK-24050 + assert(q2.recentProgress.count(_.numInputRows != 0) == 1) + q2.recentProgress.foreach { p => + assert(p.sources.exists(_.description.startsWith(testSource.sourceName))) + } + checkAnswer(sql(s"SELECT * from parquet.`$targetDir`"), (1 to 12).map(_.toString).toDF()) + } finally { + q2.stop() + } + } + } + } + + Seq( + new TestSource, + new TestSourceWithAdmissionControl, + new TestMicroBatchStream + ).foreach { testSource => + test(s"TriggerAvailableNow for single source with ${testSource.getClass}") { + val tableName = "trigger_available_now_test_table" + withTable(tableName) { + val df = testSource.toDF + + def startQuery(): StreamingQuery = { + df.writeStream + .format("memory") + .queryName(tableName) + .trigger(Trigger.AvailableNow) + .start() + } + + testSource.incrementAvailableOffset(3) + + val q = startQuery() + + try { + assert(q.awaitTermination(streamingTimeout.toMillis)) + assert(q.recentProgress.count(_.numInputRows != 0) == 1) + q.recentProgress.foreach { p => + assert(p.sources.exists(_.description.startsWith(testSource.sourceName))) + } + checkAnswer(spark.table(tableName), (1 to 3).toDF()) + } finally { + q.stop() + } + + testSource.incrementAvailableOffset(3) + + // run a second query + val q2 = startQuery() + try { + assert(q2.awaitTermination(streamingTimeout.toMillis)) + assert(q2.recentProgress.count(_.numInputRows != 0) == 1) + q2.recentProgress.foreach { p => + assert(p.sources.exists(_.description.startsWith(testSource.sourceName))) + } + checkAnswer(spark.table(tableName), (1 to 6).toDF()) + } finally { + q2.stop() + } + } + } + } +}