You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2019/10/14 22:43:05 UTC

[GitHub] [spark] qudade opened a new pull request #26118: [SPARK-24915][PySpark] Fix Row handling with Schema.

qudade opened a new pull request #26118: [SPARK-24915][PySpark] Fix Row handling with Schema.
URL: https://github.com/apache/spark/pull/26118
 
 
   
   ### What changes were proposed in this pull request?
   
   This change implements a special handling of `pyspark.sql.Row` within the conversion of a `StructType` into an internal SQL object (`toInternal()`).
   
   Previously, `Row` was processed as `tuple` (since it inherits from `tuple`). In particular, it was expected that values come in the "right" order. This works if the internal order of the `Row` (sorted by key in the current implementation) corresponds to the order of fields in the schema. If the fields have a different order *and* need special treatment (e.g. `_needConversion` is `True`) then exceptions happened when creating dataframes.
   
   With this change, it will be processed as a `dict`. 
   
   ## Design
   
   Re `asDict`: I first had an implementation for `Row` as type. However, that implementation would fail for fields that are unknown to the `Row` object, this is inconsistent with the handling of `dict`s. The most consistent implementation is to convert the `Row` to `dict`.
   
   Note: The underlying problem is that `Row` inherits from `tuple`. This is visible in the tests, too. for `assertEqual` the `Row`s `Row(a=..., b=...)` and `Row(b=..., a=...)` are *not* equal because they are compared as lists (and the order is wrong) while a direct comparison returns `True` (For this reason the tests compare based on `asDict`).
   
   ### Why are the changes needed?
   
   This fixes SPARK-24915, creating data frames from (pyspark.sql.)Rows. This failed if the order of fields in the schema differed from the (internal) order of fields in the Row and the schema is "complicated".
   
   Complicated can be if one type of the schema is nested (as in the JIRA issue) or one field needs conversion (e.g. `DateType()`)
   
   Without the change, the following examples fail:
   
   *From JIRA issue:*
   
   ```
   from pyspark import SparkConf, SparkContext
   from pyspark.sql import SparkSession
   from pyspark.sql.types import StructType, StructField, StringType, Row
   
   conf = SparkConf().setMaster("local").setAppName("repro") 
   context = SparkContext(conf=conf) 
   session = SparkSession(context)
   schema = StructType([
       StructField('field2', StructType([StructField('sub_field', StringType(), False)]), False),
       StructField('field1', StringType(), False),
   ])
   data = [Row(field1="Hello", field2=Row(sub_field='world'))]
   df = session.createDataFrame(data, schema=schema) # this will throw a ValueError
   df.show()
   ```
   
   *Date example:*
   
   ```
   import datetime as dt
   from pyspark.sql import Row, SparkSession
   from pyspark.sql.types import StringType, DateType, StructField, StructType
   spark = SparkSession.builder.master("local").getOrCreate()
   schema = StructType([
       StructField("join_date", DateType(), False),
       StructField("name", StringType(),False),
   ])
   rows = [Row(name='Alice', age=23, join_date=dt.datetime(2019,10,1,23,45,56)),]
   spark.createDataFrame(rows, schema=schema).collect()
   ```
   
   ### Does this PR introduce any user-facing change?
   
   This change is not introducing User-facing changes for existing, working pyspark code.
   
   Code that previously caused exceptions b/c of the fixed bug will now work (which - technically - is a user-facing change).
   
   ### How was this patch tested?
   
   #### Standard Tests
   
   `ARROW_PRE_0_15_IPC_FORMAT=1 ./dev/run-tests` succeeded on my machine
   
   Python: 3.7.4  
   Spark: master (`a42d894a4090c97a90ce23b0989163909ebf548d`)  
   OS: MacOS 10.14.6. 
   
   #### New Tests
   
   I added the following tests in module `pyspark.sql.tests.test_types`:
   
   - `test_create_dataframe_from_rows_mixed_with_datetype`
   - `test_create_dataframe_from_rows_with_nested_row`
   
   The latter corresponds to the test case from SPARK_24915.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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