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

[deprecated] [SPARK-5821] [SPARK-5746] [SQL] JSON external data source INSERT operation improvements #4607

Closed
wants to merge 9 commits into from

Conversation

yanboliang
Copy link
Contributor

This PR was deprecated, please move to #4610 .
JSON external data source INSERT operation improvements and bug fix:
1, The path in "CREATE TABLE AS SELECT" must be a directory. Because in this scenario we need to write or append files to the existed table, underlying directory is more reasonable for append operation, authentication and authorization.
For SPARK-5821, if we don't have write permission for the parent directory, the CTAS command will failure.
Another reason is that we can't append to HDFS files which represent RDD, if we want to implement append semantics, we need new files and add to a specific directory.
2, New INSERT OVERWRITE implementation.
First insert the new generated table to a temporary directory which named as "_temporary" under the path directory. After insert finished, we deleted the original files. At last we rename "_temporary" for "data".
This can fix the bug which mentioned at SPARK-5746.
3, Why to rename "_temporary" for "data" rather than move all files in "_temporary" to path and then delete "_temporary"? Because that spark RDD.saveAsTextFile(path) related operation will store the whole RDD to HDFS files which named as "part-* " like files under the path. If the original files were produced by this mean, and then we use "INSERT" without overwrite, the new generated table files are also named as "part-* " which will produce corrupted table.

This is the initial draft and need optimization. Looking forward your opinions and comments.

@SparkQA
Copy link

SparkQA commented Feb 14, 2015

Test build #27494 has started for PR 4607 at commit 0812dd1.

  • This patch merges cleanly.

@yanboliang yanboliang changed the title [SPARK-5821] [SPARK-5746] [SQL] JSON data source refactor initial draft [SPARK-5821] [SPARK-5746] [SQL] JSON data source refactor Feb 14, 2015
@SparkQA
Copy link

SparkQA commented Feb 14, 2015

Test build #27494 has finished for PR 4607 at commit 0812dd1.

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

@AmplabJenkins
Copy link

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27494/
Test FAILed.

@yanboliang yanboliang changed the title [SPARK-5821] [SPARK-5746] [SQL] JSON data source refactor [SPARK-5821] [SPARK-5746] [SQL] JSON external data source INSERT operation improvements Feb 14, 2015
@SparkQA
Copy link

SparkQA commented Feb 14, 2015

Test build #27495 has started for PR 4607 at commit 29e138a.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Feb 14, 2015

Test build #27495 has finished for PR 4607 at commit 29e138a.

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

@AmplabJenkins
Copy link

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27495/
Test FAILed.

gasparms and others added 6 commits February 14, 2015 20:10
…ed language

Currently, Spark Streaming Programming Guide after updateStateByKey  explanation links to file stateful_network_wordcount.py and note "For the complete Scala code ..." for any language tab selected. This is an incoherence.

I've changed the guide and link its pertinent example file. JavaStatefulNetworkWordCount.java example was not created so I added to the commit.

Author: gasparms <gmunoz@stratio.com>

Closes apache#4589 from gasparms/feature/streaming-guide and squashes the following commits:

7f37f89 [gasparms] More style changes
ec202b0 [gasparms] Follow spark style guide
f527328 [gasparms] Improve example to look like scala example
4d8785c [gasparms] Remove throw exception
e92e6b8 [gasparms] Fix incoherence
92db405 [gasparms] Fix Streaming Programming Guide. Change files according the selected language
… eclipse as source folder

   When import the whole project into eclipse as maven project, found that the
   src/main/scala & src/test/scala can not be set as source folder as default
   behavior, so add a "add-source" goal in scala-maven-plugin to let this work.

Author: gli <gli@redhat.com>

Closes apache#4531 from ligangty/addsource and squashes the following commits:

4e4db4c [gli] [IDE] cannot import src/main/scala & src/test/scala into eclipse as source folder
…into JSONDataSourceRefactor

Conflicts:
	sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala
@SparkQA
Copy link

SparkQA commented Feb 15, 2015

Test build #27502 has started for PR 4607 at commit 41307cd.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Feb 15, 2015

Test build #27502 has finished for PR 4607 at commit 41307cd.

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

@AmplabJenkins
Copy link

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27502/
Test FAILed.

@yhuai
Copy link
Contributor

yhuai commented Feb 15, 2015

@yanbohappy Thank you for working on it! For SPARK-5746, I think it is better to add an analysis rule to do a check and throw an exception when you find that users try to write to a table while reading it. Actually, I have been working on it and will have a PR soon. How about we use this PR to address the issue of SPARK-5821 in JSONRelation? Can you also try the parquet data source and see if SPARK-5821 also affects that?

@yhuai
Copy link
Contributor

yhuai commented Feb 15, 2015

Actually, I think we just need to throw an exception if the delete returns false when we try to delete the existing data for OVERWRITE (we also need to make the change at https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala#L70).

@yanboliang
Copy link
Contributor Author

@yhuai Thank you for your reply. Add analysis rule and throw an exception is reasonable and looking forward your PR.
I can address the issue of SPARK-5821, I'm working on another PR #4610 not only resolve SPARK-5821 but also with some improvements.
Could I close this PR and discuss JSON data source improvement related problem at #4610

@yanboliang
Copy link
Contributor Author

@yanboliang yanboliang changed the title [SPARK-5821] [SPARK-5746] [SQL] JSON external data source INSERT operation improvements [deprecated] [SPARK-5821] [SPARK-5746] [SQL] JSON external data source INSERT operation improvements Feb 15, 2015
@yhuai
Copy link
Contributor

yhuai commented Feb 15, 2015

Yes, please close it.

insert is used by INSERT INTO/OVERWRITE and DataFrame.insertInto.

@yanboliang yanboliang closed this Feb 15, 2015
@yanboliang yanboliang deleted the JSONDataSourceRefactor branch February 19, 2015 14:20
LuciferYang pushed a commit that referenced this pull request Jul 9, 2024
### What changes were proposed in this pull request?

This PR amis to upgrade `fasterxml.jackson` from 2.17.1 to 2.17.2.

### Why are the changes needed?

There are some bug fixes about [Databind](https://github.com/FasterXML/jackson-databind):
[#4561](FasterXML/jackson-databind#4561): Issues using jackson-databind 2.17.1 with Reactor (wrt DeserializerCache and ReentrantLock)
[#4575](FasterXML/jackson-databind#4575): StdDelegatingSerializer does not consider a Converter that may return null for a non-null input
[#4577](FasterXML/jackson-databind#4577): Cannot deserialize value of type java.math.BigDecimal from String "3." (not a valid representation)
[#4595](FasterXML/jackson-databind#4595): No way to explicitly disable wrapping in custom annotation processor
[#4607](FasterXML/jackson-databind#4607): MismatchedInput: No Object Id found for an instance of X to assign to property 'id'
[#4610](FasterXML/jackson-databind#4610): DeserializationFeature.FAIL_ON_UNRESOLVED_OBJECT_IDS does not work when used with Polymorphic type handling

The full release note of 2.17.2:
https://github.com/FasterXML/jackson/wiki/Jackson-Release-2.17.2

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Pass GA.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #47241 from wayneguow/upgrade_jackson.

Authored-by: Wei Guo <guow93@gmail.com>
Signed-off-by: yangjie01 <yangjie01@baidu.com>
ericm-db pushed a commit to ericm-db/spark that referenced this pull request Jul 10, 2024
### What changes were proposed in this pull request?

This PR amis to upgrade `fasterxml.jackson` from 2.17.1 to 2.17.2.

### Why are the changes needed?

There are some bug fixes about [Databind](https://github.com/FasterXML/jackson-databind):
[apache#4561](FasterXML/jackson-databind#4561): Issues using jackson-databind 2.17.1 with Reactor (wrt DeserializerCache and ReentrantLock)
[apache#4575](FasterXML/jackson-databind#4575): StdDelegatingSerializer does not consider a Converter that may return null for a non-null input
[apache#4577](FasterXML/jackson-databind#4577): Cannot deserialize value of type java.math.BigDecimal from String "3." (not a valid representation)
[apache#4595](FasterXML/jackson-databind#4595): No way to explicitly disable wrapping in custom annotation processor
[apache#4607](FasterXML/jackson-databind#4607): MismatchedInput: No Object Id found for an instance of X to assign to property 'id'
[apache#4610](FasterXML/jackson-databind#4610): DeserializationFeature.FAIL_ON_UNRESOLVED_OBJECT_IDS does not work when used with Polymorphic type handling

The full release note of 2.17.2:
https://github.com/FasterXML/jackson/wiki/Jackson-Release-2.17.2

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Pass GA.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes apache#47241 from wayneguow/upgrade_jackson.

Authored-by: Wei Guo <guow93@gmail.com>
Signed-off-by: yangjie01 <yangjie01@baidu.com>
jingz-db pushed a commit to jingz-db/spark that referenced this pull request Jul 22, 2024
### What changes were proposed in this pull request?

This PR amis to upgrade `fasterxml.jackson` from 2.17.1 to 2.17.2.

### Why are the changes needed?

There are some bug fixes about [Databind](https://github.com/FasterXML/jackson-databind):
[apache#4561](FasterXML/jackson-databind#4561): Issues using jackson-databind 2.17.1 with Reactor (wrt DeserializerCache and ReentrantLock)
[apache#4575](FasterXML/jackson-databind#4575): StdDelegatingSerializer does not consider a Converter that may return null for a non-null input
[apache#4577](FasterXML/jackson-databind#4577): Cannot deserialize value of type java.math.BigDecimal from String "3." (not a valid representation)
[apache#4595](FasterXML/jackson-databind#4595): No way to explicitly disable wrapping in custom annotation processor
[apache#4607](FasterXML/jackson-databind#4607): MismatchedInput: No Object Id found for an instance of X to assign to property 'id'
[apache#4610](FasterXML/jackson-databind#4610): DeserializationFeature.FAIL_ON_UNRESOLVED_OBJECT_IDS does not work when used with Polymorphic type handling

The full release note of 2.17.2:
https://github.com/FasterXML/jackson/wiki/Jackson-Release-2.17.2

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Pass GA.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes apache#47241 from wayneguow/upgrade_jackson.

Authored-by: Wei Guo <guow93@gmail.com>
Signed-off-by: yangjie01 <yangjie01@baidu.com>
attilapiros pushed a commit to attilapiros/spark that referenced this pull request Oct 4, 2024
### What changes were proposed in this pull request?

This PR amis to upgrade `fasterxml.jackson` from 2.17.1 to 2.17.2.

### Why are the changes needed?

There are some bug fixes about [Databind](https://github.com/FasterXML/jackson-databind):
[apache#4561](FasterXML/jackson-databind#4561): Issues using jackson-databind 2.17.1 with Reactor (wrt DeserializerCache and ReentrantLock)
[apache#4575](FasterXML/jackson-databind#4575): StdDelegatingSerializer does not consider a Converter that may return null for a non-null input
[apache#4577](FasterXML/jackson-databind#4577): Cannot deserialize value of type java.math.BigDecimal from String "3." (not a valid representation)
[apache#4595](FasterXML/jackson-databind#4595): No way to explicitly disable wrapping in custom annotation processor
[apache#4607](FasterXML/jackson-databind#4607): MismatchedInput: No Object Id found for an instance of X to assign to property 'id'
[apache#4610](FasterXML/jackson-databind#4610): DeserializationFeature.FAIL_ON_UNRESOLVED_OBJECT_IDS does not work when used with Polymorphic type handling

The full release note of 2.17.2:
https://github.com/FasterXML/jackson/wiki/Jackson-Release-2.17.2

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Pass GA.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes apache#47241 from wayneguow/upgrade_jackson.

Authored-by: Wei Guo <guow93@gmail.com>
Signed-off-by: yangjie01 <yangjie01@baidu.com>
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.

7 participants