-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-18141][SQL] Fix to quote column names in the predicate clause of the JDBC RDD generated sql statement #15662
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 #67662 has finished for PR 15662 at commit
|
|
Test failed is org.apache.spark.sql.streaming.StreamingQuerySuite, unrelated to this change. Might have been fixed in commit 79fd0cc |
|
retest this please |
|
Test build #67669 has finished for PR 15662 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.
Add a nested function in compileFilter
def quote(colName: String): String = dialect.quoteIdentifier(colName)Then, your code changes can look cleaner.
0944e05 to
2afe990
Compare
|
Thank you very much for the feed back @gatorsmile . Addressed the review comments. |
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.
remove this empty line
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 will fix it.
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.
What is the purpose of the above two statements?
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.
Those two statements test String StartsWith , and Contains filters. They are pushed to jdbc data source, and mapped to SQL LIKE expression.
I will fix the inconsistent column name in above two statements.
|
This sounds a right and critical fix to me; otherwise we are unable to resolve the columns of predicates in the case sensitive JDBC sources. @sureshthalamati Could you post the following exception in your PR description? cc @srowen Could you please check it? Any comment? Thanks! |
|
Test build #68044 has finished for PR 15662 at commit
|
|
Thank you for reviewing, @gatorsmile . Updated the PR description , and addressed all the review comments. |
|
Test build #68056 has finished for PR 15662 at commit
|
|
@gatorsmile I addressed all the review comments , can you please take a look. |
|
@srowen Any comment 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.
This is an unnecessary change, right?
|
@sureshthalamati Could you resolve the conflict? Thanks! |
…of the JDBC RDD generated sql statement
4e22e3c to
2178e3f
Compare
|
Test build #69269 has finished for PR 15662 at commit
|
|
Thanks, @gatorsmile . Resolved the conflicts, and also added test case for empty in clause with mixed case column name. |
| assert(sql("SELECT * FROM mixedCaseCols WHERE Name IS NULL").collect().size == 1) | ||
| assert(sql("SELECT * FROM mixedCaseCols WHERE Name IS NOT NULL").collect().size == 2) | ||
| assert(sql("SELECT * FROM mixedCaseCols") | ||
| .filter($"Name".isin(Array[String]() : _*)).collect().size == 0) |
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.
.filter($"Name".isin(Array[String]() : _*)).collect().size == 0)->
.filter($"Name".isin()).collect().size == 0)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 , @gatorsmile . Fixed it.
|
LGTM except a minor comment cc @cloud-fan |
|
Test build #69427 has finished for PR 15662 at commit
|
|
retest this please |
|
Test build #69434 has finished for PR 15662 at commit
|
|
LGTM |
…of the JDBC RDD generated sql statement
## What changes were proposed in this pull request?
SQL query generated for the JDBC data source is not quoting columns in the predicate clause. When the source table has quoted column names, spark jdbc read fails with column not found error incorrectly.
Error:
org.h2.jdbc.JdbcSQLException: Column "ID" not found;
Source SQL statement:
SELECT "Name","Id" FROM TEST."mixedCaseCols" WHERE (Id < 1)
This PR fixes by quoting column names in the generated SQL for predicate clause when filters are pushed down to the data source.
Source SQL statement after the fix:
SELECT "Name","Id" FROM TEST."mixedCaseCols" WHERE ("Id" < 1)
## How was this patch tested?
Added new test case to the JdbcSuite
Author: sureshthalamati <[email protected]>
Closes #15662 from sureshthalamati/filter_quoted_cols-SPARK-18141.
(cherry picked from commit 70c5549)
Signed-off-by: gatorsmile <[email protected]>
|
Merging to master/2.1! Thanks! |
…of the JDBC RDD generated sql statement
## What changes were proposed in this pull request?
SQL query generated for the JDBC data source is not quoting columns in the predicate clause. When the source table has quoted column names, spark jdbc read fails with column not found error incorrectly.
Error:
org.h2.jdbc.JdbcSQLException: Column "ID" not found;
Source SQL statement:
SELECT "Name","Id" FROM TEST."mixedCaseCols" WHERE (Id < 1)
This PR fixes by quoting column names in the generated SQL for predicate clause when filters are pushed down to the data source.
Source SQL statement after the fix:
SELECT "Name","Id" FROM TEST."mixedCaseCols" WHERE ("Id" < 1)
## How was this patch tested?
Added new test case to the JdbcSuite
Author: sureshthalamati <[email protected]>
Closes apache#15662 from sureshthalamati/filter_quoted_cols-SPARK-18141.
|
Thank you , @gatorsmile @cloud-fan |
…of the JDBC RDD generated sql statement
## What changes were proposed in this pull request?
SQL query generated for the JDBC data source is not quoting columns in the predicate clause. When the source table has quoted column names, spark jdbc read fails with column not found error incorrectly.
Error:
org.h2.jdbc.JdbcSQLException: Column "ID" not found;
Source SQL statement:
SELECT "Name","Id" FROM TEST."mixedCaseCols" WHERE (Id < 1)
This PR fixes by quoting column names in the generated SQL for predicate clause when filters are pushed down to the data source.
Source SQL statement after the fix:
SELECT "Name","Id" FROM TEST."mixedCaseCols" WHERE ("Id" < 1)
## How was this patch tested?
Added new test case to the JdbcSuite
Author: sureshthalamati <[email protected]>
Closes apache#15662 from sureshthalamati/filter_quoted_cols-SPARK-18141.
…of the JDBC RDD generated sql statement
## What changes were proposed in this pull request?
SQL query generated for the JDBC data source is not quoting columns in the predicate clause. When the source table has quoted column names, spark jdbc read fails with column not found error incorrectly.
Error:
org.h2.jdbc.JdbcSQLException: Column "ID" not found;
Source SQL statement:
SELECT "Name","Id" FROM TEST."mixedCaseCols" WHERE (Id < 1)
This PR fixes by quoting column names in the generated SQL for predicate clause when filters are pushed down to the data source.
Source SQL statement after the fix:
SELECT "Name","Id" FROM TEST."mixedCaseCols" WHERE ("Id" < 1)
## How was this patch tested?
Added new test case to the JdbcSuite
Author: sureshthalamati <[email protected]>
Closes apache#15662 from sureshthalamati/filter_quoted_cols-SPARK-18141.
What changes were proposed in this pull request?
SQL query generated for the JDBC data source is not quoting columns in the predicate clause. When the source table has quoted column names, spark jdbc read fails with column not found error incorrectly.
Error:
org.h2.jdbc.JdbcSQLException: Column "ID" not found;
Source SQL statement:
SELECT "Name","Id" FROM TEST."mixedCaseCols" WHERE (Id < 1)
This PR fixes by quoting column names in the generated SQL for predicate clause when filters are pushed down to the data source.
Source SQL statement after the fix:
SELECT "Name","Id" FROM TEST."mixedCaseCols" WHERE ("Id" < 1)
How was this patch tested?
Added new test case to the JdbcSuite