-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-21365][PYTHON] Deduplicate logics parsing DDL type/schema definition #18590
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
Conversation
| import org.apache.spark.sql.types.DataType | ||
|
|
||
| private[sql] object PythonSQLUtils { | ||
| def parseDataType(typeText: String): DataType = CatalystSqlParser.parseDataType(typeText) |
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.
Without this, I should do something like ...
getattr(getattr(sc._jvm.org.apache.spark.sql.catalyst.parser, "CatalystSqlParser$"), "MODULE$").parseDataType("a")
| def test_parse_datatype_string(self): | ||
| from pyspark.sql.types import _all_atomic_types, _parse_datatype_string | ||
| for k, t in _all_atomic_types.items(): | ||
| if t != NullType: |
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.
So, if I haven't missed anything, this PR drops the support the type parsing null. I guess it is almost seldom that we explicitly set the type with null. Also, IIRC, we will support NullType via void (SPARK-20680) soon as a workaround.
|
cc @cloud-fan, @felixcheung and @zero323 who I remember I talked with about similar issues before. |
|
Add |
|
Test build #79470 has finished for PR 18590 at commit
|
|
retest this please |
|
Test build #79481 has finished for PR 18590 at commit
|
|
Test build #79492 has finished for PR 18590 at commit
|
| return from_ddl_datatype(s) | ||
| except: | ||
| try: | ||
| # For backwards compatibility, "fieldname: datatype, fieldname: datatype" case. |
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.
won't fieldname: datatype, fieldname: datatype be parsed as DDL schema?
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.
I tested few cases but it looks not:
scala> StructType.fromDDL("a struct<a: INT, b: STRING>")
res5: org.apache.spark.sql.types.StructType = StructType(StructField(a,StructType(StructField(a,IntegerType,true), StructField(b,StringType,true)),true))
scala> StructType.fromDDL("a INT, b STRING")
res6: org.apache.spark.sql.types.StructType = StructType(StructField(a,IntegerType,true), StructField(b,StringType,true))
scala> StructType.fromDDL("a: INT, b: STRING")
org.apache.spark.sql.catalyst.parser.ParseException:
extraneous input ':' expecting ...
holdenk
left a comment
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.
Thanks for working on this, unifiying the parsing logic to be Scala side seems like a good idea.
| else func.__class__.__name__) | ||
|
|
||
| @property | ||
| def returnType(self): |
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.
We have pretty similar logic bellow, would it make sense to think about if there is a nicer more general way to handle these delayed iniatilization classes?
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.
hmm.. I tried several ways I could think at my best but I could not figure out ...
|
LGTM, merging to master! |
What changes were proposed in this pull request?
This PR deals with four points as below:
Reuse existing DDL parser APIs rather than reimplementing within PySpark
Support DDL formatted string,
field type, field type.Support case-insensitivity for parsing.
Support nested data types as below:
Before
After
How was this patch tested?