-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-42941][SS][CONNECT] Python StreamingQueryListener #42116
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from all commits
Commits
Show all changes
50 commits
Select commit
Hold shift + click to select a range
d363260
foreachbatch spark connect
pengzhon-db 508928a
add streaming_worker.py
pengzhon-db 6cb7d01
python proto
pengzhon-db 74ad159
use same python process for one streaming query
pengzhon-db 1f73c6f
wip
WweiL 2dec59c
working
WweiL 40bf120
latest change
WweiL 57b80ff
resolve conflicts
WweiL c5c8e85
streaming function declaration pyi & rdd
WweiL 8b9dfe5
this won't work, still throws None obj doesn't have craeteDataFrame
WweiL 341d588
same error, this also doesn't work
WweiL 4d8eec3
this worked
WweiL 3a43d6c
first revision
WweiL 38d76c0
file cleanup
WweiL 1923840
doc update
WweiL 8bcf605
is this breaking change also?
WweiL 56665c3
remove doc test for now
WweiL cb0caea
add remove listener
WweiL 9c4f6e6
gen proto
WweiL 3724992
ticket update
WweiL 2e3b8d2
before resolving merge conflict
WweiL 4a6a15f
resolve conflict
WweiL 0040b70
documentation to PythonStreamingQueryListener
WweiL d404f9f
this works on manual test but in unit test it shows No module named '…
WweiL a2041ec
works now
WweiL 4a2d184
minor
WweiL 5d04245
minor
WweiL 8c97ecc
doesn't need to make q stateful
WweiL 494c243
fmt
WweiL 28d1495
why is there a unused import
WweiL b7cc36f
minor
WweiL 20b87e6
try to resolve breaking change, will address other comments tmr
WweiL 72552ed
scala client send ids
WweiL 0640fdf
remove return NOne
WweiL b99301b
merge master, remove println log
WweiL 004e181
add streamingPythonEval
WweiL 640ab23
minor:
WweiL 270363b
remove eval type, create two worker files
WweiL 487bfa1
lint
WweiL ed5f101
retrigger tests
WweiL 8c5569f
Merge remote-tracking branch 'spark/master' into listener-poc-newest
WweiL 13f50bf
minor
WweiL 3f83d07
lint
WweiL eb4d2b5
lint
WweiL 95b0111
Merge remote-tracking branch 'spark/master' into listener-poc-newest
WweiL fb4415b
lint
WweiL baf791c
address comments, move worker files to sql/connect/streaming/worker
WweiL 8c520ba
minor, remove redundant log
WweiL aa22c3b
add init
WweiL c9ffa52
add new pkg to setup.py
WweiL File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
69 changes: 69 additions & 0 deletions
69
...er/src/main/scala/org/apache/spark/sql/connect/planner/StreamingQueryListenerHelper.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,69 @@ | ||
| /* | ||
| * 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.{PythonRDD, SimplePythonFunction, StreamingPythonRunner} | ||
| import org.apache.spark.sql.connect.service.{SessionHolder, SparkConnectService} | ||
| import org.apache.spark.sql.streaming.StreamingQueryListener | ||
|
|
||
| /** | ||
| * A helper class for handling StreamingQueryListener related functionality in Spark Connect. Each | ||
| * instance of this class starts a python process, inside which has the python handling logic. | ||
| * When new a event is received, it is serialized to json, and passed to the python process. | ||
| */ | ||
| class PythonStreamingQueryListener( | ||
WweiL marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| listener: SimplePythonFunction, | ||
| sessionHolder: SessionHolder, | ||
| pythonExec: String) | ||
| extends StreamingQueryListener { | ||
|
|
||
| val port = SparkConnectService.localPort | ||
| val connectUrl = s"sc://localhost:$port/;user_id=${sessionHolder.userId}" | ||
| val runner = StreamingPythonRunner(listener, connectUrl) | ||
|
|
||
| val (dataOut, _) = | ||
| runner.init(sessionHolder.sessionId, "pyspark.sql.connect.streaming.worker.listener_worker") | ||
|
|
||
| override def onQueryStarted(event: StreamingQueryListener.QueryStartedEvent): Unit = { | ||
| PythonRDD.writeUTF(event.json, dataOut) | ||
| dataOut.writeInt(0) | ||
| dataOut.flush() | ||
| } | ||
|
|
||
| override def onQueryProgress(event: StreamingQueryListener.QueryProgressEvent): Unit = { | ||
| PythonRDD.writeUTF(event.json, dataOut) | ||
| dataOut.writeInt(1) | ||
| dataOut.flush() | ||
| } | ||
|
|
||
| override def onQueryIdle(event: StreamingQueryListener.QueryIdleEvent): Unit = { | ||
| PythonRDD.writeUTF(event.json, dataOut) | ||
| dataOut.writeInt(2) | ||
| dataOut.flush() | ||
| } | ||
|
|
||
| override def onQueryTerminated(event: StreamingQueryListener.QueryTerminatedEvent): Unit = { | ||
| PythonRDD.writeUTF(event.json, dataOut) | ||
| dataOut.writeInt(3) | ||
| dataOut.flush() | ||
| } | ||
|
|
||
| // TODO(SPARK-44433)(SPARK-44516): Improve termination of Processes. | ||
| // Similar to foreachBatch when we need to exit the process when the query ends. | ||
| // In listener semantics, we need to exit the process when removeListener is called. | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for doing this change!