You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "bogao007 (via GitHub)" <gi...@apache.org> on 2023/09/18 22:02:32 UTC

[GitHub] [spark] bogao007 opened a new pull request, #42986: [SPARK-44463][SS][CONNECT] Improve error handling for Connect steaming Python worker

bogao007 opened a new pull request, #42986:
URL: https://github.com/apache/spark/pull/42986

   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
     8. If you want to add or modify an error type or message, please read the guideline first in
        'core/src/main/resources/error/README.md'.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   Handle errors inside streaming Python workers (foreach_batch_worker and listener_worker) and propagate to server side.
   - Write 0 to Python worker's outfile if no error occurs.
   - Write -2 and traceback to outfile if there's an error which can be read from the server side.
   
   I was referring to the code [here](https://github.com/apache/spark/blob/981312284f0776ca847c8d21411f74a72c639b22/python/pyspark/sql/worker/analyze_udtf.py#L157-L160) from another existing Python worker.
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   Without this change, there's no error handling in streaming python workers. The server side is [expecting](https://github.com/apache/spark/blob/981312284f0776ca847c8d21411f74a72c639b22/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelper.scala#L128-L129) 0 being written in [python worker's](https://github.com/apache/spark/blob/981312284f0776ca847c8d21411f74a72c639b22/python/pyspark/sql/connect/streaming/worker/foreach_batch_worker.py#L73) output stream. But if any error is thrown in the worker, the 0 is not written anymore which would lead to an `EOFException` since we are trying to read from an empty stream.
   
   If we remove the [lines](https://github.com/apache/spark/blob/981312284f0776ca847c8d21411f74a72c639b22/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelper.scala#L128-L129) reading python worker's output. The streaming query would succeed even if there's an error in foreachBatch function which is not the desired behavior we want.
   
   With this PR, we are propagating the errors from Python worker to the server so it would fail the streaming query.
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   Yes
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   If benchmark tests were added, please run the benchmarks in GitHub Actions for the consistent environment, and the instructions could accord to: https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
   -->
   Enabled `test_streaming_foreach_batch_propagates_python_errors` test.
   
   Did manual testing
   ForeachBatch:
   ```
   >>> def collectBatch(df, id):
   ...             raise RuntimeError("this should fail the query")
   >>> df = spark.readStream.format("text").load("python/test_support/sql/streaming")
   >>> q = df.writeStream.foreachBatch(collectBatch).start()
   ```
   
   ```
   23/09/18 14:21:12 ERROR MicroBatchExecution: Query [id = 8168dc4d-02cc-4ddd-996c-96667d928b88, runId = 04829434-767e-4d13-b4c2-e45ce8932223] terminated with error
   java.lang.IllegalStateException: Found error inside foreachBatch Python process: Traceback (most recent call last):
     File "/Users/bo.gao/workplace/spark/python/lib/pyspark.zip/pyspark/sql/connect/streaming/worker/foreach_batch_worker.py", line 76, in main
     File "/Users/bo.gao/workplace/spark/python/lib/pyspark.zip/pyspark/sql/connect/streaming/worker/foreach_batch_worker.py", line 69, in process
     File "<stdin>", line 2, in collectBatch
   RuntimeError: this should fail the query
   
   	at org.apache.spark.sql.connect.planner.StreamingForeachBatchHelper$.$anonfun$pythonForeachBatchWrapper$1(StreamingForeachBatchHelper.scala:137)
   	at org.apache.spark.sql.connect.planner.StreamingForeachBatchHelper$.$anonfun$pythonForeachBatchWrapper$1$adapted(StreamingForeachBatchHelper.scala:115)
   	at org.apache.spark.sql.connect.planner.StreamingForeachBatchHelper$.$anonfun$dataFrameCachingWrapper$1(StreamingForeachBatchHelper.scala:70)
   	at org.apache.spark.sql.connect.planner.StreamingForeachBatchHelper$.$anonfun$dataFrameCachingWrapper$1$adapted(StreamingForeachBatchHelper.scala:60)
   	at org.apache.spark.sql.execution.streaming.sources.ForeachBatchSink.addBatch(ForeachBatchSink.scala:34)
   	at org.apache.spark.sql.execution.streaming.MicroBatchExecution.$anonfun$runBatch$17(MicroBatchExecution.scala:732)
   	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$6(SQLExecution.scala:150)
   	at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:241)
   	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$1(SQLExecution.scala:116)
   	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
   	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId0(SQLExecution.scala:72)
   	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:196)
   	at org.apache.spark.sql.execution.streaming.MicroBatchExecution.$anonfun$runBatch$16(MicroBatchExecution.scala:729)
   	at org.apache.spark.sql.execution.streaming.ProgressReporter.reportTimeTaken(ProgressReporter.scala:427)
   	at org.apache.spark.sql.execution.streaming.ProgressReporter.reportTimeTaken$(ProgressReporter.scala:425)
   	at org.apache.spark.sql.execution.streaming.StreamExecution.reportTimeTaken(StreamExecution.scala:67)
   	at org.apache.spark.sql.execution.streaming.MicroBatchExecution.runBatch(MicroBatchExecution.scala:729)
   	at org.apache.spark.sql.execution.streaming.MicroBatchExecution.$anonfun$runActivatedStream$2(MicroBatchExecution.scala:286)
   	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
   	at org.apache.spark.sql.execution.streaming.ProgressReporter.reportTimeTaken(ProgressReporter.scala:427)
   	at org.apache.spark.sql.execution.streaming.ProgressReporter.reportTimeTaken$(ProgressReporter.scala:425)
   	at org.apache.spark.sql.execution.streaming.StreamExecution.reportTimeTaken(StreamExecution.scala:67)
   	at org.apache.spark.sql.execution.streaming.MicroBatchExecution.$anonfun$runActivatedStream$1(MicroBatchExecution.scala:249)
   	at org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:67)
   	at org.apache.spark.sql.execution.streaming.MicroBatchExecution.runActivatedStream(MicroBatchExecution.scala:239)
   	at org.apache.spark.sql.execution.streaming.StreamExecution.$anonfun$runStream$1(StreamExecution.scala:311)
   	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
   	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
   	at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runStream(StreamExecution.scala:289)
   	at org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.$anonfun$run$1(StreamExecution.scala:211)
   	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
   	at org.apache.spark.JobArtifactSet$.withActiveJobArtifactState(JobArtifactSet.scala:94)
   	at org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:211)
   ```
   StreamingQueryListener:
   ```
   >>> class TestListener(StreamingQueryListener):
   ...     def onQueryStarted(self, event):
   ...         raise RuntimeError("this should fail the listener")
   ...     def onQueryProgress(self, event):
   ...         pass
   ...     def onQueryIdle(self, event):
   ...         pass
   ...     def onQueryTerminated(self, event):
   ...         pass
   ...
   >>> test_listener = TestListener()
   >>> spark.streams.addListener(test_listener)
   >>> df = spark.readStream.format("rate").option("rowsPerSecond", 10).load()
   >>> query = df.writeStream.format("noop").queryName("test").start()
   >>> query.stop()
   ```
   
   ```
   23/09/18 14:18:56 ERROR StreamingQueryListenerBus: Listener PythonStreamingQueryListener threw an exception
   java.lang.IllegalStateException: Found error inside Streaming query listener Python process for function onQueryStarted: Traceback (most recent call last):
     File "/Users/bo.gao/workplace/spark/python/lib/pyspark.zip/pyspark/sql/connect/streaming/worker/listener_worker.py", line 90, in main
     File "/Users/bo.gao/workplace/spark/python/lib/pyspark.zip/pyspark/sql/connect/streaming/worker/listener_worker.py", line 78, in process
     File "<stdin>", line 3, in onQueryStarted
   RuntimeError: this should fail the listener
   
   	at org.apache.spark.sql.connect.planner.PythonStreamingQueryListener.handlePythonWorkerError(StreamingQueryListenerHelper.scala:88)
   	at org.apache.spark.sql.connect.planner.PythonStreamingQueryListener.onQueryStarted(StreamingQueryListenerHelper.scala:50)
   	at org.apache.spark.sql.execution.streaming.StreamingQueryListenerBus.doPostEvent(StreamingQueryListenerBus.scala:131)
   	at org.apache.spark.sql.execution.streaming.StreamingQueryListenerBus.doPostEvent(StreamingQueryListenerBus.scala:43)
   	at org.apache.spark.util.ListenerBus.postToAll(ListenerBus.scala:117)
   	at org.apache.spark.util.ListenerBus.postToAll$(ListenerBus.scala:101)
   	at org.apache.spark.sql.execution.streaming.StreamingQueryListenerBus.postToAll(StreamingQueryListenerBus.scala:88)
   	at org.apache.spark.sql.execution.streaming.StreamingQueryListenerBus.post(StreamingQueryListenerBus.scala:77)
   	at org.apache.spark.sql.streaming.StreamingQueryManager.postListenerEvent(StreamingQueryManager.scala:231)
   	at org.apache.spark.sql.execution.streaming.StreamExecution.postEvent(StreamExecution.scala:408)
   	at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runStream(StreamExecution.scala:283)
   	at org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.$anonfun$run$1(StreamExecution.scala:211)
   	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
   	at org.apache.spark.JobArtifactSet$.withActiveJobArtifactState(JobArtifactSet.scala:94)
   	at org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:211)
   ```
   
   ### Was this patch authored or co-authored using generative AI tooling?
   <!--
   If generative AI tooling has been used in the process of authoring this patch, please include the
   phrase: 'Generated-by: ' followed by the name of the tool and its version.
   If no, write 'No'.
   Please refer to the [ASF Generative Tooling Guidance](https://www.apache.org/legal/generative-tooling.html) for details.
   -->
   No


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] rangadi commented on a diff in pull request #42986: [SPARK-44463][SS][CONNECT] Improve error handling for Connect steaming Python worker

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #42986:
URL: https://github.com/apache/spark/pull/42986#discussion_r1329448797


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelper.scala:
##########
@@ -125,8 +128,21 @@ object StreamingForeachBatchHelper extends Logging {
       dataOut.writeLong(args.batchId)
       dataOut.flush()
 
-      val ret = dataIn.readInt()
-      logInfo(s"Python foreach batch for dfId ${args.dfId} completed (ret: $ret)")
+      try {
+        dataIn.readInt() match {
+          case ret if ret == 0 =>

Review Comment:
   Minor: Can simply be `case 0 => `



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingQueryListenerHelper.scala:
##########
@@ -38,33 +43,56 @@ class PythonStreamingQueryListener(listener: SimplePythonFunction, sessionHolder
     sessionHolder.sessionId,
     "pyspark.sql.connect.streaming.worker.listener_worker")
 
-  val (dataOut, _) = runner.init()
+  val (dataOut, dataIn) = runner.init()
 
   override def onQueryStarted(event: StreamingQueryListener.QueryStartedEvent): Unit = {
     PythonRDD.writeUTF(event.json, dataOut)
     dataOut.writeInt(0)
     dataOut.flush()
+    handlePythonWorkerError("onQueryStarted")
   }
 
   override def onQueryProgress(event: StreamingQueryListener.QueryProgressEvent): Unit = {
     PythonRDD.writeUTF(event.json, dataOut)
     dataOut.writeInt(1)
     dataOut.flush()
+    handlePythonWorkerError("onQueryProgress")
   }
 
   override def onQueryIdle(event: StreamingQueryListener.QueryIdleEvent): Unit = {
     PythonRDD.writeUTF(event.json, dataOut)
     dataOut.writeInt(2)
     dataOut.flush()
+    handlePythonWorkerError("onQueryIdle")
   }
 
   override def onQueryTerminated(event: StreamingQueryListener.QueryTerminatedEvent): Unit = {
     PythonRDD.writeUTF(event.json, dataOut)
     dataOut.writeInt(3)
     dataOut.flush()
+    handlePythonWorkerError("onQueryTerminated")
   }
 
   private[spark] def stopListenerProcess(): Unit = {
     runner.stop()
   }
+
+  private def handlePythonWorkerError(functionName: String): Unit = {
+    try {
+      dataIn.readInt() match {

Review Comment:
   Can you add a comment to reuse this code? 



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelper.scala:
##########
@@ -125,8 +128,21 @@ object StreamingForeachBatchHelper extends Logging {
       dataOut.writeLong(args.batchId)
       dataOut.flush()
 
-      val ret = dataIn.readInt()
-      logInfo(s"Python foreach batch for dfId ${args.dfId} completed (ret: $ret)")
+      try {
+        dataIn.readInt() match {
+          case ret if ret == 0 =>
+            logInfo(s"Python foreach batch for dfId ${args.dfId} completed (ret: $ret)")
+          case SpecialLengths.PYTHON_EXCEPTION_THROWN =>
+            val exLength = dataIn.readInt()
+            val obj = new Array[Byte](exLength)
+            dataIn.readFully(obj)
+            val msg = new String(obj, StandardCharsets.UTF_8)
+            throw new PythonException(s"Found error inside foreachBatch Python process: $msg", null)

Review Comment:
   Handle other return here.



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingQueryListenerHelper.scala:
##########
@@ -38,33 +43,56 @@ class PythonStreamingQueryListener(listener: SimplePythonFunction, sessionHolder
     sessionHolder.sessionId,
     "pyspark.sql.connect.streaming.worker.listener_worker")
 
-  val (dataOut, _) = runner.init()
+  val (dataOut, dataIn) = runner.init()
 
   override def onQueryStarted(event: StreamingQueryListener.QueryStartedEvent): Unit = {
     PythonRDD.writeUTF(event.json, dataOut)
     dataOut.writeInt(0)
     dataOut.flush()
+    handlePythonWorkerError("onQueryStarted")
   }
 
   override def onQueryProgress(event: StreamingQueryListener.QueryProgressEvent): Unit = {
     PythonRDD.writeUTF(event.json, dataOut)
     dataOut.writeInt(1)
     dataOut.flush()
+    handlePythonWorkerError("onQueryProgress")
   }
 
   override def onQueryIdle(event: StreamingQueryListener.QueryIdleEvent): Unit = {
     PythonRDD.writeUTF(event.json, dataOut)
     dataOut.writeInt(2)
     dataOut.flush()
+    handlePythonWorkerError("onQueryIdle")
   }
 
   override def onQueryTerminated(event: StreamingQueryListener.QueryTerminatedEvent): Unit = {
     PythonRDD.writeUTF(event.json, dataOut)
     dataOut.writeInt(3)
     dataOut.flush()
+    handlePythonWorkerError("onQueryTerminated")
   }
 
   private[spark] def stopListenerProcess(): Unit = {
     runner.stop()
   }
+
+  private def handlePythonWorkerError(functionName: String): Unit = {
+    try {
+      dataIn.readInt() match {
+        case ret if ret == 0 =>
+          logInfo(s"Streaming query listener function $functionName completed (ret: $ret)")
+        case SpecialLengths.PYTHON_EXCEPTION_THROWN =>
+          val exLength = dataIn.readInt()
+          val obj = new Array[Byte](exLength)
+          dataIn.readFully(obj)
+          val msg = new String(obj, StandardCharsets.UTF_8)
+          throw new PythonException(s"Found error inside Streaming query listener Python " +
+            s"process for function $functionName: $msg", null)
+      }
+    } catch {
+      case eof: EOFException =>
+        throw new SparkException("Python worker exited unexpectedly (crashed)", eof)

Review Comment:
   Always include the base exception (same for foreach batch side).



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingQueryListenerHelper.scala:
##########
@@ -76,16 +78,21 @@ class PythonStreamingQueryListener(listener: SimplePythonFunction, sessionHolder
   }
 
   private def handlePythonWorkerError(functionName: String): Unit = {
-    dataIn.readInt() match {
-      case ret if ret == 0 =>
-        logInfo(s"Streaming query listener function $functionName completed (ret: $ret)")
-      case SpecialLengths.PYTHON_EXCEPTION_THROWN =>
-        val exLength = dataIn.readInt()
-        val obj = new Array[Byte](exLength)
-        dataIn.readFully(obj)
-        val msg = new String(obj, StandardCharsets.UTF_8)
-        throw new IllegalStateException(s"Found error inside Streaming query listener Python " +
-          s"process for function $functionName: $msg")
+    try {
+      dataIn.readInt() match {
+        case ret if ret == 0 =>
+          logInfo(s"Streaming query listener function $functionName completed (ret: $ret)")
+        case SpecialLengths.PYTHON_EXCEPTION_THROWN =>
+          val exLength = dataIn.readInt()
+          val obj = new Array[Byte](exLength)
+          dataIn.readFully(obj)
+          val msg = new String(obj, StandardCharsets.UTF_8)
+          throw new PythonException(s"Found error inside Streaming query listener Python " +
+            s"process for function $functionName: $msg", null)
+      }
+    } catch {
+      case eof: EOFException =>

Review Comment:
   Yeah, we can't handle the retries in the PR. we need to implement that feature properly.
   @bogao007 can catch any `NonFatal(ex)` here. 



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] bogao007 commented on a diff in pull request #42986: [SPARK-44463][SS][CONNECT] Improve error handling for Connect steaming Python worker

Posted by "bogao007 (via GitHub)" <gi...@apache.org>.
bogao007 commented on code in PR #42986:
URL: https://github.com/apache/spark/pull/42986#discussion_r1329447168


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingQueryListenerHelper.scala:
##########
@@ -76,16 +78,21 @@ class PythonStreamingQueryListener(listener: SimplePythonFunction, sessionHolder
   }
 
   private def handlePythonWorkerError(functionName: String): Unit = {
-    dataIn.readInt() match {
-      case ret if ret == 0 =>
-        logInfo(s"Streaming query listener function $functionName completed (ret: $ret)")
-      case SpecialLengths.PYTHON_EXCEPTION_THROWN =>
-        val exLength = dataIn.readInt()
-        val obj = new Array[Byte](exLength)
-        dataIn.readFully(obj)
-        val msg = new String(obj, StandardCharsets.UTF_8)
-        throw new IllegalStateException(s"Found error inside Streaming query listener Python " +
-          s"process for function $functionName: $msg")
+    try {
+      dataIn.readInt() match {
+        case ret if ret == 0 =>
+          logInfo(s"Streaming query listener function $functionName completed (ret: $ret)")
+        case SpecialLengths.PYTHON_EXCEPTION_THROWN =>
+          val exLength = dataIn.readInt()
+          val obj = new Array[Byte](exLength)
+          dataIn.readFully(obj)
+          val msg = new String(obj, StandardCharsets.UTF_8)
+          throw new PythonException(s"Found error inside Streaming query listener Python " +
+            s"process for function $functionName: $msg", null)
+      }
+    } catch {
+      case eof: EOFException =>

Review Comment:
   If we want to add retry logic, adding it here would not solve the problem. If the EOF is due to an IOException during `write_` functions, we might need to add retries inside the workers. But I'm not sure how high would the possibility be for that scenario, since if IOException happens while writing, it's highly possible that the socket is closed, and retry would no help in that case.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] bogao007 commented on a diff in pull request #42986: [SPARK-44463][SS][CONNECT] Improve error handling for Connect steaming Python worker

Posted by "bogao007 (via GitHub)" <gi...@apache.org>.
bogao007 commented on code in PR #42986:
URL: https://github.com/apache/spark/pull/42986#discussion_r1329574367


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingQueryListenerHelper.scala:
##########
@@ -76,16 +78,21 @@ class PythonStreamingQueryListener(listener: SimplePythonFunction, sessionHolder
   }
 
   private def handlePythonWorkerError(functionName: String): Unit = {
-    dataIn.readInt() match {
-      case ret if ret == 0 =>
-        logInfo(s"Streaming query listener function $functionName completed (ret: $ret)")
-      case SpecialLengths.PYTHON_EXCEPTION_THROWN =>
-        val exLength = dataIn.readInt()
-        val obj = new Array[Byte](exLength)
-        dataIn.readFully(obj)
-        val msg = new String(obj, StandardCharsets.UTF_8)
-        throw new IllegalStateException(s"Found error inside Streaming query listener Python " +
-          s"process for function $functionName: $msg")
+    try {
+      dataIn.readInt() match {
+        case ret if ret == 0 =>
+          logInfo(s"Streaming query listener function $functionName completed (ret: $ret)")
+        case SpecialLengths.PYTHON_EXCEPTION_THROWN =>
+          val exLength = dataIn.readInt()
+          val obj = new Array[Byte](exLength)
+          dataIn.readFully(obj)
+          val msg = new String(obj, StandardCharsets.UTF_8)
+          throw new PythonException(s"Found error inside Streaming query listener Python " +
+            s"process for function $functionName: $msg", null)
+      }
+    } catch {
+      case eof: EOFException =>

Review Comment:
   Not sure if catching any `NonFatal(ex)` is a good solution here since if errors occurs, it's highly possible that something went wrong in the python worker, and better to fail the query in this case. I'll leave it as what it is for now and also added a TODO in `StreamingForeachBatchHelper` to improve handling this scenario later.



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelper.scala:
##########
@@ -125,8 +128,21 @@ object StreamingForeachBatchHelper extends Logging {
       dataOut.writeLong(args.batchId)
       dataOut.flush()
 
-      val ret = dataIn.readInt()
-      logInfo(s"Python foreach batch for dfId ${args.dfId} completed (ret: $ret)")
+      try {
+        dataIn.readInt() match {
+          case ret if ret == 0 =>
+            logInfo(s"Python foreach batch for dfId ${args.dfId} completed (ret: $ret)")
+          case SpecialLengths.PYTHON_EXCEPTION_THROWN =>
+            val exLength = dataIn.readInt()
+            val obj = new Array[Byte](exLength)
+            dataIn.readFully(obj)
+            val msg = new String(obj, StandardCharsets.UTF_8)
+            throw new PythonException(s"Found error inside foreachBatch Python process: $msg", null)

Review Comment:
   Updated.



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingQueryListenerHelper.scala:
##########
@@ -38,33 +43,56 @@ class PythonStreamingQueryListener(listener: SimplePythonFunction, sessionHolder
     sessionHolder.sessionId,
     "pyspark.sql.connect.streaming.worker.listener_worker")
 
-  val (dataOut, _) = runner.init()
+  val (dataOut, dataIn) = runner.init()
 
   override def onQueryStarted(event: StreamingQueryListener.QueryStartedEvent): Unit = {
     PythonRDD.writeUTF(event.json, dataOut)
     dataOut.writeInt(0)
     dataOut.flush()
+    handlePythonWorkerError("onQueryStarted")
   }
 
   override def onQueryProgress(event: StreamingQueryListener.QueryProgressEvent): Unit = {
     PythonRDD.writeUTF(event.json, dataOut)
     dataOut.writeInt(1)
     dataOut.flush()
+    handlePythonWorkerError("onQueryProgress")
   }
 
   override def onQueryIdle(event: StreamingQueryListener.QueryIdleEvent): Unit = {
     PythonRDD.writeUTF(event.json, dataOut)
     dataOut.writeInt(2)
     dataOut.flush()
+    handlePythonWorkerError("onQueryIdle")
   }
 
   override def onQueryTerminated(event: StreamingQueryListener.QueryTerminatedEvent): Unit = {
     PythonRDD.writeUTF(event.json, dataOut)
     dataOut.writeInt(3)
     dataOut.flush()
+    handlePythonWorkerError("onQueryTerminated")
   }
 
   private[spark] def stopListenerProcess(): Unit = {
     runner.stop()
   }
+
+  private def handlePythonWorkerError(functionName: String): Unit = {
+    try {
+      dataIn.readInt() match {

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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] bogao007 commented on a diff in pull request #42986: [SPARK-44463][SS][CONNECT] Improve error handling for Connect steaming Python worker

Posted by "bogao007 (via GitHub)" <gi...@apache.org>.
bogao007 commented on code in PR #42986:
URL: https://github.com/apache/spark/pull/42986#discussion_r1329574367


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingQueryListenerHelper.scala:
##########
@@ -76,16 +78,21 @@ class PythonStreamingQueryListener(listener: SimplePythonFunction, sessionHolder
   }
 
   private def handlePythonWorkerError(functionName: String): Unit = {
-    dataIn.readInt() match {
-      case ret if ret == 0 =>
-        logInfo(s"Streaming query listener function $functionName completed (ret: $ret)")
-      case SpecialLengths.PYTHON_EXCEPTION_THROWN =>
-        val exLength = dataIn.readInt()
-        val obj = new Array[Byte](exLength)
-        dataIn.readFully(obj)
-        val msg = new String(obj, StandardCharsets.UTF_8)
-        throw new IllegalStateException(s"Found error inside Streaming query listener Python " +
-          s"process for function $functionName: $msg")
+    try {
+      dataIn.readInt() match {
+        case ret if ret == 0 =>
+          logInfo(s"Streaming query listener function $functionName completed (ret: $ret)")
+        case SpecialLengths.PYTHON_EXCEPTION_THROWN =>
+          val exLength = dataIn.readInt()
+          val obj = new Array[Byte](exLength)
+          dataIn.readFully(obj)
+          val msg = new String(obj, StandardCharsets.UTF_8)
+          throw new PythonException(s"Found error inside Streaming query listener Python " +
+            s"process for function $functionName: $msg", null)
+      }
+    } catch {
+      case eof: EOFException =>

Review Comment:
   Not sure if catching any `NonFatal(ex)` is a good solution here since if errors occurs, it's highly possible that something went wrong in the python worker, and better to fail the query in this case. I'll leave it as what it is for now and also added a TODO in `StreamingForeachBatchHelper` to improve the handling of this scenario later.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] WweiL commented on a diff in pull request #42986: [SPARK-44463][SS][CONNECT] Improve error handling for Connect steaming Python worker

Posted by "WweiL (via GitHub)" <gi...@apache.org>.
WweiL commented on code in PR #42986:
URL: https://github.com/apache/spark/pull/42986#discussion_r1329441472


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingQueryListenerHelper.scala:
##########
@@ -76,16 +78,21 @@ class PythonStreamingQueryListener(listener: SimplePythonFunction, sessionHolder
   }
 
   private def handlePythonWorkerError(functionName: String): Unit = {
-    dataIn.readInt() match {
-      case ret if ret == 0 =>
-        logInfo(s"Streaming query listener function $functionName completed (ret: $ret)")
-      case SpecialLengths.PYTHON_EXCEPTION_THROWN =>
-        val exLength = dataIn.readInt()
-        val obj = new Array[Byte](exLength)
-        dataIn.readFully(obj)
-        val msg = new String(obj, StandardCharsets.UTF_8)
-        throw new IllegalStateException(s"Found error inside Streaming query listener Python " +
-          s"process for function $functionName: $msg")
+    try {
+      dataIn.readInt() match {
+        case ret if ret == 0 =>
+          logInfo(s"Streaming query listener function $functionName completed (ret: $ret)")
+        case SpecialLengths.PYTHON_EXCEPTION_THROWN =>
+          val exLength = dataIn.readInt()
+          val obj = new Array[Byte](exLength)
+          dataIn.readFully(obj)
+          val msg = new String(obj, StandardCharsets.UTF_8)
+          throw new PythonException(s"Found error inside Streaming query listener Python " +
+            s"process for function $functionName: $msg", null)
+      }
+    } catch {
+      case eof: EOFException =>

Review Comment:
   @HyukjinKwon @taku-k Guys, do you know if the EOFException would happen frequently in existing pyspark tests? I was wondering if this is possible:
   Say in common PythonRunner, EOF occurs, then the task would likely be failing but spark nicely handles the retry.
   
   But here if the socket has some error, we don't have any retry logic, and the stream query would just fail with our issue.
   
   Given that streamingPythonRunners are long-running processes, I was wondering if you also think having a retry-logic would be beneficial as a followup.
   
   cc @rangadi 



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] rangadi commented on a diff in pull request #42986: [SPARK-44463][SS][CONNECT] Improve error handling for Connect steaming Python worker

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #42986:
URL: https://github.com/apache/spark/pull/42986#discussion_r1329577300


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingQueryListenerHelper.scala:
##########
@@ -76,16 +78,21 @@ class PythonStreamingQueryListener(listener: SimplePythonFunction, sessionHolder
   }
 
   private def handlePythonWorkerError(functionName: String): Unit = {
-    dataIn.readInt() match {
-      case ret if ret == 0 =>
-        logInfo(s"Streaming query listener function $functionName completed (ret: $ret)")
-      case SpecialLengths.PYTHON_EXCEPTION_THROWN =>
-        val exLength = dataIn.readInt()
-        val obj = new Array[Byte](exLength)
-        dataIn.readFully(obj)
-        val msg = new String(obj, StandardCharsets.UTF_8)
-        throw new IllegalStateException(s"Found error inside Streaming query listener Python " +
-          s"process for function $functionName: $msg")
+    try {
+      dataIn.readInt() match {
+        case ret if ret == 0 =>
+          logInfo(s"Streaming query listener function $functionName completed (ret: $ret)")
+        case SpecialLengths.PYTHON_EXCEPTION_THROWN =>
+          val exLength = dataIn.readInt()
+          val obj = new Array[Byte](exLength)
+          dataIn.readFully(obj)
+          val msg = new String(obj, StandardCharsets.UTF_8)
+          throw new PythonException(s"Found error inside Streaming query listener Python " +
+            s"process for function $functionName: $msg", null)
+      }
+    } catch {
+      case eof: EOFException =>

Review Comment:
   Absolutely. It should fail the query. I thought this code was wrapping it in a SparkException and throwing it.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] bogao007 commented on a diff in pull request #42986: [SPARK-44463][SS][CONNECT] Improve error handling for Connect steaming Python worker

Posted by "bogao007 (via GitHub)" <gi...@apache.org>.
bogao007 commented on code in PR #42986:
URL: https://github.com/apache/spark/pull/42986#discussion_r1329356792


##########
python/pyspark/sql/connect/streaming/worker/listener_worker.py:
##########
@@ -83,7 +86,14 @@ def process(listener_event_str, listener_event_type):  # type: ignore[no-untyped
     while True:
         event = utf8_deserializer.loads(infile)
         event_type = read_int(infile)
-        process(event, int(event_type))  # TODO(SPARK-44463): Propagate error to the user.
+        # Handle errors inside Python worker. Write 0 to outfile if no errors and write -2 with
+        # traceback string if error occurs.
+        try:
+            process(event, int(event_type))
+            write_int(0, outfile)

Review Comment:
   I think that's the only way to propagate something out of the Python worker, if it fails, then I'm not sure if there's anything else that could propagate that error out.
   
   From the [existing worker](https://github.com/apache/spark/blob/981312284f0776ca847c8d21411f74a72c639b22/python/pyspark/sql/worker/analyze_udtf.py#L161-L163), it does not do anything if there's an IOException but close the python process. But let me update the code to use the same way they did, which should be slightly better.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] bogao007 commented on a diff in pull request #42986: [SPARK-44463][SS][CONNECT] Improve error handling for Connect steaming Python worker

Posted by "bogao007 (via GitHub)" <gi...@apache.org>.
bogao007 commented on code in PR #42986:
URL: https://github.com/apache/spark/pull/42986#discussion_r1329422764


##########
python/pyspark/sql/connect/streaming/worker/foreach_batch_worker.py:
##########
@@ -69,8 +73,32 @@ def process(df_id, batch_id):  # type: ignore[no-untyped-def]
     while True:
         df_ref_id = utf8_deserializer.loads(infile)
         batch_id = read_long(infile)
-        process(df_ref_id, int(batch_id))  # TODO(SPARK-44463): Propagate error to the user.
-        write_int(0, outfile)
+        # Handle errors inside Python worker. Write 0 to outfile if no errors and write -2 with
+        # traceback string if error occurs.
+        try:
+            process(df_ref_id, int(batch_id))
+            write_int(0, outfile)
+        except BaseException as e:

Review Comment:
   done.



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelper.scala:
##########
@@ -125,8 +128,21 @@ object StreamingForeachBatchHelper extends Logging {
       dataOut.writeLong(args.batchId)
       dataOut.flush()
 
-      val ret = dataIn.readInt()
-      logInfo(s"Python foreach batch for dfId ${args.dfId} completed (ret: $ret)")
+      try {
+        dataIn.readInt() match {
+          case ret if ret == 0 =>
+            logInfo(s"Python foreach batch for dfId ${args.dfId} completed (ret: $ret)")
+          case SpecialLengths.PYTHON_EXCEPTION_THROWN =>
+            val exLength = dataIn.readInt()
+            val obj = new Array[Byte](exLength)
+            dataIn.readFully(obj)
+            val msg = new String(obj, StandardCharsets.UTF_8)
+            throw new IllegalStateException(s"Found error inside foreachBatch Python process: $msg")

Review Comment:
   updated.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #42986: [SPARK-44463][SS][CONNECT] Improve error handling for Connect steaming Python worker

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #42986:
URL: https://github.com/apache/spark/pull/42986#discussion_r1329393005


##########
python/pyspark/sql/connect/streaming/worker/foreach_batch_worker.py:
##########
@@ -69,8 +73,32 @@ def process(df_id, batch_id):  # type: ignore[no-untyped-def]
     while True:
         df_ref_id = utf8_deserializer.loads(infile)
         batch_id = read_long(infile)
-        process(df_ref_id, int(batch_id))  # TODO(SPARK-44463): Propagate error to the user.
-        write_int(0, outfile)
+        # Handle errors inside Python worker. Write 0 to outfile if no errors and write -2 with
+        # traceback string if error occurs.
+        try:
+            process(df_ref_id, int(batch_id))
+            write_int(0, outfile)
+        except BaseException as e:

Review Comment:
   you could put that into `pyspark.util`



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] HyukjinKwon commented on pull request #42986: [SPARK-44463][SS][CONNECT] Improve error handling for Connect steaming Python worker

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on PR #42986:
URL: https://github.com/apache/spark/pull/42986#issuecomment-1724629485

   Otherwise looks sane to me. cc @ueshin 


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #42986: [SPARK-44463][SS][CONNECT] Improve error handling for Connect steaming Python worker

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #42986:
URL: https://github.com/apache/spark/pull/42986#discussion_r1329392786


##########
python/pyspark/sql/connect/streaming/worker/foreach_batch_worker.py:
##########
@@ -69,8 +73,32 @@ def process(df_id, batch_id):  # type: ignore[no-untyped-def]
     while True:
         df_ref_id = utf8_deserializer.loads(infile)
         batch_id = read_long(infile)
-        process(df_ref_id, int(batch_id))  # TODO(SPARK-44463): Propagate error to the user.
-        write_int(0, outfile)
+        # Handle errors inside Python worker. Write 0 to outfile if no errors and write -2 with
+        # traceback string if error occurs.
+        try:
+            process(df_ref_id, int(batch_id))
+            write_int(0, outfile)
+        except BaseException as e:

Review Comment:
   I think we should probably put this handling into a util and reuse it in the regular `worker.py` too.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] bogao007 commented on a diff in pull request #42986: [SPARK-44463][SS][CONNECT] Improve error handling for Connect steaming Python worker

Posted by "bogao007 (via GitHub)" <gi...@apache.org>.
bogao007 commented on code in PR #42986:
URL: https://github.com/apache/spark/pull/42986#discussion_r1329447168


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingQueryListenerHelper.scala:
##########
@@ -76,16 +78,21 @@ class PythonStreamingQueryListener(listener: SimplePythonFunction, sessionHolder
   }
 
   private def handlePythonWorkerError(functionName: String): Unit = {
-    dataIn.readInt() match {
-      case ret if ret == 0 =>
-        logInfo(s"Streaming query listener function $functionName completed (ret: $ret)")
-      case SpecialLengths.PYTHON_EXCEPTION_THROWN =>
-        val exLength = dataIn.readInt()
-        val obj = new Array[Byte](exLength)
-        dataIn.readFully(obj)
-        val msg = new String(obj, StandardCharsets.UTF_8)
-        throw new IllegalStateException(s"Found error inside Streaming query listener Python " +
-          s"process for function $functionName: $msg")
+    try {
+      dataIn.readInt() match {
+        case ret if ret == 0 =>
+          logInfo(s"Streaming query listener function $functionName completed (ret: $ret)")
+        case SpecialLengths.PYTHON_EXCEPTION_THROWN =>
+          val exLength = dataIn.readInt()
+          val obj = new Array[Byte](exLength)
+          dataIn.readFully(obj)
+          val msg = new String(obj, StandardCharsets.UTF_8)
+          throw new PythonException(s"Found error inside Streaming query listener Python " +
+            s"process for function $functionName: $msg", null)
+      }
+    } catch {
+      case eof: EOFException =>

Review Comment:
   If we want to add retry logic, adding it here would not solve the problem. If the EOF is due to an IOException during `write_` functions, we might need to add retries inside the workers. But I'm not sure how high would the possibility be for that scenario to happen, since if IOException happens while writing, it's highly possible that the socket is closed, and retry would no help in that case.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] WweiL commented on a diff in pull request #42986: [SPARK-44463][SS][CONNECT] Improve error handling for Connect steaming Python worker

Posted by "WweiL (via GitHub)" <gi...@apache.org>.
WweiL commented on code in PR #42986:
URL: https://github.com/apache/spark/pull/42986#discussion_r1329337505


##########
python/pyspark/sql/connect/streaming/worker/listener_worker.py:
##########
@@ -83,7 +86,14 @@ def process(listener_event_str, listener_event_type):  # type: ignore[no-untyped
     while True:
         event = utf8_deserializer.loads(infile)
         event_type = read_int(infile)
-        process(event, int(event_type))  # TODO(SPARK-44463): Propagate error to the user.
+        # Handle errors inside Python worker. Write 0 to outfile if no errors and write -2 with
+        # traceback string if error occurs.
+        try:
+            process(event, int(event_type))
+            write_int(0, outfile)

Review Comment:
   Do you think it's possible that the `read_` and `write_` method could throw exceptions?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #42986: [SPARK-44463][SS][CONNECT] Improve error handling for Connect steaming Python worker

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #42986:
URL: https://github.com/apache/spark/pull/42986#discussion_r1329392064


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelper.scala:
##########
@@ -125,8 +128,21 @@ object StreamingForeachBatchHelper extends Logging {
       dataOut.writeLong(args.batchId)
       dataOut.flush()
 
-      val ret = dataIn.readInt()
-      logInfo(s"Python foreach batch for dfId ${args.dfId} completed (ret: $ret)")
+      try {
+        dataIn.readInt() match {
+          case ret if ret == 0 =>
+            logInfo(s"Python foreach batch for dfId ${args.dfId} completed (ret: $ret)")
+          case SpecialLengths.PYTHON_EXCEPTION_THROWN =>
+            val exLength = dataIn.readInt()
+            val obj = new Array[Byte](exLength)
+            dataIn.readFully(obj)
+            val msg = new String(obj, StandardCharsets.UTF_8)
+            throw new IllegalStateException(s"Found error inside foreachBatch Python process: $msg")

Review Comment:
   Should it be `PythonException`? To match with `PythonRunner.handlePythonException`. Or it'd be great to see if we can reuse them.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] HyukjinKwon closed pull request #42986: [SPARK-44463][SS][CONNECT] Improve error handling for Connect steaming Python worker

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon closed pull request #42986: [SPARK-44463][SS][CONNECT] Improve error handling for Connect steaming Python worker
URL: https://github.com/apache/spark/pull/42986


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] HyukjinKwon commented on pull request #42986: [SPARK-44463][SS][CONNECT] Improve error handling for Connect steaming Python worker

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on PR #42986:
URL: https://github.com/apache/spark/pull/42986#issuecomment-1726838303

   Merged to master.


-- 
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-unsubscribe@spark.apache.org

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


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