You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
[SPARK-28074][SS] Log warn message on possible correctness issue for multiple stateful operations in single query
## What changes were proposed in this pull request?
Please refer [the link on dev. mailing list](https://lists.apache.org/thread.html/cc6489a19316e7382661d305fabd8c21915e5faf6a928b4869ac2b4a%3Cdev.spark.apache.org%3E) to see rationalization of this patch.
This patch adds the functionality to detect the possible correct issue on multiple stateful operations in single streaming query and logs warning message to inform end users.
This patch also documents some notes to inform caveats when using multiple stateful operations in single query, and provide one known alternative.
## How was this patch tested?
Added new UTs in UnsupportedOperationsSuite to test various combination of stateful operators on streaming query.
Closes#24890 from HeartSaVioR/SPARK-28074.
Authored-by: Jungtaek Lim (HeartSaVioR) <[email protected]>
Signed-off-by: Sean Owen <[email protected]>
Copy file name to clipboardExpand all lines: docs/structured-streaming-programming-guide.md
+22-1Lines changed: 22 additions & 1 deletion
Display the source diff
Display the rich diff
Original file line number
Diff line number
Diff line change
@@ -1505,7 +1505,6 @@ Additional details on supported joins:
1505
1505
1506
1506
- Cannot use mapGroupsWithState and flatMapGroupsWithState in Update mode before joins.
1507
1507
1508
-
1509
1508
### Streaming Deduplication
1510
1509
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.
1511
1510
@@ -1616,6 +1615,8 @@ this configuration judiciously.
1616
1615
### Arbitrary Stateful Operations
1617
1616
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)).
1618
1617
1618
+
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.
1619
+
1619
1620
### Unsupported Operations
1620
1621
There are a few DataFrame/Dataset operations that are not supported with streaming DataFrames/Datasets.
1621
1622
Some of them are as follows.
@@ -1647,6 +1648,26 @@ For example, sorting on the input stream is not supported, as it requires keepin
1647
1648
track of all the data received in the stream. This is therefore fundamentally hard to execute
1648
1649
efficiently.
1649
1650
1651
+
### Limitation of global watermark
1652
+
1653
+
In Append mode, if a stateful operation emits rows older than current watermark plus allowed late record delay,
1654
+
they will be "late rows" in downstream stateful operations (as Spark uses global watermark). Note that these rows may be discarded.
1655
+
This is a limitation of a global watermark, and it could potentially cause a correctness issue.
1656
+
1657
+
Spark will check the logical plan of query and log a warning when Spark detects such a pattern.
1658
+
1659
+
Any of the stateful operation(s) after any of below stateful operations can have this issue:
1660
+
1661
+
* streaming aggregation in Append mode
1662
+
* stream-stream outer join
1663
+
*`mapGroupsWithState` and `flatMapGroupsWithState` in Append mode (depending on the implementation of the state function)
1664
+
1665
+
As Spark cannot check the state function of `mapGroupsWithState`/`flatMapGroupsWithState`, Spark assumes that the state function
1666
+
emits late rows if the operator uses Append mode.
1667
+
1668
+
There's a known workaround: split your streaming query into multiple queries per stateful operator, and ensure
1669
+
end-to-end exactly once per query. Ensuring end-to-end exactly once for the last query is optional.
1670
+
1650
1671
## Starting Streaming Queries
1651
1672
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`
0 commit comments