You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "WweiL (via GitHub)" <gi...@apache.org> on 2023/07/24 01:58:51 UTC

[GitHub] [spark] WweiL opened a new pull request, #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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

   <!--
   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.
   -->
   Implement the python streaming query listener and only the addListener method, follow up filed in: SPARK-44516. We might need to change the proto file to implement removeListener
   
   ### 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.
   -->
   SS Connect development
   
   ### 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 now they can use connect listener
   
   ### 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 test for now. Will update tests 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 #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -3047,18 +3047,29 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging {
         respBuilder.setResetTerminated(true)
 
       case StreamingQueryManagerCommand.CommandCase.ADD_LISTENER =>
-        val listenerPacket = Utils
-          .deserialize[StreamingListenerPacket](
-            command.getAddListener.getListenerPayload.toByteArray,
-            Utils.getContextOrSparkClassLoader)
-        val listener: StreamingQueryListener = listenerPacket.listener
-          .asInstanceOf[StreamingQueryListener]
-        val id: String = listenerPacket.id
-        sessionHolder.cacheListenerById(id, listener)
+        val listener = if (command.getAddListener.hasListenerPayload) {
+          val listenerPacket = Utils
+            .deserialize[StreamingListenerPacket](
+              command.getAddListener.getListenerPayload.toByteArray,
+              Utils.getContextOrSparkClassLoader)
+          val listener: StreamingQueryListener = listenerPacket.listener
+            .asInstanceOf[StreamingQueryListener]
+          val id: String = listenerPacket.id
+          sessionHolder.cacheListenerById(id, listener)
+          listener
+        } else {
+          val listener = new PythonStreamingQueryListener(
+            transformPythonFunction(command.getAddListener.getPythonListenerPayload),

Review Comment:
   Ah yea this is left deliberately. I thought I need to do the process termination as part of `removeListener` But then it seems that we already have a ticket filed for `foreachBatch`. [SPARK-44433](https://issues.apache.org/jira/browse/SPARK-44433) I'll just push the change for `removeListener`



-- 
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] ueshin commented on a diff in pull request #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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


##########
core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala:
##########
@@ -48,9 +48,8 @@ private[spark] class StreamingPythonRunner(func: PythonFunction, connectUrl: Str
    * Initializes the Python worker for streaming functions. Sets up Spark Connect session
    * to be used with the functions.
    */
-  def init(sessionId: String): (DataOutputStream, DataInputStream) = {
-    logInfo(s"Initializing Python runner (session: $sessionId ,pythonExec: $pythonExec")
-
+  def init(sessionId: String, workerModule: String): (DataOutputStream, DataInputStream) = {
+    log.info(s"Initializing Python runner (session: $sessionId ,pythonExec: $pythonExec")

Review Comment:
   should use `logInfo`?



-- 
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 pull request #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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

   @HyukjinKwon Can you take another look? Thanks! This also needs to goto 3.5 sorry for the trouble!


-- 
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 #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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


##########
python/pyspark/sql/streaming/listener.py:
##########
@@ -62,6 +63,19 @@ class StreamingQueryListener(ABC):
     >>> spark.streams.addListener(MyListener())
     """
 
+    def _set_spark_session(self, spark):
+        self._sparkSession = spark
+
+    @property
+    def spark(self) -> Optional["SparkSession"]:
+        if hasattr(self, "_sparkSession"):
+            return self._sparkSession
+        else:
+            return None
+
+    def _init_listener_id(self):

Review Comment:
   This is used for `removeListener`. We initialize a listener id with each listener, when removing, only send the id so the server side recognizes it (server side has a id-listener mapping cache) and removes 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] WweiL commented on a diff in pull request #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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


##########
python/pyspark/sql/streaming/query.py:
##########
@@ -618,12 +618,23 @@ def addListener(self, listener: StreamingQueryListener) -> None:
 
         .. versionadded:: 3.4.0
 
+        .. versionchanged:: 3.5.0
+            Supports Spark Connect.
+
         Parameters
         ----------
         listener : :class:`StreamingQueryListener`
             A :class:`StreamingQueryListener` to receive up-calls for life cycle events of
             :class:`~pyspark.sql.streaming.StreamingQuery`.
 
+        Notes
+        -----
+        This function behaves differently in Spark Connect mode.
+        In Connect, the provided functions doesn't have access to variables defined outside of it.

Review Comment:
   Yes, similar to python foreachBatch, because now the code is running on the server, so if you define a variable outside of the function, even if it will be picked by pickle, any modification to that variable won't be sent back to client. ForeachBatch side might have a clearer doc for this:
   https://github.com/apache/spark/pull/42096/files#r1269909249



-- 
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 #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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


##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -230,21 +233,26 @@ def resetTerminated(self) -> None:
 
     resetTerminated.__doc__ = PySparkStreamingQueryManager.resetTerminated.__doc__
 
-    def addListener(self, listener: Any) -> None:
-        # TODO(SPARK-42941): Change listener type to Connect StreamingQueryListener
-        # and implement below
-        raise NotImplementedError("addListener() is not implemented.")
+    def addListener(self, listener: StreamingQueryListener) -> None:
+        listener._init_listener_id()
+        cmd = pb2.StreamingQueryManagerCommand()
+        expr = proto.PythonUDF()
+        expr.command = CloudPickleSerializer().dumps(listener)
+        expr.python_ver = "%d.%d" % sys.version_info[:2]
+        cmd.add_listener.python_listener_payload.CopyFrom(expr)
+        cmd.add_listener.id = listener._id
+        self._execute_streaming_query_manager_cmd(cmd)
+        return None

Review Comment:
   can be removed - all of `return None`.



-- 
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 #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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


##########
core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala:
##########
@@ -73,7 +72,10 @@ private[spark] class StreamingPythonRunner(func: PythonFunction, connectUrl: Str
     // Send sessionId

Review Comment:
   I really prefer to match the implementation to https://github.com/apache/spark/pull/41948 first. Otherwise, it's difficult to review, and would be more difficult to refactor 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 pull request #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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

   Hi Takuya @ueshin, could you check if this could be merged? Thanks! 


-- 
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 #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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


##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -230,21 +233,26 @@ def resetTerminated(self) -> None:
 
     resetTerminated.__doc__ = PySparkStreamingQueryManager.resetTerminated.__doc__
 
-    def addListener(self, listener: Any) -> None:
-        # TODO(SPARK-42941): Change listener type to Connect StreamingQueryListener
-        # and implement below
-        raise NotImplementedError("addListener() is not implemented.")
+    def addListener(self, listener: StreamingQueryListener) -> None:
+        listener._init_listener_id()
+        cmd = pb2.StreamingQueryManagerCommand()
+        expr = proto.PythonUDF()
+        expr.command = CloudPickleSerializer().dumps(listener)
+        expr.python_ver = "%d.%d" % sys.version_info[:2]
+        cmd.add_listener.python_listener_payload.CopyFrom(expr)
+        cmd.add_listener.id = listener._id
+        self._execute_streaming_query_manager_cmd(cmd)
+        return None

Review Comment:
   https://github.com/apache/spark/pull/40785#issuecomment-1515737988



-- 
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 #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -3066,23 +3066,27 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging {
         respBuilder.setResetTerminated(true)
 
       case StreamingQueryManagerCommand.CommandCase.ADD_LISTENER =>
-        val listenerPacket = Utils
-          .deserialize[StreamingListenerPacket](
-            command.getAddListener.getListenerPayload.toByteArray,
-            Utils.getContextOrSparkClassLoader)
-        val listener: StreamingQueryListener = listenerPacket.listener
-          .asInstanceOf[StreamingQueryListener]
-        val id: String = listenerPacket.id
+        val listener = if (command.getAddListener.hasPythonListenerPayload) {
+          new PythonStreamingQueryListener(
+            transformPythonFunction(command.getAddListener.getPythonListenerPayload),
+            sessionHolder,
+            pythonExec)
+        } else {
+          val listenerPacket = Utils
+            .deserialize[StreamingListenerPacket](
+              command.getAddListener.getListenerPayload.toByteArray,
+              Utils.getContextOrSparkClassLoader)
+
+          listenerPacket.listener.asInstanceOf[StreamingQueryListener]
+        }
+
+        val id = command.getAddListener.getId
         sessionHolder.cacheListenerById(id, listener)
         session.streams.addListener(listener)
         respBuilder.setAddListener(true)
 
       case StreamingQueryManagerCommand.CommandCase.REMOVE_LISTENER =>
-        val listenerId = Utils
-          .deserialize[StreamingListenerPacket](
-            command.getRemoveListener.getListenerPayload.toByteArray,
-            Utils.getContextOrSparkClassLoader)
-          .id
+        val listenerId = command.getRemoveListener.getId

Review Comment:
   Thanks for doing this change!



-- 
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 pull request #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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

   > @WweiL There was a conflict with 3.5. Could you submit another PR to backport this? Thanks.
   
   @ueshin Created a backport PR to 3.5 branch https://github.com/apache/spark/pull/42249, could you help take a look? Thanks!


-- 
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 #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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


##########
core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala:
##########
@@ -48,9 +48,8 @@ private[spark] class StreamingPythonRunner(func: PythonFunction, connectUrl: Str
    * Initializes the Python worker for streaming functions. Sets up Spark Connect session
    * to be used with the functions.
    */
-  def init(sessionId: String): (DataOutputStream, DataInputStream) = {
-    logInfo(s"Initializing Python runner (session: $sessionId ,pythonExec: $pythonExec")
-
+  def init(sessionId: String, workerModule: String): (DataOutputStream, DataInputStream) = {
+    log.info(s"Initializing Python runner (session: $sessionId ,pythonExec: $pythonExec")

Review Comment:
   Ah yea this is probably because of a concurrent PR of mine changing the same thing.. Thanks! 



-- 
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 #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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


##########
python/pyspark/sql/connect/streaming/worker/__init__.py:
##########


Review Comment:
   ah thanks! New thing to learn



-- 
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 #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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


##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -230,21 +233,26 @@ def resetTerminated(self) -> None:
 
     resetTerminated.__doc__ = PySparkStreamingQueryManager.resetTerminated.__doc__
 
-    def addListener(self, listener: Any) -> None:
-        # TODO(SPARK-42941): Change listener type to Connect StreamingQueryListener
-        # and implement below
-        raise NotImplementedError("addListener() is not implemented.")
+    def addListener(self, listener: StreamingQueryListener) -> None:
+        listener._init_listener_id()
+        cmd = pb2.StreamingQueryManagerCommand()
+        expr = proto.PythonUDF()
+        expr.command = CloudPickleSerializer().dumps(listener)
+        expr.python_ver = "%d.%d" % sys.version_info[:2]
+        cmd.add_listener.python_listener_payload.CopyFrom(expr)
+        cmd.add_listener.id = listener._id
+        self._execute_streaming_query_manager_cmd(cmd)
+        return None

Review Comment:
   Actually I do remember before I tried to remove these `return None`s but in the end it fails some python lint test



-- 
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 pull request #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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

   @HyukjinKwon Can you also take a look at this? Thank you!


-- 
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] ueshin closed pull request #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

Posted by "ueshin (via GitHub)" <gi...@apache.org>.
ueshin closed pull request #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener
URL: https://github.com/apache/spark/pull/42116


-- 
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] ueshin commented on pull request #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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

   Thanks! merging to master/3.5.


-- 
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 #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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


##########
python/pyspark/sql/tests/streaming/test_streaming_listener.py:
##########
@@ -304,6 +192,120 @@ def check_sink_progress(self, progress):
         self.assertTrue(isinstance(progress.numOutputRows, int))
         self.assertTrue(isinstance(progress.metrics, dict))
 
+
+class StreamingListenerTests(StreamingListenerTestsMixin, ReusedSQLTestCase):

Review Comment:
   This test is refactored. I only keep the check_xx_evenet method in the Mixin class. Because of the behavior change, previous checks won't work in Connect.



-- 
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 #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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

   cc @ueshin 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] ueshin commented on a diff in pull request #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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


##########
python/pyspark/sql/connect/streaming/query.py:
##########
@@ -230,21 +233,26 @@ def resetTerminated(self) -> None:
 
     resetTerminated.__doc__ = PySparkStreamingQueryManager.resetTerminated.__doc__
 
-    def addListener(self, listener: Any) -> None:
-        # TODO(SPARK-42941): Change listener type to Connect StreamingQueryListener
-        # and implement below
-        raise NotImplementedError("addListener() is not implemented.")
+    def addListener(self, listener: StreamingQueryListener) -> None:
+        listener._init_listener_id()
+        cmd = pb2.StreamingQueryManagerCommand()
+        expr = proto.PythonUDF()
+        expr.command = CloudPickleSerializer().dumps(listener)
+        expr.python_ver = "%d.%d" % sys.version_info[:2]
+        cmd.add_listener.python_listener_payload.CopyFrom(expr)
+        cmd.add_listener.id = listener._id
+        self._execute_streaming_query_manager_cmd(cmd)
+        return None

Review Comment:
   We can omit `return None` (or `return`) when the return type is `None`.



-- 
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] ueshin commented on a diff in pull request #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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


##########
python/pyspark/sql/connect/streaming/worker/__init__.py:
##########


Review Comment:
   I guess we need to add this package to `setup.py`?



-- 
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] ueshin commented on pull request #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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

   @WweiL There was a conflict with 3.5. Could you submit another PR to backport this? Thanks.


-- 
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 #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -3047,18 +3047,29 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging {
         respBuilder.setResetTerminated(true)
 
       case StreamingQueryManagerCommand.CommandCase.ADD_LISTENER =>
-        val listenerPacket = Utils
-          .deserialize[StreamingListenerPacket](
-            command.getAddListener.getListenerPayload.toByteArray,
-            Utils.getContextOrSparkClassLoader)
-        val listener: StreamingQueryListener = listenerPacket.listener
-          .asInstanceOf[StreamingQueryListener]
-        val id: String = listenerPacket.id
-        sessionHolder.cacheListenerById(id, listener)
+        val listener = if (command.getAddListener.hasListenerPayload) {
+          val listenerPacket = Utils
+            .deserialize[StreamingListenerPacket](
+              command.getAddListener.getListenerPayload.toByteArray,
+              Utils.getContextOrSparkClassLoader)
+          val listener: StreamingQueryListener = listenerPacket.listener
+            .asInstanceOf[StreamingQueryListener]
+          val id: String = listenerPacket.id
+          sessionHolder.cacheListenerById(id, listener)
+          listener
+        } else {
+          val listener = new PythonStreamingQueryListener(
+            transformPythonFunction(command.getAddListener.getPythonListenerPayload),

Review Comment:
   Ah yea this is left deliberately. I thought I need to do the process termination as part of `removeListener` But then it seems that we already have a ticket filed for `foreachBatch`. [SPARK-44433](https://issues.apache.org/jira/browse/SPARK-44433) I'll just push the change for `removeListener`
   
   I actually modify this part in the newest revision. Sorry for the trouble!



-- 
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 #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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


##########
core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala:
##########
@@ -59,6 +59,9 @@ private[spark] object PythonEvalType {
   val SQL_TABLE_UDF = 300
   val SQL_ARROW_TABLE_UDF = 301
 
+  val SQL_STREAMING_FOREACH_BATCH = 400
+  val SQL_STREAMING_LISTENER = 401

Review Comment:
   can we have a separate worker file instead of using eval? event handling is async so it might conflict with other existing running Python workers. At least they would affect each other on execution time I susepct?



-- 
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 #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingQueryListenerHelper.scala:
##########
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.connect.planner
+
+import org.apache.spark.api.python.{PythonEvalType, PythonRDD, SimplePythonFunction, StreamingPythonRunner}
+import org.apache.spark.sql.connect.service.{SessionHolder, SparkConnectService}
+import org.apache.spark.sql.streaming.StreamingQueryListener
+
+class PythonStreamingQueryListener(

Review Comment:
   Can we add documentations on why we need this class?



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala:
##########
@@ -3047,18 +3047,29 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging {
         respBuilder.setResetTerminated(true)
 
       case StreamingQueryManagerCommand.CommandCase.ADD_LISTENER =>
-        val listenerPacket = Utils
-          .deserialize[StreamingListenerPacket](
-            command.getAddListener.getListenerPayload.toByteArray,
-            Utils.getContextOrSparkClassLoader)
-        val listener: StreamingQueryListener = listenerPacket.listener
-          .asInstanceOf[StreamingQueryListener]
-        val id: String = listenerPacket.id
-        sessionHolder.cacheListenerById(id, listener)
+        val listener = if (command.getAddListener.hasListenerPayload) {
+          val listenerPacket = Utils
+            .deserialize[StreamingListenerPacket](
+              command.getAddListener.getListenerPayload.toByteArray,
+              Utils.getContextOrSparkClassLoader)
+          val listener: StreamingQueryListener = listenerPacket.listener
+            .asInstanceOf[StreamingQueryListener]
+          val id: String = listenerPacket.id
+          sessionHolder.cacheListenerById(id, listener)
+          listener
+        } else {
+          val listener = new PythonStreamingQueryListener(
+            transformPythonFunction(command.getAddListener.getPythonListenerPayload),

Review Comment:
   We might need to cache the id -> listener mapping for Python as well, but this can be done when you implement removeListener.



##########
python/pyspark/sql/streaming/query.py:
##########
@@ -618,12 +618,23 @@ def addListener(self, listener: StreamingQueryListener) -> None:
 
         .. versionadded:: 3.4.0
 
+        .. versionchanged:: 3.5.0
+            Supports Spark Connect.
+
         Parameters
         ----------
         listener : :class:`StreamingQueryListener`
             A :class:`StreamingQueryListener` to receive up-calls for life cycle events of
             :class:`~pyspark.sql.streaming.StreamingQuery`.
 
+        Notes
+        -----
+        This function behaves differently in Spark Connect mode.
+        In Connect, the provided functions doesn't have access to variables defined outside of it.

Review Comment:
   Is this a limitation for Python that variables outside of the functions cannot be accessed?



-- 
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 #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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


##########
python/pyspark/sql/connect/session.py:
##########
@@ -10,7 +10,7 @@
 #
 # Unless required by applicable law or agreed to in writing, software
 # distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# WITHOUT WARRANTIES OR CONDITIONS OF A[attr-defined]NY KIND, either express or implied.

Review Comment:
   Ah thanks so much for pointing that out! 



-- 
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] ueshin commented on a diff in pull request #42116: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener

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


##########
core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala:
##########
@@ -48,9 +48,8 @@ private[spark] class StreamingPythonRunner(func: PythonFunction, connectUrl: Str
    * Initializes the Python worker for streaming functions. Sets up Spark Connect session
    * to be used with the functions.
    */
-  def init(sessionId: String): (DataOutputStream, DataInputStream) = {
-    logInfo(s"Initializing Python runner (session: $sessionId ,pythonExec: $pythonExec")
-
+  def init(sessionId: String, workerModule: String): (DataOutputStream, DataInputStream) = {
+    log.info(s"Initializing Python runner (session: $sessionId ,pythonExec: $pythonExec")

Review Comment:
   `logInfo`?



##########
python/pyspark/connect_streaming_foreachBatch.py:
##########


Review Comment:
   ditto.



##########
python/pyspark/connect_streaming_listener.py:
##########


Review Comment:
   How about putting the worker files in some package instead of a top level?



##########
python/pyspark/sql/connect/session.py:
##########
@@ -10,7 +10,7 @@
 #
 # Unless required by applicable law or agreed to in writing, software
 # distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# WITHOUT WARRANTIES OR CONDITIONS OF A[attr-defined]NY KIND, either express or implied.

Review Comment:
   Seems to be a mistake? Could you revert this?



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