Repository: spark
Updated Branches:
  refs/heads/master 1a6815cd9 -> a853a8020


[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 
https://github.com/apache/spark/pull/20163 and 
https://github.com/apache/spark/pull/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 <gurwls...@apache.org>
Signed-off-by: hyukjinkwon <gurwls...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a853a802
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a853a802
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a853a802

Branch: refs/heads/master
Commit: a853a80202032083ad411eec5ec97b304f732a61
Parents: 1a6815c
Author: hyukjinkwon <gurwls...@apache.org>
Authored: Mon Oct 8 15:47:15 2018 +0800
Committer: hyukjinkwon <gurwls...@apache.org>
Committed: Mon Oct 8 15:47:15 2018 +0800

----------------------------------------------------------------------
 python/pyspark/sql/functions.py | 33 +++++++++++++++++++++++++++++++++
 1 file changed, 33 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/a853a802/python/pyspark/sql/functions.py
----------------------------------------------------------------------
diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py
index be089ee..5425d31 100644
--- a/python/pyspark/sql/functions.py
+++ b/python/pyspark/sql/functions.py
@@ -2733,6 +2733,39 @@ def udf(f=None, returnType=StringType()):
     |         8|      JOHN DOE|          22|
     +----------+--------------+------------+
     """
+
+    # The following table shows most of Python data and SQL type conversions 
in normal UDFs that
+    # are not yet visible to the user. Some of behaviors are buggy and might 
be changed in the near
+    # future. The table might have to be eventually documented externally.
+    # Please see SPARK-25666's PR to see the codes in order to generate the 
table below.
+    #
+    # 
+-----------------------------+--------------+----------+------+-------+---------------+---------------+--------------------+-----------------------------+----------+----------------------+---------+--------------------+-----------------+------------+--------------+------------------+----------------------+
  # noqa
+    # |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
+    # 
+-----------------------------+--------------+----------+------+-------+---------------+---------------+--------------------+-----------------------------+----------+----------------------+---------+--------------------+-----------------+------------+--------------+------------------+----------------------+
  # noqa
+    # |                      boolean|          None|      True|  None|   None| 
          None|           None|                None|                         
None|      None|                  None|     None|                None|          
   None|        None|          None|                 X|                     X|  
# noqa
+    # |                      tinyint|          None|      None|     1|      1| 
          None|           None|                None|                         
None|      None|                  None|     None|                None|          
   None|        None|          None|                 X|                     X|  
# noqa
+    # |                     smallint|          None|      None|     1|      1| 
          None|           None|                None|                         
None|      None|                  None|     None|                None|          
   None|        None|          None|                 X|                     X|  
# noqa
+    # |                          int|          None|      None|     1|      1| 
          None|           None|                None|                         
None|      None|                  None|     None|                None|          
   None|        None|          None|                 X|                     X|  
# noqa
+    # |                       bigint|          None|      None|     1|      1| 
          None|           None|                None|                         
None|      None|                  None|     None|                None|          
   None|        None|          None|                 X|                     X|  
# noqa
+    # |                       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
+    # |                         date|          None|         X|     X|      X| 
             X|              X|datetime.date(197...|         
datetime.date(197...|         X|                     X|        X|               
    X|                X|           X|             X|                 X|         
            X|  # noqa
+    # |                    timestamp|          None|         X|     X|      X| 
             X|              X|                   X|         
datetime.datetime...|         X|                     X|        X|               
    X|                X|           X|             X|                 X|         
            X|  # noqa
+    # |                        float|          None|      None|  None|   None| 
          None|           None|                None|                         
None|       1.0|                  None|     None|                None|          
   None|        None|          None|                 X|                     X|  
# noqa
+    # |                       double|          None|      None|  None|   None| 
          None|           None|                None|                         
None|       1.0|                  None|     None|                None|          
   None|        None|          None|                 X|                     X|  
# noqa
+    # |                   array<int>|          None|      None|  None|   None| 
          None|           None|                None|                         
None|      None|                   [1]|      [1]|                 [1]|     [65, 
66, 67]|        None|          None|                 X|                     X|  
# noqa
+    # |                       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
+    # |                decimal(10,0)|          None|      None|  None|   None| 
          None|           None|                None|                         
None|      None|                  None|     None|                None|          
   None|Decimal('1')|          None|                 X|                     X|  
# noqa
+    # |              map<string,int>|          None|      None|  None|   None| 
          None|           None|                None|                         
None|      None|                  None|     None|                None|          
   None|        None|     {u'a': 1}|                 X|                     X|  
# noqa
+    # |               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
+    # 
+-----------------------------+--------------+----------+------+-------+---------------+---------------+--------------------+-----------------------------+----------+----------------------+---------+--------------------+-----------------+------------+--------------+------------------+----------------------+
  # noqa
+    #
+    # Note: DDL formatted string is used for 'SQL Type' for simplicity. This 
string can be
+    #       used in `returnType`.
+    # Note: The values inside of the table are generated by `repr`.
+    # Note: Python 2 is used to generate this table since it is used to check 
the backward
+    #       compatibility often in practice.
+    # Note: 'X' means it throws an exception during the conversion.
+
     # decorator @udf, @udf(), @udf(dataType())
     if f is None or isinstance(f, (str, DataType)):
         # If DataType has been passed as a positional argument


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to