[SPARK-29041][PYTHON] Allows createDataFrame to accept bytes as binary type
### What changes were proposed in this pull request? This PR proposes to allow `bytes` as an acceptable type for binary type for `createDataFrame`. ### Why are the changes needed? `bytes` is a standard type for binary in Python. This should be respected in PySpark side. ### Does this PR introduce any user-facing change? Yes, _when specified type is binary_, we will allow `bytes` as a binary type. Previously this was not allowed in both Python 2 and Python 3 as below: ```python spark.createDataFrame([[b"abcd"]], "col binary") ``` in Python 3 ``` Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/session.py", line 787, in createDataFrame rdd, schema = self._createFromLocal(map(prepare, data), schema) File "/.../spark/python/pyspark/sql/session.py", line 442, in _createFromLocal data = list(data) File "/.../spark/python/pyspark/sql/session.py", line 769, in prepare verify_func(obj) File "/.../forked/spark/python/pyspark/sql/types.py", line 1403, in verify verify_value(obj) File "/.../spark/python/pyspark/sql/types.py", line 1384, in verify_struct verifier(v) File "/.../spark/python/pyspark/sql/types.py", line 1403, in verify verify_value(obj) File "/.../spark/python/pyspark/sql/types.py", line 1397, in verify_default verify_acceptable_types(obj) File "/.../spark/python/pyspark/sql/types.py", line 1282, in verify_acceptable_types % (dataType, obj, type(obj)))) TypeError: field col: BinaryType can not accept object b'abcd' in type <class 'bytes'> ``` in Python 2: ``` Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/session.py", line 787, in createDataFrame rdd, schema = self._createFromLocal(map(prepare, data), schema) File "/.../spark/python/pyspark/sql/session.py", line 442, in _createFromLocal data = list(data) File "/.../spark/python/pyspark/sql/session.py", line 769, in prepare verify_func(obj) File "/.../spark/python/pyspark/sql/types.py", line 1403, in verify verify_value(obj) File "/.../spark/python/pyspark/sql/types.py", line 1384, in verify_struct verifier(v) File "/.../spark/python/pyspark/sql/types.py", line 1403, in verify verify_value(obj) File "/.../spark/python/pyspark/sql/types.py", line 1397, in verify_default verify_acceptable_types(obj) File "/.../spark/python/pyspark/sql/types.py", line 1282, in verify_acceptable_types % (dataType, obj, type(obj)))) TypeError: field col: BinaryType can not accept object 'abcd' in type <type 'str'> ``` So, it won't break anything. ### How was this patch tested? Unittests were added and also manually tested as below. ```bash ./run-tests --python-executables=python2,python3 --testnames "pyspark.sql.tests.test_serde" ``` Closes #25749 from HyukjinKwon/SPARK-29041. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
This commit is contained in:
parent
c18f849d76
commit
7ce0f2b499
|
@ -132,6 +132,10 @@ class SerdeTests(ReusedSQLTestCase):
|
||||||
df = self.spark.createDataFrame(data, "array<integer>")
|
df = self.spark.createDataFrame(data, "array<integer>")
|
||||||
self.assertEqual(len(list(filter(lambda r: None in r.value, df.collect()))), 0)
|
self.assertEqual(len(list(filter(lambda r: None in r.value, df.collect()))), 0)
|
||||||
|
|
||||||
|
def test_bytes_as_binary_type(self):
|
||||||
|
df = self.spark.createDataFrame([[b"abcd"]], "col binary")
|
||||||
|
self.assertEqual(df.first().col, bytearray(b'abcd'))
|
||||||
|
|
||||||
|
|
||||||
if __name__ == "__main__":
|
if __name__ == "__main__":
|
||||||
import unittest
|
import unittest
|
||||||
|
|
|
@ -1190,7 +1190,7 @@ _acceptable_types = {
|
||||||
DoubleType: (float,),
|
DoubleType: (float,),
|
||||||
DecimalType: (decimal.Decimal,),
|
DecimalType: (decimal.Decimal,),
|
||||||
StringType: (str, unicode),
|
StringType: (str, unicode),
|
||||||
BinaryType: (bytearray,),
|
BinaryType: (bytearray, bytes),
|
||||||
DateType: (datetime.date, datetime.datetime),
|
DateType: (datetime.date, datetime.datetime),
|
||||||
TimestampType: (datetime.datetime,),
|
TimestampType: (datetime.datetime,),
|
||||||
ArrayType: (list, tuple, array),
|
ArrayType: (list, tuple, array),
|
||||||
|
|
Loading…
Reference in a new issue