GitHub user HyukjinKwon opened a pull request:

    https://github.com/apache/spark/pull/22655

    [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 array
    import datetime
    from decimal import Decimal
    
    from pyspark.sql import Row
    from pyspark.sql.types import *
    from pyspark.sql.functions import udf
    
    
    data = [
        None,
        True,
        1,
        1L, # Python 2 only
        "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(a=1),
        Row("a")(1),
    ]
    
    types =  [
        NullType(),
        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 = []
    for t in types:
        result = []
        for v in data:
            try:
                row = df.select(udf(lambda: v, t)()).first()
                result.append(row[0])
            except Exception:
                result.append("X")
        results.append([t.simpleString()] + map(str, result))
    
    schema = ["SQL Type \\ Python Value(Type)"] + 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"))))
    ```
    
    ## How was this patch tested?
    
    Manually tested and lint check.


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/HyukjinKwon/spark SPARK-25666

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/22655.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #22655
    
----
commit 3084be1de3ff58a9258dacfb8d7cf575df3fb3c9
Author: hyukjinkwon <gurwls223@...>
Date:   2018-10-06T10:59:46Z

    Internally document type conversion between Python data and SQL types in 
UDFs

----


---

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

Reply via email to