-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-18123][SQL] Use db column names instead of RDD column ones during JDBC Writing #15664
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
|
Test build #67673 has finished for PR 15664 at commit
|
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, this is an approach similar to normalizePartitionSpec in PartitioningUtils.scala.
|
Retest this please. |
|
Test build #68459 has finished for PR 15664 at commit
|
|
Two failures seems to be irrelevant. |
|
Retest this please. |
|
Test build #68481 has finished for PR 15664 at commit
|
|
Test build #69584 has finished for PR 15664 at commit
|
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.
nameMap -> lowercaseNameMap.
|
Thank you for review, @viirya . |
|
This is a bug fix, right? Will review this tomorrow. |
|
Yes, right! Thank you, @gatorsmile ! |
|
Test build #69658 has finished for PR 15664 at commit
|
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 the table schema won't change during inserting all data (or it is possible?). You ask table schema for every insert statement now. Can we do this once in caller side (i.e., savePartition) and reuse the schema then?
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.
Thank you for review, @viirya . I'll try to update like that.
|
Test build #70577 has finished for PR 15664 at commit
|
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 can get the table schema when we checking whether the table exists.
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.
Thank you for review, @gatorsmile .
Yes. That looks great! We can use getSchemaQuery instead of tableExist.
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.
The name resolution should be still controlled by spark.sql.caseSensitive, right?
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.
Yep. I'll fix that.
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.
Can we build the INSERT SQL statement in saveTable based on the schema? No need to prepare the generated statement in saveTable.
|
The PR is updated to
For |
|
Test build #70642 has finished for PR 15664 at commit
|
|
Test build #70643 has finished for PR 15664 at commit
|
| case Success(v) => | ||
| Some(v) | ||
| case Failure(e) => | ||
| None |
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.
Please do not use Try/Success/Failure. https://github.com/databricks/scala-style-guide#exception-handling-try-vs-try
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 see. Thank you for review! I wrote that by keeping the same logic in tableExists because I thought the guideline is about using return Try as a return value before this PR.
Sure, I'll remove the usage of Try/Success/Failure.
| } | ||
| } | ||
|
|
||
| /** |
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.
saving schema is not right. We need a better name here.
rddSchema's sequence and tableSchema's name -> rddSchema's column sequence and tableSchema's column names
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.
Here, we need to explain why we need to use the column sequences in rddSchema and why we need to use the column names in tableSchema
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.
Yep. I'll add more correct details here.
| if (nameMap.isDefinedAt(f.name)) { | ||
| // identical names | ||
| schema = schema.add(nameMap(f.name)) | ||
| } else if (!caseSensitive && lowercaseNameMap.isDefinedAt(f.name.toLowerCase)) { |
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.
Need to improve the comments. Actually, we return case sensitive column names.
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.
My bad. I meant case-insensitively identical names. I'll revise this.
| } else if (!caseSensitive && lowercaseNameMap.isDefinedAt(f.name.toLowerCase)) { | ||
| // case-insensitive identical names | ||
| schema = schema.add(lowercaseNameMap(f.name.toLowerCase)) | ||
| } else { |
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.
org.apache.spark.SparkException -> AnalysisException
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!
| saveTable(df, url, table, jdbcOptions) | ||
| df.schema | ||
| } | ||
| saveTable(df, url, table, savingSchema, jdbcOptions) |
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.
How about passing the table schema and resolve/merge the schemas inside saveTable? It might simplify the codes
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.
That could be. But, to do that, JdbcUtil.saveTable need to understand SaveMode, too. Is it okay?
Currently, JdbcUtil only provides somewhat primitive APIs.
|
Test build #70662 has finished for PR 15664 at commit
|
|
Test build #70663 has finished for PR 15664 at commit
|
|
The failure is not related to this PR. |
…Failure. Add comments.
| // In this case, we should truncate table and then load. | ||
| truncateTable(conn, table) | ||
| saveTable(df, url, table, jdbcOptions) | ||
| val tableSchema = JdbcUtils.getSchemaOption(conn, url, table) |
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 moved this into case statements.
Since JdbcUtils.tableExists is used, getSchemaOption can be skipped for the other SaveMode.
| isCaseSensitive: Boolean, | ||
| dialect: JdbcDialect): String = { | ||
| val columns = if (tableSchema.isEmpty) { | ||
| rddSchema.fields.map(x => dialect.quoteIdentifier(x.name)).mkString(",") |
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.
The legacy behavior is used when tableSchema is None.
|
Test build #70746 has finished for PR 15664 at commit
|
|
The only failure is irrelevant to this PR. |
|
Retest this please. |
|
Test build #70749 has finished for PR 15664 at commit
|
|
LGTM |
|
Merging to master. Thanks! |
|
Thank you, @gatorsmile . |
…ing JDBC Writing
## What changes were proposed in this pull request?
Apache Spark supports the following cases **by quoting RDD column names** while saving through JDBC.
- Allow reserved keyword as a column name, e.g., 'order'.
- Allow mixed-case colume names like the following, e.g., `[a: int, A: int]`.
``` scala
scala> val df = sql("select 1 a, 1 A")
df: org.apache.spark.sql.DataFrame = [a: int, A: int]
...
scala> df.write.mode("overwrite").format("jdbc").options(option).save()
scala> df.write.mode("append").format("jdbc").options(option).save()
```
This PR aims to use **database column names** instead of RDD column ones in order to support the following additionally.
Note that this case succeeds with `MySQL`, but fails on `Postgres`/`Oracle` before.
``` scala
val df1 = sql("select 1 a")
val df2 = sql("select 1 A")
...
df1.write.mode("overwrite").format("jdbc").options(option).save()
df2.write.mode("append").format("jdbc").options(option).save()
```
## How was this patch tested?
Pass the Jenkins test with a new testcase.
Author: Dongjoon Hyun <[email protected]>
Author: gatorsmile <[email protected]>
Closes apache#15664 from dongjoon-hyun/SPARK-18123.
…ing JDBC Writing
## What changes were proposed in this pull request?
Apache Spark supports the following cases **by quoting RDD column names** while saving through JDBC.
- Allow reserved keyword as a column name, e.g., 'order'.
- Allow mixed-case colume names like the following, e.g., `[a: int, A: int]`.
``` scala
scala> val df = sql("select 1 a, 1 A")
df: org.apache.spark.sql.DataFrame = [a: int, A: int]
...
scala> df.write.mode("overwrite").format("jdbc").options(option).save()
scala> df.write.mode("append").format("jdbc").options(option).save()
```
This PR aims to use **database column names** instead of RDD column ones in order to support the following additionally.
Note that this case succeeds with `MySQL`, but fails on `Postgres`/`Oracle` before.
``` scala
val df1 = sql("select 1 a")
val df2 = sql("select 1 A")
...
df1.write.mode("overwrite").format("jdbc").options(option).save()
df2.write.mode("append").format("jdbc").options(option).save()
```
## How was this patch tested?
Pass the Jenkins test with a new testcase.
Author: Dongjoon Hyun <[email protected]>
Author: gatorsmile <[email protected]>
Closes apache#15664 from dongjoon-hyun/SPARK-18123.
What changes were proposed in this pull request?
Apache Spark supports the following cases by quoting RDD column names while saving through JDBC.
Allow reserved keyword as a column name, e.g., 'order'.
Allow mixed-case colume names like the following, e.g.,
[a: int, A: int].This PR aims to use database column names instead of RDD column ones in order to support the following additionally.
Note that this case succeeds with
MySQL, but fails onPostgres/Oraclebefore.How was this patch tested?
Pass the Jenkins test with a new testcase.