Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-22566][PYTHON] Better error message for _merge_type in Pandas to Spark DF conversion #19792

Closed
wants to merge 15 commits into from
Closed
Show file tree
Hide file tree
Changes from 11 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
24 changes: 10 additions & 14 deletions python/pyspark/sql/session.py
Original file line number Diff line number Diff line change
Expand Up @@ -324,11 +324,12 @@ def range(self, start, end=None, step=1, numPartitions=None):

return DataFrame(jdf, self._wrapped)

def _inferSchemaFromList(self, data):
def _inferSchemaFromList(self, data, names=None):
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we set the names for _createFromRDD -> _inferSchema too?:

>>> spark.createDataFrame(spark.sparkContext.parallelize([[None, 1], ["a", None], [1, 1]]), schema=["a", "b"], samplingRatio=0.99)
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../spark/python/pyspark/sql/session.py", line 644, in createDataFrame
    rdd, schema = self._createFromRDD(data.map(prepare), schema, samplingRatio)
  File "/.../spark/python/pyspark/sql/session.py", line 383, in _createFromRDD
    struct = self._inferSchema(rdd, samplingRatio)
  File "/.../spark/python/pyspark/sql/session.py", line 375, in _inferSchema
    schema = rdd.map(_infer_schema).reduce(_merge_type)
  File "/.../spark/python/pyspark/rdd.py", line 852, in reduce
    return reduce(f, vals)
  File "/.../spark/python/pyspark/sql/types.py", line 1133, in _merge_type
    for f in a.fields]
  File "/.../spark/python/pyspark/sql/types.py", line 1126, in _merge_type
    raise TypeError(new_msg("Can not merge type %s and %s" % (type(a), type(b))))
TypeError: field _1: Can not merge type <class 'pyspark.sql.types.StringType'> and <class 'pyspark.sql.types.LongType'>

"""
Infer schema from list of Row or tuple.

:param data: list of Row or tuple
:param names: list of column names
:return: :class:`pyspark.sql.types.StructType`
"""
if not data:
Expand All @@ -337,12 +338,12 @@ def _inferSchemaFromList(self, data):
if type(first) is dict:
warnings.warn("inferring schema from dict is deprecated,"
"please use pyspark.sql.Row instead")
schema = reduce(_merge_type, map(_infer_schema, data))
schema = reduce(_merge_type, (_infer_schema(row, names) for row in data))
if _has_nulltype(schema):
raise ValueError("Some of types cannot be determined after inferring")
return schema

def _inferSchema(self, rdd, samplingRatio=None):
def _inferSchema(self, rdd, samplingRatio=None, names=None):
"""
Infer schema from an RDD of Row or tuple.

Expand All @@ -359,10 +360,10 @@ def _inferSchema(self, rdd, samplingRatio=None):
"Use pyspark.sql.Row instead")

if samplingRatio is None:
schema = _infer_schema(first)
schema = _infer_schema(first, names=names)
if _has_nulltype(schema):
for row in rdd.take(100)[1:]:
schema = _merge_type(schema, _infer_schema(row))
schema = _merge_type(schema, _infer_schema(row, names=names))
if not _has_nulltype(schema):
break
else:
Expand All @@ -371,22 +372,17 @@ def _inferSchema(self, rdd, samplingRatio=None):
else:
if samplingRatio < 0.99:
rdd = rdd.sample(False, float(samplingRatio))
schema = rdd.map(_infer_schema).reduce(_merge_type)
schema = rdd.map(lambda row: _infer_schema(row, names)).reduce(_merge_type)
return schema

def _createFromRDD(self, rdd, schema, samplingRatio):
"""
Create an RDD for DataFrame from an existing RDD, returns the RDD and schema.
"""
if schema is None or isinstance(schema, (list, tuple)):
struct = self._inferSchema(rdd, samplingRatio)
converter = _create_converter(struct)
schema = self._inferSchema(rdd, samplingRatio, names=schema)
converter = _create_converter(schema)
rdd = rdd.map(converter)
if isinstance(schema, (list, tuple)):
for i, name in enumerate(schema):
struct.fields[i].name = name
struct.names[i] = name
schema = struct

elif not isinstance(schema, StructType):
raise TypeError("schema should be StructType or list or None, but got: %s" % schema)
Expand All @@ -405,7 +401,7 @@ def _createFromLocal(self, data, schema):
data = list(data)

if schema is None or isinstance(schema, (list, tuple)):
struct = self._inferSchemaFromList(data)
struct = self._inferSchemaFromList(data, names=schema)
converter = _create_converter(struct)
data = map(converter, data)
if isinstance(schema, (list, tuple)):
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@gberger, could we remove this branch too?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

removed with commit 5131db2

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

>>> spark.createDataFrame([{'a': 1}], ["b"])
DataFrame[a: bigint]

Hm.. sorry actually I think we should not remove this one and L385 because we should primarily respect user's input and it should be DataFrame[b: bigint].

Expand Down
92 changes: 92 additions & 0 deletions python/pyspark/sql/tests.py
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@
from pyspark.sql.types import UserDefinedType, _infer_type, _make_type_verifier
from pyspark.sql.types import _array_signed_int_typecode_ctype_mappings, _array_type_mappings
from pyspark.sql.types import _array_unsigned_int_typecode_ctype_mappings
from pyspark.sql.types import _merge_type
from pyspark.tests import QuietTest, ReusedPySparkTestCase, SparkSubmitTests
from pyspark.sql.functions import UserDefinedFunction, sha2, lit
from pyspark.sql.window import Window
Expand Down Expand Up @@ -846,6 +847,11 @@ def test_infer_schema(self):
result = self.spark.sql("SELECT l[0].a from test2 where d['key'].d = '2'")
self.assertEqual(1, result.head()[0])

def test_infer_schema_fails(self):
with self.assertRaisesRegexp(TypeError, 'field a'):
self.spark.createDataFrame(self.spark.sparkContext.parallelize([[1, 1], ["x", 1]]),
schema=["a", "b"], samplingRatio=0.99)

def test_infer_nested_schema(self):
NestedRow = Row("f1", "f2")
nestedRdd1 = self.sc.parallelize([NestedRow([1, 2], {"row1": 1.0}),
Expand Down Expand Up @@ -1722,6 +1728,92 @@ def test_infer_long_type(self):
self.assertEqual(_infer_type(2**61), LongType())
self.assertEqual(_infer_type(2**71), LongType())

def test_merge_type(self):
self.assertEqual(_merge_type(LongType(), NullType()), LongType())
self.assertEqual(_merge_type(NullType(), LongType()), LongType())

self.assertEqual(_merge_type(LongType(), LongType()), LongType())

self.assertEqual(_merge_type(
ArrayType(LongType()),
ArrayType(LongType())
), ArrayType(LongType()))
with self.assertRaisesRegexp(TypeError, 'element in array'):
_merge_type(ArrayType(LongType()), ArrayType(DoubleType()))

self.assertEqual(_merge_type(
MapType(StringType(), LongType()),
MapType(StringType(), LongType())
), MapType(StringType(), LongType()))
with self.assertRaisesRegexp(TypeError, 'key of map'):
_merge_type(
MapType(StringType(), LongType()),
MapType(DoubleType(), LongType()))
with self.assertRaisesRegexp(TypeError, 'value of map'):
_merge_type(
MapType(StringType(), LongType()),
MapType(StringType(), DoubleType()))

self.assertEqual(_merge_type(
StructType([StructField("f1", LongType()), StructField("f2", StringType())]),
StructType([StructField("f1", LongType()), StructField("f2", StringType())])
), StructType([StructField("f1", LongType()), StructField("f2", StringType())]))
with self.assertRaisesRegexp(TypeError, 'field f1'):
_merge_type(
StructType([StructField("f1", LongType()), StructField("f2", StringType())]),
StructType([StructField("f1", DoubleType()), StructField("f2", StringType())]))

self.assertEqual(_merge_type(
StructType([StructField("f1", StructType([StructField("f2", LongType())]))]),
StructType([StructField("f1", StructType([StructField("f2", LongType())]))])
), StructType([StructField("f1", StructType([StructField("f2", LongType())]))]))
with self.assertRaisesRegexp(TypeError, 'field f2 in field f1'):
_merge_type(
StructType([StructField("f1", StructType([StructField("f2", LongType())]))]),
StructType([StructField("f1", StructType([StructField("f2", StringType())]))]))

self.assertEqual(_merge_type(
StructType([StructField("f1", ArrayType(LongType())), StructField("f2", StringType())]),
StructType([StructField("f1", ArrayType(LongType())), StructField("f2", StringType())])
), StructType([StructField("f1", ArrayType(LongType())), StructField("f2", StringType())]))
with self.assertRaisesRegexp(TypeError, 'element in array field f1'):
_merge_type(
StructType([
StructField("f1", ArrayType(LongType())),
StructField("f2", StringType())]),
StructType([
StructField("f1", ArrayType(DoubleType())),
StructField("f2", StringType())]))

self.assertEqual(_merge_type(
StructType([
StructField("f1", MapType(StringType(), LongType())),
StructField("f2", StringType())]),
StructType([
StructField("f1", MapType(StringType(), LongType())),
StructField("f2", StringType())])
), StructType([
StructField("f1", MapType(StringType(), LongType())),
StructField("f2", StringType())]))
with self.assertRaisesRegexp(TypeError, 'value of map field f1'):
_merge_type(
StructType([
StructField("f1", MapType(StringType(), LongType())),
StructField("f2", StringType())]),
StructType([
StructField("f1", MapType(StringType(), DoubleType())),
StructField("f2", StringType())]))

self.assertEqual(_merge_type(
StructType([StructField("f1", ArrayType(MapType(StringType(), LongType())))]),
StructType([StructField("f1", ArrayType(MapType(StringType(), LongType())))])
), StructType([StructField("f1", ArrayType(MapType(StringType(), LongType())))]))
with self.assertRaisesRegexp(TypeError, 'key of map element in array field f1'):
_merge_type(
StructType([StructField("f1", ArrayType(MapType(StringType(), LongType())))]),
StructType([StructField("f1", ArrayType(MapType(DoubleType(), LongType())))])
)

def test_filter_with_datetime(self):
time = datetime.datetime(2015, 4, 17, 23, 1, 2, 3000)
date = time.date()
Expand Down
26 changes: 18 additions & 8 deletions python/pyspark/sql/types.py
Original file line number Diff line number Diff line change
Expand Up @@ -1072,7 +1072,7 @@ def _infer_type(obj):
raise TypeError("not supported type: %s" % type(obj))


def _infer_schema(row):
def _infer_schema(row, names=None):
"""Infer the schema from dict/namedtuple/object"""
if isinstance(row, dict):
items = sorted(row.items())
Expand All @@ -1083,7 +1083,8 @@ def _infer_schema(row):
elif hasattr(row, "_fields"): # namedtuple
items = zip(row._fields, tuple(row))
else:
names = ['_%d' % i for i in range(1, len(row) + 1)]
if names is None:
names = ['_%d' % i for i in range(1, len(row) + 1)]
Copy link
Member

@HyukjinKwon HyukjinKwon Dec 12, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@gberger, Let's revert this change too. Seems it's going to introduce a behaviour change:

Before

>>> spark.createDataFrame([["a", "b"]], ["col1"]).show()
+----+---+
|col1| _2|
+----+---+
|   a|  b|
+----+---+

After

>>> spark.createDataFrame([["a", "b"]], ["col1"]).show()
...
java.lang.IllegalStateException: Input row doesn't have expected number of values required by the schema. 1 fields are required while 2 values are provided.
	at org.apache.spark.sql.execution.python.EvaluatePython$.fromJava(EvaluatePython.scala:148)
        ...

Copy link
Author

@gberger gberger Dec 12, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You're right, but by reverting we lose the nice message. Notice below reverted it says field _1, where it could have said field col1.

Instead, I am adding a new elif branch where we check len(names) vs len(row). If we have fewer names than we have columns, we extend the names list, completing it with entries such as "_2".
I have included a test for this as well.

Reverted

>>> spark.createDataFrame([["a", "b"], [1, 2]], ["col1"]).show()
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/Users/gberger/Projects/spark/python/pyspark/sql/session.py", line 646, in createDataFrame
    rdd, schema = self._createFromLocal(map(prepare, data), schema)
  File "/Users/gberger/Projects/spark/python/pyspark/sql/session.py", line 409, in _createFromLocal
    struct = self._inferSchemaFromList(data, names=schema)
  File "/Users/gberger/Projects/spark/python/pyspark/sql/session.py", line 341, in _inferSchemaFromList
    schema = reduce(_merge_type, (_infer_schema(row, names) for row in data))
  File "/Users/gberger/Projects/spark/python/pyspark/sql/types.py", line 1132, in _merge_type
    for f in a.fields]
  File "/Users/gberger/Projects/spark/python/pyspark/sql/types.py", line 1125, in _merge_type
    raise TypeError(new_msg("Can not merge type %s and %s" % (type(a), type(b))))
TypeError: field _1: Can not merge type <class 'pyspark.sql.types.StringType'> and <class 'pyspark.sql.types.LongType'>

Modified

>>> spark.createDataFrame([["a", "b"]], ["col1"])
DataFrame[col1: string, _2: string]
>>> spark.createDataFrame([["a", "b"], [1, 2]], ["col1"]).show()
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/Users/gberger/Projects/spark/python/pyspark/sql/session.py", line 646, in createDataFrame
    rdd, schema = self._createFromLocal(map(prepare, data), schema)
  File "/Users/gberger/Projects/spark/python/pyspark/sql/session.py", line 409, in _createFromLocal
    struct = self._inferSchemaFromList(data, names=schema)
  File "/Users/gberger/Projects/spark/python/pyspark/sql/session.py", line 341, in _inferSchemaFromList
    schema = reduce(_merge_type, (_infer_schema(row, names) for row in data))
  File "/Users/gberger/Projects/spark/python/pyspark/sql/types.py", line 1132, in _merge_type
    for f in a.fields]
  File "/Users/gberger/Projects/spark/python/pyspark/sql/types.py", line 1125, in _merge_type
    raise TypeError(new_msg("Can not merge type %s and %s" % (type(a), type(b))))
TypeError: field col1: Can not merge type <class 'pyspark.sql.types.StringType'> and <class 'pyspark.sql.types.LongType'>

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah, yup. I noticed it too but I think the same thing applies to other cases, for example:

>>> spark.createDataFrame([{"a": 1}, {"a": []}], ["col1"])
...
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../spark/python/pyspark/sql/session.py", line 646, in createDataFrame
    rdd, schema = self._createFromLocal(map(prepare, data), schema)
  File "/.../spark/python/pyspark/sql/session.py", line 409, in _createFromLocal
    struct = self._inferSchemaFromList(data, names=schema)
  File "/.../spark/python/pyspark/sql/session.py", line 341, in _inferSchemaFromList
    schema = reduce(_merge_type, (_infer_schema(row, names) for row in data))
  File "/.../spark/python/pyspark/sql/types.py", line 1133, in _merge_type
    for f in a.fields]
  File "/.../spark/python/pyspark/sql/types.py", line 1126, in _merge_type
    raise TypeError(new_msg("Can not merge type %s and %s" % (type(a), type(b))))
TypeError: field a: Can not merge type <class 'pyspark.sql.types.LongType'> and <class 'pyspark.sql.types.ArrayType'>

So, let's revert this change here for now. There are some subtleties here but I think it's fine.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we revert it, then the original purpose of the PR is lost:

>>> df = pd.DataFrame(data={'a':[1,2,3], 'b': [4, 5, 'hello']})
>>> spark.createDataFrame(df)
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/Users/gberger/Projects/spark/python/pyspark/sql/session.py", line 646, in createDataFrame
    rdd, schema = self._createFromLocal(map(prepare, data), schema)
  File "/Users/gberger/Projects/spark/python/pyspark/sql/session.py", line 409, in _createFromLocal
    struct = self._inferSchemaFromList(data, names=schema)
  File "/Users/gberger/Projects/spark/python/pyspark/sql/session.py", line 341, in _inferSchemaFromList
    schema = reduce(_merge_type, (_infer_schema(row, names) for row in data))
  File "/Users/gberger/Projects/spark/python/pyspark/sql/types.py", line 1132, in _merge_type
    for f in a.fields]
  File "/Users/gberger/Projects/spark/python/pyspark/sql/types.py", line 1125, in _merge_type
    raise TypeError(new_msg("Can not merge type %s and %s" % (type(a), type(b))))
TypeError: field _2: Can not merge type <class 'pyspark.sql.types.LongType'> and <class 'pyspark.sql.types.StringType'>

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@HyukjinKwon just pushed the elif branch change that I talked about above, please see if it is suitable

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yup, will take another look soon anyway.

items = zip(names, row)

elif hasattr(row, "__dict__"): # object
Expand All @@ -1108,19 +1109,27 @@ def _has_nulltype(dt):
return isinstance(dt, NullType)


def _merge_type(a, b):
def _merge_type(a, b, name=None):
if name is None:
new_msg = lambda msg: msg
new_name = lambda n: "field %s" % n
else:
new_msg = lambda msg: "%s: %s" % (name, msg)
new_name = lambda n: "field %s in %s" % (n, name)

if isinstance(a, NullType):
return b
elif isinstance(b, NullType):
return a
elif type(a) is not type(b):
# TODO: type cast (such as int -> long)
raise TypeError("Can not merge type %s and %s" % (type(a), type(b)))
raise TypeError(new_msg("Can not merge type %s and %s" % (type(a), type(b))))

# same type
if isinstance(a, StructType):
nfs = dict((f.name, f.dataType) for f in b.fields)
fields = [StructField(f.name, _merge_type(f.dataType, nfs.get(f.name, NullType())))
fields = [StructField(f.name, _merge_type(f.dataType, nfs.get(f.name, NullType()),
name=new_name(f.name)))
for f in a.fields]
names = set([f.name for f in fields])
for n in nfs:
Expand All @@ -1129,11 +1138,12 @@ def _merge_type(a, b):
return StructType(fields)

elif isinstance(a, ArrayType):
return ArrayType(_merge_type(a.elementType, b.elementType), True)
return ArrayType(_merge_type(a.elementType, b.elementType,
name='element in array %s' % name), True)

elif isinstance(a, MapType):
return MapType(_merge_type(a.keyType, b.keyType),
_merge_type(a.valueType, b.valueType),
return MapType(_merge_type(a.keyType, b.keyType, name='key of map %s' % name),
_merge_type(a.valueType, b.valueType, name='value of map %s' % name),
True)
else:
return a
Expand Down