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

Conversation

gberger
Copy link

@gberger gberger commented Nov 21, 2017

What changes were proposed in this pull request?

It provides a better error message when doing spark_session.createDataFrame(pandas_df) with no schema and an error occurs in the schema inference due to incompatible types.

The Pandas column names are propagated down and the error message mentions which column had the merging error.

https://issues.apache.org/jira/browse/SPARK-22566

How was this patch tested?

Manually in the ./bin/pyspark console, and with new tests: ./python/run-tests

screen shot 2017-11-21 at 13 29 49

I state that the contribution is my original work and that I license the work to the Apache Spark project under the project’s open source license.

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)))
if name is not None:
Copy link
Contributor

Choose a reason for hiding this comment

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

Easier to read as:

if name is None:
    raise TypeError("Can not merge type %s and %s" % (type(a), type(b)))
else:
    raise TypeError("Can not merge type %s and %s in column %s" % (type(a), type(b), name))

@ueshin
Copy link
Member

ueshin commented Nov 22, 2017

ok to test

@SparkQA
Copy link

SparkQA commented Nov 22, 2017

Test build #84095 has finished for PR 19792 at commit 518fdd4.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@ueshin
Copy link
Member

ueshin commented Nov 22, 2017

What if ArrayType or MapType, or deeply nested StructType?

@gberger
Copy link
Author

gberger commented Nov 22, 2017

@ebuildy fixed

@gberger
Copy link
Author

gberger commented Nov 22, 2017

@ueshin

The reason that I modified the case for StructType is that, in session.py#341, for each Pandas DF row we obtain a StructType with StructFields mapping column names to value type; these are reduced with _merge_types.

I do appreciate that it could be the case that a Pandas DF contains lists or dicts as values. I pushed a new commit where the name property gets passed down when we are recursing via ArrayType or MapType.

Here is what it looks like when we use lists or dicts inside a Pandas DF:

>>> df = pd.DataFrame(data={
...     'a': [[1, 2], [3, 4]],
...     'b': [[5, 'hello'], [7, 8]]
... })
>>> sdf = sql.createDataFrame(df)
>>> sdf
DataFrame[a: array<bigint>, b: array<bigint>]
>>> sdf.show()
+------+---------+
|     a|        b|
+------+---------+
|[1, 2]|[5, null]|
|[3, 4]|   [7, 8]|
+------+---------+
>>> df = pd.DataFrame(data={
...     'a': [{1: 2}, {3: 4}],
...     'b': [{5: 'hello'}, {7: 8}]
... })
>>> sdf = sql.createDataFrame(df)
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/Users/gberger/Projects/spark/python/pyspark/sql/context.py", line 354, in createDataFrame
    return self.sparkSession.createDataFrame(data, schema, samplingRatio, verifySchema)
  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 1128, in _merge_type
    for f in a.fields]
  File "/Users/gberger/Projects/spark/python/pyspark/sql/types.py", line 1128, in <listcomp>
    for f in a.fields]
  File "/Users/gberger/Projects/spark/python/pyspark/sql/types.py", line 1140, in _merge_type
    _merge_type(a.valueType, b.valueType, name=name),
  File "/Users/gberger/Projects/spark/python/pyspark/sql/types.py", line 1122, in _merge_type
    raise TypeError("Can not merge type %s and %s in field '%s'" % (type(a), type(b), name))
TypeError: Can not merge type <class 'pyspark.sql.types.StringType'> and <class 'pyspark.sql.types.LongType'> in field b

@SparkQA
Copy link

SparkQA commented Nov 22, 2017

Test build #84105 has finished for PR 19792 at commit b29434e.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Nov 22, 2017

Test build #84106 has finished for PR 19792 at commit 6aa9963.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds no public classes.

@gberger
Copy link
Author

gberger commented Nov 22, 2017

@ueshin I think this build fail was an outage, can we retest?

Could not load hsdis-amd64.so; library not loadable; PrintAssembly is disabled

@ueshin
Copy link
Member

ueshin commented Nov 22, 2017

Jenkins, retest this please.

@SparkQA
Copy link

SparkQA commented Nov 22, 2017

Test build #84108 has finished for PR 19792 at commit 6aa9963.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@ueshin
Copy link
Member

ueshin commented Nov 22, 2017

Can you add tests for this?
Maybe we need some tests for simple case and some nested type cases.
I'd also cc @HyukjinKwon. Thanks.

@HyukjinKwon
Copy link
Member

HyukjinKwon commented Nov 23, 2017

Thanks @ueshin. Yup, +1 for adding some tests.

I just wonder if we could have a similar form of error message (maybe prettier if possible) in type verification. I remember I fixed a similar issue for type verification - #18521 (see the links in "Before" and "After") although this PR includes performance improvement by avoiding type dispatch, for example:

field c in field b in field a: Can not merge type IntegerType and StringType
element in array element in array field a: Can not merge type IntegerType and StringType
key of map field a: Can not merge type IntegerType and StringType
value of map field a: Can not merge type IntegerType and StringType
field a in value of map field a: Can not merge type IntegerType and StringType

Let's make sure there is no performance regression as well (even I was about to make the mistake before).

@HyukjinKwon
Copy link
Member

@gberger, BTW, just to be clear, IIRC the type inference and merging code path here are shared for other data types, for example, dict, namedtuple, row and etc.

@gberger
Copy link
Author

gberger commented Nov 23, 2017

For sure @ueshin, I will add tests.

@HyukjinKwon understood!
How do we go about testing performance regression?

@HyukjinKwon
Copy link
Member

I think you can add a new test case somewhere around here - https://github.com/apache/spark/blob/master/python/pyspark/sql/tests.py#L1724 maybe dealing with some combinations of StructType, MapType, ArrayType and one of non-nested type (e.g., IntegerType), checking if the error message is really improved.

For running tests, you could refer https://spark.apache.org/docs/latest/building-spark.html#pyspark-tests-with-maven.

I personally test them locally first and then add and run some tests. Maybe running ./python/run-tests --modules=pyspark-sql could be enough. Also, make sure running ./dev/lint-python to check if it complies PEP8 and etc.

When you finish them, commit and push which will trigger the build via Jenkins here. See also "Pull Request" in http://spark.apache.org/contributing.html

@HyukjinKwon
Copy link
Member

HyukjinKwon commented Nov 23, 2017

D'oh, you mean performance regression test. Manual tests should be fine. When you share some codes you ran, maybe we can double check. I also did this manually in #18521.

@gberger
Copy link
Author

gberger commented Nov 23, 2017

Hey all,

Error message

I revamped the error message and made it "recursive" similar to @HyukjinKwon. Here's an example:

>>> _merge_type(
...                 StructType([StructField("f1", ArrayType(MapType(StringType(), LongType())))]),
...                 StructType([StructField("f1", ArrayType(MapType(DoubleType(), LongType())))])
...             )
Traceback (most recent call last):
  File "<stdin>", line 3, in <module>
  File "/Users/gberger/Projects/spark/python/pyspark/sql/types.py", line 1129, in _merge_type
    for f in a.fields]
  File "/Users/gberger/Projects/spark/python/pyspark/sql/types.py", line 1129, in <listcomp>
    for f in a.fields]
  File "/Users/gberger/Projects/spark/python/pyspark/sql/types.py", line 1137, in _merge_type
    return ArrayType(_merge_type(a.elementType, b.elementType, field=field+'.arrayElement'), True)
  File "/Users/gberger/Projects/spark/python/pyspark/sql/types.py", line 1140, in _merge_type
    return MapType(_merge_type(a.keyType, b.keyType, field=field+'.mapKey'),
  File "/Users/gberger/Projects/spark/python/pyspark/sql/types.py", line 1122, in _merge_type
    raise TypeError("%s: Can not merge type %s and %s" % (field, type(a), type(b)))
TypeError: .structField("f1").arrayElement.mapKey: Can not merge type <class 'pyspark.sql.types.StringType'> and <class 'pyspark.sql.types.DoubleType'>

Happy to iterate on the exact formatting or wording of the path shown.

Tests

I wrote a bunch of tests too, hope they are comprehensive enough but happy to add more if not. @ueshin

Benchmark

It seems that the time it takes for a nested _merge_type on my machine has increased for ~2.75 microseconds to ~2.85 microseconds, around a 3% increase.

This can be attributed to the string concatenation that goes on every time _merge_type goes one level down from a StructType, ArrayType or MapType. I'm not sure if there's a better way to propagate this information down the stack, maybe a tuple?

Code used:

from pyspark.sql.types import *
from pyspark.sql.types import _merge_type
import time


def test_f():
    return _merge_type(
        StructType([StructField("f1", ArrayType(MapType(StringType(), LongType())))]),
        StructType([StructField("f1", ArrayType(MapType(StringType(), LongType())))])
    )

def timing(f):
    def wrap(*args):
        time1 = time.time()
        for __ in range(100000):
            ret = f(*args)
        time2 = time.time()
        print('took %0.3f ms' % ((time2-time1)*1000.0))
        return ret
    return wrap

for _ in range(10):
    timing(test_f)()

Before:

took 2701.337 ms
took 2905.867 ms
took 2725.119 ms
took 2796.098 ms
took 2718.981 ms
took 2773.560 ms
took 2717.995 ms
took 2796.466 ms
took 2716.173 ms
took 2744.121 ms

After:

took 2865.038 ms
took 2836.403 ms
took 2871.871 ms
took 2827.625 ms
took 2820.170 ms
took 2873.976 ms
took 2833.609 ms
took 2909.599 ms
took 3162.108 ms
took 2940.864 ms

@SparkQA
Copy link

SparkQA commented Nov 23, 2017

Test build #84139 has finished for PR 19792 at commit 61ace28.

  • This patch fails Python style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@gberger
Copy link
Author

gberger commented Nov 23, 2017

Maybe a more performant way to do the path in the error message would be to propagate it up the stack via try/catching the errors and adding the paths as it goes.

But this way seems really weird to me...

    elif isinstance(a, ArrayType):
        try: 
            return ArrayType(_merge_type(a.elementType, b.elementType), True)
        catch TypeError as e:
            raise TypeError("arrayElement." + str(e))

@SparkQA
Copy link

SparkQA commented Nov 23, 2017

Test build #84140 has finished for PR 19792 at commit 3346a6c.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

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, r'structField\("f1"\)'):
Copy link
Member

Choose a reason for hiding this comment

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

nit: We don't need r prefix for each regex for assertRaisesRegexp.

Copy link
Author

Choose a reason for hiding this comment

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

But then double backslashes? :(

Copy link
Member

Choose a reason for hiding this comment

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

Hmm, I don't think we need double backslashes here.

Copy link
Author

Choose a reason for hiding this comment

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

Oh, you are right. Fixed

@@ -1108,19 +1109,23 @@ def _has_nulltype(dt):
return isinstance(dt, NullType)


def _merge_type(a, b):
def _merge_type(a, b, path=''):
Copy link
Member

Choose a reason for hiding this comment

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

Should we have the default path name for the case we don't have names?
Otherwise, the error message would be like TypeError: .arrayElement: Can not merge type <class 'pyspark.sql.types.LongType'> and <class 'pyspark.sql.types.DoubleType'>. WDYT? @HyukjinKwon

Copy link
Member

Choose a reason for hiding this comment

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

Yup, I agree with it for the current status ..

it's kind of difficult to come up with a good message format in such cases. To me, I actually kind of gave up a pretty format and just chose prose before (in the PR #18521).

I still don't have a good idea to show the nested structure in the error message to be honest. I was thinking one of prose, kind of piece of codes (like schema['f1'].dataType.elementType.keyType instead of #19792 (comment)), or somehow pretty one like printSchema() ... ?

Maybe, I am thinking of referring other formats in Spark or somewhere like Pandas or piece of codes for now.

So, to cut it short, here are what are on my mind for the example of #19792 (comment):

  1. Prose
TypeError: key in map field in array element in field f1: Can not blabla
  1. Piece of codes with StructType
TypeError: schema(?)['f1'].dataType.elementType.keyType: Can not blabla
TypeError: struct(?)['f1'].dataType.elementType.keyType: Can not blabla
  1. printSchema()
TypeError: 
root
 |-- f1: array (nullable = false)
 |    |-- element: map (containsNull = false)
 |    |    |-- key: integer*
 |    |    |-- value: long (valueContainsNull = false)
: *Can not blabla

Copy link
Member

Choose a reason for hiding this comment

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

I really don't have a strong opinion on this. @ueshin do you maybe have a preference or another option, or are you okay with the current format in general?

Copy link
Member

Choose a reason for hiding this comment

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

Maybe, just printing the string in SQL could be an option too to be consistent .. it's a bit messy for deeply nested schema though.:

TypeError: struct<f1:array<map<int,string>>> and struct<f1:array<map<int,int>>> Can not blabla

Copy link
Member

Choose a reason for hiding this comment

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

I prefer the format we can know the path to the error and I guess we don't need the pretty one like the printSchema().
Maybe following the #18521 format is good enough, and the current one is good as well but I wanted something in front of the path string instead of starting with ..

Copy link
Member

Choose a reason for hiding this comment

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

Yup, looks we need something ahead anyway. To me, I am okay with any format.

I think we could fix later again. Just I wondered if you have a preference.

Copy link
Author

Choose a reason for hiding this comment

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

Hi folks,

Great options proposed by @HyukjinKwon - though I did not comprehend what was the conclusion of the discussion with @ueshin :)

Which format should we employ? And do you want me to use this format right now or is it something you'll fix later?

Thanks!

Copy link
Member

Choose a reason for hiding this comment

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

Well, can you follow the #18521 format for now? Thanks!

@SparkQA
Copy link

SparkQA commented Nov 28, 2017

Test build #84272 has finished for PR 19792 at commit 8665115.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@@ -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

@SparkQA
Copy link

SparkQA commented Dec 6, 2017

Test build #84558 has finished for PR 19792 at commit 5131db2.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Member

@HyukjinKwon HyukjinKwon left a comment

Choose a reason for hiding this comment

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

Latest changes LGTM except for one comment.

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.

>>> 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].

@gberger
Copy link
Author

gberger commented Dec 11, 2017

Good catch @HyukjinKwon! I reverted those changes and added a test to cover this regression.

@SparkQA
Copy link

SparkQA commented Dec 11, 2017

Test build #84708 has finished for PR 19792 at commit 44a1879.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@@ -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.

@SparkQA
Copy link

SparkQA commented Dec 12, 2017

Test build #84764 has finished for PR 19792 at commit 404fdbb.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

if names is None:
names = ['_%d' % i for i in range(1, len(row) + 1)]
elif len(names) < len(row):
names = names[:]
Copy link
Member

Choose a reason for hiding this comment

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

Hm .. why we do this? to copy?

Copy link
Author

Choose a reason for hiding this comment

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

Yes, I did not want to modify the original list since .extend is an in-place operation. However, session.py#602 already creates a copy of the list passed by the user, so this copying in _infer_schema is actually not necessary. Removing now.

@SparkQA
Copy link

SparkQA commented Dec 14, 2017

Test build #84918 has finished for PR 19792 at commit 6d171dd.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member

Will take another look soon tomorrow. Sorry that is getting delayed again and again but I just realised this code path is a little bit tricky ..

@gberger
Copy link
Author

gberger commented Dec 14, 2017

@HyukjinKwon no worries, I understand. We gotta be 100% thorough here. Thanks for the help

@HyukjinKwon
Copy link
Member

ok to test

@SparkQA
Copy link

SparkQA commented Dec 14, 2017

Test build #84932 has finished for PR 19792 at commit 6d171dd.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member

retest this please

@SparkQA
Copy link

SparkQA commented Jan 6, 2018

Test build #85752 has finished for PR 19792 at commit 6d171dd.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds no public classes.

@ueshin
Copy link
Member

ueshin commented Jan 8, 2018

Jenkins, retest this please.

@HyukjinKwon
Copy link
Member

To be clear (as I was reviewing this too), I am okay with going ahead, @ueshin if this looks good to you.

@ueshin
Copy link
Member

ueshin commented Jan 8, 2018

@HyukjinKwon Thanks, I'll take another look soon.

@ueshin
Copy link
Member

ueshin commented Jan 8, 2018

LGTM, pending Jenkins.

@SparkQA
Copy link

SparkQA commented Jan 8, 2018

Test build #85784 has finished for PR 19792 at commit 6d171dd.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@ueshin
Copy link
Member

ueshin commented Jan 8, 2018

Thanks! merging to master/2.3.

asfgit pushed a commit that referenced this pull request Jan 8, 2018
…s to Spark DF conversion

## What changes were proposed in this pull request?

It provides a better error message when doing `spark_session.createDataFrame(pandas_df)` with no schema and an error occurs in the schema inference due to incompatible types.

The Pandas column names are propagated down and the error message mentions which column had the merging error.

https://issues.apache.org/jira/browse/SPARK-22566

## How was this patch tested?

Manually in the `./bin/pyspark` console, and with new tests: `./python/run-tests`

<img width="873" alt="screen shot 2017-11-21 at 13 29 49" src="https://user-images.githubusercontent.com/3977115/33080121-382274e0-cecf-11e7-808f-057a65bb7b00.png">

I state that the contribution is my original work and that I license the work to the Apache Spark project under the project’s open source license.

Author: Guilherme Berger <[email protected]>

Closes #19792 from gberger/master.

(cherry picked from commit 3e40eb3)
Signed-off-by: Takuya UESHIN <[email protected]>
@asfgit asfgit closed this in 3e40eb3 Jan 8, 2018
@gberger
Copy link
Author

gberger commented Jan 10, 2018

Great! Thanks all

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

6 participants