From ac5d83c4aecb4989da086629ab95bab854a0f665 Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Fri, 26 Sep 2025 10:44:44 +0000 Subject: [PATCH 01/26] first commit --- python/pyspark/sql/connect/dataframe.py | 6 +++--- python/pyspark/sql/conversion.py | 20 +++++++++---------- python/pyspark/sql/pandas/serializers.py | 8 ++++++-- python/pyspark/worker.py | 11 ++++++++-- .../apache/spark/sql/internal/SQLConf.scala | 8 ++++++++ .../execution/python/ArrowPythonRunner.scala | 5 ++++- 6 files changed, 40 insertions(+), 18 deletions(-) diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index ab7fdc90ba3c5..608c90b1285bc 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -1823,7 +1823,7 @@ def collect(self) -> List[Row]: assert schema is not None and isinstance(schema, StructType) - return ArrowTableToRowsConversion.convert(table, schema) + return ArrowTableToRowsConversion.convert(table, schema, binary_as_bytes = False) def _to_table(self) -> Tuple["pa.Table", Optional[StructType]]: query = self._plan.to_proto(self._session.client) @@ -2075,7 +2075,7 @@ def toLocalIterator(self, prefetchPartitions: bool = False) -> Iterator[Row]: table = schema_or_table if schema is None: schema = from_arrow_schema(table.schema, prefer_timestamp_ntz=True) - yield from ArrowTableToRowsConversion.convert(table, schema) + yield from ArrowTableToRowsConversion.convert(table, schema, binary_as_bytes = False) def pandas_api( self, index_col: Optional[Union[str, List[str]]] = None @@ -2162,7 +2162,7 @@ def foreach_func(row: Any) -> None: def foreachPartition(self, f: Callable[[Iterator[Row]], None]) -> None: schema = self._schema field_converters = [ - ArrowTableToRowsConversion._create_converter(f.dataType) for f in schema.fields + ArrowTableToRowsConversion._create_converter(f.dataType, binary_as_bytes = False) for f in schema.fields ] def foreach_partition_func(itr: Iterable[pa.RecordBatch]) -> Iterable[pa.RecordBatch]: diff --git a/python/pyspark/sql/conversion.py b/python/pyspark/sql/conversion.py index f1aa55c2039ac..613323e2b4614 100644 --- a/python/pyspark/sql/conversion.py +++ b/python/pyspark/sql/conversion.py @@ -518,13 +518,13 @@ def _create_converter(dataType: DataType) -> Callable: @overload @staticmethod def _create_converter( - dataType: DataType, *, none_on_identity: bool = True + dataType: DataType, *, none_on_identity: bool = True, binary_as_bytes: bool = True ) -> Optional[Callable]: pass @staticmethod def _create_converter( - dataType: DataType, *, none_on_identity: bool = False + dataType: DataType, *, none_on_identity: bool = False, binary_as_bytes: bool = True ) -> Optional[Callable]: assert dataType is not None and isinstance(dataType, DataType) @@ -542,7 +542,7 @@ def _create_converter( dedup_field_names = _dedup_names(field_names) field_convs = [ - ArrowTableToRowsConversion._create_converter(f.dataType, none_on_identity=True) + ArrowTableToRowsConversion._create_converter(f.dataType, none_on_identity=True, binary_as_bytes=binary_as_bytes) for f in dataType.fields ] @@ -564,7 +564,7 @@ def convert_struct(value: Any) -> Any: elif isinstance(dataType, ArrayType): element_conv = ArrowTableToRowsConversion._create_converter( - dataType.elementType, none_on_identity=True + dataType.elementType, none_on_identity=True, binary_as_bytes=binary_as_bytes ) if element_conv is None: @@ -589,10 +589,10 @@ def convert_array(value: Any) -> Any: elif isinstance(dataType, MapType): key_conv = ArrowTableToRowsConversion._create_converter( - dataType.keyType, none_on_identity=True + dataType.keyType, none_on_identity=True, binary_as_bytes=binary_as_bytes ) value_conv = ArrowTableToRowsConversion._create_converter( - dataType.valueType, none_on_identity=True + dataType.valueType, none_on_identity=True, binary_as_bytes=binary_as_bytes ) if key_conv is None: @@ -646,7 +646,7 @@ def convert_binary(value: Any) -> Any: return None else: assert isinstance(value, bytes) - return bytearray(value) + return value if binary_as_bytes else bytearray(value) return convert_binary @@ -676,7 +676,7 @@ def convert_timestample_ntz(value: Any) -> Any: udt: UserDefinedType = dataType conv = ArrowTableToRowsConversion._create_converter( - udt.sqlType(), none_on_identity=True + udt.sqlType(), none_on_identity=True, binary_as_bytes=binary_as_bytes ) if conv is None: @@ -735,7 +735,7 @@ def convert( @staticmethod # type: ignore[misc] def convert( - table: "pa.Table", schema: StructType, *, return_as_tuples: bool = False + table: "pa.Table", schema: StructType, *, return_as_tuples: bool = False, binary_as_bytes: bool = True ) -> List[Union[Row, tuple]]: require_minimum_pyarrow_version() import pyarrow as pa @@ -748,7 +748,7 @@ def convert( if len(fields) > 0: field_converters = [ - ArrowTableToRowsConversion._create_converter(f.dataType, none_on_identity=True) + ArrowTableToRowsConversion._create_converter(f.dataType, none_on_identity=True, binary_as_bytes=binary_as_bytes) for f in schema.fields ] diff --git a/python/pyspark/sql/pandas/serializers.py b/python/pyspark/sql/pandas/serializers.py index 801a87c06cc5d..56c410ccb5831 100644 --- a/python/pyspark/sql/pandas/serializers.py +++ b/python/pyspark/sql/pandas/serializers.py @@ -854,6 +854,8 @@ class ArrowBatchUDFSerializer(ArrowStreamArrowUDFSerializer): int_to_decimal_coercion_enabled : bool If True, applies additional coercions in Python before converting to Arrow This has performance penalties. + binary_as_bytes : bool + If True, binary type will be deserialized as bytes, otherwise as bytearray. """ def __init__( @@ -861,7 +863,8 @@ def __init__( timezone, safecheck, input_types, - int_to_decimal_coercion_enabled=False, + int_to_decimal_coercion_enabled, + binary_as_bytes, ): super().__init__( timezone=timezone, @@ -871,6 +874,7 @@ def __init__( ) self._input_types = input_types self._int_to_decimal_coercion_enabled = int_to_decimal_coercion_enabled + self._binary_as_bytes = binary_as_bytes def load_stream(self, stream): """ @@ -887,7 +891,7 @@ def load_stream(self, stream): List of columns containing list of Python values. """ converters = [ - ArrowTableToRowsConversion._create_converter(dt, none_on_identity=True) + ArrowTableToRowsConversion._create_converter(dt, none_on_identity=True, binary_as_bytes=self._binary_as_bytes) for dt in self._input_types ] diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 25eaf26243917..03410f032e08a 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -1288,6 +1288,7 @@ def use_legacy_pandas_udf_conversion(runner_conf): def read_udtf(pickleSer, infile, eval_type): prefers_large_var_types = False legacy_pandas_conversion = False + binary_as_bytes = True if eval_type == PythonEvalType.SQL_ARROW_TABLE_UDF: runner_conf = {} @@ -1304,6 +1305,9 @@ def read_udtf(pickleSer, infile, eval_type): ).lower() == "true" ) + binary_as_bytes = ( + runner_conf.get("spark.sql.execution.arrow.pyspark.binaryAsBytes", "true").lower() == "true" + ) input_types = [ field.dataType for field in _parse_datatype_json_string(utf8_deserializer.loads(infile)) ] @@ -2248,7 +2252,7 @@ def evaluate(*args: list, num_rows=1): def mapper(_, it): try: converters = [ - ArrowTableToRowsConversion._create_converter(dt, none_on_identity=True) + ArrowTableToRowsConversion._create_converter(dt, none_on_identity=True, binary_as_bytes=binary_as_bytes) for dt in input_types ] for a in it: @@ -2545,6 +2549,9 @@ def read_udfs(pickleSer, infile, eval_type): ).lower() == "true" ) + binary_as_bytes = ( + runner_conf.get("spark.sql.execution.arrow.pyspark.binaryAsBytes", "true").lower() == "true" + ) _assign_cols_by_name = assign_cols_by_name(runner_conf) if eval_type == PythonEvalType.SQL_GROUPED_MAP_ARROW_UDF: @@ -2636,7 +2643,7 @@ def read_udfs(pickleSer, infile, eval_type): f.dataType for f in _parse_datatype_json_string(utf8_deserializer.loads(infile)) ] ser = ArrowBatchUDFSerializer( - timezone, safecheck, input_types, int_to_decimal_coercion_enabled + timezone, safecheck, input_types, int_to_decimal_coercion_enabled, binary_as_bytes ) else: # Scalar Pandas UDF handles struct type arguments as pandas DataFrames instead of diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 17b8dd493cf80..d904d7a5b8de5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -3724,6 +3724,14 @@ object SQLConf { .booleanConf .createWithDefault(false) + val ARROW_PYSPARK_BINARY_TYPE_AS_BYTES = + buildConf("spark.sql.execution.arrow.pyspark.binaryAsBytes") + .doc("To be added.") + .version("4.1.0") + .booleanConf + .createWithDefault(true) + + val ARROW_LOCAL_RELATION_THRESHOLD = buildConf("spark.sql.execution.arrow.localRelationThreshold") .doc( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala index 609fa218f1288..9f75682b6ddd0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala @@ -155,9 +155,12 @@ object ArrowPythonRunner { val intToDecimalCoercion = Seq( SQLConf.PYTHON_UDF_PANDAS_INT_TO_DECIMAL_COERCION_ENABLED.key -> conf.getConf(SQLConf.PYTHON_UDF_PANDAS_INT_TO_DECIMAL_COERCION_ENABLED, false).toString) + val binaryAsBytes = Seq( + SQLConf.ARROW_PYSPARK_BINARY_TYPE_AS_BYTES.key -> + conf.getConf(SQLConf.ARROW_PYSPARK_BINARY_TYPE_AS_BYTES).toString) Map(timeZoneConf ++ pandasColsByName ++ arrowSafeTypeCheck ++ arrowAyncParallelism ++ useLargeVarTypes ++ - intToDecimalCoercion ++ + intToDecimalCoercion ++ binaryAsBytes ++ legacyPandasConversion ++ legacyPandasConversionUDF: _*) } } From 1e702c90266e718be4eedd2a228483bcea29bb76 Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Fri, 26 Sep 2025 11:13:10 +0000 Subject: [PATCH 02/26] data source related --- python/pyspark/sql/worker/data_source_pushdown_filters.py | 4 +++- python/pyspark/sql/worker/plan_data_source_read.py | 5 ++++- python/pyspark/sql/worker/write_into_data_source.py | 3 ++- .../datasources/v2/python/UserDefinedPythonDataSource.scala | 3 +++ 4 files changed, 12 insertions(+), 3 deletions(-) diff --git a/python/pyspark/sql/worker/data_source_pushdown_filters.py b/python/pyspark/sql/worker/data_source_pushdown_filters.py index ac6f84e617150..b523cab7c49e7 100644 --- a/python/pyspark/sql/worker/data_source_pushdown_filters.py +++ b/python/pyspark/sql/worker/data_source_pushdown_filters.py @@ -27,7 +27,7 @@ from pyspark.accumulators import _accumulatorRegistry from pyspark.errors import PySparkAssertionError, PySparkValueError from pyspark.errors.exceptions.base import PySparkNotImplementedError -from pyspark.serializers import SpecialLengths, UTF8Deserializer, read_int, write_int +from pyspark.serializers import SpecialLengths, UTF8Deserializer, read_int, read_bool, write_int from pyspark.sql.datasource import ( DataSource, DataSourceReader, @@ -232,6 +232,7 @@ def main(infile: IO, outfile: IO) -> None: "The maximum arrow batch size should be greater than 0, but got " f"'{max_arrow_batch_size}'" ) + binary_as_bytes = read_bool(infile) # Return the read function and partitions. Doing this in the same worker as filter pushdown # helps reduce the number of Python worker calls. @@ -241,6 +242,7 @@ def main(infile: IO, outfile: IO) -> None: data_source=data_source, schema=schema, max_arrow_batch_size=max_arrow_batch_size, + binary_as_bytes=binary_as_bytes, ) # Return the supported filter indices. diff --git a/python/pyspark/sql/worker/plan_data_source_read.py b/python/pyspark/sql/worker/plan_data_source_read.py index 7c14ebfc53e4d..15b0e610cdaf3 100644 --- a/python/pyspark/sql/worker/plan_data_source_read.py +++ b/python/pyspark/sql/worker/plan_data_source_read.py @@ -175,11 +175,12 @@ def write_read_func_and_partitions( data_source: DataSource, schema: StructType, max_arrow_batch_size: int, + binary_as_bytes: bool, ) -> None: is_streaming = isinstance(reader, DataSourceStreamReader) # Create input converter. - converter = ArrowTableToRowsConversion._create_converter(BinaryType()) + converter = ArrowTableToRowsConversion._create_converter(BinaryType(), binary_as_bytes=binary_as_bytes) # Create output converter. return_type = schema @@ -352,6 +353,7 @@ def main(infile: IO, outfile: IO) -> None: enable_pushdown = read_bool(infile) is_streaming = read_bool(infile) + binary_as_bytes = read_bool(infile) # Instantiate data source reader. if is_streaming: @@ -390,6 +392,7 @@ def main(infile: IO, outfile: IO) -> None: data_source=data_source, schema=schema, max_arrow_batch_size=max_arrow_batch_size, + binary_as_bytes=binary_as_bytes, ) except BaseException as e: handle_worker_exception(e, outfile) diff --git a/python/pyspark/sql/worker/write_into_data_source.py b/python/pyspark/sql/worker/write_into_data_source.py index 3e772031225d5..a009aa1b591d1 100644 --- a/python/pyspark/sql/worker/write_into_data_source.py +++ b/python/pyspark/sql/worker/write_into_data_source.py @@ -171,6 +171,7 @@ def main(infile: IO, outfile: IO) -> None: overwrite = read_bool(infile) is_streaming = read_bool(infile) + binary_as_bytes = read_bool(infile) # Instantiate a data source. data_source = data_source_cls(options=options) # type: ignore @@ -205,7 +206,7 @@ def main(infile: IO, outfile: IO) -> None: import pyarrow as pa converters = [ - ArrowTableToRowsConversion._create_converter(f.dataType) for f in schema.fields + ArrowTableToRowsConversion._create_converter(f.dataType, binary_as_bytes=binary_as_bytes) for f in schema.fields ] fields = schema.fieldNames() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/UserDefinedPythonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/UserDefinedPythonDataSource.scala index 14aeba92dafe1..f41caaa5d687b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/UserDefinedPythonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/UserDefinedPythonDataSource.scala @@ -458,6 +458,7 @@ private class UserDefinedPythonDataSourceFilterPushdownRunner( // Send configurations dataOut.writeInt(SQLConf.get.arrowMaxRecordsPerBatch) + dataOut.writeBoolean(SQLConf.get.arrowPySparkBinaryAsBytes) } override protected def receiveFromPython(dataIn: DataInputStream): PythonFilterPushdownResult = { @@ -550,6 +551,7 @@ private class UserDefinedPythonDataSourceReadRunner( dataOut.writeBoolean(SQLConf.get.pythonFilterPushDown) dataOut.writeBoolean(isStreaming) + dataOut.writeBoolean(SQLConf.get.arrowPySparkBinaryAsBytes) } override protected def receiveFromPython(dataIn: DataInputStream): PythonDataSourceReadInfo = { @@ -600,6 +602,7 @@ private class UserDefinedPythonDataSourceWriteRunner( dataOut.writeBoolean(overwrite) dataOut.writeBoolean(isStreaming) + dataOut.writeBoolean(SQLConf.get.arrowPySparkBinaryAsBytes) } override protected def receiveFromPython( From 659fe4e9acbce1fabcc01a11d7710f73618c9199 Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Fri, 26 Sep 2025 11:23:40 +0000 Subject: [PATCH 03/26] add conf on connect --- python/pyspark/sql/connect/dataframe.py | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index 608c90b1285bc..9f1e1c9352eae 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -1823,7 +1823,11 @@ def collect(self) -> List[Row]: assert schema is not None and isinstance(schema, StructType) - return ArrowTableToRowsConversion.convert(table, schema, binary_as_bytes = False) + return ArrowTableToRowsConversion.convert(table, schema, binary_as_bytes=self._get_binary_as_bytes()) + + def _get_binary_as_bytes(self) -> bool: + """Get the binary_as_bytes configuration value from Spark session.""" + return self._session.conf.get("spark.sql.execution.arrow.pyspark.binaryAsBytes", "true").lower() == "true" def _to_table(self) -> Tuple["pa.Table", Optional[StructType]]: query = self._plan.to_proto(self._session.client) @@ -2075,7 +2079,7 @@ def toLocalIterator(self, prefetchPartitions: bool = False) -> Iterator[Row]: table = schema_or_table if schema is None: schema = from_arrow_schema(table.schema, prefer_timestamp_ntz=True) - yield from ArrowTableToRowsConversion.convert(table, schema, binary_as_bytes = False) + yield from ArrowTableToRowsConversion.convert(table, schema, binary_as_bytes=self._get_binary_as_bytes()) def pandas_api( self, index_col: Optional[Union[str, List[str]]] = None @@ -2161,8 +2165,9 @@ def foreach_func(row: Any) -> None: def foreachPartition(self, f: Callable[[Iterator[Row]], None]) -> None: schema = self._schema + binary_as_bytes = self._get_binary_as_bytes() field_converters = [ - ArrowTableToRowsConversion._create_converter(f.dataType, binary_as_bytes = False) for f in schema.fields + ArrowTableToRowsConversion._create_converter(f.dataType, binary_as_bytes=binary_as_bytes) for f in schema.fields ] def foreach_partition_func(itr: Iterable[pa.RecordBatch]) -> Iterable[pa.RecordBatch]: From e79153224ee8a342ce4cad474a35910865b51527 Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Fri, 26 Sep 2025 13:24:04 +0000 Subject: [PATCH 04/26] add partial tests --- .../arrow/test_arrow_binary_as_bytes_udf.py | 145 ++++++++++++++++++ python/pyspark/sql/tests/test_conversion.py | 61 ++++++++ .../apache/spark/sql/internal/SQLConf.scala | 4 +- .../execution/python/ArrowPythonRunner.scala | 4 +- 4 files changed, 211 insertions(+), 3 deletions(-) create mode 100644 python/pyspark/sql/tests/arrow/test_arrow_binary_as_bytes_udf.py diff --git a/python/pyspark/sql/tests/arrow/test_arrow_binary_as_bytes_udf.py b/python/pyspark/sql/tests/arrow/test_arrow_binary_as_bytes_udf.py new file mode 100644 index 0000000000000..b18364aacc3e6 --- /dev/null +++ b/python/pyspark/sql/tests/arrow/test_arrow_binary_as_bytes_udf.py @@ -0,0 +1,145 @@ +# +# 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. +# + +import unittest + +from pyspark.sql import Row +from pyspark.sql.functions import udf, col +from pyspark.sql.types import ( + ArrayType, + BinaryType, + IntegerType, + MapType, + StringType, + StructType, + StructField, +) +from pyspark.testing.sqlutils import ( + have_pandas, + have_pyarrow, + pandas_requirement_message, + pyarrow_requirement_message, + ReusedSQLTestCase, +) +from pyspark.util import PythonEvalType + + +@unittest.skipIf( + not have_pandas or not have_pyarrow, pandas_requirement_message or pyarrow_requirement_message +) +class BinaryAsBytesUDFTests(ReusedSQLTestCase): + def test_arrow_batched_udf_binary_type(self): + def get_binary_type(x): + return type(x).__name__ + + binary_udf = udf(get_binary_type, returnType="string", useArrow=True) + + df = self.spark.createDataFrame([ + Row(b=b"hello"), + Row(b=b"world") + ], schema=StructType([StructField("b", BinaryType())])) + + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): + result = df.select(binary_udf(col("b")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytes") + self.assertEqual(result[1]["type_name"], "bytes") + + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): + result = df.select(binary_udf(col("b")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytearray") + self.assertEqual(result[1]["type_name"], "bytearray") + + def test_arrow_batched_udf_array_binary_type(self): + """Test SQL_ARROW_BATCHED_UDF with array of binary""" + def check_array_binary_types(arr): + return [type(x).__name__ for x in arr] + + array_binary_udf = udf(check_array_binary_types, returnType="array", useArrow=True) + + df = self.spark.createDataFrame([ + Row(arr_b=[b"a", b"b"]), + Row(arr_b=[b"c", b"d"]) + ], schema=StructType([StructField("arr_b", ArrayType(BinaryType()))])) + + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): + result = df.select(array_binary_udf(col("arr_b")).alias("types")).collect() + self.assertEqual(result[0]["types"], ["bytes", "bytes"]) + self.assertEqual(result[1]["types"], ["bytes", "bytes"]) + + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): + result = df.select(array_binary_udf(col("arr_b")).alias("types")).collect() + self.assertEqual(result[0]["types"], ["bytearray", "bytearray"]) + self.assertEqual(result[1]["types"], ["bytearray", "bytearray"]) + + def test_arrow_batched_udf_map_binary_type(self): + def check_map_binary_types(m): + return [type(v).__name__ for v in m.values()] + + map_binary_udf = udf(check_map_binary_types, returnType="array", useArrow=True) + + df = self.spark.createDataFrame([ + Row(map_b={"k1": b"v1", "k2": b"v2"}), + Row(map_b={"k3": b"v3"}) + ], schema=StructType([StructField("map_b", MapType(StringType(), BinaryType()))])) + + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): + result = df.select(map_binary_udf(col("map_b")).alias("types")).collect() + self.assertEqual(set(result[0]["types"]), {"bytes"}) + self.assertEqual(result[1]["types"], ["bytes"]) + + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): + result = df.select(map_binary_udf(col("map_b")).alias("types")).collect() + self.assertEqual(set(result[0]["types"]), {"bytearray"}) + self.assertEqual(result[1]["types"], ["bytearray"]) + + def test_arrow_batched_udf_struct_binary_type(self): + def check_struct_binary_type(s): + return type(s.b).__name__ + + struct_binary_udf = udf(check_struct_binary_type, returnType="string", useArrow=True) + + struct_schema = StructType([ + StructField("i", IntegerType()), + StructField("b", BinaryType()) + ]) + + df = self.spark.createDataFrame([ + Row(struct_b=Row(i=1, b=b"data1")), + Row(struct_b=Row(i=2, b=b"data2")) + ], schema=StructType([StructField("struct_b", struct_schema)])) + + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): + result = df.select(struct_binary_udf(col("struct_b")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytes") + self.assertEqual(result[1]["type_name"], "bytes") + + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): + result = df.select(struct_binary_udf(col("struct_b")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytearray") + self.assertEqual(result[1]["type_name"], "bytearray") + + +if __name__ == "__main__": + from pyspark.sql.tests.arrow.test_arrow_binary_as_bytes_udf import * # noqa: F401 + + try: + import xmlrunner # type: ignore + + testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2) + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) \ No newline at end of file diff --git a/python/pyspark/sql/tests/test_conversion.py b/python/pyspark/sql/tests/test_conversion.py index 2b18fe8d04d7a..7928a8e74d4f0 100644 --- a/python/pyspark/sql/tests/test_conversion.py +++ b/python/pyspark/sql/tests/test_conversion.py @@ -109,6 +109,67 @@ def test_conversion(self): with self.subTest(expected=e): self.assertEqual(a, e) + def test_binary_as_bytes_conversion(self): + """Test binary type conversion with binary_as_bytes parameter""" + data = [ + ( + str(i).encode(), # simple binary + [str(j).encode() for j in range(3)], # array of binary + {str(j): str(j).encode() for j in range(2)}, # map with binary values + {"b": str(i).encode()}, # struct with binary + ) + for i in range(2) + ] + schema = ( + StructType() + .add("b", BinaryType()) + .add("arr_b", ArrayType(BinaryType())) + .add("map_b", MapType(StringType(), BinaryType())) + .add("struct_b", StructType().add("b", BinaryType())) + ) + + tbl = LocalDataToArrowConversion.convert(data, schema, use_large_var_types=False) + + # Test binary_as_bytes=True (default) - should return bytes + actual_bytes = ArrowTableToRowsConversion.convert(tbl, schema, binary_as_bytes=True) + + for row in actual_bytes: + # Simple binary field should be bytes + self.assertIsInstance(row.b, bytes) + # Array elements should be bytes + for elem in row.arr_b: + self.assertIsInstance(elem, bytes) + # Map values should be bytes + for value in row.map_b.values(): + self.assertIsInstance(value, bytes) + # Struct field should be bytes + self.assertIsInstance(row.struct_b.b, bytes) + + # Test binary_as_bytes=False - should return bytearray + actual_bytearray = ArrowTableToRowsConversion.convert(tbl, schema, binary_as_bytes=False) + + for row in actual_bytearray: + # Simple binary field should be bytearray + self.assertIsInstance(row.b, bytearray) + # Array elements should be bytearray + for elem in row.arr_b: + self.assertIsInstance(elem, bytearray) + # Map values should be bytearray + for value in row.map_b.values(): + self.assertIsInstance(value, bytearray) + # Struct field should be bytearray + self.assertIsInstance(row.struct_b.b, bytearray) + + # Verify the actual content is the same, just different types + for bytes_row, bytearray_row in zip(actual_bytes, actual_bytearray): + self.assertEqual(bytes_row.b, bytes(bytearray_row.b)) + self.assertEqual([bytes(ba) for ba in bytearray_row.arr_b], bytes_row.arr_b) + self.assertEqual( + {k: bytes(v) for k, v in bytearray_row.map_b.items()}, + bytes_row.map_b + ) + self.assertEqual(bytes(bytearray_row.struct_b.b), bytes_row.struct_b.b) + if __name__ == "__main__": from pyspark.sql.tests.test_conversion import * # noqa: F401 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index d904d7a5b8de5..2a5f0e62853ab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -3724,7 +3724,7 @@ object SQLConf { .booleanConf .createWithDefault(false) - val ARROW_PYSPARK_BINARY_TYPE_AS_BYTES = + val ARROW_PYSPARK_BINARY_AS_BYTES = buildConf("spark.sql.execution.arrow.pyspark.binaryAsBytes") .doc("To be added.") .version("4.1.0") @@ -7109,6 +7109,8 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def arrowPySparkSelfDestructEnabled: Boolean = getConf(ARROW_PYSPARK_SELF_DESTRUCT_ENABLED) + def arrowPySparkBinaryAsBytes: Boolean = getConf(ARROW_PYSPARK_BINARY_AS_BYTES) + def pysparkJVMStacktraceEnabled: Boolean = getConf(PYSPARK_JVM_STACKTRACE_ENABLED) def pythonUDFProfiler: Option[String] = getConf(PYTHON_UDF_PROFILER) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala index 9f75682b6ddd0..f9a6b8389f104 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala @@ -156,8 +156,8 @@ object ArrowPythonRunner { SQLConf.PYTHON_UDF_PANDAS_INT_TO_DECIMAL_COERCION_ENABLED.key -> conf.getConf(SQLConf.PYTHON_UDF_PANDAS_INT_TO_DECIMAL_COERCION_ENABLED, false).toString) val binaryAsBytes = Seq( - SQLConf.ARROW_PYSPARK_BINARY_TYPE_AS_BYTES.key -> - conf.getConf(SQLConf.ARROW_PYSPARK_BINARY_TYPE_AS_BYTES).toString) + SQLConf.ARROW_PYSPARK_BINARY_AS_BYTES.key -> + conf.arrowPySparkBinaryAsBytes.toString) Map(timeZoneConf ++ pandasColsByName ++ arrowSafeTypeCheck ++ arrowAyncParallelism ++ useLargeVarTypes ++ intToDecimalCoercion ++ binaryAsBytes ++ From 6787bfc13cfd397029afbe23399ef459957f2eba Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Mon, 29 Sep 2025 08:00:37 +0900 Subject: [PATCH 05/26] Update python/pyspark/sql/tests/arrow/test_arrow_binary_as_bytes_udf.py --- .../pyspark/sql/tests/arrow/test_arrow_binary_as_bytes_udf.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/tests/arrow/test_arrow_binary_as_bytes_udf.py b/python/pyspark/sql/tests/arrow/test_arrow_binary_as_bytes_udf.py index b18364aacc3e6..d71e13284b9ce 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_binary_as_bytes_udf.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_binary_as_bytes_udf.py @@ -142,4 +142,5 @@ def check_struct_binary_type(s): testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2) except ImportError: testRunner = None - unittest.main(testRunner=testRunner, verbosity=2) \ No newline at end of file + unittest.main(testRunner=testRunner, verbosity=2) + \ No newline at end of file From 453298d7e2438d372e20516ce5b744d55a4cf5d7 Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Fri, 26 Sep 2025 14:52:41 +0000 Subject: [PATCH 06/26] add tests for spark connect --- .../tests/connect/test_connect_collection.py | 81 +++++++++++++++++++ 1 file changed, 81 insertions(+) diff --git a/python/pyspark/sql/tests/connect/test_connect_collection.py b/python/pyspark/sql/tests/connect/test_connect_collection.py index 61932c38733b7..e7b5ae8e82d47 100644 --- a/python/pyspark/sql/tests/connect/test_connect_collection.py +++ b/python/pyspark/sql/tests/connect/test_connect_collection.py @@ -291,6 +291,87 @@ def test_collect_nested_type(self): ).collect(), ) + def test_collect_binary_type(self): + """Test that df.collect() respects binary_as_bytes configuration for server-side data""" + # Use SQL to create data with binary type on the server side + # This ensures the data goes through Arrow conversion from server to client + query = """ + SELECT * FROM VALUES + (CAST('hello' AS BINARY)), + (CAST('world' AS BINARY)), + (CAST('test' AS BINARY)) + AS tab(b) + """ + + # Test with binary_as_bytes=True (default) + self.connect.conf.set("spark.sql.execution.arrow.pyspark.binaryAsBytes", "true") + crows = self.connect.sql(query).collect() + for row in crows: + self.assertIsInstance(row.b, bytes) + + # Test with binary_as_bytes=False + self.connect.conf.set("spark.sql.execution.arrow.pyspark.binaryAsBytes", "false") + crows = self.connect.sql(query).collect() + + # Connect should return bytearray when configured + for row in crows: + self.assertIsInstance(row.b, bytearray) + + def test_to_local_iterator_binary_type(self): + """Test that df.toLocalIterator() respects binary_as_bytes configuration""" + # Use server-side query that creates binary data + query = """ + SELECT * FROM VALUES + (CAST('data1' AS BINARY)), + (CAST('data2' AS BINARY)) + AS tab(b) + """ + + # Test with binary_as_bytes=True + self.connect.conf.set("spark.sql.execution.arrow.pyspark.binaryAsBytes", "true") + for row in self.connect.sql(query).toLocalIterator(): + self.assertIsInstance(row.b, bytes) + + # Test with binary_as_bytes=False + self.connect.conf.set("spark.sql.execution.arrow.pyspark.binaryAsBytes", "false") + for row in self.connect.sql(query).toLocalIterator(): + self.assertIsInstance(row.b, bytearray) + + def test_foreach_partition_binary_type(self): + """Test that df.foreachPartition() respects binary_as_bytes configuration""" + # Use server-side query that creates binary data + query = """ + SELECT * FROM VALUES + (CAST('partition1' AS BINARY)), + (CAST('partition2' AS BINARY)) + AS tab(b) + """ + + # Collect data types in each partition + collected_types = [] + + def collect_binary_types(iterator): + for row in iterator: + collected_types.append(type(row.b).__name__) + + # Test with binary_as_bytes=True + self.connect.conf.set("spark.sql.execution.arrow.pyspark.binaryAsBytes", "true") + collected_types.clear() + self.connect.sql(query).foreachPartition(collect_binary_types) + + # All should be bytes + for type_name in collected_types: + self.assertEqual(type_name, "bytes") + + # Test with binary_as_bytes=False + self.connect.conf.set("spark.sql.execution.arrow.pyspark.binaryAsBytes", "false") + collected_types.clear() + self.connect.sql(query).foreachPartition(collect_binary_types) + + # All should be bytearray + for type_name in collected_types: + self.assertEqual(type_name, "bytearray") + if __name__ == "__main__": from pyspark.sql.tests.connect.test_connect_collection import * # noqa: F401 From c859ce75f6b7ae824dc8b74957b7b2505d888c88 Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Mon, 29 Sep 2025 09:31:50 +0000 Subject: [PATCH 07/26] resolve with remote --- python/pyspark/sql/tests/arrow/test_arrow_binary_as_bytes_udf.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/pyspark/sql/tests/arrow/test_arrow_binary_as_bytes_udf.py b/python/pyspark/sql/tests/arrow/test_arrow_binary_as_bytes_udf.py index d71e13284b9ce..3639af1844bdc 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_binary_as_bytes_udf.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_binary_as_bytes_udf.py @@ -143,4 +143,3 @@ def check_struct_binary_type(s): except ImportError: testRunner = None unittest.main(testRunner=testRunner, verbosity=2) - \ No newline at end of file From 09bfef3fdb188fe8efd2f77f96c2b60a81ff7e04 Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Fri, 26 Sep 2025 15:22:15 +0000 Subject: [PATCH 08/26] lint --- python/pyspark/sql/connect/dataframe.py | 20 +++++-- python/pyspark/sql/conversion.py | 14 ++++- python/pyspark/sql/pandas/serializers.py | 4 +- .../arrow/test_arrow_binary_as_bytes_udf.py | 60 ++++++++++++------- .../sql/tests/connect/client/test_client.py | 10 ++++ .../tests/connect/test_connect_collection.py | 54 ++++++++--------- python/pyspark/sql/tests/test_conversion.py | 5 +- .../sql/worker/plan_data_source_read.py | 4 +- .../sql/worker/write_into_data_source.py | 5 +- python/pyspark/worker.py | 10 +++- 10 files changed, 122 insertions(+), 64 deletions(-) diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index 9f1e1c9352eae..d1bdfc1aa974e 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -1823,11 +1823,18 @@ def collect(self) -> List[Row]: assert schema is not None and isinstance(schema, StructType) - return ArrowTableToRowsConversion.convert(table, schema, binary_as_bytes=self._get_binary_as_bytes()) + return ArrowTableToRowsConversion.convert( + table, schema, binary_as_bytes=self._get_binary_as_bytes() + ) def _get_binary_as_bytes(self) -> bool: """Get the binary_as_bytes configuration value from Spark session.""" - return self._session.conf.get("spark.sql.execution.arrow.pyspark.binaryAsBytes", "true").lower() == "true" + return ( + self._session.conf.get( + "spark.sql.execution.arrow.pyspark.binaryAsBytes", "true" + ).lower() + == "true" + ) def _to_table(self) -> Tuple["pa.Table", Optional[StructType]]: query = self._plan.to_proto(self._session.client) @@ -2079,7 +2086,9 @@ def toLocalIterator(self, prefetchPartitions: bool = False) -> Iterator[Row]: table = schema_or_table if schema is None: schema = from_arrow_schema(table.schema, prefer_timestamp_ntz=True) - yield from ArrowTableToRowsConversion.convert(table, schema, binary_as_bytes=self._get_binary_as_bytes()) + yield from ArrowTableToRowsConversion.convert( + table, schema, binary_as_bytes=self._get_binary_as_bytes() + ) def pandas_api( self, index_col: Optional[Union[str, List[str]]] = None @@ -2167,7 +2176,10 @@ def foreachPartition(self, f: Callable[[Iterator[Row]], None]) -> None: schema = self._schema binary_as_bytes = self._get_binary_as_bytes() field_converters = [ - ArrowTableToRowsConversion._create_converter(f.dataType, binary_as_bytes=binary_as_bytes) for f in schema.fields + ArrowTableToRowsConversion._create_converter( + f.dataType, binary_as_bytes=binary_as_bytes + ) + for f in schema.fields ] def foreach_partition_func(itr: Iterable[pa.RecordBatch]) -> Iterable[pa.RecordBatch]: diff --git a/python/pyspark/sql/conversion.py b/python/pyspark/sql/conversion.py index 613323e2b4614..068964b5ea586 100644 --- a/python/pyspark/sql/conversion.py +++ b/python/pyspark/sql/conversion.py @@ -542,7 +542,9 @@ def _create_converter( dedup_field_names = _dedup_names(field_names) field_convs = [ - ArrowTableToRowsConversion._create_converter(f.dataType, none_on_identity=True, binary_as_bytes=binary_as_bytes) + ArrowTableToRowsConversion._create_converter( + f.dataType, none_on_identity=True, binary_as_bytes=binary_as_bytes + ) for f in dataType.fields ] @@ -735,7 +737,11 @@ def convert( @staticmethod # type: ignore[misc] def convert( - table: "pa.Table", schema: StructType, *, return_as_tuples: bool = False, binary_as_bytes: bool = True + table: "pa.Table", + schema: StructType, + *, + return_as_tuples: bool = False, + binary_as_bytes: bool = True, ) -> List[Union[Row, tuple]]: require_minimum_pyarrow_version() import pyarrow as pa @@ -748,7 +754,9 @@ def convert( if len(fields) > 0: field_converters = [ - ArrowTableToRowsConversion._create_converter(f.dataType, none_on_identity=True, binary_as_bytes=binary_as_bytes) + ArrowTableToRowsConversion._create_converter( + f.dataType, none_on_identity=True, binary_as_bytes=binary_as_bytes + ) for f in schema.fields ] diff --git a/python/pyspark/sql/pandas/serializers.py b/python/pyspark/sql/pandas/serializers.py index 56c410ccb5831..9030cd89776d7 100644 --- a/python/pyspark/sql/pandas/serializers.py +++ b/python/pyspark/sql/pandas/serializers.py @@ -891,7 +891,9 @@ def load_stream(self, stream): List of columns containing list of Python values. """ converters = [ - ArrowTableToRowsConversion._create_converter(dt, none_on_identity=True, binary_as_bytes=self._binary_as_bytes) + ArrowTableToRowsConversion._create_converter( + dt, none_on_identity=True, binary_as_bytes=self._binary_as_bytes + ) for dt in self._input_types ] diff --git a/python/pyspark/sql/tests/arrow/test_arrow_binary_as_bytes_udf.py b/python/pyspark/sql/tests/arrow/test_arrow_binary_as_bytes_udf.py index 3639af1844bdc..aa0f9a661aa53 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_binary_as_bytes_udf.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_binary_as_bytes_udf.py @@ -48,10 +48,9 @@ def get_binary_type(x): binary_udf = udf(get_binary_type, returnType="string", useArrow=True) - df = self.spark.createDataFrame([ - Row(b=b"hello"), - Row(b=b"world") - ], schema=StructType([StructField("b", BinaryType())])) + df = self.spark.createDataFrame( + [Row(b=b"hello"), Row(b=b"world")], schema=StructType([StructField("b", BinaryType())]) + ) with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): result = df.select(binary_udf(col("b")).alias("type_name")).collect() @@ -65,15 +64,16 @@ def get_binary_type(x): def test_arrow_batched_udf_array_binary_type(self): """Test SQL_ARROW_BATCHED_UDF with array of binary""" + def check_array_binary_types(arr): return [type(x).__name__ for x in arr] array_binary_udf = udf(check_array_binary_types, returnType="array", useArrow=True) - df = self.spark.createDataFrame([ - Row(arr_b=[b"a", b"b"]), - Row(arr_b=[b"c", b"d"]) - ], schema=StructType([StructField("arr_b", ArrayType(BinaryType()))])) + df = self.spark.createDataFrame( + [Row(arr_b=[b"a", b"b"]), Row(arr_b=[b"c", b"d"])], + schema=StructType([StructField("arr_b", ArrayType(BinaryType()))]), + ) with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): result = df.select(array_binary_udf(col("arr_b")).alias("types")).collect() @@ -91,10 +91,10 @@ def check_map_binary_types(m): map_binary_udf = udf(check_map_binary_types, returnType="array", useArrow=True) - df = self.spark.createDataFrame([ - Row(map_b={"k1": b"v1", "k2": b"v2"}), - Row(map_b={"k3": b"v3"}) - ], schema=StructType([StructField("map_b", MapType(StringType(), BinaryType()))])) + df = self.spark.createDataFrame( + [Row(map_b={"k1": b"v1", "k2": b"v2"}), Row(map_b={"k3": b"v3"})], + schema=StructType([StructField("map_b", MapType(StringType(), BinaryType()))]), + ) with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): result = df.select(map_binary_udf(col("map_b")).alias("types")).collect() @@ -112,15 +112,14 @@ def check_struct_binary_type(s): struct_binary_udf = udf(check_struct_binary_type, returnType="string", useArrow=True) - struct_schema = StructType([ - StructField("i", IntegerType()), - StructField("b", BinaryType()) - ]) + struct_schema = StructType( + [StructField("i", IntegerType()), StructField("b", BinaryType())] + ) - df = self.spark.createDataFrame([ - Row(struct_b=Row(i=1, b=b"data1")), - Row(struct_b=Row(i=2, b=b"data2")) - ], schema=StructType([StructField("struct_b", struct_schema)])) + df = self.spark.createDataFrame( + [Row(struct_b=Row(i=1, b=b"data1")), Row(struct_b=Row(i=2, b=b"data2"))], + schema=StructType([StructField("struct_b", struct_schema)]), + ) with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): result = df.select(struct_binary_udf(col("struct_b")).alias("type_name")).collect() @@ -132,6 +131,27 @@ def check_struct_binary_type(s): self.assertEqual(result[0]["type_name"], "bytearray") self.assertEqual(result[1]["type_name"], "bytearray") + def test_arrow_table_udf_binary_type(self): + """Test SQL_ARROW_TABLE_UDF with binary type""" + from pyspark.sql.functions import udtf, lit + + @udtf(returnType="b: binary", useArrow=True) + class BinaryTableUDF: + def eval(self, b): + # Return the binary data for testing + yield (b,) + + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): + # Call UDTF directly with binary literal + result = BinaryTableUDF(lit(b"test_bytes")).collect() + for row in result: + self.assertIsInstance(row.b, bytes) + + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): + result = BinaryTableUDF(lit(b"test_bytearray")).collect() + for row in result: + self.assertIsInstance(row.b, bytearray) + if __name__ == "__main__": from pyspark.sql.tests.arrow.test_arrow_binary_as_bytes_udf import * # noqa: F401 diff --git a/python/pyspark/sql/tests/connect/client/test_client.py b/python/pyspark/sql/tests/connect/client/test_client.py index b0a7f3f381274..cf10026b50157 100644 --- a/python/pyspark/sql/tests/connect/client/test_client.py +++ b/python/pyspark/sql/tests/connect/client/test_client.py @@ -163,6 +163,16 @@ def Interrupt(self, req: proto.InterruptRequest, metadata): resp.session_id = self._session_id return resp + def Config(self, req: proto.ConfigRequest, metadata): + resp = proto.ConfigResponse() + resp.session_id = self._session_id + # Return default value for spark.sql.execution.arrow.pyspark.binaryAsBytes + if req.operation.HasField("get"): + pair = resp.pairs.add() + pair.key = req.operation.get.keys[0] + pair.value = "true" # Default value + return resp + # The _cleanup_ml_cache invocation will hang in this test (no valid spark cluster) # and it blocks the test process exiting because it is registered as the atexit handler # in `SparkConnectClient` constructor. To bypass the issue, patch the method in the test. diff --git a/python/pyspark/sql/tests/connect/test_connect_collection.py b/python/pyspark/sql/tests/connect/test_connect_collection.py index e7b5ae8e82d47..83f334895cd4b 100644 --- a/python/pyspark/sql/tests/connect/test_connect_collection.py +++ b/python/pyspark/sql/tests/connect/test_connect_collection.py @@ -304,18 +304,18 @@ def test_collect_binary_type(self): """ # Test with binary_as_bytes=True (default) - self.connect.conf.set("spark.sql.execution.arrow.pyspark.binaryAsBytes", "true") - crows = self.connect.sql(query).collect() - for row in crows: - self.assertIsInstance(row.b, bytes) + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): + crows = self.connect.sql(query).collect() + for row in crows: + self.assertIsInstance(row.b, bytes) # Test with binary_as_bytes=False - self.connect.conf.set("spark.sql.execution.arrow.pyspark.binaryAsBytes", "false") - crows = self.connect.sql(query).collect() + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): + crows = self.connect.sql(query).collect() - # Connect should return bytearray when configured - for row in crows: - self.assertIsInstance(row.b, bytearray) + # Connect should return bytearray when configured + for row in crows: + self.assertIsInstance(row.b, bytearray) def test_to_local_iterator_binary_type(self): """Test that df.toLocalIterator() respects binary_as_bytes configuration""" @@ -328,14 +328,14 @@ def test_to_local_iterator_binary_type(self): """ # Test with binary_as_bytes=True - self.connect.conf.set("spark.sql.execution.arrow.pyspark.binaryAsBytes", "true") - for row in self.connect.sql(query).toLocalIterator(): - self.assertIsInstance(row.b, bytes) + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): + for row in self.connect.sql(query).toLocalIterator(): + self.assertIsInstance(row.b, bytes) # Test with binary_as_bytes=False - self.connect.conf.set("spark.sql.execution.arrow.pyspark.binaryAsBytes", "false") - for row in self.connect.sql(query).toLocalIterator(): - self.assertIsInstance(row.b, bytearray) + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): + for row in self.connect.sql(query).toLocalIterator(): + self.assertIsInstance(row.b, bytearray) def test_foreach_partition_binary_type(self): """Test that df.foreachPartition() respects binary_as_bytes configuration""" @@ -355,22 +355,22 @@ def collect_binary_types(iterator): collected_types.append(type(row.b).__name__) # Test with binary_as_bytes=True - self.connect.conf.set("spark.sql.execution.arrow.pyspark.binaryAsBytes", "true") - collected_types.clear() - self.connect.sql(query).foreachPartition(collect_binary_types) + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): + collected_types.clear() + self.connect.sql(query).foreachPartition(collect_binary_types) - # All should be bytes - for type_name in collected_types: - self.assertEqual(type_name, "bytes") + # All should be bytes + for type_name in collected_types: + self.assertEqual(type_name, "bytes") # Test with binary_as_bytes=False - self.connect.conf.set("spark.sql.execution.arrow.pyspark.binaryAsBytes", "false") - collected_types.clear() - self.connect.sql(query).foreachPartition(collect_binary_types) + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): + collected_types.clear() + self.connect.sql(query).foreachPartition(collect_binary_types) - # All should be bytearray - for type_name in collected_types: - self.assertEqual(type_name, "bytearray") + # All should be bytearray + for type_name in collected_types: + self.assertEqual(type_name, "bytearray") if __name__ == "__main__": diff --git a/python/pyspark/sql/tests/test_conversion.py b/python/pyspark/sql/tests/test_conversion.py index 7928a8e74d4f0..7caec6f2c9545 100644 --- a/python/pyspark/sql/tests/test_conversion.py +++ b/python/pyspark/sql/tests/test_conversion.py @@ -164,10 +164,7 @@ def test_binary_as_bytes_conversion(self): for bytes_row, bytearray_row in zip(actual_bytes, actual_bytearray): self.assertEqual(bytes_row.b, bytes(bytearray_row.b)) self.assertEqual([bytes(ba) for ba in bytearray_row.arr_b], bytes_row.arr_b) - self.assertEqual( - {k: bytes(v) for k, v in bytearray_row.map_b.items()}, - bytes_row.map_b - ) + self.assertEqual({k: bytes(v) for k, v in bytearray_row.map_b.items()}, bytes_row.map_b) self.assertEqual(bytes(bytearray_row.struct_b.b), bytes_row.struct_b.b) diff --git a/python/pyspark/sql/worker/plan_data_source_read.py b/python/pyspark/sql/worker/plan_data_source_read.py index 15b0e610cdaf3..f341fdf7ce2cf 100644 --- a/python/pyspark/sql/worker/plan_data_source_read.py +++ b/python/pyspark/sql/worker/plan_data_source_read.py @@ -180,7 +180,9 @@ def write_read_func_and_partitions( is_streaming = isinstance(reader, DataSourceStreamReader) # Create input converter. - converter = ArrowTableToRowsConversion._create_converter(BinaryType(), binary_as_bytes=binary_as_bytes) + converter = ArrowTableToRowsConversion._create_converter( + BinaryType(), binary_as_bytes=binary_as_bytes + ) # Create output converter. return_type = schema diff --git a/python/pyspark/sql/worker/write_into_data_source.py b/python/pyspark/sql/worker/write_into_data_source.py index a009aa1b591d1..3d27f2c779520 100644 --- a/python/pyspark/sql/worker/write_into_data_source.py +++ b/python/pyspark/sql/worker/write_into_data_source.py @@ -206,7 +206,10 @@ def main(infile: IO, outfile: IO) -> None: import pyarrow as pa converters = [ - ArrowTableToRowsConversion._create_converter(f.dataType, binary_as_bytes=binary_as_bytes) for f in schema.fields + ArrowTableToRowsConversion._create_converter( + f.dataType, binary_as_bytes=binary_as_bytes + ) + for f in schema.fields ] fields = schema.fieldNames() diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 03410f032e08a..efcf2683bf385 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -1306,7 +1306,8 @@ def read_udtf(pickleSer, infile, eval_type): == "true" ) binary_as_bytes = ( - runner_conf.get("spark.sql.execution.arrow.pyspark.binaryAsBytes", "true").lower() == "true" + runner_conf.get("spark.sql.execution.arrow.pyspark.binaryAsBytes", "true").lower() + == "true" ) input_types = [ field.dataType for field in _parse_datatype_json_string(utf8_deserializer.loads(infile)) @@ -2252,7 +2253,9 @@ def evaluate(*args: list, num_rows=1): def mapper(_, it): try: converters = [ - ArrowTableToRowsConversion._create_converter(dt, none_on_identity=True, binary_as_bytes=binary_as_bytes) + ArrowTableToRowsConversion._create_converter( + dt, none_on_identity=True, binary_as_bytes=binary_as_bytes + ) for dt in input_types ] for a in it: @@ -2550,7 +2553,8 @@ def read_udfs(pickleSer, infile, eval_type): == "true" ) binary_as_bytes = ( - runner_conf.get("spark.sql.execution.arrow.pyspark.binaryAsBytes", "true").lower() == "true" + runner_conf.get("spark.sql.execution.arrow.pyspark.binaryAsBytes", "true").lower() + == "true" ) _assign_cols_by_name = assign_cols_by_name(runner_conf) From 238e2b7399e446b6dfb1f059b58bb6c8fc7b8aab Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Mon, 29 Sep 2025 09:30:04 +0000 Subject: [PATCH 09/26] fix ci --- python/pyspark/sql/tests/connect/client/test_client.py | 6 +++++- .../udf_type_tests/golden_udf_input_types_arrow_enabled.txt | 4 ++-- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/python/pyspark/sql/tests/connect/client/test_client.py b/python/pyspark/sql/tests/connect/client/test_client.py index cf10026b50157..c189f996cbe43 100644 --- a/python/pyspark/sql/tests/connect/client/test_client.py +++ b/python/pyspark/sql/tests/connect/client/test_client.py @@ -164,13 +164,17 @@ def Interrupt(self, req: proto.InterruptRequest, metadata): return resp def Config(self, req: proto.ConfigRequest, metadata): + self.req = req resp = proto.ConfigResponse() resp.session_id = self._session_id - # Return default value for spark.sql.execution.arrow.pyspark.binaryAsBytes if req.operation.HasField("get"): pair = resp.pairs.add() pair.key = req.operation.get.keys[0] pair.value = "true" # Default value + elif req.operation.HasField("get_with_default"): + pair = resp.pairs.add() + pair.key = req.operation.get_with_default.pairs[0].key + pair.value = req.operation.get_with_default.pairs[0].value or "true" return resp # The _cleanup_ml_cache invocation will hang in this test (no valid spark cluster) diff --git a/python/pyspark/sql/tests/udf_type_tests/golden_udf_input_types_arrow_enabled.txt b/python/pyspark/sql/tests/udf_type_tests/golden_udf_input_types_arrow_enabled.txt index 2572d48dbec7c..e0e8b1ea88cf8 100644 --- a/python/pyspark/sql/tests/udf_type_tests/golden_udf_input_types_arrow_enabled.txt +++ b/python/pyspark/sql/tests/udf_type_tests/golden_udf_input_types_arrow_enabled.txt @@ -17,8 +17,8 @@ |decimal_null |decimal(3,2) |[None, Decimal('9.99')] |['NoneType', 'Decimal'] |['None', '9.99'] | |string_values |string |['abc', '', 'hello'] |['str', 'str', 'str'] |['abc', '', 'hello'] | |string_null |string |[None, 'test'] |['NoneType', 'str'] |['None', 'test'] | -|binary_values |binary |[bytearray(b'abc'), bytearray(b''), bytearray(b'ABC')] |['bytearray', 'bytearray', 'bytearray'] |["bytearray(b'abc')", "bytearray(b'')", "bytearray(b'ABC')"] | -|binary_null |binary |[None, bytearray(b'test')] |['NoneType', 'bytearray'] |['None', "bytearray(b'test')"] | +|binary_values |binary |[bytearray(b'abc'), bytearray(b''), bytearray(b'ABC')] |['bytes', 'bytes', 'bytes'] |["b'abc'", "b''", "b'ABC'"] | +|binary_null |binary |[None, bytearray(b'test')] |['NoneType', 'bytes'] |['None', "b'test'"] | |boolean_values |boolean |[True, False] |['bool', 'bool'] |['True', 'False'] | |boolean_null |boolean |[None, True] |['NoneType', 'bool'] |['None', 'True'] | |date_values |date |[datetime.date(2020, 2, 2), datetime.date(1970, 1, 1)] |['date', 'date'] |['2020-02-02', '1970-01-01'] | From 8db164ce7ea81490529f6c0fc16c386b1253d803 Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Mon, 29 Sep 2025 09:52:30 +0000 Subject: [PATCH 10/26] add docs --- docs/sql-ref-datatypes.md | 2 +- python/docs/source/tutorial/sql/type_conversions.rst | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/sql-ref-datatypes.md b/docs/sql-ref-datatypes.md index 25f847508ce01..49df4f807e762 100644 --- a/docs/sql-ref-datatypes.md +++ b/docs/sql-ref-datatypes.md @@ -131,7 +131,7 @@ from pyspark.sql.types import * |**StringType**|str|StringType()| |**CharType(length)**|str|CharType(length)| |**VarcharType(length)**|str|VarcharType(length)| -|**BinaryType**|bytearray|BinaryType()| +|**BinaryType**|bytearray
**Note:** When Arrow is enabled (`spark.sql.execution.arrow.pyspark.enabled=true`), BinaryType maps to `bytes` instead of `bytearray`.|BinaryType()| |**BooleanType**|bool|BooleanType()| |**TimestampType**|datetime.datetime|TimestampType()| |**TimestampNTZType**|datetime.datetime|TimestampNTZType()| diff --git a/python/docs/source/tutorial/sql/type_conversions.rst b/python/docs/source/tutorial/sql/type_conversions.rst index 2f13701995ef2..14b0ae51120ec 100644 --- a/python/docs/source/tutorial/sql/type_conversions.rst +++ b/python/docs/source/tutorial/sql/type_conversions.rst @@ -58,6 +58,9 @@ are listed below: * - spark.sql.execution.pythonUDF.arrow.enabled - Enable PyArrow in PySpark. See more `here `_. - False + * - spark.sql.execution.arrow.pyspark.enabled + - Enable PyArrow for PySpark operations. When enabled, affects binary type conversions (BinaryType maps to ``bytes`` instead of ``bytearray``). + - False * - spark.sql.pyspark.inferNestedDictAsStruct.enabled - When enabled, nested dictionaries are inferred as StructType. Otherwise, they are inferred as MapType. - False @@ -106,6 +109,7 @@ All Conversions - StringType() * - **BinaryType** - bytearray + .. note:: When Arrow is enabled (``spark.sql.execution.arrow.pyspark.enabled=true``), BinaryType maps to ``bytes`` instead of ``bytearray``. - BinaryType() * - **BooleanType** - bool From b10fc9d23ec526e4ff3b3853b594c5ebf10eef27 Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Mon, 29 Sep 2025 12:10:20 +0000 Subject: [PATCH 11/26] add tests for arrow python udtf --- dev/sparktestsupport/modules.py | 1 + .../arrow/test_arrow_binary_as_bytes_udf.py | 165 ------------------ .../sql/tests/arrow/test_arrow_python_udf.py | 101 ++++++++++- .../sql/tests/arrow/test_arrow_python_udtf.py | 57 ++++++ 4 files changed, 157 insertions(+), 167 deletions(-) delete mode 100644 python/pyspark/sql/tests/arrow/test_arrow_binary_as_bytes_udf.py create mode 100644 python/pyspark/sql/tests/arrow/test_arrow_python_udtf.py diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index a8bbf6c0eef63..fe3838d6b175a 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -560,6 +560,7 @@ def __hash__(self): "pyspark.sql.tests.arrow.test_arrow_udf_window", "pyspark.sql.tests.arrow.test_arrow_udf_typehints", "pyspark.sql.tests.arrow.test_arrow_udtf", + "pyspark.sql.tests.arrow.test_arrow_python_udtf", "pyspark.sql.tests.pandas.test_pandas_cogrouped_map", "pyspark.sql.tests.pandas.test_pandas_grouped_map", "pyspark.sql.tests.pandas.test_pandas_grouped_map_with_state", diff --git a/python/pyspark/sql/tests/arrow/test_arrow_binary_as_bytes_udf.py b/python/pyspark/sql/tests/arrow/test_arrow_binary_as_bytes_udf.py deleted file mode 100644 index aa0f9a661aa53..0000000000000 --- a/python/pyspark/sql/tests/arrow/test_arrow_binary_as_bytes_udf.py +++ /dev/null @@ -1,165 +0,0 @@ -# -# 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. -# - -import unittest - -from pyspark.sql import Row -from pyspark.sql.functions import udf, col -from pyspark.sql.types import ( - ArrayType, - BinaryType, - IntegerType, - MapType, - StringType, - StructType, - StructField, -) -from pyspark.testing.sqlutils import ( - have_pandas, - have_pyarrow, - pandas_requirement_message, - pyarrow_requirement_message, - ReusedSQLTestCase, -) -from pyspark.util import PythonEvalType - - -@unittest.skipIf( - not have_pandas or not have_pyarrow, pandas_requirement_message or pyarrow_requirement_message -) -class BinaryAsBytesUDFTests(ReusedSQLTestCase): - def test_arrow_batched_udf_binary_type(self): - def get_binary_type(x): - return type(x).__name__ - - binary_udf = udf(get_binary_type, returnType="string", useArrow=True) - - df = self.spark.createDataFrame( - [Row(b=b"hello"), Row(b=b"world")], schema=StructType([StructField("b", BinaryType())]) - ) - - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): - result = df.select(binary_udf(col("b")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytes") - self.assertEqual(result[1]["type_name"], "bytes") - - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): - result = df.select(binary_udf(col("b")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytearray") - self.assertEqual(result[1]["type_name"], "bytearray") - - def test_arrow_batched_udf_array_binary_type(self): - """Test SQL_ARROW_BATCHED_UDF with array of binary""" - - def check_array_binary_types(arr): - return [type(x).__name__ for x in arr] - - array_binary_udf = udf(check_array_binary_types, returnType="array", useArrow=True) - - df = self.spark.createDataFrame( - [Row(arr_b=[b"a", b"b"]), Row(arr_b=[b"c", b"d"])], - schema=StructType([StructField("arr_b", ArrayType(BinaryType()))]), - ) - - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): - result = df.select(array_binary_udf(col("arr_b")).alias("types")).collect() - self.assertEqual(result[0]["types"], ["bytes", "bytes"]) - self.assertEqual(result[1]["types"], ["bytes", "bytes"]) - - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): - result = df.select(array_binary_udf(col("arr_b")).alias("types")).collect() - self.assertEqual(result[0]["types"], ["bytearray", "bytearray"]) - self.assertEqual(result[1]["types"], ["bytearray", "bytearray"]) - - def test_arrow_batched_udf_map_binary_type(self): - def check_map_binary_types(m): - return [type(v).__name__ for v in m.values()] - - map_binary_udf = udf(check_map_binary_types, returnType="array", useArrow=True) - - df = self.spark.createDataFrame( - [Row(map_b={"k1": b"v1", "k2": b"v2"}), Row(map_b={"k3": b"v3"})], - schema=StructType([StructField("map_b", MapType(StringType(), BinaryType()))]), - ) - - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): - result = df.select(map_binary_udf(col("map_b")).alias("types")).collect() - self.assertEqual(set(result[0]["types"]), {"bytes"}) - self.assertEqual(result[1]["types"], ["bytes"]) - - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): - result = df.select(map_binary_udf(col("map_b")).alias("types")).collect() - self.assertEqual(set(result[0]["types"]), {"bytearray"}) - self.assertEqual(result[1]["types"], ["bytearray"]) - - def test_arrow_batched_udf_struct_binary_type(self): - def check_struct_binary_type(s): - return type(s.b).__name__ - - struct_binary_udf = udf(check_struct_binary_type, returnType="string", useArrow=True) - - struct_schema = StructType( - [StructField("i", IntegerType()), StructField("b", BinaryType())] - ) - - df = self.spark.createDataFrame( - [Row(struct_b=Row(i=1, b=b"data1")), Row(struct_b=Row(i=2, b=b"data2"))], - schema=StructType([StructField("struct_b", struct_schema)]), - ) - - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): - result = df.select(struct_binary_udf(col("struct_b")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytes") - self.assertEqual(result[1]["type_name"], "bytes") - - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): - result = df.select(struct_binary_udf(col("struct_b")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytearray") - self.assertEqual(result[1]["type_name"], "bytearray") - - def test_arrow_table_udf_binary_type(self): - """Test SQL_ARROW_TABLE_UDF with binary type""" - from pyspark.sql.functions import udtf, lit - - @udtf(returnType="b: binary", useArrow=True) - class BinaryTableUDF: - def eval(self, b): - # Return the binary data for testing - yield (b,) - - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): - # Call UDTF directly with binary literal - result = BinaryTableUDF(lit(b"test_bytes")).collect() - for row in result: - self.assertIsInstance(row.b, bytes) - - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): - result = BinaryTableUDF(lit(b"test_bytearray")).collect() - for row in result: - self.assertIsInstance(row.b, bytearray) - - -if __name__ == "__main__": - from pyspark.sql.tests.arrow.test_arrow_binary_as_bytes_udf import * # noqa: F401 - - try: - import xmlrunner # type: ignore - - testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2) - except ImportError: - testRunner = None - unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py b/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py index a054261304c6f..b71f1ee7a9c21 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py @@ -19,9 +19,19 @@ from pyspark.errors import AnalysisException, PythonException, PySparkNotImplementedError from pyspark.sql import Row -from pyspark.sql.functions import udf +from pyspark.sql.functions import udf, col from pyspark.sql.tests.test_udf import BaseUDFTestsMixin -from pyspark.sql.types import DayTimeIntervalType, VarcharType, StructType, StructField, StringType +from pyspark.sql.types import ( + ArrayType, + BinaryType, + DayTimeIntervalType, + IntegerType, + MapType, + StringType, + StructField, + StructType, + VarcharType, +) from pyspark.testing.sqlutils import ( have_pandas, have_pyarrow, @@ -422,6 +432,93 @@ def tearDownClass(cls): finally: super(ArrowPythonUDFNonLegacyTests, cls).tearDownClass() + def test_arrow_batched_udf_binary_type(self): + def get_binary_type(x): + return type(x).__name__ + + binary_udf = udf(get_binary_type, returnType="string", useArrow=True) + + df = self.spark.createDataFrame( + [Row(b=b"hello"), Row(b=b"world")], schema=StructType([StructField("b", BinaryType())]) + ) + + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): + result = df.select(binary_udf(col("b")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytes") + self.assertEqual(result[1]["type_name"], "bytes") + + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): + result = df.select(binary_udf(col("b")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytearray") + self.assertEqual(result[1]["type_name"], "bytearray") + + def test_arrow_batched_udf_array_binary_type(self): + def check_array_binary_types(arr): + return [type(x).__name__ for x in arr] + + array_binary_udf = udf(check_array_binary_types, returnType="array", useArrow=True) + + df = self.spark.createDataFrame( + [Row(arr_b=[b"a", b"b"]), Row(arr_b=[b"c", b"d"])], + schema=StructType([StructField("arr_b", ArrayType(BinaryType()))]), + ) + + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): + result = df.select(array_binary_udf(col("arr_b")).alias("types")).collect() + self.assertEqual(result[0]["types"], ["bytes", "bytes"]) + self.assertEqual(result[1]["types"], ["bytes", "bytes"]) + + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): + result = df.select(array_binary_udf(col("arr_b")).alias("types")).collect() + self.assertEqual(result[0]["types"], ["bytearray", "bytearray"]) + self.assertEqual(result[1]["types"], ["bytearray", "bytearray"]) + + def test_arrow_batched_udf_map_binary_type(self): + def check_map_binary_types(m): + return [type(v).__name__ for v in m.values()] + + map_binary_udf = udf(check_map_binary_types, returnType="array", useArrow=True) + + df = self.spark.createDataFrame( + [Row(map_b={"k1": b"v1", "k2": b"v2"}), Row(map_b={"k3": b"v3"})], + schema=StructType([StructField("map_b", MapType(StringType(), BinaryType()))]), + ) + + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): + result = df.select(map_binary_udf(col("map_b")).alias("types")).collect() + self.assertEqual(set(result[0]["types"]), {"bytes"}) + self.assertEqual(result[1]["types"], ["bytes"]) + + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): + result = df.select(map_binary_udf(col("map_b")).alias("types")).collect() + self.assertEqual(set(result[0]["types"]), {"bytearray"}) + self.assertEqual(result[1]["types"], ["bytearray"]) + + def test_arrow_batched_udf_struct_binary_type(self): + def check_struct_binary_type(s): + return type(s.b).__name__ + + struct_binary_udf = udf(check_struct_binary_type, returnType="string", useArrow=True) + + struct_schema = StructType( + [StructField("i", IntegerType()), StructField("b", BinaryType())] + ) + + df = self.spark.createDataFrame( + [Row(struct_b=Row(i=1, b=b"data1")), Row(struct_b=Row(i=2, b=b"data2"))], + schema=StructType([StructField("struct_b", struct_schema)]), + ) + + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): + result = df.select(struct_binary_udf(col("struct_b")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytes") + self.assertEqual(result[1]["type_name"], "bytes") + + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): + result = df.select(struct_binary_udf(col("struct_b")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytearray") + self.assertEqual(result[1]["type_name"], "bytearray") + if __name__ == "__main__": from pyspark.sql.tests.arrow.test_arrow_python_udf import * # noqa: F401 diff --git a/python/pyspark/sql/tests/arrow/test_arrow_python_udtf.py b/python/pyspark/sql/tests/arrow/test_arrow_python_udtf.py new file mode 100644 index 0000000000000..a5548a682c43c --- /dev/null +++ b/python/pyspark/sql/tests/arrow/test_arrow_python_udtf.py @@ -0,0 +1,57 @@ +# +# 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. +# + +import unittest + +from pyspark.sql.types import BinaryType +from pyspark.testing.sqlutils import ( + have_pyarrow, + pyarrow_requirement_message, + ReusedSQLTestCase, +) + + +@unittest.skipIf(not have_pyarrow, pyarrow_requirement_message) +class ArrowPythonUDTFTests(ReusedSQLTestCase): + def test_arrow_table_udf_binary_type(self): + from pyspark.sql.functions import udtf, lit + + @udtf(returnType="type_name: string", useArrow=True) + class BinaryTypeUDF: + def eval(self, b): + # Check the type of the binary input and return type name as string + yield (type(b).__name__,) + + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): + result = BinaryTypeUDF(lit(b"test_bytes")).collect() + self.assertEqual(result[0]["type_name"], "bytes") + + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): + result = BinaryTypeUDF(lit(b"test_bytearray")).collect() + self.assertEqual(result[0]["type_name"], "bytearray") + + +if __name__ == "__main__": + from pyspark.sql.tests.arrow.test_arrow_python_udtf import * # noqa: F401 + + try: + import xmlrunner # type: ignore + + testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2) + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) From 7d3cf56ca1cb8d3a1a6f2514806a79ffa59dc56f Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Mon, 29 Sep 2025 13:41:29 +0000 Subject: [PATCH 12/26] add a shield --- .../sql/tests/arrow/test_arrow_python_udtf.py | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/python/pyspark/sql/tests/arrow/test_arrow_python_udtf.py b/python/pyspark/sql/tests/arrow/test_arrow_python_udtf.py index a5548a682c43c..d3ef0a0f2c90b 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_python_udtf.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_python_udtf.py @@ -36,13 +36,14 @@ def eval(self, b): # Check the type of the binary input and return type name as string yield (type(b).__name__,) - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): - result = BinaryTypeUDF(lit(b"test_bytes")).collect() - self.assertEqual(result[0]["type_name"], "bytes") - - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): - result = BinaryTypeUDF(lit(b"test_bytearray")).collect() - self.assertEqual(result[0]["type_name"], "bytearray") + with self.sql_conf({"spark.sql.legacy.execution.pythonUDF.pandas.conversion.enabled": "false"}): + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): + result = BinaryTypeUDF(lit(b"test_bytes")).collect() + self.assertEqual(result[0]["type_name"], "bytes") + + with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): + result = BinaryTypeUDF(lit(b"test_bytearray")).collect() + self.assertEqual(result[0]["type_name"], "bytearray") if __name__ == "__main__": From 922c1a4c2ae14a1ad9164206f4157fbcc1e4f1c1 Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Tue, 30 Sep 2025 11:32:53 +0000 Subject: [PATCH 13/26] rename conf' also make classic dataframe API work with the conf --- dev/sparktestsupport/modules.py | 1 - python/pyspark/sql/connect/dataframe.py | 2 +- .../sql/tests/arrow/test_arrow_python_udf.py | 16 +-- .../sql/tests/arrow/test_arrow_python_udtf.py | 58 ---------- .../tests/connect/test_connect_collection.py | 100 ++++++++++++------ python/pyspark/sql/tests/test_udtf.py | 17 +++ python/pyspark/worker.py | 6 +- .../apache/spark/sql/internal/SQLConf.scala | 9 +- .../python/UserDefinedPythonDataSource.scala | 6 +- .../execution/python/ArrowPythonRunner.scala | 4 +- .../sql/execution/python/EvaluatePython.scala | 47 ++++++++ 11 files changed, 155 insertions(+), 111 deletions(-) delete mode 100644 python/pyspark/sql/tests/arrow/test_arrow_python_udtf.py diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index fe3838d6b175a..a8bbf6c0eef63 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -560,7 +560,6 @@ def __hash__(self): "pyspark.sql.tests.arrow.test_arrow_udf_window", "pyspark.sql.tests.arrow.test_arrow_udf_typehints", "pyspark.sql.tests.arrow.test_arrow_udtf", - "pyspark.sql.tests.arrow.test_arrow_python_udtf", "pyspark.sql.tests.pandas.test_pandas_cogrouped_map", "pyspark.sql.tests.pandas.test_pandas_grouped_map", "pyspark.sql.tests.pandas.test_pandas_grouped_map_with_state", diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index d1bdfc1aa974e..c6eabca1d9441 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -1831,7 +1831,7 @@ def _get_binary_as_bytes(self) -> bool: """Get the binary_as_bytes configuration value from Spark session.""" return ( self._session.conf.get( - "spark.sql.execution.arrow.pyspark.binaryAsBytes", "true" + "spark.sql.execution.pyspark.binaryAsBytes", "true" ).lower() == "true" ) diff --git a/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py b/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py index b71f1ee7a9c21..39c2f4592ae4a 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py @@ -442,12 +442,12 @@ def get_binary_type(x): [Row(b=b"hello"), Row(b=b"world")], schema=StructType([StructField("b", BinaryType())]) ) - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): result = df.select(binary_udf(col("b")).alias("type_name")).collect() self.assertEqual(result[0]["type_name"], "bytes") self.assertEqual(result[1]["type_name"], "bytes") - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): result = df.select(binary_udf(col("b")).alias("type_name")).collect() self.assertEqual(result[0]["type_name"], "bytearray") self.assertEqual(result[1]["type_name"], "bytearray") @@ -463,12 +463,12 @@ def check_array_binary_types(arr): schema=StructType([StructField("arr_b", ArrayType(BinaryType()))]), ) - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): result = df.select(array_binary_udf(col("arr_b")).alias("types")).collect() self.assertEqual(result[0]["types"], ["bytes", "bytes"]) self.assertEqual(result[1]["types"], ["bytes", "bytes"]) - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): result = df.select(array_binary_udf(col("arr_b")).alias("types")).collect() self.assertEqual(result[0]["types"], ["bytearray", "bytearray"]) self.assertEqual(result[1]["types"], ["bytearray", "bytearray"]) @@ -484,12 +484,12 @@ def check_map_binary_types(m): schema=StructType([StructField("map_b", MapType(StringType(), BinaryType()))]), ) - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): result = df.select(map_binary_udf(col("map_b")).alias("types")).collect() self.assertEqual(set(result[0]["types"]), {"bytes"}) self.assertEqual(result[1]["types"], ["bytes"]) - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): result = df.select(map_binary_udf(col("map_b")).alias("types")).collect() self.assertEqual(set(result[0]["types"]), {"bytearray"}) self.assertEqual(result[1]["types"], ["bytearray"]) @@ -509,12 +509,12 @@ def check_struct_binary_type(s): schema=StructType([StructField("struct_b", struct_schema)]), ) - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): result = df.select(struct_binary_udf(col("struct_b")).alias("type_name")).collect() self.assertEqual(result[0]["type_name"], "bytes") self.assertEqual(result[1]["type_name"], "bytes") - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): result = df.select(struct_binary_udf(col("struct_b")).alias("type_name")).collect() self.assertEqual(result[0]["type_name"], "bytearray") self.assertEqual(result[1]["type_name"], "bytearray") diff --git a/python/pyspark/sql/tests/arrow/test_arrow_python_udtf.py b/python/pyspark/sql/tests/arrow/test_arrow_python_udtf.py deleted file mode 100644 index d3ef0a0f2c90b..0000000000000 --- a/python/pyspark/sql/tests/arrow/test_arrow_python_udtf.py +++ /dev/null @@ -1,58 +0,0 @@ -# -# 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. -# - -import unittest - -from pyspark.sql.types import BinaryType -from pyspark.testing.sqlutils import ( - have_pyarrow, - pyarrow_requirement_message, - ReusedSQLTestCase, -) - - -@unittest.skipIf(not have_pyarrow, pyarrow_requirement_message) -class ArrowPythonUDTFTests(ReusedSQLTestCase): - def test_arrow_table_udf_binary_type(self): - from pyspark.sql.functions import udtf, lit - - @udtf(returnType="type_name: string", useArrow=True) - class BinaryTypeUDF: - def eval(self, b): - # Check the type of the binary input and return type name as string - yield (type(b).__name__,) - - with self.sql_conf({"spark.sql.legacy.execution.pythonUDF.pandas.conversion.enabled": "false"}): - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): - result = BinaryTypeUDF(lit(b"test_bytes")).collect() - self.assertEqual(result[0]["type_name"], "bytes") - - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): - result = BinaryTypeUDF(lit(b"test_bytearray")).collect() - self.assertEqual(result[0]["type_name"], "bytearray") - - -if __name__ == "__main__": - from pyspark.sql.tests.arrow.test_arrow_python_udtf import * # noqa: F401 - - try: - import xmlrunner # type: ignore - - testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2) - except ImportError: - testRunner = None - unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/sql/tests/connect/test_connect_collection.py b/python/pyspark/sql/tests/connect/test_connect_collection.py index 83f334895cd4b..d4731a97adbb1 100644 --- a/python/pyspark/sql/tests/connect/test_connect_collection.py +++ b/python/pyspark/sql/tests/connect/test_connect_collection.py @@ -16,6 +16,7 @@ # import unittest +from contextlib import contextmanager, ExitStack from pyspark.testing.connectutils import should_test_connect, ReusedMixedTestCase from pyspark.testing.pandasutils import PandasOnSparkTestUtils @@ -25,6 +26,38 @@ class SparkConnectCollectionTests(ReusedMixedTestCase, PandasOnSparkTestUtils): + def connect_conf(self, conf_dict): + """Context manager to set configuration on Spark Connect session""" + + @contextmanager + def _connect_conf(): + old_values = {} + for key, value in conf_dict.items(): + old_values[key] = self.connect.conf.get(key, None) + self.connect.conf.set(key, value) + try: + yield + finally: + for key, old_value in old_values.items(): + if old_value is None: + self.connect.conf.unset(key) + else: + self.connect.conf.set(key, old_value) + + return _connect_conf() + + def both_conf(self, conf_dict): + """Context manager to set configuration on both classic and Connect sessions""" + + @contextmanager + def _both_conf(): + with ExitStack() as stack: + stack.enter_context(self.sql_conf(conf_dict)) + stack.enter_context(self.connect_conf(conf_dict)) + yield + + return _both_conf() + def test_collect(self): query = "SELECT id, CAST(id AS STRING) AS name FROM RANGE(100)" cdf = self.connect.sql(query) @@ -304,17 +337,17 @@ def test_collect_binary_type(self): """ # Test with binary_as_bytes=True (default) - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): - crows = self.connect.sql(query).collect() - for row in crows: + with self.both_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): + for row in self.connect.sql(query).collect(): + self.assertIsInstance(row.b, bytes) + for row in self.spark.sql(query).collect(): self.assertIsInstance(row.b, bytes) # Test with binary_as_bytes=False - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): - crows = self.connect.sql(query).collect() - - # Connect should return bytearray when configured - for row in crows: + with self.both_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): + for row in self.connect.sql(query).collect(): + self.assertIsInstance(row.b, bytearray) + for row in self.spark.sql(query).collect(): self.assertIsInstance(row.b, bytearray) def test_to_local_iterator_binary_type(self): @@ -328,17 +361,25 @@ def test_to_local_iterator_binary_type(self): """ # Test with binary_as_bytes=True - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): + with self.both_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): for row in self.connect.sql(query).toLocalIterator(): self.assertIsInstance(row.b, bytes) + for row in self.spark.sql(query).toLocalIterator(): + self.assertIsInstance(row.b, bytes) # Test with binary_as_bytes=False - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): + with self.both_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): for row in self.connect.sql(query).toLocalIterator(): self.assertIsInstance(row.b, bytearray) + for row in self.spark.sql(query).toLocalIterator(): + self.assertIsInstance(row.b, bytearray) def test_foreach_partition_binary_type(self): - """Test that df.foreachPartition() respects binary_as_bytes configuration""" + """Test that df.foreachPartition() respects binary_as_bytes configuration + + Since foreachPartition() runs on executors and cannot return data to the driver, + we test by ensuring the function doesn't throw exceptions when it expects the correct types. + """ # Use server-side query that creates binary data query = """ SELECT * FROM VALUES @@ -347,30 +388,29 @@ def test_foreach_partition_binary_type(self): AS tab(b) """ - # Collect data types in each partition - collected_types = [] + # Test with binary_as_bytes=True - should get bytes objects + with self.both_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - def collect_binary_types(iterator): - for row in iterator: - collected_types.append(type(row.b).__name__) + def assert_bytes_type(iterator): + for row in iterator: + # This will raise an exception if the type is not bytes + assert isinstance(row.b, bytes), f"Expected bytes, got {type(row.b).__name__}" - # Test with binary_as_bytes=True - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "true"}): - collected_types.clear() - self.connect.sql(query).foreachPartition(collect_binary_types) + self.connect.sql(query).foreachPartition(assert_bytes_type) + self.spark.sql(query).foreachPartition(assert_bytes_type) - # All should be bytes - for type_name in collected_types: - self.assertEqual(type_name, "bytes") + # Test with binary_as_bytes=False - should get bytearray objects + with self.both_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - # Test with binary_as_bytes=False - with self.sql_conf({"spark.sql.execution.arrow.pyspark.binaryAsBytes": "false"}): - collected_types.clear() - self.connect.sql(query).foreachPartition(collect_binary_types) + def assert_bytearray_type(iterator): + for row in iterator: + # This will raise an exception if the type is not bytearray + assert isinstance( + row.b, bytearray + ), f"Expected bytearray, got {type(row.b).__name__}" - # All should be bytearray - for type_name in collected_types: - self.assertEqual(type_name, "bytearray") + self.connect.sql(query).foreachPartition(assert_bytearray_type) + # self.spark.sql(query).foreachPartition(assert_bytearray_type) if __name__ == "__main__": diff --git a/python/pyspark/sql/tests/test_udtf.py b/python/pyspark/sql/tests/test_udtf.py index b006ac6c14d4a..5504a1f55f79a 100644 --- a/python/pyspark/sql/tests/test_udtf.py +++ b/python/pyspark/sql/tests/test_udtf.py @@ -3628,6 +3628,23 @@ def tearDownClass(cls): finally: super(UDTFArrowTests, cls).tearDownClass() + def test_arrow_table_udf_binary_type(self): + from pyspark.sql.functions import udtf, lit + + @udtf(returnType="type_name: string", useArrow=True) + class BinaryTypeUDF: + def eval(self, b): + # Check the type of the binary input and return type name as string + yield (type(b).__name__,) + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): + result = BinaryTypeUDF(lit(b"test_bytes")).collect() + self.assertEqual(result[0]["type_name"], "bytes") + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): + result = BinaryTypeUDF(lit(b"test_bytearray")).collect() + self.assertEqual(result[0]["type_name"], "bytearray") + if __name__ == "__main__": from pyspark.sql.tests.test_udtf import * # noqa: F401 diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index efcf2683bf385..de00696b41149 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -1306,8 +1306,7 @@ def read_udtf(pickleSer, infile, eval_type): == "true" ) binary_as_bytes = ( - runner_conf.get("spark.sql.execution.arrow.pyspark.binaryAsBytes", "true").lower() - == "true" + runner_conf.get("spark.sql.execution.pyspark.binaryAsBytes", "true").lower() == "true" ) input_types = [ field.dataType for field in _parse_datatype_json_string(utf8_deserializer.loads(infile)) @@ -2553,8 +2552,7 @@ def read_udfs(pickleSer, infile, eval_type): == "true" ) binary_as_bytes = ( - runner_conf.get("spark.sql.execution.arrow.pyspark.binaryAsBytes", "true").lower() - == "true" + runner_conf.get("spark.sql.execution.pyspark.binaryAsBytes", "true").lower() == "true" ) _assign_cols_by_name = assign_cols_by_name(runner_conf) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 2a5f0e62853ab..3384c13acd9d9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -3724,9 +3724,10 @@ object SQLConf { .booleanConf .createWithDefault(false) - val ARROW_PYSPARK_BINARY_AS_BYTES = - buildConf("spark.sql.execution.arrow.pyspark.binaryAsBytes") - .doc("To be added.") + val PYSPARK_BINARY_AS_BYTES = + buildConf("spark.sql.execution.pyspark.binaryAsBytes") + .doc("When true, BINARY type columns are returned as bytes in PySpark. " + + "When false, restores pre 4.1 behavior.") .version("4.1.0") .booleanConf .createWithDefault(true) @@ -7109,7 +7110,7 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def arrowPySparkSelfDestructEnabled: Boolean = getConf(ARROW_PYSPARK_SELF_DESTRUCT_ENABLED) - def arrowPySparkBinaryAsBytes: Boolean = getConf(ARROW_PYSPARK_BINARY_AS_BYTES) + def pysparkBinaryAsBytes: Boolean = getConf(PYSPARK_BINARY_AS_BYTES) def pysparkJVMStacktraceEnabled: Boolean = getConf(PYSPARK_JVM_STACKTRACE_ENABLED) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/UserDefinedPythonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/UserDefinedPythonDataSource.scala index f41caaa5d687b..26bd5368e6f9c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/UserDefinedPythonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/UserDefinedPythonDataSource.scala @@ -458,7 +458,7 @@ private class UserDefinedPythonDataSourceFilterPushdownRunner( // Send configurations dataOut.writeInt(SQLConf.get.arrowMaxRecordsPerBatch) - dataOut.writeBoolean(SQLConf.get.arrowPySparkBinaryAsBytes) + dataOut.writeBoolean(SQLConf.get.pysparkBinaryAsBytes) } override protected def receiveFromPython(dataIn: DataInputStream): PythonFilterPushdownResult = { @@ -551,7 +551,7 @@ private class UserDefinedPythonDataSourceReadRunner( dataOut.writeBoolean(SQLConf.get.pythonFilterPushDown) dataOut.writeBoolean(isStreaming) - dataOut.writeBoolean(SQLConf.get.arrowPySparkBinaryAsBytes) + dataOut.writeBoolean(SQLConf.get.pysparkBinaryAsBytes) } override protected def receiveFromPython(dataIn: DataInputStream): PythonDataSourceReadInfo = { @@ -602,7 +602,7 @@ private class UserDefinedPythonDataSourceWriteRunner( dataOut.writeBoolean(overwrite) dataOut.writeBoolean(isStreaming) - dataOut.writeBoolean(SQLConf.get.arrowPySparkBinaryAsBytes) + dataOut.writeBoolean(SQLConf.get.pysparkBinaryAsBytes) } override protected def receiveFromPython( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala index f9a6b8389f104..77aec2a35f21d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala @@ -156,8 +156,8 @@ object ArrowPythonRunner { SQLConf.PYTHON_UDF_PANDAS_INT_TO_DECIMAL_COERCION_ENABLED.key -> conf.getConf(SQLConf.PYTHON_UDF_PANDAS_INT_TO_DECIMAL_COERCION_ENABLED, false).toString) val binaryAsBytes = Seq( - SQLConf.ARROW_PYSPARK_BINARY_AS_BYTES.key -> - conf.arrowPySparkBinaryAsBytes.toString) + SQLConf.PYSPARK_BINARY_AS_BYTES.key -> + conf.pysparkBinaryAsBytes.toString) Map(timeZoneConf ++ pandasColsByName ++ arrowSafeTypeCheck ++ arrowAyncParallelism ++ useLargeVarTypes ++ intToDecimalCoercion ++ binaryAsBytes ++ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala index 5d117a67e6bee..2d78c4e11767d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala @@ -30,11 +30,18 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData, MapData} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{UTF8String, VariantVal} object EvaluatePython { + /** + * Wrapper class for byte arrays that should be pickled as Python bytes instead of bytearray. + * This is a marker class that tells the pickler to use bytes() constructor. + */ + private[python] class BytesWrapper(val data: Array[Byte]) + def needConversionInPython(dt: DataType): Boolean = dt match { case DateType | TimestampType | TimestampNTZType | VariantType | _: DayTimeIntervalType | _: TimeType => true @@ -81,6 +88,14 @@ object EvaluatePython { case (s: UTF8String, _: StringType) => s.toString + case (bytes: Array[Byte], BinaryType) => + // Check config to decide whether to wrap bytes for conversion to Python bytes + if (SQLConf.get.pysparkBinaryAsBytes) { + new BytesWrapper(bytes) + } else { + bytes + } + case (other, _) => other } @@ -248,6 +263,37 @@ object EvaluatePython { } } + /** + * Pickler for BytesWrapper that pickles byte arrays as Python bytes using bytes() builtin. + * Structure: bytes(bytearray_data) where bytearray_data is pickled by razorvine's + * default pickler. + */ + private class BytesWrapperPickler extends IObjectPickler { + + private val cls = classOf[BytesWrapper] + + def register(): Unit = { + Pickler.registerCustomPickler(cls, this) + } + + def pickle(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + // Pickle structure: bytes(bytearray_value) + // GLOBAL 'builtins' 'bytes' + out.write(Opcodes.GLOBAL) + out.write("builtins\nbytes\n".getBytes(StandardCharsets.UTF_8)) + + // Pickle the wrapped byte array data using razorvine's built-in pickler + val wrapper = obj.asInstanceOf[BytesWrapper] + pickler.save(wrapper.data) + + // TUPLE1 creates a 1-tuple: (bytearray_value,) + out.write(Opcodes.TUPLE1) + + // REDUCE calls bytes(bytearray_value) + out.write(Opcodes.REDUCE) + } + } + /** * Pickler for external row. */ @@ -299,6 +345,7 @@ object EvaluatePython { SerDeUtil.initialize() new StructTypePickler().register() new RowPickler().register() + new BytesWrapperPickler().register() registered = true } } From 16e84285457f9e5f60e330054996a1876da0981e Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Tue, 30 Sep 2025 12:06:09 +0000 Subject: [PATCH 14/26] add size --- .../tests/connect/test_connect_collection.py | 43 ++++++++++++++++--- 1 file changed, 37 insertions(+), 6 deletions(-) diff --git a/python/pyspark/sql/tests/connect/test_connect_collection.py b/python/pyspark/sql/tests/connect/test_connect_collection.py index d4731a97adbb1..0505ddd30c881 100644 --- a/python/pyspark/sql/tests/connect/test_connect_collection.py +++ b/python/pyspark/sql/tests/connect/test_connect_collection.py @@ -331,23 +331,32 @@ def test_collect_binary_type(self): query = """ SELECT * FROM VALUES (CAST('hello' AS BINARY)), - (CAST('world' AS BINARY)), - (CAST('test' AS BINARY)) + (CAST('world' AS BINARY)) AS tab(b) """ # Test with binary_as_bytes=True (default) with self.both_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - for row in self.connect.sql(query).collect(): + connect_rows = self.connect.sql(query).collect() + self.assertEqual(len(connect_rows), 2) + for row in connect_rows: self.assertIsInstance(row.b, bytes) - for row in self.spark.sql(query).collect(): + + spark_rows = self.spark.sql(query).collect() + self.assertEqual(len(spark_rows), 2) + for row in spark_rows: self.assertIsInstance(row.b, bytes) # Test with binary_as_bytes=False with self.both_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - for row in self.connect.sql(query).collect(): + connect_rows = self.connect.sql(query).collect() + self.assertEqual(len(connect_rows), 2) + for row in connect_rows: self.assertIsInstance(row.b, bytearray) - for row in self.spark.sql(query).collect(): + + spark_rows = self.spark.sql(query).collect() + self.assertEqual(len(spark_rows), 2) + for row in spark_rows: self.assertIsInstance(row.b, bytearray) def test_to_local_iterator_binary_type(self): @@ -362,17 +371,31 @@ def test_to_local_iterator_binary_type(self): # Test with binary_as_bytes=True with self.both_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): + connect_count = 0 for row in self.connect.sql(query).toLocalIterator(): self.assertIsInstance(row.b, bytes) + connect_count += 1 + self.assertEqual(connect_count, 2) + + spark_count = 0 for row in self.spark.sql(query).toLocalIterator(): self.assertIsInstance(row.b, bytes) + spark_count += 1 + self.assertEqual(spark_count, 2) # Test with binary_as_bytes=False with self.both_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): + connect_count = 0 for row in self.connect.sql(query).toLocalIterator(): self.assertIsInstance(row.b, bytearray) + connect_count += 1 + self.assertEqual(connect_count, 2) + + spark_count = 0 for row in self.spark.sql(query).toLocalIterator(): self.assertIsInstance(row.b, bytearray) + spark_count += 1 + self.assertEqual(spark_count, 2) def test_foreach_partition_binary_type(self): """Test that df.foreachPartition() respects binary_as_bytes configuration @@ -392,9 +415,13 @@ def test_foreach_partition_binary_type(self): with self.both_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): def assert_bytes_type(iterator): + count = 0 for row in iterator: # This will raise an exception if the type is not bytes assert isinstance(row.b, bytes), f"Expected bytes, got {type(row.b).__name__}" + count += 1 + # Ensure we actually processed rows + assert count > 0, "No rows were processed" self.connect.sql(query).foreachPartition(assert_bytes_type) self.spark.sql(query).foreachPartition(assert_bytes_type) @@ -403,11 +430,15 @@ def assert_bytes_type(iterator): with self.both_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): def assert_bytearray_type(iterator): + count = 0 for row in iterator: # This will raise an exception if the type is not bytearray assert isinstance( row.b, bytearray ), f"Expected bytearray, got {type(row.b).__name__}" + count += 1 + # Ensure we actually processed rows + assert count > 0, "No rows were processed" self.connect.sql(query).foreachPartition(assert_bytearray_type) # self.spark.sql(query).foreachPartition(assert_bytearray_type) From fd5fcb21a058b1e8c108a0e33ce757169e250b15 Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Tue, 30 Sep 2025 14:59:35 +0000 Subject: [PATCH 15/26] apply binary as bytes consistently in all cases --- docs/sql-ref-datatypes.md | 2 +- .../source/tutorial/sql/type_conversions.rst | 8 +- .../sql/tests/arrow/test_arrow_python_udf.py | 8 +- python/pyspark/sql/tests/test_udf.py | 88 +++++++++++++++++++ python/pyspark/sql/tests/test_udtf.py | 32 ++++--- .../golden_pandas_udf_input_types.txt | 4 +- ...golden_pandas_udf_return_type_coercion.txt | 2 +- .../golden_udf_input_types_arrow_disabled.txt | 4 +- .../golden_udf_input_types_arrow_enabled.txt | 4 +- ...en_udf_input_types_arrow_legacy_pandas.txt | 4 +- ...df_return_type_coercion_arrow_disabled.txt | 2 +- ...udf_return_type_coercion_arrow_enabled.txt | 2 +- ...turn_type_coercion_arrow_legacy_pandas.txt | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 5 +- 14 files changed, 126 insertions(+), 41 deletions(-) diff --git a/docs/sql-ref-datatypes.md b/docs/sql-ref-datatypes.md index 49df4f807e762..fd3e0b7a3164f 100644 --- a/docs/sql-ref-datatypes.md +++ b/docs/sql-ref-datatypes.md @@ -131,7 +131,7 @@ from pyspark.sql.types import * |**StringType**|str|StringType()| |**CharType(length)**|str|CharType(length)| |**VarcharType(length)**|str|VarcharType(length)| -|**BinaryType**|bytearray
**Note:** When Arrow is enabled (`spark.sql.execution.arrow.pyspark.enabled=true`), BinaryType maps to `bytes` instead of `bytearray`.|BinaryType()| +|**BinaryType**|bytes|BinaryType()| |**BooleanType**|bool|BooleanType()| |**TimestampType**|datetime.datetime|TimestampType()| |**TimestampNTZType**|datetime.datetime|TimestampNTZType()| diff --git a/python/docs/source/tutorial/sql/type_conversions.rst b/python/docs/source/tutorial/sql/type_conversions.rst index 14b0ae51120ec..a7bd0d16a6147 100644 --- a/python/docs/source/tutorial/sql/type_conversions.rst +++ b/python/docs/source/tutorial/sql/type_conversions.rst @@ -58,9 +58,6 @@ are listed below: * - spark.sql.execution.pythonUDF.arrow.enabled - Enable PyArrow in PySpark. See more `here `_. - False - * - spark.sql.execution.arrow.pyspark.enabled - - Enable PyArrow for PySpark operations. When enabled, affects binary type conversions (BinaryType maps to ``bytes`` instead of ``bytearray``). - - False * - spark.sql.pyspark.inferNestedDictAsStruct.enabled - When enabled, nested dictionaries are inferred as StructType. Otherwise, they are inferred as MapType. - False @@ -70,6 +67,8 @@ are listed below: * - spark.sql.execution.pandas.inferPandasDictAsMap - When enabled, Pandas dictionaries are inferred as MapType. Otherwise, they are inferred as StructType. - False + * - spark.sql.execution.pyspark.binaryAsBytes + - Introduced in Spark 4.1.0. When enabled, BinaryType is mapped consistently to Python bytes; when disabled, matches the PySpark default behavior before 4.1.0. All Conversions --------------- @@ -108,8 +107,7 @@ All Conversions - string - StringType() * - **BinaryType** - - bytearray - .. note:: When Arrow is enabled (``spark.sql.execution.arrow.pyspark.enabled=true``), BinaryType maps to ``bytes`` instead of ``bytearray``. + - bytes - BinaryType() * - **BooleanType** - bool diff --git a/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py b/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py index 39c2f4592ae4a..130135ae3ee1d 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py @@ -432,7 +432,7 @@ def tearDownClass(cls): finally: super(ArrowPythonUDFNonLegacyTests, cls).tearDownClass() - def test_arrow_batched_udf_binary_type(self): + def test_udf_binary_type(self): def get_binary_type(x): return type(x).__name__ @@ -452,7 +452,7 @@ def get_binary_type(x): self.assertEqual(result[0]["type_name"], "bytearray") self.assertEqual(result[1]["type_name"], "bytearray") - def test_arrow_batched_udf_array_binary_type(self): + def test_udf_array_binary_type(self): def check_array_binary_types(arr): return [type(x).__name__ for x in arr] @@ -473,7 +473,7 @@ def check_array_binary_types(arr): self.assertEqual(result[0]["types"], ["bytearray", "bytearray"]) self.assertEqual(result[1]["types"], ["bytearray", "bytearray"]) - def test_arrow_batched_udf_map_binary_type(self): + def test_udf_map_binary_type(self): def check_map_binary_types(m): return [type(v).__name__ for v in m.values()] @@ -494,7 +494,7 @@ def check_map_binary_types(m): self.assertEqual(set(result[0]["types"]), {"bytearray"}) self.assertEqual(result[1]["types"], ["bytearray"]) - def test_arrow_batched_udf_struct_binary_type(self): + def test_udf_struct_binary_type(self): def check_struct_binary_type(s): return type(s.b).__name__ diff --git a/python/pyspark/sql/tests/test_udf.py b/python/pyspark/sql/tests/test_udf.py index 9d130a7f525ac..7bad159b42180 100644 --- a/python/pyspark/sql/tests/test_udf.py +++ b/python/pyspark/sql/tests/test_udf.py @@ -32,6 +32,7 @@ from pyspark.sql.types import ( StringType, IntegerType, + BinaryType, BooleanType, DoubleType, LongType, @@ -1480,6 +1481,93 @@ def test_udf_with_various_buffer_size(self): rets = [x["ret"] for x in df.collect()] self.assertEqual(rets, list(range(4, 1004))) + def test_udf_binary_type(self): + def get_binary_type(x): + return type(x).__name__ + + binary_udf = udf(get_binary_type, returnType="string", useArrow=True) + + df = self.spark.createDataFrame( + [Row(b=b"hello"), Row(b=b"world")], schema=StructType([StructField("b", BinaryType())]) + ) + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): + result = df.select(binary_udf(col("b")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytes") + self.assertEqual(result[1]["type_name"], "bytes") + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): + result = df.select(binary_udf(col("b")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytearray") + self.assertEqual(result[1]["type_name"], "bytearray") + + def test_udf_array_binary_type(self): + def check_array_binary_types(arr): + return [type(x).__name__ for x in arr] + + array_binary_udf = udf(check_array_binary_types, returnType="array", useArrow=True) + + df = self.spark.createDataFrame( + [Row(arr_b=[b"a", b"b"]), Row(arr_b=[b"c", b"d"])], + schema=StructType([StructField("arr_b", ArrayType(BinaryType()))]), + ) + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): + result = df.select(array_binary_udf(col("arr_b")).alias("types")).collect() + self.assertEqual(result[0]["types"], ["bytes", "bytes"]) + self.assertEqual(result[1]["types"], ["bytes", "bytes"]) + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): + result = df.select(array_binary_udf(col("arr_b")).alias("types")).collect() + self.assertEqual(result[0]["types"], ["bytearray", "bytearray"]) + self.assertEqual(result[1]["types"], ["bytearray", "bytearray"]) + + def test_udf_map_binary_type(self): + def check_map_binary_types(m): + return [type(v).__name__ for v in m.values()] + + map_binary_udf = udf(check_map_binary_types, returnType="array", useArrow=True) + + df = self.spark.createDataFrame( + [Row(map_b={"k1": b"v1", "k2": b"v2"}), Row(map_b={"k3": b"v3"})], + schema=StructType([StructField("map_b", MapType(StringType(), BinaryType()))]), + ) + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): + result = df.select(map_binary_udf(col("map_b")).alias("types")).collect() + self.assertEqual(set(result[0]["types"]), {"bytes"}) + self.assertEqual(result[1]["types"], ["bytes"]) + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): + result = df.select(map_binary_udf(col("map_b")).alias("types")).collect() + self.assertEqual(set(result[0]["types"]), {"bytearray"}) + self.assertEqual(result[1]["types"], ["bytearray"]) + + def test_udf_struct_binary_type(self): + def check_struct_binary_type(s): + return type(s.b).__name__ + + struct_binary_udf = udf(check_struct_binary_type, returnType="string", useArrow=True) + + struct_schema = StructType( + [StructField("i", IntegerType()), StructField("b", BinaryType())] + ) + + df = self.spark.createDataFrame( + [Row(struct_b=Row(i=1, b=b"data1")), Row(struct_b=Row(i=2, b=b"data2"))], + schema=StructType([StructField("struct_b", struct_schema)]), + ) + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): + result = df.select(struct_binary_udf(col("struct_b")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytes") + self.assertEqual(result[1]["type_name"], "bytes") + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): + result = df.select(struct_binary_udf(col("struct_b")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytearray") + self.assertEqual(result[1]["type_name"], "bytearray") + class UDFInitializationTests(unittest.TestCase): def tearDown(self): diff --git a/python/pyspark/sql/tests/test_udtf.py b/python/pyspark/sql/tests/test_udtf.py index 5504a1f55f79a..3f04fa256bf0d 100644 --- a/python/pyspark/sql/tests/test_udtf.py +++ b/python/pyspark/sql/tests/test_udtf.py @@ -3044,6 +3044,21 @@ def eval(self, v1, v2, v3, v4): for idx, field in enumerate(result_df.schema.fields): self.assertEqual(field.dataType, expected_output_types[idx]) + def test_arrow_table_udf_binary_type(self): + @udtf(returnType="type_name: string") + class BinaryTypeUDF: + def eval(self, b): + # Check the type of the binary input and return type name as string + yield (type(b).__name__,) + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): + result = BinaryTypeUDF(lit(b"test_bytes")).collect() + self.assertEqual(result[0]["type_name"], "bytes") + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): + result = BinaryTypeUDF(lit(b"test_bytearray")).collect() + self.assertEqual(result[0]["type_name"], "bytearray") + class UDTFTests(BaseUDTFTestsMixin, ReusedSQLTestCase): @classmethod @@ -3628,23 +3643,6 @@ def tearDownClass(cls): finally: super(UDTFArrowTests, cls).tearDownClass() - def test_arrow_table_udf_binary_type(self): - from pyspark.sql.functions import udtf, lit - - @udtf(returnType="type_name: string", useArrow=True) - class BinaryTypeUDF: - def eval(self, b): - # Check the type of the binary input and return type name as string - yield (type(b).__name__,) - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = BinaryTypeUDF(lit(b"test_bytes")).collect() - self.assertEqual(result[0]["type_name"], "bytes") - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = BinaryTypeUDF(lit(b"test_bytearray")).collect() - self.assertEqual(result[0]["type_name"], "bytearray") - if __name__ == "__main__": from pyspark.sql.tests.test_udtf import * # noqa: F401 diff --git a/python/pyspark/sql/tests/udf_type_tests/golden_pandas_udf_input_types.txt b/python/pyspark/sql/tests/udf_type_tests/golden_pandas_udf_input_types.txt index d21e7f2eb24a1..2b87384586237 100644 --- a/python/pyspark/sql/tests/udf_type_tests/golden_pandas_udf_input_types.txt +++ b/python/pyspark/sql/tests/udf_type_tests/golden_pandas_udf_input_types.txt @@ -17,8 +17,8 @@ |decimal_null |decimal(3,2) |[None, Decimal('9.99')] |['object', 'object'] |[None, Decimal('9.99')] | |string_values |string |['abc', '', 'hello'] |['object', 'object', 'object'] |['abc', '', 'hello'] | |string_null |string |[None, 'test'] |['object', 'object'] |[None, 'test'] | -|binary_values |binary |[bytearray(b'abc'), bytearray(b''), bytearray(b'ABC')] |['object', 'object', 'object'] |[bytearray(b'abc'), bytearray(b''), bytearray(b'ABC')] | -|binary_null |binary |[None, bytearray(b'test')] |['object', 'object'] |[None, bytearray(b'test')] | +|binary_values |binary |[b'abc', b'', b'ABC'] |['object', 'object', 'object'] |[b'abc', b'', b'ABC'] | +|binary_null |binary |[None, b'test'] |['object', 'object'] |[None, b'test'] | |boolean_values |boolean |[True, False] |['bool', 'bool'] |[True, False] | |boolean_null |boolean |[None, True] |['object', 'object'] |[None, True] | |date_values |date |[datetime.date(2020, 2, 2), datetime.date(1970, 1, 1)] |['object', 'object'] |[datetime.date(2020, 2, 2), datetime.date(1970, 1, 1)] | diff --git a/python/pyspark/sql/tests/udf_type_tests/golden_pandas_udf_return_type_coercion.txt b/python/pyspark/sql/tests/udf_type_tests/golden_pandas_udf_return_type_coercion.txt index 7719d1805d9e9..052e930c2e1f2 100644 --- a/python/pyspark/sql/tests/udf_type_tests/golden_pandas_udf_return_type_coercion.txt +++ b/python/pyspark/sql/tests/udf_type_tests/golden_pandas_udf_return_type_coercion.txt @@ -12,7 +12,7 @@ |float |[None, None] |[1.0, 0.0] |[1.0, 2.0] |[1.0, 2.0] |[1.0, 2.0] |[1.0, 2.0] |[1.0, 2.0] |[1.0, 2.0] |[1.0, 2.0] |[1.0, 2.0] |[1.0, 2.0] |[1.0, 2.0] |[1.0, 2.0] |X |X |X |X |[12.0, 34.0] |X |[1.0, 2.0] |X |X |X |X |X |X | |double |[None, None] |[1.0, 0.0] |[1.0, 2.0] |[1.0, 2.0] |[1.0, 2.0] |[1.0, 2.0] |[1.0, 2.0] |[1.0, 2.0] |[1.0, 2.0] |[1.0, 2.0] |[1.0, 2.0] |[1.0, 2.0] |[1.0, 2.0] |X |X |X |X |[12.0, 34.0] |X |[1.0, 2.0] |X |X |X |X |X |X | |array |[None, None] |X |X |X |X |X |X |X |X |X |X |X |X |X |X |X |X |[[1, 2], [3, 4]] |[[1, 2, 3], [1, 2, 3]] |X |X |X |X |X |X |X | -|binary |[None, None] |[bytearray(b'\x01'), bytearray |[bytearray(b'\x01'), bytearray |[bytearray(b'\x01'), bytearray |[bytearray(b'\x01'), bytearray |[bytearray(b'\x01'), bytearray |[bytearray(b'\x01'), bytearray |[bytearray(b'\x01'), bytearray |[bytearray(b'\x01'), bytearray |[bytearray(b'\x01'), bytearray |[bytearray(b''), bytearray(b'' |[bytearray(b''), bytearray(b'' |[bytearray(b''), bytearray(b'' |[bytearray(b''), bytearray(b'' |[bytearray(b''), bytearray(b'' |[bytearray(b''), bytearray(b'' |[bytearray(b'a'), bytearray(b' |[bytearray(b'12'), bytearray(b |X |X |[bytearray(b''), bytearray(b'' |[bytearray(b''), bytearray(b'' |[bytearray(b''), bytearray(b'' |[bytearray(b'A'), bytearray(b' |X |X | +|binary |[None, None] |[b'\x01', b''] |[b'\x01', b'\x02'] |[b'\x01', b'\x02'] |[b'\x01', b'\x02'] |[b'\x01', b'\x02'] |[b'\x01', b'\x02'] |[b'\x01', b'\x02'] |[b'\x01', b'\x02'] |[b'\x01', b'\x02'] |[b'', b''] |[b'', b''] |[b'', b''] |[b'', b''] |[b'', b''] |[b'', b''] |[b'a', b'b'] |[b'12', b'34'] |X |X |[b'', b''] |[b'', b''] |[b'', b''] |[b'A', b'B'] |X |X | |decimal(10,0) |[None, None] |X |[Decimal('1'), Decimal('2')] |[Decimal('1'), Decimal('2')] |[Decimal('1'), Decimal('2')] |X |[Decimal('1'), Decimal('2')] |[Decimal('1'), Decimal('2')] |[Decimal('1'), Decimal('2')] |X |X |[Decimal('1'), Decimal('2')] |[Decimal('1'), Decimal('2')] |X |X |X |X |X |X |[Decimal('1'), Decimal('2')] |X |X |X |X |X |X | |map |[None, None] |X |X |X |X |X |X |X |X |X |X |X |X |X |X |X |X |X |X |X |X |X |X |X |X |[{'a': 1}, {'b': 2}] | |struct<_1:int> |X |X |X |X |X |X |X |X |X |X |X |X |X |X |X |X |X |X |X |X |X |X |X |X |[Row(_1=1), Row(_1=2)] |X | diff --git a/python/pyspark/sql/tests/udf_type_tests/golden_udf_input_types_arrow_disabled.txt b/python/pyspark/sql/tests/udf_type_tests/golden_udf_input_types_arrow_disabled.txt index 2572d48dbec7c..a3727dfd5d6b7 100644 --- a/python/pyspark/sql/tests/udf_type_tests/golden_udf_input_types_arrow_disabled.txt +++ b/python/pyspark/sql/tests/udf_type_tests/golden_udf_input_types_arrow_disabled.txt @@ -17,8 +17,8 @@ |decimal_null |decimal(3,2) |[None, Decimal('9.99')] |['NoneType', 'Decimal'] |['None', '9.99'] | |string_values |string |['abc', '', 'hello'] |['str', 'str', 'str'] |['abc', '', 'hello'] | |string_null |string |[None, 'test'] |['NoneType', 'str'] |['None', 'test'] | -|binary_values |binary |[bytearray(b'abc'), bytearray(b''), bytearray(b'ABC')] |['bytearray', 'bytearray', 'bytearray'] |["bytearray(b'abc')", "bytearray(b'')", "bytearray(b'ABC')"] | -|binary_null |binary |[None, bytearray(b'test')] |['NoneType', 'bytearray'] |['None', "bytearray(b'test')"] | +|binary_values |binary |[b'abc', b'', b'ABC'] |['bytes', 'bytes', 'bytes'] |["b'abc'", "b''", "b'ABC'"] | +|binary_null |binary |[None, b'test'] |['NoneType', 'bytes'] |['None', "b'test'"] | |boolean_values |boolean |[True, False] |['bool', 'bool'] |['True', 'False'] | |boolean_null |boolean |[None, True] |['NoneType', 'bool'] |['None', 'True'] | |date_values |date |[datetime.date(2020, 2, 2), datetime.date(1970, 1, 1)] |['date', 'date'] |['2020-02-02', '1970-01-01'] | diff --git a/python/pyspark/sql/tests/udf_type_tests/golden_udf_input_types_arrow_enabled.txt b/python/pyspark/sql/tests/udf_type_tests/golden_udf_input_types_arrow_enabled.txt index e0e8b1ea88cf8..a3727dfd5d6b7 100644 --- a/python/pyspark/sql/tests/udf_type_tests/golden_udf_input_types_arrow_enabled.txt +++ b/python/pyspark/sql/tests/udf_type_tests/golden_udf_input_types_arrow_enabled.txt @@ -17,8 +17,8 @@ |decimal_null |decimal(3,2) |[None, Decimal('9.99')] |['NoneType', 'Decimal'] |['None', '9.99'] | |string_values |string |['abc', '', 'hello'] |['str', 'str', 'str'] |['abc', '', 'hello'] | |string_null |string |[None, 'test'] |['NoneType', 'str'] |['None', 'test'] | -|binary_values |binary |[bytearray(b'abc'), bytearray(b''), bytearray(b'ABC')] |['bytes', 'bytes', 'bytes'] |["b'abc'", "b''", "b'ABC'"] | -|binary_null |binary |[None, bytearray(b'test')] |['NoneType', 'bytes'] |['None', "b'test'"] | +|binary_values |binary |[b'abc', b'', b'ABC'] |['bytes', 'bytes', 'bytes'] |["b'abc'", "b''", "b'ABC'"] | +|binary_null |binary |[None, b'test'] |['NoneType', 'bytes'] |['None', "b'test'"] | |boolean_values |boolean |[True, False] |['bool', 'bool'] |['True', 'False'] | |boolean_null |boolean |[None, True] |['NoneType', 'bool'] |['None', 'True'] | |date_values |date |[datetime.date(2020, 2, 2), datetime.date(1970, 1, 1)] |['date', 'date'] |['2020-02-02', '1970-01-01'] | diff --git a/python/pyspark/sql/tests/udf_type_tests/golden_udf_input_types_arrow_legacy_pandas.txt b/python/pyspark/sql/tests/udf_type_tests/golden_udf_input_types_arrow_legacy_pandas.txt index 92f8af100e743..576af5f12102d 100644 --- a/python/pyspark/sql/tests/udf_type_tests/golden_udf_input_types_arrow_legacy_pandas.txt +++ b/python/pyspark/sql/tests/udf_type_tests/golden_udf_input_types_arrow_legacy_pandas.txt @@ -17,8 +17,8 @@ |decimal_null |decimal(3,2) |[None, Decimal('9.99')] |['NoneType', 'Decimal'] |['None', '9.99'] | |string_values |string |['abc', '', 'hello'] |['str', 'str', 'str'] |['abc', '', 'hello'] | |string_null |string |[None, 'test'] |['NoneType', 'str'] |['None', 'test'] | -|binary_values |binary |[bytearray(b'abc'), bytearray(b''), bytearray(b'ABC')] |['bytes', 'bytes', 'bytes'] |["b'abc'", "b''", "b'ABC'"] | -|binary_null |binary |[None, bytearray(b'test')] |['NoneType', 'bytes'] |['None', "b'test'"] | +|binary_values |binary |[b'abc', b'', b'ABC'] |['bytes', 'bytes', 'bytes'] |["b'abc'", "b''", "b'ABC'"] | +|binary_null |binary |[None, b'test'] |['NoneType', 'bytes'] |['None', "b'test'"] | |boolean_values |boolean |[True, False] |['bool', 'bool'] |['True', 'False'] | |boolean_null |boolean |[None, True] |['NoneType', 'bool'] |['None', 'True'] | |date_values |date |[datetime.date(2020, 2, 2), datetime.date(1970, 1, 1)] |['date', 'date'] |['2020-02-02', '1970-01-01'] | diff --git a/python/pyspark/sql/tests/udf_type_tests/golden_udf_return_type_coercion_arrow_disabled.txt b/python/pyspark/sql/tests/udf_type_tests/golden_udf_return_type_coercion_arrow_disabled.txt index 7f87b89a5fcf6..46dd597d926f4 100644 --- a/python/pyspark/sql/tests/udf_type_tests/golden_udf_return_type_coercion_arrow_disabled.txt +++ b/python/pyspark/sql/tests/udf_type_tests/golden_udf_return_type_coercion_arrow_disabled.txt @@ -12,7 +12,7 @@ |float |None |None |None |None |None |None |1.0 |None |None |None |None |None |None |X |X | |double |None |None |None |None |None |None |1.0 |None |None |None |None |None |None |X |X | |array |None |None |None |None |None |None |None |[1] |[1] |[1] |[65, 66, 67] |None |None |X |X | -|binary |None |None |None |bytearray(b'a') |None |None |None |None |None |None |bytearray(b'ABC') |None |None |X |X | +|binary |None |None |None |b'a' |None |None |None |None |None |None |b'ABC' |None |None |X |X | |decimal(10,0) |None |None |None |None |None |None |None |None |None |None |None |Decimal('1') |None |X |X | |map |None |None |None |None |None |None |None |None |None |None |None |None |{'a': 1} |X |X | |struct<_1:int> |None |X |X |X |X |X |X |X |Row(_1=1) |Row(_1=1) |X |X |Row(_1=None) |Row(_1=1) |Row(_1=1) | diff --git a/python/pyspark/sql/tests/udf_type_tests/golden_udf_return_type_coercion_arrow_enabled.txt b/python/pyspark/sql/tests/udf_type_tests/golden_udf_return_type_coercion_arrow_enabled.txt index c117113369e56..f08f03d6d7d71 100644 --- a/python/pyspark/sql/tests/udf_type_tests/golden_udf_return_type_coercion_arrow_enabled.txt +++ b/python/pyspark/sql/tests/udf_type_tests/golden_udf_return_type_coercion_arrow_enabled.txt @@ -12,7 +12,7 @@ |float |None |1.0 |1.0 |X |X |X |1.0 |X |X |X |X |1.0 |X |X |X | |double |None |1.0 |1.0 |X |X |X |1.0 |X |X |X |X |1.0 |X |X |X | |array |None |X |X |X |X |X |X |[1] |[1] |[1] |[65, 66, 67] |X |X |[1] |[1] | -|binary |None |X |X |X |X |X |X |X |X |X |bytearray(b'ABC') |X |X |X |X | +|binary |None |X |X |X |X |X |X |X |X |X |b'ABC' |X |X |X |X | |decimal(10,0) |None |X |X |X |X |X |X |X |X |X |X |Decimal('1') |X |X |X | |map |None |X |X |X |X |X |X |X |X |X |X |X |{'a': 1} |X |X | |struct<_1:int> |None |X |X |X |X |X |X |X |X |Row(_1=1) |X |X |Row(_1=None) |Row(_1=1) |Row(_1=1) | diff --git a/python/pyspark/sql/tests/udf_type_tests/golden_udf_return_type_coercion_arrow_legacy_pandas.txt b/python/pyspark/sql/tests/udf_type_tests/golden_udf_return_type_coercion_arrow_legacy_pandas.txt index a1809dfa9aab6..8b68abb412e58 100644 --- a/python/pyspark/sql/tests/udf_type_tests/golden_udf_return_type_coercion_arrow_legacy_pandas.txt +++ b/python/pyspark/sql/tests/udf_type_tests/golden_udf_return_type_coercion_arrow_legacy_pandas.txt @@ -12,7 +12,7 @@ |float |None |1.0 |1.0 |X |X |X |1.0 |X |X |X |X |1.0 |X |X |X | |double |None |1.0 |1.0 |X |X |X |1.0 |X |X |X |X |1.0 |X |X |X | |array |None |X |X |X |X |X |X |[1] |[1] |[1] |[65, 66, 67] |X |X |[1] |[1] | -|binary |None |bytearray(b'\x00') |bytearray(b'\x00') |X |X |X |X |bytearray(b'\x01\x00\x00\x00') |bytearray(b'\x01') |bytearray(b'\x01') |bytearray(b'ABC') |X |X |bytearray(b'\x01') |bytearray(b'\x01') | +|binary |None |b'\x00' |b'\x00' |X |X |X |X |b'\x01\x00\x00\x00' |b'\x01' |b'\x01' |b'ABC' |X |X |b'\x01' |b'\x01' | |decimal(10,0) |None |X |X |X |X |X |Decimal('1') |X |X |X |X |Decimal('1') |X |X |X | |map |None |X |X |X |X |X |X |X |X |X |X |X |{'a': 1} |X |X | |struct<_1:int> |None |X |X |X |X |X |X |Row(_1=1) |Row(_1=1) |Row(_1=1) |Row(_1=65) |X |Row(_1=None) |Row(_1=1) |Row(_1=1) | diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 3384c13acd9d9..bc47f07db813f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -3726,8 +3726,9 @@ object SQLConf { val PYSPARK_BINARY_AS_BYTES = buildConf("spark.sql.execution.pyspark.binaryAsBytes") - .doc("When true, BINARY type columns are returned as bytes in PySpark. " + - "When false, restores pre 4.1 behavior.") + .doc("When true, BinaryType is mapped consistently to bytes in PySpark." + + "When false, matches the PySpark behavior before 4.1.0, where BinaryType is mapped to " + + "to bytes or bytearray in different cases.") .version("4.1.0") .booleanConf .createWithDefault(true) From fd687f225d059ba52dc4ea9df46f4c4f0596372e Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Tue, 30 Sep 2025 15:08:45 +0000 Subject: [PATCH 16/26] reformat --- python/pyspark/sql/connect/dataframe.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index c6eabca1d9441..aface86a8fd79 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -1830,9 +1830,7 @@ def collect(self) -> List[Row]: def _get_binary_as_bytes(self) -> bool: """Get the binary_as_bytes configuration value from Spark session.""" return ( - self._session.conf.get( - "spark.sql.execution.pyspark.binaryAsBytes", "true" - ).lower() + self._session.conf.get("spark.sql.execution.pyspark.binaryAsBytes", "true").lower() == "true" ) From ee5d2c239f1153b017a39e1147ffd4e53483bc75 Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Tue, 30 Sep 2025 15:55:05 +0000 Subject: [PATCH 17/26] remove usearrow --- python/pyspark/sql/tests/test_udf.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql/tests/test_udf.py b/python/pyspark/sql/tests/test_udf.py index 7bad159b42180..9f13388c4ad2e 100644 --- a/python/pyspark/sql/tests/test_udf.py +++ b/python/pyspark/sql/tests/test_udf.py @@ -1485,7 +1485,7 @@ def test_udf_binary_type(self): def get_binary_type(x): return type(x).__name__ - binary_udf = udf(get_binary_type, returnType="string", useArrow=True) + binary_udf = udf(get_binary_type, returnType="string") df = self.spark.createDataFrame( [Row(b=b"hello"), Row(b=b"world")], schema=StructType([StructField("b", BinaryType())]) @@ -1505,7 +1505,7 @@ def test_udf_array_binary_type(self): def check_array_binary_types(arr): return [type(x).__name__ for x in arr] - array_binary_udf = udf(check_array_binary_types, returnType="array", useArrow=True) + array_binary_udf = udf(check_array_binary_types, returnType="array") df = self.spark.createDataFrame( [Row(arr_b=[b"a", b"b"]), Row(arr_b=[b"c", b"d"])], @@ -1526,7 +1526,7 @@ def test_udf_map_binary_type(self): def check_map_binary_types(m): return [type(v).__name__ for v in m.values()] - map_binary_udf = udf(check_map_binary_types, returnType="array", useArrow=True) + map_binary_udf = udf(check_map_binary_types, returnType="array") df = self.spark.createDataFrame( [Row(map_b={"k1": b"v1", "k2": b"v2"}), Row(map_b={"k3": b"v3"})], @@ -1547,7 +1547,7 @@ def test_udf_struct_binary_type(self): def check_struct_binary_type(s): return type(s.b).__name__ - struct_binary_udf = udf(check_struct_binary_type, returnType="string", useArrow=True) + struct_binary_udf = udf(check_struct_binary_type, returnType="string") struct_schema = StructType( [StructField("i", IntegerType()), StructField("b", BinaryType())] From 31f52c8d40862d49d67f0edde157cb903f190e85 Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Wed, 1 Oct 2025 11:56:08 +0000 Subject: [PATCH 18/26] udf test restructure --- python/pyspark/sql/connect/dataframe.py | 8 +- python/pyspark/sql/conversion.py | 9 +- .../sql/tests/arrow/test_arrow_python_udf.py | 107 ++++-------------- python/pyspark/sql/tests/test_udf.py | 107 ++++-------------- python/pyspark/sql/tests/test_udtf.py | 23 +++- .../sql/worker/plan_data_source_read.py | 2 +- .../sql/worker/write_into_data_source.py | 2 +- 7 files changed, 75 insertions(+), 183 deletions(-) diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index aface86a8fd79..9d083fd459f5c 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -1829,10 +1829,8 @@ def collect(self) -> List[Row]: def _get_binary_as_bytes(self) -> bool: """Get the binary_as_bytes configuration value from Spark session.""" - return ( - self._session.conf.get("spark.sql.execution.pyspark.binaryAsBytes", "true").lower() - == "true" - ) + conf_value = self._session.conf.get("spark.sql.execution.pyspark.binaryAsBytes", "true") + return (conf_value or "true").lower() == "true" def _to_table(self) -> Tuple["pa.Table", Optional[StructType]]: query = self._plan.to_proto(self._session.client) @@ -2175,7 +2173,7 @@ def foreachPartition(self, f: Callable[[Iterator[Row]], None]) -> None: binary_as_bytes = self._get_binary_as_bytes() field_converters = [ ArrowTableToRowsConversion._create_converter( - f.dataType, binary_as_bytes=binary_as_bytes + f.dataType, none_on_identity=False, binary_as_bytes=binary_as_bytes ) for f in schema.fields ] diff --git a/python/pyspark/sql/conversion.py b/python/pyspark/sql/conversion.py index 068964b5ea586..8d8e700ed5876 100644 --- a/python/pyspark/sql/conversion.py +++ b/python/pyspark/sql/conversion.py @@ -518,7 +518,7 @@ def _create_converter(dataType: DataType) -> Callable: @overload @staticmethod def _create_converter( - dataType: DataType, *, none_on_identity: bool = True, binary_as_bytes: bool = True + dataType: DataType, *, none_on_identity: bool = True ) -> Optional[Callable]: pass @@ -728,6 +728,13 @@ def convert( # type: ignore[overload-overlap] ) -> List[Row]: pass + @overload + @staticmethod + def convert( + table: "pa.Table", schema: StructType, *, binary_as_bytes: bool = True + ) -> List[Row]: + pass + @overload @staticmethod def convert( diff --git a/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py b/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py index 130135ae3ee1d..53f3c3e767f75 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py @@ -374,6 +374,26 @@ def tearDownClass(cls): finally: super().tearDownClass() + def test_udf_binary_type(self): + def get_binary_type(x): + return type(x).__name__ + + binary_udf = udf(get_binary_type, returnType="string", useArrow=True) + + df = self.spark.createDataFrame( + [Row(b=b"hello"), Row(b=b"world")], schema=StructType([StructField("b", BinaryType())]) + ) + # For Arrow Python UDF with legacy conversion BinaryType is always mapped to bytes + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): + result = df.select(binary_udf(col("b")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytes") + self.assertEqual(result[1]["type_name"], "bytes") + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): + result = df.select(binary_udf(col("b")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytes") + self.assertEqual(result[1]["type_name"], "bytes") + class ArrowPythonUDFNonLegacyTestsMixin(ArrowPythonUDFTestsMixin): @classmethod @@ -432,93 +452,6 @@ def tearDownClass(cls): finally: super(ArrowPythonUDFNonLegacyTests, cls).tearDownClass() - def test_udf_binary_type(self): - def get_binary_type(x): - return type(x).__name__ - - binary_udf = udf(get_binary_type, returnType="string", useArrow=True) - - df = self.spark.createDataFrame( - [Row(b=b"hello"), Row(b=b"world")], schema=StructType([StructField("b", BinaryType())]) - ) - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = df.select(binary_udf(col("b")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytes") - self.assertEqual(result[1]["type_name"], "bytes") - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = df.select(binary_udf(col("b")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytearray") - self.assertEqual(result[1]["type_name"], "bytearray") - - def test_udf_array_binary_type(self): - def check_array_binary_types(arr): - return [type(x).__name__ for x in arr] - - array_binary_udf = udf(check_array_binary_types, returnType="array", useArrow=True) - - df = self.spark.createDataFrame( - [Row(arr_b=[b"a", b"b"]), Row(arr_b=[b"c", b"d"])], - schema=StructType([StructField("arr_b", ArrayType(BinaryType()))]), - ) - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = df.select(array_binary_udf(col("arr_b")).alias("types")).collect() - self.assertEqual(result[0]["types"], ["bytes", "bytes"]) - self.assertEqual(result[1]["types"], ["bytes", "bytes"]) - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = df.select(array_binary_udf(col("arr_b")).alias("types")).collect() - self.assertEqual(result[0]["types"], ["bytearray", "bytearray"]) - self.assertEqual(result[1]["types"], ["bytearray", "bytearray"]) - - def test_udf_map_binary_type(self): - def check_map_binary_types(m): - return [type(v).__name__ for v in m.values()] - - map_binary_udf = udf(check_map_binary_types, returnType="array", useArrow=True) - - df = self.spark.createDataFrame( - [Row(map_b={"k1": b"v1", "k2": b"v2"}), Row(map_b={"k3": b"v3"})], - schema=StructType([StructField("map_b", MapType(StringType(), BinaryType()))]), - ) - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = df.select(map_binary_udf(col("map_b")).alias("types")).collect() - self.assertEqual(set(result[0]["types"]), {"bytes"}) - self.assertEqual(result[1]["types"], ["bytes"]) - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = df.select(map_binary_udf(col("map_b")).alias("types")).collect() - self.assertEqual(set(result[0]["types"]), {"bytearray"}) - self.assertEqual(result[1]["types"], ["bytearray"]) - - def test_udf_struct_binary_type(self): - def check_struct_binary_type(s): - return type(s.b).__name__ - - struct_binary_udf = udf(check_struct_binary_type, returnType="string", useArrow=True) - - struct_schema = StructType( - [StructField("i", IntegerType()), StructField("b", BinaryType())] - ) - - df = self.spark.createDataFrame( - [Row(struct_b=Row(i=1, b=b"data1")), Row(struct_b=Row(i=2, b=b"data2"))], - schema=StructType([StructField("struct_b", struct_schema)]), - ) - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = df.select(struct_binary_udf(col("struct_b")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytes") - self.assertEqual(result[1]["type_name"], "bytes") - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = df.select(struct_binary_udf(col("struct_b")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytearray") - self.assertEqual(result[1]["type_name"], "bytearray") - if __name__ == "__main__": from pyspark.sql.tests.arrow.test_arrow_python_udf import * # noqa: F401 diff --git a/python/pyspark/sql/tests/test_udf.py b/python/pyspark/sql/tests/test_udf.py index 9f13388c4ad2e..93f41f685f23a 100644 --- a/python/pyspark/sql/tests/test_udf.py +++ b/python/pyspark/sql/tests/test_udf.py @@ -1447,6 +1447,26 @@ def my_udf(): self.spark.range(1).select(my_udf().alias("result")).show() + def test_udf_binary_type(self): + def get_binary_type(x): + return type(x).__name__ + + binary_udf = udf(get_binary_type, returnType="string") + + df = self.spark.createDataFrame( + [Row(b=b"hello"), Row(b=b"world")], schema=StructType([StructField("b", BinaryType())]) + ) + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): + result = df.select(binary_udf(col("b")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytes") + self.assertEqual(result[1]["type_name"], "bytes") + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): + result = df.select(binary_udf(col("b")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytearray") + self.assertEqual(result[1]["type_name"], "bytearray") + class UDFTests(BaseUDFTestsMixin, ReusedSQLTestCase): @classmethod @@ -1481,93 +1501,6 @@ def test_udf_with_various_buffer_size(self): rets = [x["ret"] for x in df.collect()] self.assertEqual(rets, list(range(4, 1004))) - def test_udf_binary_type(self): - def get_binary_type(x): - return type(x).__name__ - - binary_udf = udf(get_binary_type, returnType="string") - - df = self.spark.createDataFrame( - [Row(b=b"hello"), Row(b=b"world")], schema=StructType([StructField("b", BinaryType())]) - ) - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = df.select(binary_udf(col("b")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytes") - self.assertEqual(result[1]["type_name"], "bytes") - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = df.select(binary_udf(col("b")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytearray") - self.assertEqual(result[1]["type_name"], "bytearray") - - def test_udf_array_binary_type(self): - def check_array_binary_types(arr): - return [type(x).__name__ for x in arr] - - array_binary_udf = udf(check_array_binary_types, returnType="array") - - df = self.spark.createDataFrame( - [Row(arr_b=[b"a", b"b"]), Row(arr_b=[b"c", b"d"])], - schema=StructType([StructField("arr_b", ArrayType(BinaryType()))]), - ) - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = df.select(array_binary_udf(col("arr_b")).alias("types")).collect() - self.assertEqual(result[0]["types"], ["bytes", "bytes"]) - self.assertEqual(result[1]["types"], ["bytes", "bytes"]) - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = df.select(array_binary_udf(col("arr_b")).alias("types")).collect() - self.assertEqual(result[0]["types"], ["bytearray", "bytearray"]) - self.assertEqual(result[1]["types"], ["bytearray", "bytearray"]) - - def test_udf_map_binary_type(self): - def check_map_binary_types(m): - return [type(v).__name__ for v in m.values()] - - map_binary_udf = udf(check_map_binary_types, returnType="array") - - df = self.spark.createDataFrame( - [Row(map_b={"k1": b"v1", "k2": b"v2"}), Row(map_b={"k3": b"v3"})], - schema=StructType([StructField("map_b", MapType(StringType(), BinaryType()))]), - ) - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = df.select(map_binary_udf(col("map_b")).alias("types")).collect() - self.assertEqual(set(result[0]["types"]), {"bytes"}) - self.assertEqual(result[1]["types"], ["bytes"]) - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = df.select(map_binary_udf(col("map_b")).alias("types")).collect() - self.assertEqual(set(result[0]["types"]), {"bytearray"}) - self.assertEqual(result[1]["types"], ["bytearray"]) - - def test_udf_struct_binary_type(self): - def check_struct_binary_type(s): - return type(s.b).__name__ - - struct_binary_udf = udf(check_struct_binary_type, returnType="string") - - struct_schema = StructType( - [StructField("i", IntegerType()), StructField("b", BinaryType())] - ) - - df = self.spark.createDataFrame( - [Row(struct_b=Row(i=1, b=b"data1")), Row(struct_b=Row(i=2, b=b"data2"))], - schema=StructType([StructField("struct_b", struct_schema)]), - ) - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = df.select(struct_binary_udf(col("struct_b")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytes") - self.assertEqual(result[1]["type_name"], "bytes") - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = df.select(struct_binary_udf(col("struct_b")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytearray") - self.assertEqual(result[1]["type_name"], "bytearray") - class UDFInitializationTests(unittest.TestCase): def tearDown(self): diff --git a/python/pyspark/sql/tests/test_udtf.py b/python/pyspark/sql/tests/test_udtf.py index 3f04fa256bf0d..3a9f3742981f0 100644 --- a/python/pyspark/sql/tests/test_udtf.py +++ b/python/pyspark/sql/tests/test_udtf.py @@ -3044,7 +3044,7 @@ def eval(self, v1, v2, v3, v4): for idx, field in enumerate(result_df.schema.fields): self.assertEqual(field.dataType, expected_output_types[idx]) - def test_arrow_table_udf_binary_type(self): + def test_udtf_binary_type(self): @udtf(returnType="type_name: string") class BinaryTypeUDF: def eval(self, b): @@ -3078,6 +3078,22 @@ def tearDownClass(cls): not have_pandas or not have_pyarrow, pandas_requirement_message or pyarrow_requirement_message ) class LegacyUDTFArrowTestsMixin(BaseUDTFTestsMixin): + def test_udtf_binary_type(self): + @udtf(returnType="type_name: string") + class BinaryTypeUDF: + def eval(self, b): + # Check the type of the binary input and return type name as string + yield (type(b).__name__,) + + # For Arrow Python UDTF with legacy conversion BinaryType is always mapped to bytes + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): + result = BinaryTypeUDF(lit(b"test_bytes")).collect() + self.assertEqual(result[0]["type_name"], "bytes") + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): + result = BinaryTypeUDF(lit(b"test_bytearray")).collect() + self.assertEqual(result[0]["type_name"], "bytes") + def test_eval_type(self): def upper(x: str): return upper(x) @@ -3404,6 +3420,11 @@ def tearDownClass(cls): class UDTFArrowTestsMixin(LegacyUDTFArrowTestsMixin): + def test_udtf_binary_type(self): + # For Arrow Python UDTF with non-legacy conversion BinaryType is mapped to bytes or bytearray + # consistently with non-Arrow Python UDTF behavior. + BaseUDTFTestsMixin.test_udtf_binary_type(self) + def test_numeric_output_type_casting(self): class TestUDTF: def eval(self): diff --git a/python/pyspark/sql/worker/plan_data_source_read.py b/python/pyspark/sql/worker/plan_data_source_read.py index f341fdf7ce2cf..08d18acda78b9 100644 --- a/python/pyspark/sql/worker/plan_data_source_read.py +++ b/python/pyspark/sql/worker/plan_data_source_read.py @@ -181,7 +181,7 @@ def write_read_func_and_partitions( # Create input converter. converter = ArrowTableToRowsConversion._create_converter( - BinaryType(), binary_as_bytes=binary_as_bytes + BinaryType(), none_on_identity=False, binary_as_bytes=binary_as_bytes ) # Create output converter. diff --git a/python/pyspark/sql/worker/write_into_data_source.py b/python/pyspark/sql/worker/write_into_data_source.py index 3d27f2c779520..d752a176bcbad 100644 --- a/python/pyspark/sql/worker/write_into_data_source.py +++ b/python/pyspark/sql/worker/write_into_data_source.py @@ -207,7 +207,7 @@ def main(infile: IO, outfile: IO) -> None: converters = [ ArrowTableToRowsConversion._create_converter( - f.dataType, binary_as_bytes=binary_as_bytes + f.dataType, none_on_identity=False, binary_as_bytes=binary_as_bytes ) for f in schema.fields ] From d63d7b3fe2e7493b4be400330158a5cbabb80381 Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Wed, 1 Oct 2025 11:56:51 +0000 Subject: [PATCH 19/26] doc --- python/docs/source/tutorial/sql/type_conversions.rst | 1 + 1 file changed, 1 insertion(+) diff --git a/python/docs/source/tutorial/sql/type_conversions.rst b/python/docs/source/tutorial/sql/type_conversions.rst index a7bd0d16a6147..82f6a428bb528 100644 --- a/python/docs/source/tutorial/sql/type_conversions.rst +++ b/python/docs/source/tutorial/sql/type_conversions.rst @@ -69,6 +69,7 @@ are listed below: - False * - spark.sql.execution.pyspark.binaryAsBytes - Introduced in Spark 4.1.0. When enabled, BinaryType is mapped consistently to Python bytes; when disabled, matches the PySpark default behavior before 4.1.0. + - True All Conversions --------------- From 91aed2062aea9c862471855f9e5165ef096f2681 Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Wed, 1 Oct 2025 15:11:42 +0000 Subject: [PATCH 20/26] fix ci --- python/pyspark/sql/avro/functions.py | 6 +++--- python/pyspark/sql/tests/arrow/test_arrow_python_udf.py | 3 --- .../tests/connect/arrow/test_parity_arrow_python_udf.py | 8 ++++++++ python/pyspark/sql/tests/test_udtf.py | 4 ++-- 4 files changed, 13 insertions(+), 8 deletions(-) diff --git a/python/pyspark/sql/avro/functions.py b/python/pyspark/sql/avro/functions.py index 5c4be6570cd6a..e1b70c5fd3597 100644 --- a/python/pyspark/sql/avro/functions.py +++ b/python/pyspark/sql/avro/functions.py @@ -69,7 +69,7 @@ def from_avro( >>> df = spark.createDataFrame(data, ("key", "value")) >>> avroDf = df.select(to_avro(df.value).alias("avro")) >>> avroDf.collect() - [Row(avro=bytearray(b'\\x00\\x00\\x04\\x00\\nAlice'))] + [Row(avro=b'\\x00\\x00\\x04\\x00\\nAlice')] >>> jsonFormatSchema = '''{"type":"record","name":"topLevelRecord","fields": ... [{"name":"avro","type":[{"type":"record","name":"value","namespace":"topLevelRecord", @@ -141,12 +141,12 @@ def to_avro(data: "ColumnOrName", jsonFormatSchema: str = "") -> Column: >>> data = ['SPADES'] >>> df = spark.createDataFrame(data, "string") >>> df.select(to_avro(df.value).alias("suite")).collect() - [Row(suite=bytearray(b'\\x00\\x0cSPADES'))] + [Row(suite=b'\\x00\\x0cSPADES')] >>> jsonFormatSchema = '''["null", {"type": "enum", "name": "value", ... "symbols": ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]}]''' >>> df.select(to_avro(df.value, jsonFormatSchema).alias("suite")).collect() - [Row(suite=bytearray(b'\\x02\\x00'))] + [Row(suite=b'\\x02\\x00')] """ from py4j.java_gateway import JVMView from pyspark.sql.classic.column import _to_java_column diff --git a/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py b/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py index 53f3c3e767f75..035b46f9e77e9 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py @@ -22,11 +22,8 @@ from pyspark.sql.functions import udf, col from pyspark.sql.tests.test_udf import BaseUDFTestsMixin from pyspark.sql.types import ( - ArrayType, BinaryType, DayTimeIntervalType, - IntegerType, - MapType, StringType, StructField, StructType, diff --git a/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_python_udf.py b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_python_udf.py index 6af55bb0b0bd1..b63cdb7fdbbef 100644 --- a/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_python_udf.py +++ b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_python_udf.py @@ -46,6 +46,10 @@ def tearDownClass(cls): finally: super().tearDownClass() + @unittest.skip("Duplicate test as it is already tested in ArrowPythonUDFLegacyTests.") + def test_udf_binary_type(self): + super().test_udf_binary_type(self) + class ArrowPythonUDFParityNonLegacyTestsMixin(ArrowPythonUDFTestsMixin): @classmethod @@ -62,6 +66,10 @@ def tearDownClass(cls): finally: super().tearDownClass() + @unittest.skip("Duplicate test as it is already tested in ArrowPythonUDFNonLegacyTests.") + def test_udf_binary_type(self): + super().test_udf_binary_type(self) + class ArrowPythonUDFParityLegacyTests(UDFParityTests, ArrowPythonUDFParityLegacyTestsMixin): @classmethod diff --git a/python/pyspark/sql/tests/test_udtf.py b/python/pyspark/sql/tests/test_udtf.py index 3a9f3742981f0..ee679d842fba6 100644 --- a/python/pyspark/sql/tests/test_udtf.py +++ b/python/pyspark/sql/tests/test_udtf.py @@ -3421,8 +3421,8 @@ def tearDownClass(cls): class UDTFArrowTestsMixin(LegacyUDTFArrowTestsMixin): def test_udtf_binary_type(self): - # For Arrow Python UDTF with non-legacy conversion BinaryType is mapped to bytes or bytearray - # consistently with non-Arrow Python UDTF behavior. + # For Arrow Python UDTF with non-legacy conversionBinaryType is mapped to + # bytes or bytearray consistently with non-Arrow Python UDTF behavior. BaseUDTFTestsMixin.test_udtf_binary_type(self) def test_numeric_output_type_casting(self): From 9ce5dfd741197886bdf4d35d2fb36400921efed8 Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Wed, 1 Oct 2025 16:30:45 +0000 Subject: [PATCH 21/26] fix foreach partition --- .../tests/connect/test_connect_collection.py | 3 +- .../apache/spark/sql/classic/Dataset.scala | 4 +- .../sql/execution/python/EvaluatePython.scala | 73 ++++++++++--------- 3 files changed, 43 insertions(+), 37 deletions(-) diff --git a/python/pyspark/sql/tests/connect/test_connect_collection.py b/python/pyspark/sql/tests/connect/test_connect_collection.py index 0505ddd30c881..fbb9e24b80367 100644 --- a/python/pyspark/sql/tests/connect/test_connect_collection.py +++ b/python/pyspark/sql/tests/connect/test_connect_collection.py @@ -428,7 +428,6 @@ def assert_bytes_type(iterator): # Test with binary_as_bytes=False - should get bytearray objects with self.both_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - def assert_bytearray_type(iterator): count = 0 for row in iterator: @@ -441,7 +440,7 @@ def assert_bytearray_type(iterator): assert count > 0, "No rows were processed" self.connect.sql(query).foreachPartition(assert_bytearray_type) - # self.spark.sql(query).foreachPartition(assert_bytearray_type) + self.spark.sql(query).foreachPartition(assert_bytearray_type) if __name__ == "__main__": diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/Dataset.scala index b8ffa09dfa05c..2e954cd7d6446 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/Dataset.scala @@ -2072,7 +2072,9 @@ class Dataset[T] private[sql]( */ private[sql] def javaToPython: JavaRDD[Array[Byte]] = { val structType = schema // capture it for closure - val rdd = queryExecution.toRdd.map(EvaluatePython.toJava(_, structType)) + val binaryAsBytes = sparkSession.sessionState.conf.pysparkBinaryAsBytes // capture config value + val rdd = queryExecution.toRdd.map(row => + EvaluatePython.toJava(row, structType, binaryAsBytes)) EvaluatePython.javaToPython(rdd) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala index 2d78c4e11767d..8e40a401f971a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala @@ -56,47 +56,52 @@ object EvaluatePython { /** * Helper for converting from Catalyst type to java type suitable for Pickle. */ - def toJava(obj: Any, dataType: DataType): Any = (obj, dataType) match { - case (null, _) => null - - case (row: InternalRow, struct: StructType) => - val values = new Array[Any](row.numFields) - var i = 0 - while (i < row.numFields) { - values(i) = toJava(row.get(i, struct.fields(i).dataType), struct.fields(i).dataType) - i += 1 - } - new GenericRowWithSchema(values, struct) + def toJava( + obj: Any, + dataType: DataType, + binaryAsBytes: Boolean = SQLConf.get.pysparkBinaryAsBytes): Any = { + (obj, dataType) match { + case (null, _) => null + + case (row: InternalRow, struct: StructType) => + val values = new Array[Any](row.numFields) + var i = 0 + while (i < row.numFields) { + val field = struct.fields(i) + values(i) = toJava(row.get(i, field.dataType), field.dataType, binaryAsBytes) + i += 1 + } + new GenericRowWithSchema(values, struct) - case (a: ArrayData, array: ArrayType) => - val values = new java.util.ArrayList[Any](a.numElements()) - a.foreach(array.elementType, (_, e) => { - values.add(toJava(e, array.elementType)) - }) - values + case (a: ArrayData, array: ArrayType) => + val values = new java.util.ArrayList[Any](a.numElements()) + a.foreach(array.elementType, (_, e) => { + values.add(toJava(e, array.elementType, binaryAsBytes)) + }) + values - case (map: MapData, mt: MapType) => - val jmap = new java.util.HashMap[Any, Any](map.numElements()) - map.foreach(mt.keyType, mt.valueType, (k, v) => { - jmap.put(toJava(k, mt.keyType), toJava(v, mt.valueType)) - }) - jmap + case (map: MapData, mt: MapType) => + val jmap = new java.util.HashMap[Any, Any](map.numElements()) + map.foreach(mt.keyType, mt.valueType, (k, v) => { + jmap.put(toJava(k, mt.keyType, binaryAsBytes), toJava(v, mt.valueType, binaryAsBytes)) + }) + jmap - case (ud, udt: UserDefinedType[_]) => toJava(ud, udt.sqlType) + case (ud, udt: UserDefinedType[_]) => toJava(ud, udt.sqlType, binaryAsBytes) - case (d: Decimal, _) => d.toJavaBigDecimal + case (d: Decimal, _) => d.toJavaBigDecimal - case (s: UTF8String, _: StringType) => s.toString + case (s: UTF8String, _: StringType) => s.toString - case (bytes: Array[Byte], BinaryType) => - // Check config to decide whether to wrap bytes for conversion to Python bytes - if (SQLConf.get.pysparkBinaryAsBytes) { - new BytesWrapper(bytes) - } else { - bytes - } + case (bytes: Array[Byte], BinaryType) => + if (binaryAsBytes) { + new BytesWrapper(bytes) + } else { + bytes + } - case (other, _) => other + case (other, _) => other + } } /** From e488e3b98e6d8257800c62de98488ff242b99103 Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Wed, 1 Oct 2025 16:53:17 +0000 Subject: [PATCH 22/26] add tests for nested structure --- .../sql/tests/arrow/test_arrow_python_udf.py | 68 ++++++++++++++++++ .../arrow/test_parity_arrow_python_udf.py | 10 +++ .../tests/connect/test_connect_collection.py | 1 + python/pyspark/sql/tests/test_udf.py | 69 +++++++++++++++++++ 4 files changed, 148 insertions(+) diff --git a/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py b/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py index 035b46f9e77e9..65d43ad92eb0d 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py @@ -391,6 +391,74 @@ def get_binary_type(x): self.assertEqual(result[0]["type_name"], "bytes") self.assertEqual(result[1]["type_name"], "bytes") + def test_udf_binary_type_in_nested_structures(self): + # For Arrow Python UDF with legacy conversion BinaryType is always mapped to bytes + # Test binary in array + def check_array_binary_type(arr): + return type(arr[0]).__name__ + + array_udf = udf(check_array_binary_type, returnType="string") + df_array = self.spark.createDataFrame( + [Row(arr=[b"hello", b"world"])], + schema=StructType([StructField("arr", ArrayType(BinaryType()))]), + ) + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): + result = df_array.select(array_udf(col("arr")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytes") + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): + result = df_array.select(array_udf(col("arr")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytes") + + # Test binary in map value + def check_map_binary_type(m): + return type(list(m.values())[0]).__name__ + + map_udf = udf(check_map_binary_type, returnType="string") + df_map = self.spark.createDataFrame( + [Row(m={"key": b"value"})], + schema=StructType([StructField("m", MapType(StringType(), BinaryType()))]), + ) + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): + result = df_map.select(map_udf(col("m")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytes") + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): + result = df_map.select(map_udf(col("m")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytes") + + # Test binary in struct + def check_struct_binary_type(s): + return type(s.binary_field).__name__ + + struct_udf = udf(check_struct_binary_type, returnType="string") + df_struct = self.spark.createDataFrame( + [Row(s=Row(binary_field=b"test", other_field="value"))], + schema=StructType( + [ + StructField( + "s", + StructType( + [ + StructField("binary_field", BinaryType()), + StructField("other_field", StringType()), + ] + ), + ) + ] + ), + ) + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): + result = df_struct.select(struct_udf(col("s")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytes") + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): + result = df_struct.select(struct_udf(col("s")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytes") + class ArrowPythonUDFNonLegacyTestsMixin(ArrowPythonUDFTestsMixin): @classmethod diff --git a/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_python_udf.py b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_python_udf.py index b63cdb7fdbbef..0f879087e7bb2 100644 --- a/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_python_udf.py +++ b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_python_udf.py @@ -15,6 +15,8 @@ # limitations under the License. # +import unittest + from pyspark.sql.tests.connect.test_parity_udf import UDFParityTests from pyspark.sql.tests.arrow.test_arrow_python_udf import ArrowPythonUDFTestsMixin @@ -50,6 +52,10 @@ def tearDownClass(cls): def test_udf_binary_type(self): super().test_udf_binary_type(self) + @unittest.skip("Duplicate test as it is already tested in ArrowPythonUDFLegacyTests.") + def test_udf_binary_type_in_nested_structures(self): + super().test_udf_binary_type_in_nested_structures(self) + class ArrowPythonUDFParityNonLegacyTestsMixin(ArrowPythonUDFTestsMixin): @classmethod @@ -70,6 +76,10 @@ def tearDownClass(cls): def test_udf_binary_type(self): super().test_udf_binary_type(self) + @unittest.skip("Duplicate test as it is already tested in ArrowPythonUDFLegacyTests.") + def test_udf_binary_type_in_nested_structures(self): + super().test_udf_binary_type_in_nested_structures(self) + class ArrowPythonUDFParityLegacyTests(UDFParityTests, ArrowPythonUDFParityLegacyTestsMixin): @classmethod diff --git a/python/pyspark/sql/tests/connect/test_connect_collection.py b/python/pyspark/sql/tests/connect/test_connect_collection.py index fbb9e24b80367..895fb25163a9f 100644 --- a/python/pyspark/sql/tests/connect/test_connect_collection.py +++ b/python/pyspark/sql/tests/connect/test_connect_collection.py @@ -428,6 +428,7 @@ def assert_bytes_type(iterator): # Test with binary_as_bytes=False - should get bytearray objects with self.both_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): + def assert_bytearray_type(iterator): count = 0 for row in iterator: diff --git a/python/pyspark/sql/tests/test_udf.py b/python/pyspark/sql/tests/test_udf.py index 93f41f685f23a..d0a4c8e900d03 100644 --- a/python/pyspark/sql/tests/test_udf.py +++ b/python/pyspark/sql/tests/test_udf.py @@ -1467,6 +1467,75 @@ def get_binary_type(x): self.assertEqual(result[0]["type_name"], "bytearray") self.assertEqual(result[1]["type_name"], "bytearray") + def test_udf_binary_type_in_nested_structures(self): + """Test that binary type in arrays, maps, and structs respects binaryAsBytes config""" + + # Test binary in array + def check_array_binary_type(arr): + return type(arr[0]).__name__ + + array_udf = udf(check_array_binary_type, returnType="string") + df_array = self.spark.createDataFrame( + [Row(arr=[b"hello", b"world"])], + schema=StructType([StructField("arr", ArrayType(BinaryType()))]), + ) + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): + result = df_array.select(array_udf(col("arr")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytes") + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): + result = df_array.select(array_udf(col("arr")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytearray") + + # Test binary in map value + def check_map_binary_type(m): + return type(list(m.values())[0]).__name__ + + map_udf = udf(check_map_binary_type, returnType="string") + df_map = self.spark.createDataFrame( + [Row(m={"key": b"value"})], + schema=StructType([StructField("m", MapType(StringType(), BinaryType()))]), + ) + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): + result = df_map.select(map_udf(col("m")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytes") + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): + result = df_map.select(map_udf(col("m")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytearray") + + # Test binary in struct + def check_struct_binary_type(s): + return type(s.binary_field).__name__ + + struct_udf = udf(check_struct_binary_type, returnType="string") + df_struct = self.spark.createDataFrame( + [Row(s=Row(binary_field=b"test", other_field="value"))], + schema=StructType( + [ + StructField( + "s", + StructType( + [ + StructField("binary_field", BinaryType()), + StructField("other_field", StringType()), + ] + ), + ) + ] + ), + ) + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): + result = df_struct.select(struct_udf(col("s")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytes") + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): + result = df_struct.select(struct_udf(col("s")).alias("type_name")).collect() + self.assertEqual(result[0]["type_name"], "bytearray") + class UDFTests(BaseUDFTestsMixin, ReusedSQLTestCase): @classmethod From 468344915b10473ab4f4b7364cda1bf2244ccbbe Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Thu, 2 Oct 2025 14:18:35 +0000 Subject: [PATCH 23/26] address comments --- python/pyspark/sql/functions/builtin.py | 8 ++-- .../sql/tests/arrow/test_arrow_python_udf.py | 45 +++++++++++-------- python/pyspark/sql/tests/test_udf.py | 44 ++++++++++-------- python/pyspark/sql/tests/test_udtf.py | 12 ++--- .../python/BatchEvalPythonExec.scala | 16 ++++--- .../python/BatchEvalPythonUDTFExec.scala | 3 +- .../sql/execution/python/EvaluatePython.scala | 2 +- .../python/UserDefinedPythonFunction.scala | 3 +- .../streaming/PythonForeachWriter.scala | 4 +- 9 files changed, 81 insertions(+), 56 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index fb5d8ea461963..512af852babcd 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -16627,14 +16627,14 @@ def to_binary(col: "ColumnOrName", format: Optional["ColumnOrName"] = None) -> C >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([("abc",)], ["e"]) >>> df.select(sf.try_to_binary(df.e, sf.lit("utf-8")).alias('r')).collect() - [Row(r=bytearray(b'abc'))] + [Row(r=b'abc')] Example 2: Convert string to a timestamp without encoding specified >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([("414243",)], ["e"]) >>> df.select(sf.try_to_binary(df.e).alias('r')).collect() - [Row(r=bytearray(b'ABC'))] + [Row(r=b'ABC')] """ if format is not None: return _invoke_function_over_columns("to_binary", col, format) @@ -17650,14 +17650,14 @@ def try_to_binary(col: "ColumnOrName", format: Optional["ColumnOrName"] = None) >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([("abc",)], ["e"]) >>> df.select(sf.try_to_binary(df.e, sf.lit("utf-8")).alias('r')).collect() - [Row(r=bytearray(b'abc'))] + [Row(r=b'abc')] Example 2: Convert string to a timestamp without encoding specified >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([("414243",)], ["e"]) >>> df.select(sf.try_to_binary(df.e).alias('r')).collect() - [Row(r=bytearray(b'ABC'))] + [Row(r=b'ABC')] Example 3: Converion failure results in NULL when ANSI mode is on diff --git a/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py b/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py index 65d43ad92eb0d..15be489e09c8c 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py @@ -22,8 +22,10 @@ from pyspark.sql.functions import udf, col from pyspark.sql.tests.test_udf import BaseUDFTestsMixin from pyspark.sql.types import ( + ArrayType, BinaryType, DayTimeIntervalType, + MapType, StringType, StructField, StructType, @@ -36,6 +38,7 @@ pyarrow_requirement_message, ReusedSQLTestCase, ) +from pyspark.testing.utils import assertDataFrameEqual from pyspark.util import PythonEvalType @@ -382,14 +385,14 @@ def get_binary_type(x): ) # For Arrow Python UDF with legacy conversion BinaryType is always mapped to bytes with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = df.select(binary_udf(col("b")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytes") - self.assertEqual(result[1]["type_name"], "bytes") + result = df.select(binary_udf(col("b")).alias("type_name")) + expected = self.spark.createDataFrame([Row(type_name="bytes"), Row(type_name="bytes")]) + assertDataFrameEqual(result, expected) with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = df.select(binary_udf(col("b")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytes") - self.assertEqual(result[1]["type_name"], "bytes") + result = df.select(binary_udf(col("b")).alias("type_name")) + expected = self.spark.createDataFrame([Row(type_name="bytes"), Row(type_name="bytes")]) + assertDataFrameEqual(result, expected) def test_udf_binary_type_in_nested_structures(self): # For Arrow Python UDF with legacy conversion BinaryType is always mapped to bytes @@ -404,12 +407,14 @@ def check_array_binary_type(arr): ) with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = df_array.select(array_udf(col("arr")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytes") + result = df_array.select(array_udf(col("arr")).alias("type_name")) + expected = self.spark.createDataFrame([Row(type_name="bytes")]) + assertDataFrameEqual(result, expected) with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = df_array.select(array_udf(col("arr")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytes") + result = df_array.select(array_udf(col("arr")).alias("type_name")) + expected = self.spark.createDataFrame([Row(type_name="bytes")]) + assertDataFrameEqual(result, expected) # Test binary in map value def check_map_binary_type(m): @@ -422,12 +427,14 @@ def check_map_binary_type(m): ) with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = df_map.select(map_udf(col("m")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytes") + result = df_map.select(map_udf(col("m")).alias("type_name")) + expected = self.spark.createDataFrame([Row(type_name="bytes")]) + assertDataFrameEqual(result, expected) with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = df_map.select(map_udf(col("m")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytes") + result = df_map.select(map_udf(col("m")).alias("type_name")) + expected = self.spark.createDataFrame([Row(type_name="bytes")]) + assertDataFrameEqual(result, expected) # Test binary in struct def check_struct_binary_type(s): @@ -452,12 +459,14 @@ def check_struct_binary_type(s): ) with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = df_struct.select(struct_udf(col("s")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytes") + result = df_struct.select(struct_udf(col("s")).alias("type_name")) + expected = self.spark.createDataFrame([Row(type_name="bytes")]) + assertDataFrameEqual(result, expected) with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = df_struct.select(struct_udf(col("s")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytes") + result = df_struct.select(struct_udf(col("s")).alias("type_name")) + expected = self.spark.createDataFrame([Row(type_name="bytes")]) + assertDataFrameEqual(result, expected) class ArrowPythonUDFNonLegacyTestsMixin(ArrowPythonUDFTestsMixin): diff --git a/python/pyspark/sql/tests/test_udf.py b/python/pyspark/sql/tests/test_udf.py index d0a4c8e900d03..59eb54e3cb3ca 100644 --- a/python/pyspark/sql/tests/test_udf.py +++ b/python/pyspark/sql/tests/test_udf.py @@ -1458,14 +1458,16 @@ def get_binary_type(x): ) with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = df.select(binary_udf(col("b")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytes") - self.assertEqual(result[1]["type_name"], "bytes") + result = df.select(binary_udf(col("b")).alias("type_name")) + expected = self.spark.createDataFrame([Row(type_name="bytes"), Row(type_name="bytes")]) + assertDataFrameEqual(result, expected) with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = df.select(binary_udf(col("b")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytearray") - self.assertEqual(result[1]["type_name"], "bytearray") + result = df.select(binary_udf(col("b")).alias("type_name")) + expected = self.spark.createDataFrame( + [Row(type_name="bytearray"), Row(type_name="bytearray")] + ) + assertDataFrameEqual(result, expected) def test_udf_binary_type_in_nested_structures(self): """Test that binary type in arrays, maps, and structs respects binaryAsBytes config""" @@ -1481,12 +1483,14 @@ def check_array_binary_type(arr): ) with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = df_array.select(array_udf(col("arr")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytes") + result = df_array.select(array_udf(col("arr")).alias("type_name")) + expected = self.spark.createDataFrame([Row(type_name="bytes")]) + assertDataFrameEqual(result, expected) with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = df_array.select(array_udf(col("arr")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytearray") + result = df_array.select(array_udf(col("arr")).alias("type_name")) + expected = self.spark.createDataFrame([Row(type_name="bytearray")]) + assertDataFrameEqual(result, expected) # Test binary in map value def check_map_binary_type(m): @@ -1499,12 +1503,14 @@ def check_map_binary_type(m): ) with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = df_map.select(map_udf(col("m")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytes") + result = df_map.select(map_udf(col("m")).alias("type_name")) + expected = self.spark.createDataFrame([Row(type_name="bytes")]) + assertDataFrameEqual(result, expected) with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = df_map.select(map_udf(col("m")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytearray") + result = df_map.select(map_udf(col("m")).alias("type_name")) + expected = self.spark.createDataFrame([Row(type_name="bytearray")]) + assertDataFrameEqual(result, expected) # Test binary in struct def check_struct_binary_type(s): @@ -1529,12 +1535,14 @@ def check_struct_binary_type(s): ) with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = df_struct.select(struct_udf(col("s")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytes") + result = df_struct.select(struct_udf(col("s")).alias("type_name")) + expected = self.spark.createDataFrame([Row(type_name="bytes")]) + assertDataFrameEqual(result, expected) with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = df_struct.select(struct_udf(col("s")).alias("type_name")).collect() - self.assertEqual(result[0]["type_name"], "bytearray") + result = df_struct.select(struct_udf(col("s")).alias("type_name")) + expected = self.spark.createDataFrame([Row(type_name="bytearray")]) + assertDataFrameEqual(result, expected) class UDFTests(BaseUDFTestsMixin, ReusedSQLTestCase): diff --git a/python/pyspark/sql/tests/test_udtf.py b/python/pyspark/sql/tests/test_udtf.py index ee679d842fba6..78355f4d34c20 100644 --- a/python/pyspark/sql/tests/test_udtf.py +++ b/python/pyspark/sql/tests/test_udtf.py @@ -3046,17 +3046,17 @@ def eval(self, v1, v2, v3, v4): def test_udtf_binary_type(self): @udtf(returnType="type_name: string") - class BinaryTypeUDF: + class BinaryTypeUDTF: def eval(self, b): # Check the type of the binary input and return type name as string yield (type(b).__name__,) with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = BinaryTypeUDF(lit(b"test_bytes")).collect() + result = BinaryTypeUDTF(lit(b"test_bytes")).collect() self.assertEqual(result[0]["type_name"], "bytes") with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = BinaryTypeUDF(lit(b"test_bytearray")).collect() + result = BinaryTypeUDTF(lit(b"test_bytearray")).collect() self.assertEqual(result[0]["type_name"], "bytearray") @@ -3080,18 +3080,18 @@ def tearDownClass(cls): class LegacyUDTFArrowTestsMixin(BaseUDTFTestsMixin): def test_udtf_binary_type(self): @udtf(returnType="type_name: string") - class BinaryTypeUDF: + class BinaryTypeUDTF: def eval(self, b): # Check the type of the binary input and return type name as string yield (type(b).__name__,) # For Arrow Python UDTF with legacy conversion BinaryType is always mapped to bytes with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = BinaryTypeUDF(lit(b"test_bytes")).collect() + result = BinaryTypeUDTF(lit(b"test_bytes")).collect() self.assertEqual(result[0]["type_name"], "bytes") with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = BinaryTypeUDF(lit(b"test_bytearray")).collect() + result = BinaryTypeUDTF(lit(b"test_bytearray")).collect() self.assertEqual(result[0]["type_name"], "bytes") def test_eval_type(self): diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala index 28318a319b088..866719122ec4c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala @@ -41,6 +41,7 @@ case class BatchEvalPythonExec(udfs: Seq[PythonUDF], resultAttrs: Seq[Attribute] override protected def evaluatorFactory: EvalPythonEvaluatorFactory = { val batchSize = conf.getConf(SQLConf.PYTHON_UDF_MAX_RECORDS_PER_BATCH) + val binaryAsBytes = conf.pysparkBinaryAsBytes new BatchEvalPythonEvaluatorFactory( child.output, udfs, @@ -48,7 +49,8 @@ case class BatchEvalPythonExec(udfs: Seq[PythonUDF], resultAttrs: Seq[Attribute] batchSize, pythonMetrics, jobArtifactUUID, - conf.pythonUDFProfiler) + conf.pythonUDFProfiler, + binaryAsBytes) } override protected def withNewChildInternal(newChild: SparkPlan): BatchEvalPythonExec = @@ -62,7 +64,8 @@ class BatchEvalPythonEvaluatorFactory( batchSize: Int, pythonMetrics: Map[String, SQLMetric], jobArtifactUUID: Option[String], - profiler: Option[String]) + profiler: Option[String], + binaryAsBytes: Boolean) extends EvalPythonEvaluatorFactory(childOutput, udfs, output) { override def evaluate( @@ -74,7 +77,7 @@ class BatchEvalPythonEvaluatorFactory( EvaluatePython.registerPicklers() // register pickler for Row // Input iterator to Python. - val inputIterator = BatchEvalPythonExec.getInputIterator(iter, schema, batchSize) + val inputIterator = BatchEvalPythonExec.getInputIterator(iter, schema, batchSize, binaryAsBytes) // Output iterator for results from Python. val outputIterator = @@ -112,7 +115,8 @@ object BatchEvalPythonExec { def getInputIterator( iter: Iterator[InternalRow], schema: StructType, - batchSize: Int): Iterator[Array[Byte]] = { + batchSize: Int, + binaryAsBytes: Boolean): Iterator[Array[Byte]] = { val dataTypes = schema.map(_.dataType) val needConversion = dataTypes.exists(EvaluatePython.needConversionInPython) @@ -133,14 +137,14 @@ object BatchEvalPythonExec { // For each row, add it to the queue. iter.map { row => if (needConversion) { - EvaluatePython.toJava(row, schema) + EvaluatePython.toJava(row, schema, binaryAsBytes) } else { // fast path for these types that does not need conversion in Python val fields = new Array[Any](row.numFields) var i = 0 while (i < row.numFields) { val dt = dataTypes(i) - fields(i) = EvaluatePython.toJava(row.get(i, dt), dt) + fields(i) = EvaluatePython.toJava(row.get(i, dt), dt, binaryAsBytes) i += 1 } fields diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonUDTFExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonUDTFExec.scala index a1358c9cd7746..7b46ab4bd34aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonUDTFExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonUDTFExec.scala @@ -65,7 +65,8 @@ case class BatchEvalPythonUDTFExec( // Input iterator to Python. // For Python UDTF, we don't have a separate configuration for the batch size yet. - val inputIterator = BatchEvalPythonExec.getInputIterator(iter, schema, 100) + val inputIterator = BatchEvalPythonExec.getInputIterator( + iter, schema, 100, conf.pysparkBinaryAsBytes) // Output iterator for results from Python. val outputIterator = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala index 8e40a401f971a..6ad70b555cf07 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala @@ -59,7 +59,7 @@ object EvaluatePython { def toJava( obj: Any, dataType: DataType, - binaryAsBytes: Boolean = SQLConf.get.pysparkBinaryAsBytes): Any = { + binaryAsBytes: Boolean): Any = { (obj, dataType) match { case (null, _) => null diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala index 037b784c9dd18..8987d6e36ff96 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala @@ -231,7 +231,8 @@ class UserDefinedPythonTableFunctionAnalyzeRunner( } if (value.foldable) { dataOut.writeBoolean(true) - val obj = pickler.dumps(EvaluatePython.toJava(value.eval(), value.dataType)) + val obj = pickler.dumps(EvaluatePython.toJava( + value.eval(), value.dataType, SQLConf.get.pysparkBinaryAsBytes)) PythonWorkerUtils.writeBytes(obj, dataOut) } else { dataOut.writeBoolean(false) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/PythonForeachWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/PythonForeachWriter.scala index b1b79946c2fba..37716d2d8413b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/PythonForeachWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/PythonForeachWriter.scala @@ -89,7 +89,9 @@ class PythonForeachWriter(func: PythonFunction, schema: StructType) private lazy val inputByteIterator = { EvaluatePython.registerPicklers() - val objIterator = inputRowIterator.map { row => EvaluatePython.toJava(row, schema) } + val objIterator = inputRowIterator.map { row => + EvaluatePython.toJava(row, schema, SQLConf.get.pysparkBinaryAsBytes) + } new SerDeUtil.AutoBatchedPickler(objIterator) } From 852f9d9c2d7e13abd06ebe6405da9f9173e04956 Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Thu, 2 Oct 2025 15:25:59 +0000 Subject: [PATCH 24/26] fix build and simplify tests --- .../sql/tests/arrow/test_arrow_python_udf.py | 59 +++----- .../tests/connect/test_connect_collection.py | 130 ++++++------------ python/pyspark/sql/tests/test_conversion.py | 51 ++----- python/pyspark/sql/tests/test_readwriter.py | 50 ++++++- python/pyspark/sql/tests/test_udf.py | 66 ++++----- python/pyspark/sql/tests/test_udtf.py | 23 ++-- .../spark/sql/api/python/PythonSQLUtils.scala | 2 +- .../apache/spark/sql/classic/Dataset.scala | 10 +- .../sql/execution/python/EvaluatePython.scala | 1 - 9 files changed, 172 insertions(+), 220 deletions(-) diff --git a/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py b/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py index 15be489e09c8c..5ea329ad5444d 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py @@ -383,16 +383,12 @@ def get_binary_type(x): df = self.spark.createDataFrame( [Row(b=b"hello"), Row(b=b"world")], schema=StructType([StructField("b", BinaryType())]) ) + expected = self.spark.createDataFrame([Row(type_name="bytes"), Row(type_name="bytes")]) # For Arrow Python UDF with legacy conversion BinaryType is always mapped to bytes - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = df.select(binary_udf(col("b")).alias("type_name")) - expected = self.spark.createDataFrame([Row(type_name="bytes"), Row(type_name="bytes")]) - assertDataFrameEqual(result, expected) - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = df.select(binary_udf(col("b")).alias("type_name")) - expected = self.spark.createDataFrame([Row(type_name="bytes"), Row(type_name="bytes")]) - assertDataFrameEqual(result, expected) + for conf_val in ["true", "false"]: + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": conf_val}): + result = df.select(binary_udf(col("b")).alias("type_name")) + assertDataFrameEqual(result, expected) def test_udf_binary_type_in_nested_structures(self): # For Arrow Python UDF with legacy conversion BinaryType is always mapped to bytes @@ -405,16 +401,11 @@ def check_array_binary_type(arr): [Row(arr=[b"hello", b"world"])], schema=StructType([StructField("arr", ArrayType(BinaryType()))]), ) - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = df_array.select(array_udf(col("arr")).alias("type_name")) - expected = self.spark.createDataFrame([Row(type_name="bytes")]) - assertDataFrameEqual(result, expected) - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = df_array.select(array_udf(col("arr")).alias("type_name")) - expected = self.spark.createDataFrame([Row(type_name="bytes")]) - assertDataFrameEqual(result, expected) + expected = self.spark.createDataFrame([Row(type_name="bytes")]) + for conf_val in ["true", "false"]: + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": conf_val}): + result = df_array.select(array_udf(col("arr")).alias("type_name")) + assertDataFrameEqual(result, expected) # Test binary in map value def check_map_binary_type(m): @@ -425,16 +416,11 @@ def check_map_binary_type(m): [Row(m={"key": b"value"})], schema=StructType([StructField("m", MapType(StringType(), BinaryType()))]), ) - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = df_map.select(map_udf(col("m")).alias("type_name")) - expected = self.spark.createDataFrame([Row(type_name="bytes")]) - assertDataFrameEqual(result, expected) - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = df_map.select(map_udf(col("m")).alias("type_name")) - expected = self.spark.createDataFrame([Row(type_name="bytes")]) - assertDataFrameEqual(result, expected) + expected = self.spark.createDataFrame([Row(type_name="bytes")]) + for conf_val in ["true", "false"]: + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": conf_val}): + result = df_map.select(map_udf(col("m")).alias("type_name")) + assertDataFrameEqual(result, expected) # Test binary in struct def check_struct_binary_type(s): @@ -457,16 +443,11 @@ def check_struct_binary_type(s): ] ), ) - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = df_struct.select(struct_udf(col("s")).alias("type_name")) - expected = self.spark.createDataFrame([Row(type_name="bytes")]) - assertDataFrameEqual(result, expected) - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = df_struct.select(struct_udf(col("s")).alias("type_name")) - expected = self.spark.createDataFrame([Row(type_name="bytes")]) - assertDataFrameEqual(result, expected) + expected = self.spark.createDataFrame([Row(type_name="bytes")]) + for conf_val in ["true", "false"]: + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": conf_val}): + result = df_struct.select(struct_udf(col("s")).alias("type_name")) + assertDataFrameEqual(result, expected) class ArrowPythonUDFNonLegacyTestsMixin(ArrowPythonUDFTestsMixin): diff --git a/python/pyspark/sql/tests/connect/test_connect_collection.py b/python/pyspark/sql/tests/connect/test_connect_collection.py index 895fb25163a9f..84696e251b1b3 100644 --- a/python/pyspark/sql/tests/connect/test_connect_collection.py +++ b/python/pyspark/sql/tests/connect/test_connect_collection.py @@ -326,8 +326,6 @@ def test_collect_nested_type(self): def test_collect_binary_type(self): """Test that df.collect() respects binary_as_bytes configuration for server-side data""" - # Use SQL to create data with binary type on the server side - # This ensures the data goes through Arrow conversion from server to client query = """ SELECT * FROM VALUES (CAST('hello' AS BINARY)), @@ -335,33 +333,21 @@ def test_collect_binary_type(self): AS tab(b) """ - # Test with binary_as_bytes=True (default) - with self.both_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - connect_rows = self.connect.sql(query).collect() - self.assertEqual(len(connect_rows), 2) - for row in connect_rows: - self.assertIsInstance(row.b, bytes) - - spark_rows = self.spark.sql(query).collect() - self.assertEqual(len(spark_rows), 2) - for row in spark_rows: - self.assertIsInstance(row.b, bytes) - - # Test with binary_as_bytes=False - with self.both_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - connect_rows = self.connect.sql(query).collect() - self.assertEqual(len(connect_rows), 2) - for row in connect_rows: - self.assertIsInstance(row.b, bytearray) - - spark_rows = self.spark.sql(query).collect() - self.assertEqual(len(spark_rows), 2) - for row in spark_rows: - self.assertIsInstance(row.b, bytearray) + for conf_value in ["true", "false"]: + expected_type = bytes if conf_value == "true" else bytearray + with self.both_conf({"spark.sql.execution.pyspark.binaryAsBytes": conf_value}): + connect_rows = self.connect.sql(query).collect() + self.assertEqual(len(connect_rows), 2) + for row in connect_rows: + self.assertIsInstance(row.b, expected_type) + + spark_rows = self.spark.sql(query).collect() + self.assertEqual(len(spark_rows), 2) + for row in spark_rows: + self.assertIsInstance(row.b, expected_type) def test_to_local_iterator_binary_type(self): """Test that df.toLocalIterator() respects binary_as_bytes configuration""" - # Use server-side query that creates binary data query = """ SELECT * FROM VALUES (CAST('data1' AS BINARY)), @@ -369,33 +355,20 @@ def test_to_local_iterator_binary_type(self): AS tab(b) """ - # Test with binary_as_bytes=True - with self.both_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - connect_count = 0 - for row in self.connect.sql(query).toLocalIterator(): - self.assertIsInstance(row.b, bytes) - connect_count += 1 - self.assertEqual(connect_count, 2) - - spark_count = 0 - for row in self.spark.sql(query).toLocalIterator(): - self.assertIsInstance(row.b, bytes) - spark_count += 1 - self.assertEqual(spark_count, 2) - - # Test with binary_as_bytes=False - with self.both_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - connect_count = 0 - for row in self.connect.sql(query).toLocalIterator(): - self.assertIsInstance(row.b, bytearray) - connect_count += 1 - self.assertEqual(connect_count, 2) - - spark_count = 0 - for row in self.spark.sql(query).toLocalIterator(): - self.assertIsInstance(row.b, bytearray) - spark_count += 1 - self.assertEqual(spark_count, 2) + for conf_value in ["true", "false"]: + expected_type = bytes if conf_value == "true" else bytearray + with self.both_conf({"spark.sql.execution.pyspark.binaryAsBytes": conf_value}): + connect_count = 0 + for row in self.connect.sql(query).toLocalIterator(): + self.assertIsInstance(row.b, expected_type) + connect_count += 1 + self.assertEqual(connect_count, 2) + + spark_count = 0 + for row in self.spark.sql(query).toLocalIterator(): + self.assertIsInstance(row.b, expected_type) + spark_count += 1 + self.assertEqual(spark_count, 2) def test_foreach_partition_binary_type(self): """Test that df.foreachPartition() respects binary_as_bytes configuration @@ -403,7 +376,6 @@ def test_foreach_partition_binary_type(self): Since foreachPartition() runs on executors and cannot return data to the driver, we test by ensuring the function doesn't throw exceptions when it expects the correct types. """ - # Use server-side query that creates binary data query = """ SELECT * FROM VALUES (CAST('partition1' AS BINARY)), @@ -411,37 +383,25 @@ def test_foreach_partition_binary_type(self): AS tab(b) """ - # Test with binary_as_bytes=True - should get bytes objects - with self.both_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - - def assert_bytes_type(iterator): - count = 0 - for row in iterator: - # This will raise an exception if the type is not bytes - assert isinstance(row.b, bytes), f"Expected bytes, got {type(row.b).__name__}" - count += 1 - # Ensure we actually processed rows - assert count > 0, "No rows were processed" - - self.connect.sql(query).foreachPartition(assert_bytes_type) - self.spark.sql(query).foreachPartition(assert_bytes_type) - - # Test with binary_as_bytes=False - should get bytearray objects - with self.both_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - - def assert_bytearray_type(iterator): - count = 0 - for row in iterator: - # This will raise an exception if the type is not bytearray - assert isinstance( - row.b, bytearray - ), f"Expected bytearray, got {type(row.b).__name__}" - count += 1 - # Ensure we actually processed rows - assert count > 0, "No rows were processed" - - self.connect.sql(query).foreachPartition(assert_bytearray_type) - self.spark.sql(query).foreachPartition(assert_bytearray_type) + for conf_value in ["true", "false"]: + expected_type = bytes if conf_value == "true" else bytearray + expected_type_name = "bytes" if conf_value == "true" else "bytearray" + + with self.both_conf({"spark.sql.execution.pyspark.binaryAsBytes": conf_value}): + + def assert_type(iterator): + count = 0 + for row in iterator: + # This will raise an exception if the type is not as expected + assert isinstance( + row.b, expected_type + ), f"Expected {expected_type_name}, got {type(row.b).__name__}" + count += 1 + # Ensure we actually processed rows + assert count > 0, "No rows were processed" + + self.connect.sql(query).foreachPartition(assert_type) + self.spark.sql(query).foreachPartition(assert_type) if __name__ == "__main__": diff --git a/python/pyspark/sql/tests/test_conversion.py b/python/pyspark/sql/tests/test_conversion.py index 7caec6f2c9545..ca3b6f6671aa5 100644 --- a/python/pyspark/sql/tests/test_conversion.py +++ b/python/pyspark/sql/tests/test_conversion.py @@ -110,7 +110,6 @@ def test_conversion(self): self.assertEqual(a, e) def test_binary_as_bytes_conversion(self): - """Test binary type conversion with binary_as_bytes parameter""" data = [ ( str(i).encode(), # simple binary @@ -130,42 +129,22 @@ def test_binary_as_bytes_conversion(self): tbl = LocalDataToArrowConversion.convert(data, schema, use_large_var_types=False) - # Test binary_as_bytes=True (default) - should return bytes - actual_bytes = ArrowTableToRowsConversion.convert(tbl, schema, binary_as_bytes=True) - - for row in actual_bytes: - # Simple binary field should be bytes - self.assertIsInstance(row.b, bytes) - # Array elements should be bytes - for elem in row.arr_b: - self.assertIsInstance(elem, bytes) - # Map values should be bytes - for value in row.map_b.values(): - self.assertIsInstance(value, bytes) - # Struct field should be bytes - self.assertIsInstance(row.struct_b.b, bytes) - - # Test binary_as_bytes=False - should return bytearray - actual_bytearray = ArrowTableToRowsConversion.convert(tbl, schema, binary_as_bytes=False) - - for row in actual_bytearray: - # Simple binary field should be bytearray - self.assertIsInstance(row.b, bytearray) - # Array elements should be bytearray - for elem in row.arr_b: - self.assertIsInstance(elem, bytearray) - # Map values should be bytearray - for value in row.map_b.values(): - self.assertIsInstance(value, bytearray) - # Struct field should be bytearray - self.assertIsInstance(row.struct_b.b, bytearray) + for binary_as_bytes, expected_type in [(True, bytes), (False, bytearray)]: + actual = ArrowTableToRowsConversion.convert( + tbl, schema, binary_as_bytes=binary_as_bytes + ) - # Verify the actual content is the same, just different types - for bytes_row, bytearray_row in zip(actual_bytes, actual_bytearray): - self.assertEqual(bytes_row.b, bytes(bytearray_row.b)) - self.assertEqual([bytes(ba) for ba in bytearray_row.arr_b], bytes_row.arr_b) - self.assertEqual({k: bytes(v) for k, v in bytearray_row.map_b.items()}, bytes_row.map_b) - self.assertEqual(bytes(bytearray_row.struct_b.b), bytes_row.struct_b.b) + for row in actual: + # Simple binary field + self.assertIsInstance(row.b, expected_type) + # Array elements + for elem in row.arr_b: + self.assertIsInstance(elem, expected_type) + # Map values + for value in row.map_b.values(): + self.assertIsInstance(value, expected_type) + # Struct field + self.assertIsInstance(row.struct_b.b, expected_type) if __name__ == "__main__": diff --git a/python/pyspark/sql/tests/test_readwriter.py b/python/pyspark/sql/tests/test_readwriter.py index 1d1cc3507f0e6..33ba2c02c63cb 100644 --- a/python/pyspark/sql/tests/test_readwriter.py +++ b/python/pyspark/sql/tests/test_readwriter.py @@ -20,9 +20,17 @@ import tempfile from pyspark.errors import AnalysisException +from pyspark.sql import Row from pyspark.sql.functions import col, lit from pyspark.sql.readwriter import DataFrameWriterV2 -from pyspark.sql.types import StructType, StructField, StringType +from pyspark.sql.types import ( + StructType, + StructField, + StringType, + BinaryType, + ArrayType, + MapType, +) from pyspark.testing import assertDataFrameEqual from pyspark.testing.sqlutils import ReusedSQLTestCase @@ -238,6 +246,46 @@ def test_cached_table(self): self.assertEqual(join2.columns, ["id", "value_1", "index", "value_2"]) + def test_binary_type(self): + """Test that binary type in data sources respects binaryAsBytes config""" + schema = StructType( + [ + StructField("id", StringType()), + StructField("bin", BinaryType()), + StructField("arr_bin", ArrayType(BinaryType())), + StructField("map_bin", MapType(StringType(), BinaryType())), + ] + ) + # Create DataFrame with binary data (can use either bytes or bytearray) + data = [Row(id="1", bin=b"hello", arr_bin=[b"a"], map_bin={"key": b"value"})] + df = self.spark.createDataFrame(data, schema) + + tmpPath = tempfile.mkdtemp() + try: + # Write to parquet + df.write.mode("overwrite").parquet(tmpPath) + + for conf_value in ["true", "false"]: + expected_type = bytes if conf_value == "true" else bytearray + expected_bin = b"hello" if conf_value == "true" else bytearray(b"hello") + expected_arr = b"a" if conf_value == "true" else bytearray(b"a") + expected_map = b"value" if conf_value == "true" else bytearray(b"value") + + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": conf_value}): + result = self.spark.read.parquet(tmpPath).collect() + row = result[0] + # Check binary field + self.assertIsInstance(row.bin, expected_type) + self.assertEqual(row.bin, expected_bin) + # Check array of binary + self.assertIsInstance(row.arr_bin[0], expected_type) + self.assertEqual(row.arr_bin[0], expected_arr) + # Check map value + self.assertIsInstance(row.map_bin["key"], expected_type) + self.assertEqual(row.map_bin["key"], expected_map) + finally: + shutil.rmtree(tmpPath) + # "[SPARK-51182]: DataFrameWriter should throw dataPathNotSpecifiedError when path is not # specified" def test_save(self): diff --git a/python/pyspark/sql/tests/test_udf.py b/python/pyspark/sql/tests/test_udf.py index 59eb54e3cb3ca..37f651a96a846 100644 --- a/python/pyspark/sql/tests/test_udf.py +++ b/python/pyspark/sql/tests/test_udf.py @@ -1454,20 +1454,15 @@ def get_binary_type(x): binary_udf = udf(get_binary_type, returnType="string") df = self.spark.createDataFrame( - [Row(b=b"hello"), Row(b=b"world")], schema=StructType([StructField("b", BinaryType())]) + [Row(b=b"hello world")], schema=StructType([StructField("b", BinaryType())]) ) - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = df.select(binary_udf(col("b")).alias("type_name")) - expected = self.spark.createDataFrame([Row(type_name="bytes"), Row(type_name="bytes")]) - assertDataFrameEqual(result, expected) - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = df.select(binary_udf(col("b")).alias("type_name")) - expected = self.spark.createDataFrame( - [Row(type_name="bytearray"), Row(type_name="bytearray")] - ) - assertDataFrameEqual(result, expected) + for conf_value in ["true", "false"]: + expected_type = "bytes" if conf_value == "true" else "bytearray" + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": conf_value}): + result = df.select(binary_udf(col("b")).alias("type_name")) + expected = self.spark.createDataFrame([Row(type_name=expected_type)]) + assertDataFrameEqual(result, expected) def test_udf_binary_type_in_nested_structures(self): """Test that binary type in arrays, maps, and structs respects binaryAsBytes config""" @@ -1478,19 +1473,16 @@ def check_array_binary_type(arr): array_udf = udf(check_array_binary_type, returnType="string") df_array = self.spark.createDataFrame( - [Row(arr=[b"hello", b"world"])], + [Row(arr=[b"hello world"])], schema=StructType([StructField("arr", ArrayType(BinaryType()))]), ) - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = df_array.select(array_udf(col("arr")).alias("type_name")) - expected = self.spark.createDataFrame([Row(type_name="bytes")]) - assertDataFrameEqual(result, expected) - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = df_array.select(array_udf(col("arr")).alias("type_name")) - expected = self.spark.createDataFrame([Row(type_name="bytearray")]) - assertDataFrameEqual(result, expected) + for conf_value in ["true", "false"]: + expected_type = "bytes" if conf_value == "true" else "bytearray" + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": conf_value}): + result = df_array.select(array_udf(col("arr")).alias("type_name")) + expected = self.spark.createDataFrame([Row(type_name=expected_type)]) + assertDataFrameEqual(result, expected) # Test binary in map value def check_map_binary_type(m): @@ -1502,15 +1494,12 @@ def check_map_binary_type(m): schema=StructType([StructField("m", MapType(StringType(), BinaryType()))]), ) - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = df_map.select(map_udf(col("m")).alias("type_name")) - expected = self.spark.createDataFrame([Row(type_name="bytes")]) - assertDataFrameEqual(result, expected) - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = df_map.select(map_udf(col("m")).alias("type_name")) - expected = self.spark.createDataFrame([Row(type_name="bytearray")]) - assertDataFrameEqual(result, expected) + for conf_value in ["true", "false"]: + expected_type = "bytes" if conf_value == "true" else "bytearray" + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": conf_value}): + result = df_map.select(map_udf(col("m")).alias("type_name")) + expected = self.spark.createDataFrame([Row(type_name=expected_type)]) + assertDataFrameEqual(result, expected) # Test binary in struct def check_struct_binary_type(s): @@ -1534,15 +1523,12 @@ def check_struct_binary_type(s): ), ) - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = df_struct.select(struct_udf(col("s")).alias("type_name")) - expected = self.spark.createDataFrame([Row(type_name="bytes")]) - assertDataFrameEqual(result, expected) - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = df_struct.select(struct_udf(col("s")).alias("type_name")) - expected = self.spark.createDataFrame([Row(type_name="bytearray")]) - assertDataFrameEqual(result, expected) + for conf_value in ["true", "false"]: + expected_type = "bytes" if conf_value == "true" else "bytearray" + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": conf_value}): + result = df_struct.select(struct_udf(col("s")).alias("type_name")) + expected = self.spark.createDataFrame([Row(type_name=expected_type)]) + assertDataFrameEqual(result, expected) class UDFTests(BaseUDFTestsMixin, ReusedSQLTestCase): diff --git a/python/pyspark/sql/tests/test_udtf.py b/python/pyspark/sql/tests/test_udtf.py index 78355f4d34c20..5cf6a0eba59c3 100644 --- a/python/pyspark/sql/tests/test_udtf.py +++ b/python/pyspark/sql/tests/test_udtf.py @@ -3051,13 +3051,11 @@ def eval(self, b): # Check the type of the binary input and return type name as string yield (type(b).__name__,) - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = BinaryTypeUDTF(lit(b"test_bytes")).collect() - self.assertEqual(result[0]["type_name"], "bytes") - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = BinaryTypeUDTF(lit(b"test_bytearray")).collect() - self.assertEqual(result[0]["type_name"], "bytearray") + for conf_value in ["true", "false"]: + expected_type = "bytes" if conf_value == "true" else "bytearray" + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": conf_value}): + result = BinaryTypeUDTF(lit(b"test")).collect() + self.assertEqual(result[0]["type_name"], expected_type) class UDTFTests(BaseUDTFTestsMixin, ReusedSQLTestCase): @@ -3086,13 +3084,10 @@ def eval(self, b): yield (type(b).__name__,) # For Arrow Python UDTF with legacy conversion BinaryType is always mapped to bytes - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "true"}): - result = BinaryTypeUDTF(lit(b"test_bytes")).collect() - self.assertEqual(result[0]["type_name"], "bytes") - - with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": "false"}): - result = BinaryTypeUDTF(lit(b"test_bytearray")).collect() - self.assertEqual(result[0]["type_name"], "bytes") + for conf_value in ["true", "false"]: + with self.sql_conf({"spark.sql.execution.pyspark.binaryAsBytes": conf_value}): + result = BinaryTypeUDTF(lit(b"test")).collect() + self.assertEqual(result[0]["type_name"], "bytes") def test_eval_type(self): def upper(x: str): diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala index 85f59c282ff55..471e376e1c22d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala @@ -117,7 +117,7 @@ private[sql] object PythonSQLUtils extends Logging { def toPyRow(row: Row): Array[Byte] = { assert(row.isInstanceOf[GenericRowWithSchema]) withInternalRowPickler(_.dumps(EvaluatePython.toJava( - CatalystTypeConverters.convertToCatalyst(row), row.schema))) + CatalystTypeConverters.convertToCatalyst(row), row.schema, SQLConf.get.pysparkBinaryAsBytes))) } def toJVMRow( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/Dataset.scala index 2e954cd7d6446..65881cf5d03b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/Dataset.scala @@ -2081,7 +2081,8 @@ class Dataset[T] private[sql]( private[sql] def collectToPython(): Array[Any] = { EvaluatePython.registerPicklers() withAction("collectToPython", queryExecution) { plan => - val toJava: (Any) => Any = EvaluatePython.toJava(_, schema) + val binaryAsBytes = sparkSession.sessionState.conf.pysparkBinaryAsBytes + val toJava: (Any) => Any = EvaluatePython.toJava(_, schema, binaryAsBytes) val iter: Iterator[Array[Byte]] = new SerDeUtil.AutoBatchedPickler( plan.executeCollect().iterator.map(toJava)) PythonRDD.serveIterator(iter, "serve-DataFrame") @@ -2091,7 +2092,8 @@ class Dataset[T] private[sql]( private[sql] def tailToPython(n: Int): Array[Any] = { EvaluatePython.registerPicklers() withAction("tailToPython", queryExecution) { plan => - val toJava: (Any) => Any = EvaluatePython.toJava(_, schema) + val binaryAsBytes = sparkSession.sessionState.conf.pysparkBinaryAsBytes + val toJava: (Any) => Any = EvaluatePython.toJava(_, schema, binaryAsBytes) val iter: Iterator[Array[Byte]] = new SerDeUtil.AutoBatchedPickler( plan.executeTail(n).iterator.map(toJava)) PythonRDD.serveIterator(iter, "serve-DataFrame") @@ -2104,7 +2106,9 @@ class Dataset[T] private[sql]( EvaluatePython.registerPicklers() val numRows = _numRows.max(0).min(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH - 1) val rows = getRows(numRows, truncate).map(_.toArray).toArray - val toJava: (Any) => Any = EvaluatePython.toJava(_, ArrayType(ArrayType(StringType))) + val binaryAsBytes = sparkSession.sessionState.conf.pysparkBinaryAsBytes + val toJava: (Any) => Any = + EvaluatePython.toJava(_, ArrayType(ArrayType(StringType)), binaryAsBytes) val iter: Iterator[Array[Byte]] = new SerDeUtil.AutoBatchedPickler( rows.iterator.map(toJava)) PythonRDD.serveIterator(iter, "serve-GetRows") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala index 6ad70b555cf07..212cc5db124ce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala @@ -30,7 +30,6 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData, MapData} -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{UTF8String, VariantVal} From b933d61df829863124571994e1c1f5c6f304675b Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Tue, 7 Oct 2025 20:09:46 +0000 Subject: [PATCH 25/26] address comments --- python/pyspark/sql/connect/dataframe.py | 2 +- .../main/scala/org/apache/spark/sql/internal/SQLConf.scala | 7 +++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index 9d083fd459f5c..b3f5947e491b6 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -1830,7 +1830,7 @@ def collect(self) -> List[Row]: def _get_binary_as_bytes(self) -> bool: """Get the binary_as_bytes configuration value from Spark session.""" conf_value = self._session.conf.get("spark.sql.execution.pyspark.binaryAsBytes", "true") - return (conf_value or "true").lower() == "true" + return conf_value is not None and conf_value.lower() == "true" def _to_table(self) -> Tuple["pa.Table", Optional[StructType]]: query = self._plan.to_proto(self._session.client) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index bc47f07db813f..80dd336f69bd6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -3727,8 +3727,11 @@ object SQLConf { val PYSPARK_BINARY_AS_BYTES = buildConf("spark.sql.execution.pyspark.binaryAsBytes") .doc("When true, BinaryType is mapped consistently to bytes in PySpark." + - "When false, matches the PySpark behavior before 4.1.0, where BinaryType is mapped to " + - "to bytes or bytearray in different cases.") + "When false, matches the PySpark behavior before 4.1.0. Before 4.1.0, BinaryType is " + + "mapped to bytearray for input of regular UDF and UDTF without arrow optimization, " + + "regular UDF and UDTF with arrow optimization and without legacy pandas conversion, " + + "Dataframe APIs, and data source; BinaryType is mapped to bytes for " + + "input of regular UDF and UDTF with arrow optimization and legacy pandas conversion.") .version("4.1.0") .booleanConf .createWithDefault(true) From fa202b1c256e0145f3c4737064aa594814047e18 Mon Sep 17 00:00:00 2001 From: xianzhe-databricks Date: Tue, 7 Oct 2025 20:15:32 +0000 Subject: [PATCH 26/26] move utils --- .../tests/connect/test_connect_collection.py | 33 ------------------- python/pyspark/testing/connectutils.py | 32 ++++++++++++++++++ 2 files changed, 32 insertions(+), 33 deletions(-) diff --git a/python/pyspark/sql/tests/connect/test_connect_collection.py b/python/pyspark/sql/tests/connect/test_connect_collection.py index 84696e251b1b3..2343e8b9cde4c 100644 --- a/python/pyspark/sql/tests/connect/test_connect_collection.py +++ b/python/pyspark/sql/tests/connect/test_connect_collection.py @@ -16,7 +16,6 @@ # import unittest -from contextlib import contextmanager, ExitStack from pyspark.testing.connectutils import should_test_connect, ReusedMixedTestCase from pyspark.testing.pandasutils import PandasOnSparkTestUtils @@ -26,38 +25,6 @@ class SparkConnectCollectionTests(ReusedMixedTestCase, PandasOnSparkTestUtils): - def connect_conf(self, conf_dict): - """Context manager to set configuration on Spark Connect session""" - - @contextmanager - def _connect_conf(): - old_values = {} - for key, value in conf_dict.items(): - old_values[key] = self.connect.conf.get(key, None) - self.connect.conf.set(key, value) - try: - yield - finally: - for key, old_value in old_values.items(): - if old_value is None: - self.connect.conf.unset(key) - else: - self.connect.conf.set(key, old_value) - - return _connect_conf() - - def both_conf(self, conf_dict): - """Context manager to set configuration on both classic and Connect sessions""" - - @contextmanager - def _both_conf(): - with ExitStack() as stack: - stack.enter_context(self.sql_conf(conf_dict)) - stack.enter_context(self.connect_conf(conf_dict)) - yield - - return _both_conf() - def test_collect(self): query = "SELECT id, CAST(id AS STRING) AS name FROM RANGE(100)" cdf = self.connect.sql(query) diff --git a/python/pyspark/testing/connectutils.py b/python/pyspark/testing/connectutils.py index 1f5c5a086abfb..b409bd12ae379 100644 --- a/python/pyspark/testing/connectutils.py +++ b/python/pyspark/testing/connectutils.py @@ -257,3 +257,35 @@ def compare_by_show(self, df1, df2, n: int = 20, truncate: int = 20): def test_assert_remote_mode(self): # no need to test this in mixed mode pass + + def connect_conf(self, conf_dict): + """Context manager to set configuration on Spark Connect session""" + + @contextlib.contextmanager + def _connect_conf(): + old_values = {} + for key, value in conf_dict.items(): + old_values[key] = self.connect.conf.get(key, None) + self.connect.conf.set(key, value) + try: + yield + finally: + for key, old_value in old_values.items(): + if old_value is None: + self.connect.conf.unset(key) + else: + self.connect.conf.set(key, old_value) + + return _connect_conf() + + def both_conf(self, conf_dict): + """Context manager to set configuration on both classic and Connect sessions""" + + @contextlib.contextmanager + def _both_conf(): + with contextlib.ExitStack() as stack: + stack.enter_context(self.sql_conf(conf_dict)) + stack.enter_context(self.connect_conf(conf_dict)) + yield + + return _both_conf()