Skip to content

Conversation

@WweiL
Copy link
Contributor

@WweiL WweiL commented Aug 4, 2023

Master Branch PR: #42283

What changes were proposed in this pull request?

This is a followup to #42116. It addresses the following issues:

  1. When removeListener is called upon one listener, before the python process is left running, now it also get stopped.
  2. When multiple removeListener is called on the same listener, in non-connect mode, subsequent calls will be noop. But before this PR, in connect it actually throws an error, which doesn't align with existing behavior, this PR addresses it.
  3. Set the socket timeout to be None (\infty) for foreachBatch_worker and listener_worker, because there could be a long time between each microbatch. If not setting this, the socket will timeout and won't be able to process new data.
scala> Streaming query listener worker is starting with url sc://localhost:15002/;user_id=wei.liu and sessionId 886191f0-2b64-4c44-b067-de511f04b42d.
Traceback (most recent call last):
  File "/usr/lib/python3.9/runpy.py", line 197, in _run_module_as_main
    return _run_code(code, main_globals, None,
  File "/usr/lib/python3.9/runpy.py", line 87, in _run_code
    exec(code, run_globals)
  File "/home/wei.liu/oss-spark/python/lib/pyspark.zip/pyspark/sql/connect/streaming/worker/listener_worker.py", line 95, in <module>
  File "/home/wei.liu/oss-spark/python/lib/pyspark.zip/pyspark/sql/connect/streaming/worker/listener_worker.py", line 82, in main
  File "/home/wei.liu/oss-spark/python/lib/pyspark.zip/pyspark/serializers.py", line 557, in loads
  File "/home/wei.liu/oss-spark/python/lib/pyspark.zip/pyspark/serializers.py", line 594, in read_int
  File "/usr/lib/python3.9/socket.py", line 704, in readinto
    return self._sock.recv_into(b)
socket.timeout: timed out

Why are the changes needed?

Necessary improvements

Does this PR introduce any user-facing change?

No

How was this patch tested?

Manual test + unit test

pythonWorkerFactory = Some(workerFactory)
} finally {
conf.set(PYTHON_USE_DAEMON, prevConf)
}
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This and the stop() method are different from master branch since the createPythonWorker method doesn't support custom modules at that time:

https://github.com/WweiL/oss-spark/blob/f8b312a22eae3ce1176da49a693182832c1f1402/core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala#L72-L74

cc @ueshin to double check this

Copy link
Member

@ueshin ueshin left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Otherwise, LGTM, pending tests.

@ueshin
Copy link
Member

ueshin commented Aug 5, 2023

Thanks! merging to 3.5.

ueshin pushed a commit that referenced this pull request Aug 5, 2023
…process with removeListener and improvements

### Master Branch PR: #42283

### What changes were proposed in this pull request?

This is a followup to #42116. It addresses the following issues:

1. When `removeListener` is called upon one listener, before the python process is left running, now it also get stopped.
2. When multiple `removeListener` is called on the same listener, in non-connect mode, subsequent calls will be noop. But before this PR, in connect it actually throws an error, which doesn't align with existing behavior, this PR addresses it.
3. Set the socket timeout to be None (\infty) for `foreachBatch_worker` and `listener_worker`, because there could be a long time between each microbatch. If not setting this, the socket will timeout and won't be able to process new data.

```
scala> Streaming query listener worker is starting with url sc://localhost:15002/;user_id=wei.liu and sessionId 886191f0-2b64-4c44-b067-de511f04b42d.
Traceback (most recent call last):
  File "/usr/lib/python3.9/runpy.py", line 197, in _run_module_as_main
    return _run_code(code, main_globals, None,
  File "/usr/lib/python3.9/runpy.py", line 87, in _run_code
    exec(code, run_globals)
  File "/home/wei.liu/oss-spark/python/lib/pyspark.zip/pyspark/sql/connect/streaming/worker/listener_worker.py", line 95, in <module>
  File "/home/wei.liu/oss-spark/python/lib/pyspark.zip/pyspark/sql/connect/streaming/worker/listener_worker.py", line 82, in main
  File "/home/wei.liu/oss-spark/python/lib/pyspark.zip/pyspark/serializers.py", line 557, in loads
  File "/home/wei.liu/oss-spark/python/lib/pyspark.zip/pyspark/serializers.py", line 594, in read_int
  File "/usr/lib/python3.9/socket.py", line 704, in readinto
    return self._sock.recv_into(b)
socket.timeout: timed out
```

### Why are the changes needed?

Necessary improvements

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Manual test + unit test

Closes #42340 from WweiL/SPARK-44433-listener-followup-3.5.

Authored-by: Wei Liu <[email protected]>
Signed-off-by: Takuya UESHIN <[email protected]>
@ueshin ueshin closed this Aug 5, 2023
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants