-
Notifications
You must be signed in to change notification settings - Fork 29.2k
[SPARK-56324] Introducing message-based communication to Spark -> PySpark communication channel #55716
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
Open
sven-weber-db
wants to merge
1
commit into
apache:master
Choose a base branch
from
sven-weber-db:sven-weber_data/spark-56324
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
[SPARK-56324] Introducing message-based communication to Spark -> PySpark communication channel #55716
Changes from all commits
Commits
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
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,16 @@ | ||
| # | ||
| # 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. | ||
| # |
64 changes: 64 additions & 0 deletions
64
python/pyspark/messages/socket/spark_socket_message_receiver.py
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,64 @@ | ||
| # | ||
| # 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. | ||
| # | ||
|
|
||
| from typing import BinaryIO | ||
|
|
||
| from pyspark.serializers import read_int, SpecialLengths | ||
| from pyspark.messages.zero_copy_byte_stream import ZeroCopyByteStream | ||
| from pyspark.messages.spark_message_receiver import ( | ||
| SparkMessageReceiver, | ||
| ) | ||
|
|
||
|
|
||
| def _assert_message_id(message_id: int, expected: int): | ||
| assert message_id == expected, ( | ||
| f"Expected message with id {expected} " + f"but got message with id {message_id} instead." | ||
| ) | ||
|
|
||
|
|
||
| class SparkSocketMessageReceiver(SparkMessageReceiver): | ||
| def __init__(self, infile: BinaryIO): | ||
| super().__init__() | ||
| self._infile = infile | ||
|
|
||
| def _do_get_init_message(self) -> ZeroCopyByteStream: | ||
| message_id = read_int(self._infile) | ||
| _assert_message_id(message_id, SpecialLengths.START_OF_INIT_MESSAGE) | ||
|
|
||
| # Read the length and init content | ||
| message_length = read_int(self._infile) | ||
| message_content = self._infile.read(message_length) | ||
|
|
||
| return ZeroCopyByteStream(memoryview(message_content)) | ||
|
|
||
| def _do_get_data_stream(self) -> BinaryIO: | ||
| # For socket communication, we just pass along the underlying socket | ||
| # for the data channel. We already stripped the initialization data | ||
| # at this state. Therefore, any bytes following this are data bytes. | ||
| # | ||
| # Note: We deliberately did not introduce a message header for | ||
| # data messages to reduce the overhead, especially for small | ||
| # batch sizes and real-time-mode (RTM). | ||
| return self._infile | ||
|
|
||
| def _do_is_stream_finished(self) -> bool: | ||
| # Check if the stream is finished. | ||
| # If everything finished properly, we should read a | ||
| # 'END_OF_STREAM'. If we read somethign else this means | ||
| # the stream has unread data and something went wrong | ||
| # during processing. | ||
| return read_int(self._infile) == SpecialLengths.END_OF_STREAM |
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,120 @@ | ||
| # | ||
| # 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. | ||
| # | ||
|
|
||
| from enum import Enum | ||
| from functools import wraps | ||
| from typing import BinaryIO | ||
| from abc import ABC, abstractmethod | ||
|
|
||
| from pyspark.messages.zero_copy_byte_stream import ZeroCopyByteStream | ||
|
|
||
|
|
||
| class MessageState(Enum): | ||
| WAITING_FOR_INIT = 1 | ||
| WAITING_FOR_DATA = 2 | ||
| WAITING_FOR_FINISH = 3 | ||
| DONE = 4 | ||
|
|
||
|
|
||
| class SparkMessageReceiver(ABC): | ||
| """ | ||
| Generic class that implements receiving messages from Spark. | ||
| Caution: This class is STATEFUL. It is expected, that the | ||
| methods of this class are called in the following order: | ||
|
|
||
| 1. Init -> 2. Data stream -> 3. Finish | ||
|
|
||
| This order is verified using assertions in the class. Each function | ||
| can be called EXACTLY ONCE in the specified order. | ||
| """ | ||
|
|
||
| def __init__(self): | ||
| self._state = MessageState.WAITING_FOR_INIT | ||
|
|
||
| @staticmethod | ||
| def _state_transition(required_state: MessageState, next_state: MessageState): | ||
| """Decorator to enforce state transitions.""" | ||
|
|
||
| def decorator(func): | ||
| @wraps(func) | ||
| def wrapper(self): | ||
| assert self._state == required_state | ||
| result = func(self) | ||
| self._state = next_state | ||
| return result | ||
|
|
||
| return wrapper | ||
|
|
||
| return decorator | ||
|
|
||
| @_state_transition(MessageState.WAITING_FOR_INIT, MessageState.WAITING_FOR_DATA) | ||
| def get_init_message(self) -> ZeroCopyByteStream: | ||
| """ | ||
| Returns: | ||
| the binary contents of the initial message as a ZeroCopyByteStream. | ||
| """ | ||
| return self._do_get_init_message() | ||
|
|
||
| @_state_transition(MessageState.WAITING_FOR_DATA, MessageState.WAITING_FOR_FINISH) | ||
| def get_data_stream(self) -> BinaryIO: | ||
| """ | ||
| Returns: | ||
| A binary stream containing the data to invoke the UDF on. | ||
| """ | ||
| return self._do_get_data_stream() | ||
|
|
||
| @_state_transition(MessageState.WAITING_FOR_FINISH, MessageState.DONE) | ||
| def is_stream_finished(self) -> bool: | ||
| """ | ||
| Checks if a finish message was received | ||
| from the JVM. The finish message itself only | ||
| has a message id and marks the end of the stream. | ||
| If bytes different from the finish id are read | ||
| this means something went wrong while consuming the stream. | ||
| """ | ||
| return self._do_is_stream_finished() | ||
|
|
||
| @abstractmethod | ||
| def _do_get_init_message(self) -> ZeroCopyByteStream: | ||
| """ | ||
| Returns the contents of the init message | ||
| as a 'ZeroCopyByteStream'. | ||
|
|
||
| To be implemented by child classes. | ||
| """ | ||
| pass | ||
|
|
||
| @abstractmethod | ||
| def _do_get_data_stream(self) -> BinaryIO: | ||
| """ | ||
| Returns the Spark data stream. | ||
|
|
||
| To be implemented by child classes. | ||
| """ | ||
| pass | ||
|
|
||
| @abstractmethod | ||
| def _do_is_stream_finished(self) -> bool: | ||
| """ | ||
| Blocking call that returns whether | ||
| the data stream from the JVM is finished. | ||
| This is implemented differently, depending | ||
| on the transport channel. | ||
|
|
||
| To be implemented by child classes. | ||
| """ | ||
| pass |
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
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -39,6 +39,7 @@ | |
| Union, | ||
| get_args, | ||
| get_origin, | ||
| BinaryIO, | ||
| ) | ||
|
|
||
| T = TypeVar("T") | ||
|
|
@@ -122,6 +123,10 @@ | |
| Conf, | ||
| ) | ||
| from pyspark.logger.worker_io import capture_outputs | ||
| from pyspark.messages import ( | ||
| SparkMessageReceiver, | ||
| SparkSocketMessageReceiver, | ||
| ) | ||
|
|
||
|
|
||
| class RunnerConf(Conf): | ||
|
|
@@ -3595,10 +3600,20 @@ def func(_, it): | |
| return func, None, ser, ser | ||
|
|
||
|
|
||
| @with_faulthandler | ||
| def main(infile, outfile): | ||
| def invoke_udf(message_receiver: SparkMessageReceiver, outfile: BinaryIO): | ||
|
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. Happy to rename this function if another name is prefered |
||
| """ | ||
| This function is the main processing function for worker.py. | ||
| It receives messages from the JVM, processes the data, and sends back results. | ||
| This method goes through three phases: | ||
|
|
||
| Initialization -> Processing -> Finish/Cleanup | ||
| """ | ||
| try: | ||
| boot_time = time.time() | ||
|
|
||
| # Initialization | ||
| infile = message_receiver.get_init_message() | ||
|
|
||
| split_index = read_int(infile) | ||
| if split_index == -1: # for unit tests | ||
| sys.exit(-1) | ||
|
|
@@ -3642,8 +3657,13 @@ def main(infile, outfile): | |
|
|
||
| init_time = time.time() | ||
|
|
||
| # Processing | ||
|
|
||
| # Fetch the input data stream | ||
| input_data_stream = message_receiver.get_data_stream() | ||
|
|
||
| def process(): | ||
| iterator = deserializer.load_stream(infile) | ||
| iterator = deserializer.load_stream(input_data_stream) | ||
| out_iter = func(split_index, iterator) | ||
| try: | ||
| serializer.dump_stream(out_iter, outfile) | ||
|
|
@@ -3659,6 +3679,7 @@ def process(): | |
| process() | ||
| processing_time_ms = int(1000 * (time.time() - processing_start_time)) | ||
|
|
||
| # Cleanup | ||
| # Reset task context to None. This is a guard code to avoid residual context when worker | ||
| # reuse. | ||
| TaskContext._setTaskContext(None) | ||
|
|
@@ -3676,14 +3697,22 @@ def process(): | |
| send_accumulator_updates(outfile) | ||
|
|
||
| # check end of stream | ||
| if read_int(infile) == SpecialLengths.END_OF_STREAM: | ||
| if message_receiver.is_stream_finished(): | ||
| write_int(SpecialLengths.END_OF_STREAM, outfile) | ||
| else: | ||
| # write a different value to tell JVM to not reuse this worker | ||
| write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) | ||
| sys.exit(-1) | ||
|
|
||
|
|
||
| @with_faulthandler | ||
| def main(infile, outfile): | ||
| # Instantiate socket message readers for executing the UDF | ||
| socket_reader = SparkSocketMessageReceiver(infile) | ||
|
|
||
| invoke_udf(socket_reader, outfile) | ||
|
|
||
|
|
||
| if __name__ == "__main__": | ||
| with get_sock_file_to_executor() as sock_file: | ||
| main(sock_file, sock_file) | ||
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.
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 change is required because we use the
ZeroCopyByteStreamclass now to read the initial message.ZeroCopyByteStreamreturnsmemoryviewobjects instead ofbytesorbytearray.s.decodeis only implemented fors∈{bytes, bytearray}.According to the Python documentation, for
bytes.decodeandcodecs.decodethe later implementation may behave different in case of errors.bytes.decodestates thatcodecs.decodestates thatAs decoding errors are unexpected and the specific Exception type that is thrown should not matter, I believe this change is safe. However, if there are concerns we can change this implementation to the following:
This alternative implementation will invoke a memory copy to copy the
memoryviewcontents into abytesobject before decoding. Given that this call is only invoked on deserialization of the initial message, this memory copy is probably acceptable as it is a one time cost.