Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
23 changes: 22 additions & 1 deletion docs/structured-streaming-programming-guide.md
Original file line number Diff line number Diff line change
Expand Up @@ -1505,7 +1505,6 @@ Additional details on supported joins:

- Cannot use mapGroupsWithState and flatMapGroupsWithState in Update mode before joins.


### Streaming Deduplication
You can deduplicate records in data streams using a unique identifier in the events. This is exactly same as deduplication on static using a unique identifier column. The query will store the necessary amount of data from previous records such that it can filter duplicate records. Similar to aggregations, you can use deduplication with or without watermarking.

Expand Down Expand Up @@ -1616,6 +1615,8 @@ this configuration judiciously.
### Arbitrary Stateful Operations
Many usecases require more advanced stateful operations than aggregations. For example, in many usecases, you have to track sessions from data streams of events. For doing such sessionization, you will have to save arbitrary types of data as state, and perform arbitrary operations on the state using the data stream events in every trigger. Since Spark 2.2, this can be done using the operation `mapGroupsWithState` and the more powerful operation `flatMapGroupsWithState`. Both operations allow you to apply user-defined code on grouped Datasets to update user-defined state. For more concrete details, take a look at the API documentation ([Scala](api/scala/index.html#org.apache.spark.sql.streaming.GroupState)/[Java](api/java/org/apache/spark/sql/streaming/GroupState.html)) and the examples ([Scala]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala)/[Java]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredSessionization.java)).

Though Spark cannot check and force it, the state function should be implemented with respect to the semantics of the output mode. For example, in Update mode Spark doesn't expect that the state function will emit rows which are older than current watermark plus allowed late record delay, whereas in Append mode the state function can emit these rows.

### Unsupported Operations
There are a few DataFrame/Dataset operations that are not supported with streaming DataFrames/Datasets.
Some of them are as follows.
Expand Down Expand Up @@ -1647,6 +1648,26 @@ For example, sorting on the input stream is not supported, as it requires keepin
track of all the data received in the stream. This is therefore fundamentally hard to execute
efficiently.

### Limitation of global watermark

In Append mode, if a stateful operation emits rows older than current watermark plus allowed late record delay,
they will be "late rows" in downstream stateful operations (as Spark uses global watermark). Note that these rows may be discarded.
This is a limitation of a global watermark, and it could potentially cause a correctness issue.

Spark will check the logical plan of query and log a warning when Spark detects such a pattern.

Any of the stateful operation(s) after any of below stateful operations can have this issue:

* streaming aggregation in Append mode
* stream-stream outer join
* `mapGroupsWithState` and `flatMapGroupsWithState` in Append mode (depending on the implementation of the state function)

As Spark cannot check the state function of `mapGroupsWithState`/`flatMapGroupsWithState`, Spark assumes that the state function
emits late rows if the operator uses Append mode.

There's a known workaround: split your streaming query into multiple queries per stateful operator, and ensure
end-to-end exactly once per query. Ensuring end-to-end exactly once for the last query is optional.

## Starting Streaming Queries
Once you have defined the final result DataFrame/Dataset, all that is left is for you to start the streaming computation. To do that, you have to use the `DataStreamWriter`
([Scala](api/scala/index.html#org.apache.spark.sql.streaming.DataStreamWriter)/[Java](api/java/org/apache/spark/sql/streaming/DataStreamWriter.html)/[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamWriter) docs)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.spark.sql.catalyst.analysis

import org.apache.spark.internal.Logging
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, AttributeSet, CurrentDate, CurrentTimestamp, MonotonicallyIncreasingID}
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
Expand All @@ -30,7 +31,7 @@ import org.apache.spark.sql.streaming.OutputMode
/**
* Analyzes the presence of unsupported operations in a logical plan.
*/
object UnsupportedOperationChecker {
object UnsupportedOperationChecker extends Logging {

def checkForBatch(plan: LogicalPlan): Unit = {
plan.foreachUp {
Expand All @@ -41,8 +42,50 @@ object UnsupportedOperationChecker {
}
}

def checkForStreaming(plan: LogicalPlan, outputMode: OutputMode): Unit = {
def checkStreamingQueryGlobalWatermarkLimit(
plan: LogicalPlan,
outputMode: OutputMode,
failWhenDetected: Boolean): Unit = {
def isStatefulOperationPossiblyEmitLateRows(p: LogicalPlan): Boolean = p match {
case s: Aggregate
if s.isStreaming && outputMode == InternalOutputModes.Append => true
case Join(left, right, joinType, _, _)
if left.isStreaming && right.isStreaming && joinType != Inner => true
case f: FlatMapGroupsWithState
if f.isStreaming && f.outputMode == OutputMode.Append() => true
case _ => false
}

def isStatefulOperation(p: LogicalPlan): Boolean = p match {
case s: Aggregate if s.isStreaming => true
case _ @ Join(left, right, _, _, _) if left.isStreaming && right.isStreaming => true
case f: FlatMapGroupsWithState if f.isStreaming => true
case d: Deduplicate if d.isStreaming => true
case _ => false
}

try {
plan.foreach { subPlan =>
if (isStatefulOperation(subPlan)) {
subPlan.find { p =>
(p ne subPlan) && isStatefulOperationPossiblyEmitLateRows(p)
}.foreach { _ =>
val errorMsg = "Detected pattern of possible 'correctness' issue " +
"due to global watermark. " +
"The query contains stateful operation which can emit rows older than " +
"the current watermark plus allowed late record delay, which are \"late rows\"" +
" in downstream stateful operations and these rows can be discarded. " +
"Please refer the programming guide doc for more details."
throwError(errorMsg)(plan)
}
}
}
} catch {
case e: AnalysisException if !failWhenDetected => logWarning(s"${e.message};\n$plan")
}
}

def checkForStreaming(plan: LogicalPlan, outputMode: OutputMode): Unit = {
if (!plan.isStreaming) {
throwError(
"Queries without streaming sources cannot be executed with writeStream.start()")(plan)
Expand Down Expand Up @@ -339,6 +382,8 @@ object UnsupportedOperationChecker {
// Check if there are unsupported expressions in streaming query plan.
checkUnsupportedExpressions(subPlan)
}

checkStreamingQueryGlobalWatermarkLimit(plan, outputMode, failWhenDetected = false)
}

def checkForContinuous(plan: LogicalPlan, outputMode: OutputMode): Unit = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -643,6 +643,153 @@ class UnsupportedOperationsSuite extends SparkFunSuite {
null,
new TestStreamingRelationV2(attribute)), OutputMode.Append())

// streaming aggregation
{
assertPassOnGlobalWatermarkLimit(
"single streaming aggregation in Append mode",
streamRelation.groupBy("a")(count("*")),
OutputMode.Append())

assertFailOnGlobalWatermarkLimit(
"chained streaming aggregations in Append mode",
streamRelation.groupBy("a")(count("*")).groupBy()(count("*")),
OutputMode.Append())

Seq(Inner, LeftOuter, RightOuter).foreach { joinType =>
val plan = streamRelation.join(streamRelation.groupBy("a")(count("*")), joinType = joinType)
assertFailOnGlobalWatermarkLimit(
s"$joinType join after streaming aggregation in Append mode",
streamRelation.join(streamRelation.groupBy("a")(count("*")), joinType = joinType),
OutputMode.Append())
}

assertFailOnGlobalWatermarkLimit(
"deduplicate after streaming aggregation in Append mode",
Deduplicate(Seq(attribute), streamRelation.groupBy("a")(count("*"))),
OutputMode.Append())

assertFailOnGlobalWatermarkLimit(
"FlatMapGroupsWithState after streaming aggregation in Append mode",
FlatMapGroupsWithState(
null, att, att, Seq(att), Seq(att), att, null, Append,
isMapGroupsWithState = false, null,
streamRelation.groupBy("a")(count("*"))),
OutputMode.Append())
}

// stream-stream join
// stream-stream inner join doesn't emit late rows, whereas outer joins could
Seq((Inner, false), (LeftOuter, true), (RightOuter, true)).map { case (joinType, expectFailure) =>
assertPassOnGlobalWatermarkLimit(
s"single $joinType join in Append mode",
streamRelation.join(streamRelation, joinType = RightOuter,
condition = Some(attributeWithWatermark === attribute)),
OutputMode.Append())

testGlobalWatermarkLimit(
s"streaming aggregation after stream-stream $joinType join in Append mode",
streamRelation.join(streamRelation, joinType = joinType,
condition = Some(attributeWithWatermark === attribute))
.groupBy("a")(count("*")),
OutputMode.Append(),
expectFailure = expectFailure)

Seq(Inner, LeftOuter, RightOuter).map { joinType2 =>
testGlobalWatermarkLimit(
s"streaming-stream $joinType2 after stream-stream $joinType join in Append mode",
streamRelation.join(
streamRelation.join(streamRelation, joinType = joinType,
condition = Some(attributeWithWatermark === attribute)),
joinType = joinType2,
condition = Some(attributeWithWatermark === attribute)),
OutputMode.Append(),
expectFailure = expectFailure)
}

testGlobalWatermarkLimit(
s"FlatMapGroupsWithState after stream-stream $joinType join in Append mode",
FlatMapGroupsWithState(
null, att, att, Seq(att), Seq(att), att, null, Append,
isMapGroupsWithState = false, null,
streamRelation.join(streamRelation, joinType = joinType,
condition = Some(attributeWithWatermark === attribute))),
OutputMode.Append(),
expectFailure = expectFailure)

testGlobalWatermarkLimit(
s"deduplicate after stream-stream $joinType join in Append mode",
Deduplicate(Seq(attribute), streamRelation.join(streamRelation, joinType = joinType,
condition = Some(attributeWithWatermark === attribute))),
OutputMode.Append(),
expectFailure = expectFailure)
}

// FlatMapGroupsWithState
{
assertPassOnGlobalWatermarkLimit(
"single FlatMapGroupsWithState in Append mode",
FlatMapGroupsWithState(
null, att, att, Seq(att), Seq(att), att, null, Append,
isMapGroupsWithState = false, null, streamRelation),
OutputMode.Append())

assertFailOnGlobalWatermarkLimit(
"streaming aggregation after FlatMapGroupsWithState in Append mode",
FlatMapGroupsWithState(
null, att, att, Seq(att), Seq(att), att, null, Append,
isMapGroupsWithState = false, null, streamRelation).groupBy("*")(count("*")),
OutputMode.Append())

Seq(Inner, LeftOuter, RightOuter).map { joinType =>
assertFailOnGlobalWatermarkLimit(
s"stream-stream $joinType after FlatMapGroupsWithState in Append mode",
streamRelation.join(
FlatMapGroupsWithState(null, att, att, Seq(att), Seq(att), att, null, Append,
isMapGroupsWithState = false, null, streamRelation), joinType = joinType,
condition = Some(attributeWithWatermark === attribute)),
OutputMode.Append())
}

assertFailOnGlobalWatermarkLimit(
"FlatMapGroupsWithState after FlatMapGroupsWithState in Append mode",
FlatMapGroupsWithState(null, att, att, Seq(att), Seq(att), att, null, Append,
isMapGroupsWithState = false, null,
FlatMapGroupsWithState(null, att, att, Seq(att), Seq(att), att, null, Append,
isMapGroupsWithState = false, null, streamRelation)),
OutputMode.Append())

assertFailOnGlobalWatermarkLimit(
s"deduplicate after FlatMapGroupsWithState in Append mode",
Deduplicate(Seq(attribute),
FlatMapGroupsWithState(null, att, att, Seq(att), Seq(att), att, null, Append,
isMapGroupsWithState = false, null, streamRelation)),
OutputMode.Append())
}

// deduplicate
{
assertPassOnGlobalWatermarkLimit(
"streaming aggregation after deduplicate in Append mode",
Deduplicate(Seq(attribute), streamRelation).groupBy("a")(count("*")),
OutputMode.Append())

Seq(Inner, LeftOuter, RightOuter).map { joinType =>
assertPassOnGlobalWatermarkLimit(
s"$joinType join after deduplicate in Append mode",
streamRelation.join(Deduplicate(Seq(attribute), streamRelation), joinType = joinType,
condition = Some(attributeWithWatermark === attribute)),
OutputMode.Append())
}

assertPassOnGlobalWatermarkLimit(
"FlatMapGroupsWithState after deduplicate in Append mode",
FlatMapGroupsWithState(
null, att, att, Seq(att), Seq(att), att, null, Append,
isMapGroupsWithState = false, null,
Deduplicate(Seq(attribute), streamRelation)),
OutputMode.Append())
}

/*
=======================================================================================
TESTING FUNCTIONS
Expand Down Expand Up @@ -839,6 +986,40 @@ class UnsupportedOperationsSuite extends SparkFunSuite {
}
}


def assertPassOnGlobalWatermarkLimit(
testNamePostfix: String,
plan: LogicalPlan,
outputMode: OutputMode): Unit = {
testGlobalWatermarkLimit(testNamePostfix, plan, outputMode, expectFailure = false)
}

def assertFailOnGlobalWatermarkLimit(
testNamePostfix: String,
plan: LogicalPlan,
outputMode: OutputMode): Unit = {
testGlobalWatermarkLimit(testNamePostfix, plan, outputMode, expectFailure = true)
}

def testGlobalWatermarkLimit(
testNamePostfix: String,
plan: LogicalPlan,
outputMode: OutputMode,
expectFailure: Boolean): Unit = {
test(s"Global watermark limit - $testNamePostfix") {
if (expectFailure) {
val e = intercept[AnalysisException] {
UnsupportedOperationChecker.checkStreamingQueryGlobalWatermarkLimit(
wrapInStreaming(plan), outputMode, failWhenDetected = true)
}
assert(e.message.contains("Detected pattern of possible 'correctness' issue"))
} else {
UnsupportedOperationChecker.checkStreamingQueryGlobalWatermarkLimit(
wrapInStreaming(plan), outputMode, failWhenDetected = true)
}
}
}

/**
* Test whether the body of code will fail. If it does fail, then check if it has expected
* messages.
Expand Down