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

   <!--
   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.
   -->
   Add support for Streaming Listener in Scala for Spark Connect
   
   ### 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.
   -->
   Streaming Listener is a key feature for monitoring Structured Streaming and 
we want it to be available in Spark Connect.
   
   ### 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.
   -->
   Manual Tested:
   - Define the custom listener class and add it
   ```
   @ import scala.collection.mutable
   import scala.collection.mutable
   
   @ import org.apache.spark.sql.streaming.StreamingQueryProgress
   import org.apache.spark.sql.streaming.StreamingQueryProgress
   
   @ import org.apache.spark.sql.streaming.StreamingQueryListener
   import org.apache.spark.sql.streaming.StreamingQueryListener
   
   @ import 
org.apache.spark.sql.streaming.StreamingQueryListener.{QueryIdleEvent, 
QueryStartedEvent, QueryTerminatedEvent}
   import 
org.apache.spark.sql.streaming.StreamingQueryListener.{QueryIdleEvent, 
QueryStartedEvent, QueryTerminatedEvent}
   
   @ class EventCollector extends StreamingQueryListener {
         @volatile var startEvent: QueryStartedEvent = null
         @volatile var terminationEvent: QueryTerminatedEvent = null
         @volatile var idleEvent: QueryIdleEvent = null
   
         private val _progressEvents = new mutable.Queue[StreamingQueryProgress]
   
         def progressEvents: Seq[StreamingQueryProgress] = 
_progressEvents.synchronized {
           _progressEvents.filter(_.numInputRows > 0).toSeq
         }
   
         def allProgressEvents: Seq[StreamingQueryProgress] = 
_progressEvents.synchronized {
           _progressEvents.clone().toSeq
         }
   
         def reset(): Unit = {
           startEvent = null
           idleEvent = null
           terminationEvent = null
           _progressEvents.clear()
         }
   
         override def onQueryStarted(event: 
StreamingQueryListener.QueryStartedEvent): Unit = {
           startEvent = event
           println("Query start: " + startEvent.json)
         }
   
         override def onQueryProgress(event: 
StreamingQueryListener.QueryProgressEvent): Unit = {
           _progressEvents += event.progress
           println("Progress: " + event.progress)
         }
   
         override def onQueryIdle(event: 
StreamingQueryListener.QueryIdleEvent): Unit = {
           idleEvent = event
         }
   
         override def onQueryTerminated(
                                         event: 
StreamingQueryListener.QueryTerminatedEvent): Unit = {
           terminationEvent = event
         }
       }
   defined class EventCollector
   
   @ val listener = new EventCollector
   listener: EventCollector = ammonite.$sess.cmd4$Helper$EventCollector@e6fbf82
   
   @ spark.streams.addListener(listener)
   
   
   @ val q = 
spark.readStream.format("rate").load().writeStream.format("console").start()
   q: org.apache.spark.sql.streaming.StreamingQuery = 
org.apache.spark.sql.streaming.RemoteStreamingQuery@71eb4ea4
   
   @ q.stop()
   ```
   - List listeners
   ```
   @ spark.streams.listListeners()
   res9: Array[StreamingQueryListener] = 
Array(org.apache.spark.sql.streaming.ui.StreamingQueryStatusListener@6cd31e5f, 
ammonite.$sess.cmd4$Helper$EventCollector@57be0d93)
   ```
   Current Problems:
   - Remove and list again, there's no difference since the class id has 
changed after serialization and deserialization
    ```
   @ spark.streams.listListeners()
   res14: Array[StreamingQueryListener] = 
Array(org.apache.spark.sql.streaming.ui.StreamingQueryStatusListener@39ebf505, 
ammonite.$sess.cmd4$Helper$EventCollector@217c7135)
   
   @ spark.streams.removeListener(listener)
   
   
   @ spark.streams.listListeners()
   res16: Array[StreamingQueryListener] = 
Array(org.apache.spark.sql.streaming.ui.StreamingQueryStatusListener@6048e171, 
ammonite.$sess.cmd4$Helper$EventCollector@75eed00f)
    ```
       
     - Custom variables defined in `EventCollector` doesn't get serialized
     ```
     @ print(listener.startEvent)
   null
   
   @ val currListener = spark.streams.listListeners()(1)
   currListener: StreamingQueryListener = 
ammonite.$sess.cmd4$Helper$EventCollector@40f53ea
   
   @ print(currListener.startEvent)
   cmd12.sc:1: value startEvent is not a member of 
org.apache.spark.sql.streaming.StreamingQueryListener
   val res12 = print(currListener.startEvent)
                                  ^
   Compilation Failed
     ```


-- 
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