You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Stephen Spencer (JIRA)" <ji...@apache.org> on 2018/07/25 09:20:00 UTC

[jira] [Created] (SPARK-24915) Calling SparkSession.createDataFrame with schema can throw exception

Stephen Spencer created SPARK-24915:
---------------------------------------

             Summary: Calling SparkSession.createDataFrame with schema can throw exception
                 Key: SPARK-24915
                 URL: https://issues.apache.org/jira/browse/SPARK-24915
             Project: Spark
          Issue Type: Bug
          Components: PySpark
    Affects Versions: 2.3.1
         Environment: Python 3.6.3

PySpark 2.3.1 (installed via pip)

OSX 10.12.6
            Reporter: Stephen Spencer


There seems to be a bug in PySpark when using the PySparkSQL session to create a dataframe with a pre-defined schema.

Code to reproduce the error:
{code:java}
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)

# Construct schema (the order of fields is important)
schema = StructType([
    StructField('field2', StructType([StructField('sub_field', StringType(), False)]), False),
    StructField('field1', StringType(), False),
])

# Create data to populate data frame
data = [
    Row(field1="Hello", field2=Row(sub_field='world'))
]

# Attempt to create the data frame supplying the schema
# this will throw a ValueError
df = session.createDataFrame(data, schema=schema)
df.show(){code}
Running this throws a ValueError
{noformat}
Traceback (most recent call last):
File "schema_bug.py", line 18, in <module>
df = session.createDataFrame(data, schema=schema)
File "/Users/stephenspencer/benevolent/ai/neat/rex/.env/lib/python3.6/site-packages/pyspark/sql/session.py", line 691, in createDataFrame
rdd, schema = self._createFromLocal(map(prepare, data), schema)
File "/Users/stephenspencer/benevolent/ai/neat/rex/.env/lib/python3.6/site-packages/pyspark/sql/session.py", line 423, in _createFromLocal
data = [schema.toInternal(row) for row in data]
File "/Users/stephenspencer/benevolent/ai/neat/rex/.env/lib/python3.6/site-packages/pyspark/sql/session.py", line 423, in <listcomp>
data = [schema.toInternal(row) for row in data]
File "/Users/stephenspencer/benevolent/ai/neat/rex/.env/lib/python3.6/site-packages/pyspark/sql/types.py", line 601, in toInternal
for f, v, c in zip(self.fields, obj, self._needConversion))
File "/Users/stephenspencer/benevolent/ai/neat/rex/.env/lib/python3.6/site-packages/pyspark/sql/types.py", line 601, in <genexpr>
for f, v, c in zip(self.fields, obj, self._needConversion))
File "/Users/stephenspencer/benevolent/ai/neat/rex/.env/lib/python3.6/site-packages/pyspark/sql/types.py", line 439, in toInternal
return self.dataType.toInternal(obj)
File "/Users/stephenspencer/benevolent/ai/neat/rex/.env/lib/python3.6/site-packages/pyspark/sql/types.py", line 619, in toInternal
raise ValueError("Unexpected tuple %r with StructType" % obj)
ValueError: Unexpected tuple 'Hello' with StructType{noformat}
The problem seems to be here:

https://github.com/apache/spark/blob/3d5c61e5fd24f07302e39b5d61294da79aa0c2f9/python/pyspark/sql/types.py#L603

specifically the bit
{code:java}
zip(self.fields, obj, self._needConversion)
{code}
This zip statement seems to assume that obj and self.fields are ordered in the same way, so that the elements of obj will correspond to the right fields in the schema. However this is not true, a Row orders its elements alphabetically but the fields in the schema are in whatever order they are specified. In this example field2 is being initialised with the field1 element 'Hello'. If you re-order the fields in the schema to go (field1, field2), the given example works without error.

The schema in the repro is specifically designed to elicit the problem, the fields are out of alphabetical order and one field is a StructType, making chema._needSerializeAnyField==True . However we encountered this in real use.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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