-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-17764][SQL] Add to_json supporting to convert nested struct column to JSON string
#15354
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
|
cc @marmbrus Could you take a look please? |
|
Test build #66356 has finished for PR 15354 at commit
|
|
Test build #66372 has finished for PR 15354 at commit
|
to_json supporting to convert nested struct column to JSON stringto_json supporting to convert nested struct column to JSON string
|
Ah, this was a known issue |
to_json supporting to convert nested struct column to JSON stringto_json supporting to convert nested struct column to JSON string
|
Test build #66383 has finished for PR 15354 at commit
|
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 :) Just did a quick look at the Python parts and some minor style changes that we might want to consider :)
python/pyspark/sql/functions.py
Outdated
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.
This is super minor, but there is a pretty consistent pattern for all of the other functions here (including from_json), it might be good to follow that same pattern for consistencies sake since there isn't an obvious reason why that wouldn't work here.
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.
@holdenk Thank you for your comment. Could you please a bit elaborate this comment? I am a bit not sure on what to fix.
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.
actually nvm my original comment, the more I look at this file the less it seems the pattern is overly consistent and this same pattern is done elsewhere within the file.
python/pyspark/sql/functions.py
Outdated
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.
Would :param col: name of column containing the struct maybe be more consistent with the other pydocs for the functions? (I only skimmed a few though so if its the other way around thats cool).
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.
Sure, let me try to double check other comments as well.
|
Test build #66484 has finished for PR 15354 at commit
|
marmbrus
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.
Overall, looks pretty good. What do you think about failing earlier and more obviously when there are unsupported datatypes?
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 realize this is a little different than from_json, but it seems it would be better to eagerly throw an AnalysisException to say the schema contains an unsupported type. We know that ahead of time, and otherwise its kind of mysterious why all the values come out as null.
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.
Sure, that makes sense. Thanks.
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 would like to leave a note. In case of CSV we are verifying the types before actually running tasks but for JSON it is not doing this. So, I made this SparkSQLJsonProcessingException which is technically a RuntimeException. However, if you want me to fix it here (adding a logic to verify the schema ahead) I will definitely do this here together.
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.
Yeah, I think it makes more sense to add a static check for this case. We know all of the types that we are able to handle. For consistency I would also add this to the write.json code path.
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.
@marmbrus Do you mind if I ask it is okay for me to create another JIRA and deal with this problem for JSON/CSV for reading/writing paths in another pr? It seems I should add this logics separatlely from JacksonGenerator instance (as it seems initiated in tasks and it is used in DataSet.toJSON, StructToJson and write.json and therefore, it seems I should add each separate test for each..)
|
Test build #66526 has finished for PR 15354 at commit
|
|
Test build #66523 has finished for PR 15354 at commit
|
|
Test build #66525 has finished for PR 15354 at commit
|
|
retest this please |
|
Test build #66558 has finished for PR 15354 at commit
|
ecdac76 to
38d89a6
Compare
|
Test build #67012 has finished for PR 15354 at commit
|
|
hi - where are we on this? |
38d89a6 to
bbbfaff
Compare
|
@felixcheung Thank you for pinging! @marmbrus Do you please mind if I handle checking the schema ahead for csv/json in read/write in another PR? |
|
Test build #67387 has finished for PR 15354 at commit
|
|
It would be really nice to fail in analysis rather than execution. What if it only fails after hours of computation? As a user I'd be upset. I'm also concerned they will think it's a spark bug. |
|
@marmbrus Sure (I didn't mean I am not going to do this..), I just handled the case in this PR for BTW, I would like to note that there are the same problems in other JSON related functionalities. For example, I might have to add override def checkInputDataTypes(): TypeCheckResult = {
...
JacksonUtils.verifySchema(child.dataType.asInstanceOf[StructType])
...
}for |
|
Test build #67407 has finished for PR 15354 at commit
|
|
Test build #67408 has finished for PR 15354 at commit
|
|
Test build #67409 has finished for PR 15354 at commit
|
marmbrus
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.
Only minor comments. Thanks for working on this!
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.
does not support to process is a little hard to parse. Maybe Unable to convert column ${name} of type ${dataType.simpleString} to JSON.
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 would avoid this change since its throwing a private exception type to the user now.
|
@marmbrus Thank you so much. |
|
Test build #67622 has finished for PR 15354 at commit
|
|
retest this please |
|
Test build #67630 has finished for PR 15354 at commit
|
|
Let me take a look into this deeper if some same tests constantly fail. |
|
retest this please |
|
Test build #67633 has finished for PR 15354 at commit
|
|
retest this please |
|
It seems the test is not related with this PR. |
|
Test build #67647 has finished for PR 15354 at commit
|
|
looks good, should we clarify output JSON is in JSON Lines format? http://jsonlines.org/ |
|
Oh nvm. I left a useless comment and removed it. It seems it'd be better to mention. |
791f802 to
4d69ab2
Compare
|
Test build #67694 has finished for PR 15354 at commit
|
marmbrus
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.
I think we are getting close. Just one more large comment and some doc changes.
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 don't think that this case really follows "JSON lines". It is a string inside of a larger dataframe. There are no newlines involved.
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.
Sorry, one final comment as I'm looking at this more closely. I don't think we should use exceptions for control flow in the common case. Specifically, verifySchema should work the same way as acceptsType above and return a boolean.
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.
Ah, yes, makes sense but if verifySchema returns a boolean, we could not find which field and type are problematic.
Maybe, I can make do one of the below:
- make this logic in
verifySchemaintocheckInputDataTypes - make
verifySchemareturn the unsupported fields. and types. - Just fix the exception message without the information of unsupported fields and types.
If you pick one, I will follow (or please let me know if there is a better way)!
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.
Oh, I see. It is for the better message. I guess its probably not worth the time to refactor in that case.
|
Test build #67736 has finished for PR 15354 at commit
|
b76a08e to
971d1c0
Compare
|
Test build #67893 has finished for PR 15354 at commit
|
|
retest this please |
|
Test build #67902 has finished for PR 15354 at commit
|
|
Thanks, I'm going to merge this to master. |
|
Thank you for merging this! |
…column to JSON string
## What changes were proposed in this pull request?
This PR proposes to add `to_json` function in contrast with `from_json` in Scala, Java and Python.
It'd be useful if we can convert a same column from/to json. Also, some datasources do not support nested types. If we are forced to save a dataframe into those data sources, we might be able to work around by this function.
The usage is as below:
``` scala
val df = Seq(Tuple1(Tuple1(1))).toDF("a")
df.select(to_json($"a").as("json")).show()
```
``` bash
+--------+
| json|
+--------+
|{"_1":1}|
+--------+
```
## How was this patch tested?
Unit tests in `JsonFunctionsSuite` and `JsonExpressionsSuite`.
Author: hyukjinkwon <[email protected]>
Closes apache#15354 from HyukjinKwon/SPARK-17764.
What changes were proposed in this pull request?
This PR proposes to add
to_jsonfunction in contrast withfrom_jsonin Scala, Java and Python.It'd be useful if we can convert a same column from/to json. Also, some datasources do not support nested types. If we are forced to save a dataframe into those data sources, we might be able to work around by this function.
The usage is as below:
How was this patch tested?
Unit tests in
JsonFunctionsSuiteandJsonExpressionsSuite.