[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 <bo.zhang@databricks.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
This commit is contained in:
Bo Zhang 2021-09-01 15:02:21 +09:00 committed by Jungtaek Lim
parent ff8cc4b800
commit e33cdfb317
12 changed files with 656 additions and 33 deletions

View file

@ -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.
</td>
</tr>
<tr>
<td><b>Available-now micro-batch</b></td>
<td>
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. <code>maxFilesPerTrigger</code> for file source), which will result
in better query scalability.
</td>
</tr>
<tr>
<td><b>Continuous with fixed checkpoint interval</b><br/><i>(experimental)</i></td>
<td>
@ -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")

View file

@ -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();
}

View file

@ -19,12 +19,13 @@ package org.apache.spark.sql.streaming;
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 class Trigger {
}
/**
* 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 class Trigger {
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.

View file

@ -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()
}
}

View file

@ -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()
}

View file

@ -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)
}

View file

@ -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)
}

View file

@ -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)))
}

View file

@ -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.
*/

View file

@ -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.

View file

@ -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()

View file

@ -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()
}
}
}
}
}