liviazhu-db commented on code in PR #47940:
URL: https://github.com/apache/spark/pull/47940#discussion_r1742864836


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala:
##########
@@ -447,8 +450,18 @@ case class FlatMapGroupsWithStateExec(
         hasTimedOut,
         watermarkPresent)
 
-      // Call function, get the returned objects and convert them to rows
-      val mappedIterator = func(keyObj, valueObjIter, groupState).map { obj =>
+      // Call function
+      val iterator = try {
+        func(keyObj, valueObjIter, groupState)
+      } catch {
+        case NonFatal(e) if !e.isInstanceOf[SparkThrowable] =>
+          throw FlatMapGroupsWithStateUserFuncException(e)
+        case f: Throwable =>
+          throw f
+      }
+
+      // Get the returned objects and convert them to rows
+      val mappedIterator = iterator.map { obj =>

Review Comment:
   Good catch, updated and added a new test case for this. 



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala:
##########
@@ -635,6 +634,25 @@ class FlatMapGroupsWithStateSuite extends 
StateStoreMetricsTest {
     )
   }
 
+  testWithAllStateVersions("[SPARK-49474] flatMapGroupsWithState - user error 
is classified") {
+    // Function to maintain running count up to 2, and then remove the count
+    // Returns the data and the count if state is defined, otherwise does not 
return anything
+    val stateFunc = (_: String, _: Iterator[String], _: 
GroupState[RunningCount]) => {
+      
Iterator(null.asInstanceOf[GroupState[RunningCount]].getCurrentProcessingTimeMs())

Review Comment:
   Done!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to