-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-25798][PYTHON] Internally document type conversion between Pandas data and SQL types in Pandas UDFs #22795
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -3023,6 +3023,42 @@ def pandas_udf(f=None, returnType=None, functionType=None): | |
| conversion on returned data. The conversion is not guaranteed to be correct and results | ||
| should be checked for accuracy by users. | ||
| """ | ||
|
|
||
| # The following table shows most of Pandas data and SQL type conversions in Pandas UDFs that | ||
| # are not yet visible to the user. Some of behaviors are buggy and might be changed in the near | ||
| # future. The table might have to be eventually documented externally. | ||
| # Please see SPARK-25798's PR to see the codes in order to generate the table below. | ||
| # | ||
| # +-----------------------------+----------------------+----------+-------+--------+--------------------+--------------------+--------+---------+---------+---------+------------+------------+------------+-----------------------------------+-----------------------------------------------------+-----------------+--------------------+-----------------------------+-------------+-----------------+------------------+-----------+--------------------------------+ # noqa | ||
| # |SQL Type \ Pandas Value(Type)|None(object(NoneType))|True(bool)|1(int8)|1(int16)| 1(int32)| 1(int64)|1(uint8)|1(uint16)|1(uint32)|1(uint64)|1.0(float16)|1.0(float32)|1.0(float64)|1970-01-01 00:00:00(datetime64[ns])|1970-01-01 00:00:00-05:00(datetime64[ns, US/Eastern])|a(object(string))| 1(object(Decimal))|[1 2 3](object(array[int32]))|1.0(float128)|(1+0j)(complex64)|(1+0j)(complex128)|A(category)|1 days 00:00:00(timedelta64[ns])| # noqa | ||
| # +-----------------------------+----------------------+----------+-------+--------+--------------------+--------------------+--------+---------+---------+---------+------------+------------+------------+-----------------------------------+-----------------------------------------------------+-----------------+--------------------+-----------------------------+-------------+-----------------+------------------+-----------+--------------------------------+ # noqa | ||
| # | boolean| None| True| True| True| True| True| True| True| True| True| False| False| False| False| False| X| X| X| False| False| False| X| False| # noqa | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Could you add a note that bool coversion is not accurate and being addressed in ARROW-3428?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think it's fine .. many conversions here look buggy, for instance,
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Sure, no prob. I just don't want to us to forget to update this and then it might look confusing and look like these are expected results |
||
| # | tinyint| None| 1| 1| 1| 1| 1| X| X| X| X| 1| 1| 1| X| X| X| X| X| X| X| X| 0| X| # noqa | ||
| # | smallint| None| 1| 1| 1| 1| 1| 1| X| X| X| 1| 1| 1| X| X| X| X| X| X| X| X| X| X| # noqa | ||
| # | int| None| 1| 1| 1| 1| 1| 1| 1| X| X| 1| 1| 1| X| X| X| X| X| X| X| X| X| X| # noqa | ||
| # | bigint| None| 1| 1| 1| 1| 1| 1| 1| 1| X| 1| 1| 1| 0| 18000000000000| X| X| X| X| X| X| X| X| # noqa | ||
| # | float| None| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| X| X| X|1.401298464324817...| X| X| X| X| X| X| # noqa | ||
| # | double| None| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| X| X| X| X| X| X| X| X| X| X| # noqa | ||
| # | date| None| X| X| X|datetime.date(197...| X| X| X| X| X| X| X| X| datetime.date(197...| X| X| X| X| X| X| X| X| X| # noqa | ||
| # | timestamp| None| X| X| X| X|datetime.datetime...| X| X| X| X| X| X| X| datetime.datetime...| datetime.datetime...| X| X| X| X| X| X| X| X| # noqa | ||
| # | string| None| u''|u'\x01'| u'\x01'| u'\x01'| u'\x01'| u'\x01'| u'\x01'| u'\x01'| u'\x01'| u''| u''| u''| X| X| u'a'| X| X| u''| u''| u''| X| X| # noqa | ||
| # | decimal(10,0)| None| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| Decimal('1')| X| X| X| X| X| X| # noqa | ||
| # | array<int>| None| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| [1, 2, 3]| X| X| X| X| X| # noqa | ||
| # | map<string,int>| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| # noqa | ||
| # | struct<_1:int>| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| # noqa | ||
|
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @BryanCutler, I think we explicitly don't support map and struct here. I hope we can just leave it since supported/unuspported types are externally documented (and it's written in the doc above as well).
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. No you can leave them, they will hopefully be supported eventually. I was just wondering if it's possible to differentiate the
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Oh I see. Hm, I think it's kind of difficult to automatically distinguish supported or unsupported programmatically.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I was thinking some of them might throw an |
||
| # | binary| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| # noqa | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: we can get rid of binary line here. There is a note below. |
||
| # +-----------------------------+----------------------+----------+-------+--------+--------------------+--------------------+--------+---------+---------+---------+------------+------------+------------+-----------------------------------+-----------------------------------------------------+-----------------+--------------------+-----------------------------+-------------+-----------------+------------------+-----------+--------------------------------+ # noqa | ||
| # | ||
| # Note: DDL formatted string is used for 'SQL Type' for simplicity. This string can be | ||
| # used in `returnType`. | ||
| # Note: The values inside of the table are generated by `repr`. | ||
| # Note: Python 2 is used to generate this table since it is used to check the backward | ||
| # compatibility often in practice. | ||
| # Note: Pandas 0.19.2 and PyArrow 0.9.0 are used. | ||
| # Note: Timezone is Singapore timezone. | ||
| # Note: 'X' means it throws an exception during the conversion. | ||
| # Note: 'binary' type is only supported with PyArrow 0.10.0+ (SPARK-23555). | ||
|
|
||
| # decorator @pandas_udf(returnType, functionType) | ||
| is_decorator = f is None or isinstance(f, (str, DataType)) | ||
|
|
||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nice code for this table generating! maybe the right place of this table could be https://github.com/apache/spark/blob/987f386588de7311b066cf0f62f0eed64d4aa7d7/docs/sql-pyspark-pandas-with-arrow.md#usage-notes
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's not supposed to be exposed and visible to users yet as I said in the comments. Let's keep it internal for now.