Skip to content

Conversation

@HyukjinKwon
Copy link
Member

@HyukjinKwon HyukjinKwon commented Jul 10, 2017

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

    >>> spark.createDataFrame([[[1]]], "struct<a: struct<b: int>>").show()
    ...
    ValueError: The strcut field string format is: 'field_name:field_type', but got: a: struct<b: int>
    
    >>> spark.createDataFrame([[[1]]], "a: struct<b: int>").show()
    ...
    ValueError: The strcut field string format is: 'field_name:field_type', but got: a: struct<b: int>
    
    >>> spark.createDataFrame([[1]], "a int").show()
    ...
    ValueError: Could not parse datatype: a int
    

    After

    >>> spark.createDataFrame([[[1]]], "struct<a: struct<b: int>>").show()
    +---+
    |  a|
    +---+
    |[1]|
    +---+
    
    >>> spark.createDataFrame([[[1]]], "a: struct<b: int>").show()
    +---+
    |  a|
    +---+
    |[1]|
    +---+
    
    >>> spark.createDataFrame([[1]], "a int").show()
    +---+
    |  a|
    +---+
    |  1|
    +---+
    

How was this patch tested?

import org.apache.spark.sql.types.DataType

private[sql] object PythonSQLUtils {
def parseDataType(typeText: String): DataType = CatalystSqlParser.parseDataType(typeText)
Copy link
Member Author

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:
Copy link
Member Author

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.

@HyukjinKwon
Copy link
Member Author

HyukjinKwon commented Jul 10, 2017

cc @cloud-fan, @felixcheung and @zero323 who I remember I talked with about similar issues before.

@felixcheung
Copy link
Member

Add
@gatorsmile
@holdenk

@SparkQA
Copy link

SparkQA commented Jul 10, 2017

Test build #79470 has finished for PR 18590 at commit 3472873.

  • This patch fails SparkR unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member Author

retest this please

@SparkQA
Copy link

SparkQA commented Jul 11, 2017

Test build #79481 has finished for PR 18590 at commit 3472873.

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

@SparkQA
Copy link

SparkQA commented Jul 11, 2017

Test build #79492 has finished for PR 18590 at commit 9d857e6.

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

return from_ddl_datatype(s)
except:
try:
# For backwards compatibility, "fieldname: datatype, fieldname: datatype" case.
Copy link
Contributor

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?

Copy link
Member Author

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

Copy link
Contributor

@holdenk holdenk left a 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):
Copy link
Contributor

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?

Copy link
Member Author

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

@cloud-fan
Copy link
Contributor

LGTM, merging to master!

@asfgit asfgit closed this in ebc124d Jul 11, 2017
@HyukjinKwon HyukjinKwon deleted the deduplicate-python-ddl branch January 2, 2018 03:41
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.

5 participants