Skip to content

Commit a853a80

Browse files
committed
[SPARK-25666][PYTHON] Internally document type conversion between Python data and SQL types in normal UDFs
### What changes were proposed in this pull request? We are facing some problems about type conversions between Python data and SQL types in UDFs (Pandas UDFs as well). It's even difficult to identify the problems (see #20163 and #22610). This PR targets to internally document the type conversion table. Some of them looks buggy and we should fix them. ```python import sys import array import datetime from decimal import Decimal from pyspark.sql import Row from pyspark.sql.types import * from pyspark.sql.functions import udf if sys.version >= '3': long = int data = [ None, True, 1, long(1), "a", u"a", datetime.date(1970, 1, 1), datetime.datetime(1970, 1, 1, 0, 0), 1.0, array.array("i", [1]), [1], (1,), bytearray([65, 66, 67]), Decimal(1), {"a": 1}, Row(kwargs=1), Row("namedtuple")(1), ] types = [ BooleanType(), ByteType(), ShortType(), IntegerType(), LongType(), StringType(), DateType(), TimestampType(), FloatType(), DoubleType(), ArrayType(IntegerType()), BinaryType(), DecimalType(10, 0), MapType(StringType(), IntegerType()), StructType([StructField("_1", IntegerType())]), ] df = spark.range(1) results = [] count = 0 total = len(types) * len(data) spark.sparkContext.setLogLevel("FATAL") for t in types: result = [] for v in data: try: row = df.select(udf(lambda: v, t)()).first() ret_str = repr(row[0]) except Exception: ret_str = "X" result.append(ret_str) progress = "SQL Type: [%s]\n Python Value: [%s(%s)]\n Result Python Value: [%s]" % ( t.simpleString(), str(v), type(v).__name__, ret_str) count += 1 print("%s/%s:\n %s" % (count, total, progress)) results.append([t.simpleString()] + list(map(str, result))) schema = ["SQL Type \\ Python Value(Type)"] + list(map(lambda v: "%s(%s)" % (str(v), type(v).__name__), data)) strings = spark.createDataFrame(results, schema=schema)._jdf.showString(20, 20, False) print("\n".join(map(lambda line: " # %s # noqa" % line, strings.strip().split("\n")))) ``` This table was generated under Python 2 but the code above is Python 3 compatible as well. ## How was this patch tested? Manually tested and lint check. Closes #22655 from HyukjinKwon/SPARK-25666. Authored-by: hyukjinkwon <[email protected]> Signed-off-by: hyukjinkwon <[email protected]>
1 parent 1a6815c commit a853a80

File tree

1 file changed

+33
-0
lines changed

1 file changed

+33
-0
lines changed

python/pyspark/sql/functions.py

Lines changed: 33 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2733,6 +2733,39 @@ def udf(f=None, returnType=StringType()):
27332733
| 8| JOHN DOE| 22|
27342734
+----------+--------------+------------+
27352735
"""
2736+
2737+
# The following table shows most of Python data and SQL type conversions in normal UDFs that
2738+
# are not yet visible to the user. Some of behaviors are buggy and might be changed in the near
2739+
# future. The table might have to be eventually documented externally.
2740+
# Please see SPARK-25666's PR to see the codes in order to generate the table below.
2741+
#
2742+
# +-----------------------------+--------------+----------+------+-------+---------------+---------------+--------------------+-----------------------------+----------+----------------------+---------+--------------------+-----------------+------------+--------------+------------------+----------------------+ # noqa
2743+
# |SQL Type \ Python Value(Type)|None(NoneType)|True(bool)|1(int)|1(long)| a(str)| a(unicode)| 1970-01-01(date)|1970-01-01 00:00:00(datetime)|1.0(float)|array('i', [1])(array)|[1](list)| (1,)(tuple)| ABC(bytearray)| 1(Decimal)|{'a': 1}(dict)|Row(kwargs=1)(Row)|Row(namedtuple=1)(Row)| # noqa
2744+
# +-----------------------------+--------------+----------+------+-------+---------------+---------------+--------------------+-----------------------------+----------+----------------------+---------+--------------------+-----------------+------------+--------------+------------------+----------------------+ # noqa
2745+
# | boolean| None| True| None| None| None| None| None| None| None| None| None| None| None| None| None| X| X| # noqa
2746+
# | tinyint| None| None| 1| 1| None| None| None| None| None| None| None| None| None| None| None| X| X| # noqa
2747+
# | smallint| None| None| 1| 1| None| None| None| None| None| None| None| None| None| None| None| X| X| # noqa
2748+
# | int| None| None| 1| 1| None| None| None| None| None| None| None| None| None| None| None| X| X| # noqa
2749+
# | bigint| None| None| 1| 1| None| None| None| None| None| None| None| None| None| None| None| X| X| # noqa
2750+
# | string| None| u'true'| u'1'| u'1'| u'a'| u'a'|u'java.util.Grego...| u'java.util.Grego...| u'1.0'| u'[I@24a83055'| u'[1]'|u'[Ljava.lang.Obj...| u'[B@49093632'| u'1'| u'{a=1}'| X| X| # noqa
2751+
# | date| None| X| X| X| X| X|datetime.date(197...| datetime.date(197...| X| X| X| X| X| X| X| X| X| # noqa
2752+
# | timestamp| None| X| X| X| X| X| X| datetime.datetime...| X| X| X| X| X| X| X| X| X| # noqa
2753+
# | float| None| None| None| None| None| None| None| None| 1.0| None| None| None| None| None| None| X| X| # noqa
2754+
# | double| None| None| None| None| None| None| None| None| 1.0| None| None| None| None| None| None| X| X| # noqa
2755+
# | array<int>| None| None| None| None| None| None| None| None| None| [1]| [1]| [1]| [65, 66, 67]| None| None| X| X| # noqa
2756+
# | binary| None| None| None| None|bytearray(b'a')|bytearray(b'a')| None| None| None| None| None| None|bytearray(b'ABC')| None| None| X| X| # noqa
2757+
# | decimal(10,0)| None| None| None| None| None| None| None| None| None| None| None| None| None|Decimal('1')| None| X| X| # noqa
2758+
# | map<string,int>| None| None| None| None| None| None| None| None| None| None| None| None| None| None| {u'a': 1}| X| X| # noqa
2759+
# | struct<_1:int>| None| X| X| X| X| X| X| X| X| X|Row(_1=1)| Row(_1=1)| X| X| Row(_1=None)| Row(_1=1)| Row(_1=1)| # noqa
2760+
# +-----------------------------+--------------+----------+------+-------+---------------+---------------+--------------------+-----------------------------+----------+----------------------+---------+--------------------+-----------------+------------+--------------+------------------+----------------------+ # noqa
2761+
#
2762+
# Note: DDL formatted string is used for 'SQL Type' for simplicity. This string can be
2763+
# used in `returnType`.
2764+
# Note: The values inside of the table are generated by `repr`.
2765+
# Note: Python 2 is used to generate this table since it is used to check the backward
2766+
# compatibility often in practice.
2767+
# Note: 'X' means it throws an exception during the conversion.
2768+
27362769
# decorator @udf, @udf(), @udf(dataType())
27372770
if f is None or isinstance(f, (str, DataType)):
27382771
# If DataType has been passed as a positional argument

0 commit comments

Comments
 (0)