You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
[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]>
0 commit comments