Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -415,13 +415,17 @@ private[spark] abstract class BasePythonRunner[IN, OUT](
*/
def writeNextInputToStream(dataOut: DataOutputStream): Boolean

def open(dataOut: DataOutputStream): Unit = Utils.logUncaughtExceptions {
def open(outputStream: DataOutputStream): Unit = Utils.logUncaughtExceptions {
val isUnixDomainSock = authHelper.conf.get(PYTHON_UNIX_DOMAIN_SOCKET_ENABLED)
lazy val sockPath = new File(
authHelper.conf.get(PYTHON_UNIX_DOMAIN_SOCKET_DIR)
.getOrElse(System.getProperty("java.io.tmpdir")),
s".${UUID.randomUUID()}.sock")
try {
// Buffer the initialization message, and send it together with its length.
val buffer = new ByteArrayOutputStream()
val dataOut = new DataOutputStream(buffer)

// Partition index
dataOut.writeInt(partitionIndex)

Expand Down Expand Up @@ -522,6 +526,13 @@ private[spark] abstract class BasePythonRunner[IN, OUT](
writeCommand(dataOut)

dataOut.flush()

// The initialization message is complete, write it to the stream with its length.
val messageBytes = buffer.toByteArray
outputStream.writeInt(SpecialLengths.START_OF_INIT_MESSAGE)
outputStream.writeInt(messageBytes.length)
outputStream.write(messageBytes)
outputStream.flush()
} catch {
case t: Throwable if NonFatal(t) || t.isInstanceOf[Exception] =>
if (context.isCompleted() || context.isInterrupted()) {
Expand Down Expand Up @@ -1085,6 +1096,7 @@ private[spark] object SpecialLengths {
val NULL = -5
val START_ARROW_STREAM = -6
val END_OF_MICRO_BATCH = -7
val START_OF_INIT_MESSAGE = -8
}

private[spark] object BarrierTaskContextMessageProtocol {
Expand Down
2 changes: 2 additions & 0 deletions python/packaging/classic/setup.py
Original file line number Diff line number Diff line change
Expand Up @@ -267,6 +267,8 @@ def run(self):
"pyspark",
"pyspark.core",
"pyspark.cloudpickle",
"pyspark.messages",
"pyspark.messages.socket",
"pyspark.mllib",
"pyspark.mllib.linalg",
"pyspark.mllib.stat",
Expand Down
2 changes: 2 additions & 0 deletions python/packaging/client/setup.py
Original file line number Diff line number Diff line change
Expand Up @@ -148,6 +148,8 @@
connect_packages = [
"pyspark",
"pyspark.cloudpickle",
"pyspark.messages",
"pyspark.messages.socket",
"pyspark.mllib",
"pyspark.mllib.linalg",
"pyspark.mllib.stat",
Expand Down
4 changes: 4 additions & 0 deletions python/pyspark/messages/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -15,8 +15,12 @@
# limitations under the License.
#

from pyspark.messages.spark_message_receiver import SparkMessageReceiver
from pyspark.messages.zero_copy_byte_stream import ZeroCopyByteStream
from pyspark.messages.socket.spark_socket_message_receiver import SparkSocketMessageReceiver

__all__ = [
"SparkMessageReceiver",
"SparkSocketMessageReceiver",
"ZeroCopyByteStream",
]
16 changes: 16 additions & 0 deletions python/pyspark/messages/socket/__init__.py
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 python/pyspark/messages/socket/spark_socket_message_receiver.py
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
120 changes: 120 additions & 0 deletions python/pyspark/messages/spark_message_receiver.py
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
4 changes: 3 additions & 1 deletion python/pyspark/serializers.py
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,7 @@
import zlib
import itertools
import pickle
import codecs

pickle_protocol = pickle.HIGHEST_PROTOCOL

Expand All @@ -84,6 +85,7 @@ class SpecialLengths:
END_OF_STREAM = -4
NULL = -5
START_ARROW_STREAM = -6
START_OF_INIT_MESSAGE = -8


class Serializer:
Expand Down Expand Up @@ -539,7 +541,7 @@ def loads(self, stream):
elif length == SpecialLengths.NULL:
return None
s = stream.read(length)
return s.decode("utf-8") if self.use_unicode else s
return codecs.decode(s, "utf-8") if self.use_unicode else s
Copy link
Copy Markdown
Contributor Author

@sven-weber-db sven-weber-db May 7, 2026

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 ZeroCopyByteStream class now to read the initial message. ZeroCopyByteStream returns memoryview objects instead of bytes or bytearray. s.decode is only implemented for s{bytes, bytearray}.

According to the Python documentation, for bytes.decode and codecs.decode the later implementation may behave different in case of errors.

bytes.decode states that

errors controls how decoding errors are handled. If 'strict' (the default), a UnicodeError exception is raised.

codecs.decode states that

The default error handler is 'strict' meaning that decoding errors raise ValueError (or a more codec specific subclass, such as UnicodeDecodeError)

As 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:

return bytes(s).decode("utf-8") if self.use_unicode else s

This alternative implementation will invoke a memory copy to copy the memoryview contents into a bytes object 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.


def load_stream(self, stream):
try:
Expand Down
37 changes: 33 additions & 4 deletions python/pyspark/worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
Union,
get_args,
get_origin,
BinaryIO,
)

T = TypeVar("T")
Expand Down Expand Up @@ -122,6 +123,10 @@
Conf,
)
from pyspark.logger.worker_io import capture_outputs
from pyspark.messages import (
SparkMessageReceiver,
SparkSocketMessageReceiver,
)


class RunnerConf(Conf):
Expand Down Expand Up @@ -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):
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The 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)
Expand Down Expand Up @@ -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)
Expand All @@ -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)
Expand All @@ -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)