[SPARK-56324] Add ZeroCopyByteStream to enable PySpark <-> Spark message-based communication#55515
[SPARK-56324] Add ZeroCopyByteStream to enable PySpark <-> Spark message-based communication#55515sven-weber-db wants to merge 1 commit intoapache:masterfrom
Conversation
84b90de to
dbaec94
Compare
dbaec94 to
caf5e47
Compare
|
|
||
| The chunk to be added cannot be None. | ||
| """ | ||
| if type(chunk) is not memoryview: |
There was a problem hiding this comment.
We used isinstance() in __init__ and is not here - I think we can allow subclass right? So maybe use isinstance here for the consistency?
| self._current_chunk = initial_view | ||
| self._current_position = 0 | ||
| self._eof = False | ||
| self._lock = threading.Lock() |
There was a problem hiding this comment.
The only reason we create this lock is for _condition right? threading.Condition() will create a RLock without any argument - is there a concern to use RLock? I don't want developers to see this lock as something that they can access, if the interface should be just self._condition.
| self._current_position += to_read | ||
|
|
||
| # If entire chunk consumed, clear it for next chunk | ||
| if self._current_position > len(self._current_chunk): |
There was a problem hiding this comment.
Let's change this to an assertion. This should never happen if our code is correct. No user data should trigger this.
| def read(self, size: int) -> memoryview: | ||
| """ | ||
| Reads size bytes. If the read failed because the underlying | ||
| stream was marked as finished (EOF), None is returned. |
There was a problem hiding this comment.
This comment is not accurate anymore right?
|
|
||
| return result | ||
|
|
||
| def read(self, size: int) -> memoryview: |
There was a problem hiding this comment.
I think I mentioned this once - should we support read() for read everything? I think in the future we probably just want to read the whole thing and parse.
What changes were proposed in this pull request?
This is the first in a series of PRs that introduce message-based communication to PySpark UDFs. This initiative is part of SPIP SPARK-55278, which proposes language-agnostic UDFs.
The goal of introducing message-based communication to PySpark is to:
The overall goal is to introduce a second communication channel while keeping the existing channel intact. Specifically, we want to introduce gRPC in addition to UDS. The existing UDS channel will not be changed, and its characteristics, including performance, will remain untouched.
As the first step to make PySpark communication message-based, this PR introduces a new class, which implements a file-like interface on top of a stream of byte arrays. This class will be used in follow-up PRs to provide raw gRPC-transmitted bytes to PySpark.
Why are the changes needed?
This is the first step toward a language-agnostic UDF protocol for Spark that enables UDF workers written in any language to communicate with the Spark engine through a well-defined specification and API boundary. The abstractions introduced here will be used to make PySpark transport layer agnostic, which is required for PySpark to support the new protocol.
Does this PR introduce any user-facing change?
No. There will be follow-up PRs to consume the introduced abstractions.
How was this patch tested?
New unit tests have been added for the new modules.
Was this patch authored or co-authored using generative AI tooling?
Partially, yes. However, the code is manually reviewed and adjusted.