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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to