Skip to content

Commit

Permalink
[SPARK-36533][SS] Trigger.AvailableNow for running streaming queries …
Browse files Browse the repository at this point in the history
…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>
  • Loading branch information
bozhang2820 authored and HeartSaVioR committed Sep 1, 2021
1 parent ff8cc4b commit e33cdfb
Show file tree
Hide file tree
Showing 12 changed files with 656 additions and 33 deletions.
21 changes: 21 additions & 0 deletions docs/structured-streaming-programming-guide.md
Expand Up @@ -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>
Expand Down Expand Up @@ -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")
Expand Down Expand Up @@ -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")
Expand Down
@@ -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();
}
21 changes: 17 additions & 4 deletions sql/core/src/main/java/org/apache/spark/sql/streaming/Trigger.java
Expand Up @@ -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]].
Expand Down Expand Up @@ -89,15 +90,27 @@ 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
*/
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.
Expand Down
@@ -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()
}
}
@@ -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()
}
@@ -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)
}
Expand Up @@ -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
Expand All @@ -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._

Expand Down Expand Up @@ -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)
}
Expand All @@ -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)
}
}
Expand Down Expand Up @@ -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)
}
Expand Down

0 comments on commit e33cdfb

Please sign in to comment.