You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Davies Liu (JIRA)" <ji...@apache.org> on 2016/08/01 20:19:20 UTC

[jira] [Commented] (SPARK-16700) StructType doesn't accept Python dicts anymore

    [ https://issues.apache.org/jira/browse/SPARK-16700?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15402739#comment-15402739 ] 

Davies Liu commented on SPARK-16700:
------------------------------------

There are two separate problems here:

1) Spark 2.0 enforce data type checking when creating a DataFrame, it's safer but slower. It makes sense to have a flag for that (on by default)

2) Row object is similar to named tuple (not dict), the columns are ordered. When it's created in a way like dict, we have no way to know the order of columns, so they are sorted by name, then it does not match with the schema provided. We should check the schema (order of columns) when create a DataFrame from RDD of Row (we assume they matched)

> StructType doesn't accept Python dicts anymore
> ----------------------------------------------
>
>                 Key: SPARK-16700
>                 URL: https://issues.apache.org/jira/browse/SPARK-16700
>             Project: Spark
>          Issue Type: Bug
>          Components: PySpark
>    Affects Versions: 2.0.0
>            Reporter: Sylvain Zimmer
>
> Hello,
> I found this issue while testing my codebase with 2.0.0-rc5
> StructType in Spark 1.6.2 accepts the Python <dict> type, which is very handy. 2.0.0-rc5 does not and throws an error.
> I don't know if this was intended but I'd advocate for this behaviour to remain the same. MapType is probably wasteful when your key names never change and switching to Python tuples would be cumbersome.
> Here is a minimal script to reproduce the issue: 
> {code}
> from pyspark import SparkContext
> from pyspark.sql import types as SparkTypes
> from pyspark.sql import SQLContext
> sc = SparkContext()
> sqlc = SQLContext(sc)
> struct_schema = SparkTypes.StructType([
>     SparkTypes.StructField("id", SparkTypes.LongType())
> ])
> rdd = sc.parallelize([{"id": 0}, {"id": 1}])
> df = sqlc.createDataFrame(rdd, struct_schema)
> print df.collect()
> # 1.6.2 prints [Row(id=0), Row(id=1)]
> # 2.0.0-rc5 raises TypeError: StructType can not accept object {'id': 0} in type <type 'dict'>
> {code}
> Thanks!



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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