-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-44433][PYTHON][CONNECT][SS][FOLLOWUP] Terminate listener process with removeListener and improvements
#42283
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -29,27 +29,36 @@ import org.apache.spark.internal.config.Python.{PYTHON_AUTH_SOCKET_TIMEOUT, PYTH | |
|
|
||
|
|
||
| private[spark] object StreamingPythonRunner { | ||
| def apply(func: PythonFunction, connectUrl: String): StreamingPythonRunner = { | ||
| new StreamingPythonRunner(func, connectUrl) | ||
| def apply( | ||
| func: PythonFunction, | ||
| connectUrl: String, | ||
| sessionId: String, | ||
| workerModule: String | ||
| ): StreamingPythonRunner = { | ||
| new StreamingPythonRunner(func, connectUrl, sessionId, workerModule) | ||
| } | ||
| } | ||
|
|
||
| private[spark] class StreamingPythonRunner(func: PythonFunction, connectUrl: String) | ||
| extends Logging { | ||
| private[spark] class StreamingPythonRunner( | ||
| func: PythonFunction, | ||
| connectUrl: String, | ||
| sessionId: String, | ||
| workerModule: String) extends Logging { | ||
| private val conf = SparkEnv.get.conf | ||
| protected val bufferSize: Int = conf.get(BUFFER_SIZE) | ||
| protected val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT) | ||
|
|
||
| private val envVars: java.util.Map[String, String] = func.envVars | ||
| private val pythonExec: String = func.pythonExec | ||
| private var pythonWorker: Option[Socket] = None | ||
| protected val pythonVer: String = func.pythonVer | ||
|
|
||
| /** | ||
| * Initializes the Python worker for streaming functions. Sets up Spark Connect session | ||
| * to be used with the functions. | ||
| */ | ||
| def init(sessionId: String, workerModule: String): (DataOutputStream, DataInputStream) = { | ||
| logInfo(s"Initializing Python runner (session: $sessionId ,pythonExec: $pythonExec") | ||
| def init(): (DataOutputStream, DataInputStream) = { | ||
| logInfo(s"Initializing Python runner (session: $sessionId, pythonExec: $pythonExec") | ||
| val env = SparkEnv.get | ||
|
|
||
| val localdir = env.blockManager.diskBlockManager.localDirs.map(f => f.getPath()).mkString(",") | ||
|
|
@@ -60,9 +69,9 @@ private[spark] class StreamingPythonRunner(func: PythonFunction, connectUrl: Str | |
| conf.set(PYTHON_USE_DAEMON, false) | ||
| envVars.put("SPARK_CONNECT_LOCAL_URL", connectUrl) | ||
|
|
||
| val pythonWorkerFactory = | ||
| new PythonWorkerFactory(pythonExec, workerModule, envVars.asScala.toMap) | ||
| val (worker: Socket, _) = pythonWorkerFactory.createSimpleWorker() | ||
| val (worker, _) = env.createPythonWorker( | ||
| pythonExec, workerModule, envVars.asScala.toMap) | ||
| pythonWorker = Some(worker) | ||
|
|
||
| val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize) | ||
| val dataOut = new DataOutputStream(stream) | ||
|
|
@@ -85,4 +94,13 @@ private[spark] class StreamingPythonRunner(func: PythonFunction, connectUrl: Str | |
|
|
||
| (dataOut, dataIn) | ||
| } | ||
|
|
||
| /** | ||
| * Stops the Python worker. | ||
| */ | ||
| def stop(): Unit = { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Minor: please add documentation since this is a public function |
||
| pythonWorker.foreach { worker => | ||
| SparkEnv.get.destroyPythonWorker(pythonExec, workerModule, envVars.asScala.toMap, worker) | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -60,6 +60,10 @@ def test_listener_events(self): | |
| try: | ||
| self.spark.streams.addListener(test_listener) | ||
|
|
||
| # This ensures the read socket on the server won't crash (i.e. because of timeout) | ||
| # when there hasn't been a new event for a long time | ||
| time.sleep(30) | ||
|
|
||
| df = self.spark.readStream.format("rate").option("rowsPerSecond", 10).load() | ||
| q = df.writeStream.format("noop").queryName("test").start() | ||
|
|
||
|
|
@@ -76,6 +80,9 @@ def test_listener_events(self): | |
| finally: | ||
| self.spark.streams.removeListener(test_listener) | ||
|
|
||
| # Remove again to verify this won't throw any error | ||
| self.spark.streams.removeListener(test_listener) | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. How does this test ensure listener worker is removed?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. That test is not to ensure the worker is removed, it is to ensure no error will be thrown when |
||
|
|
||
|
|
||
| if __name__ == "__main__": | ||
| import unittest | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.
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.
This is not updated in this PR, but should we set this back to the original value after creating the Python worker?
As the
confis visible from other part in the Driver, it could affect the behavior.It can be done in a separate PR.
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.
Btw, do we need to need to change this at all? It might be simpler to keep this unchanged.