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

[SPARK-18141][SQL] Fix to quote column names in the predicate clause of the JDBC RDD generated sql statement #15662

Conversation

sureshthalamati
Copy link
Contributor

@sureshthalamati sureshthalamati commented Oct 27, 2016

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

@SparkQA
Copy link

SparkQA commented Oct 27, 2016

Test build #67662 has finished for PR 15662 at commit 0944e05.

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

@sureshthalamati
Copy link
Contributor Author

Test failed is org.apache.spark.sql.streaming.StreamingQuerySuite, unrelated to this change. Might have been fixed in commit 79fd0cc

@sureshthalamati
Copy link
Contributor Author

retest this please

@SparkQA
Copy link

SparkQA commented Oct 27, 2016

Test build #67669 has finished for PR 15662 at commit 0944e05.

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

@sureshthalamati
Copy link
Contributor Author

@rxin @gatorsmile

Option(f match {
case EqualTo(attr, value) => s"$attr = ${compileValue(value)}"
case EqualTo(attr, value) => s"${dialect.quoteIdentifier(attr)} = ${compileValue(value)}"
Copy link
Member

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.

@sureshthalamati sureshthalamati force-pushed the filter_quoted_cols-SPARK-18141 branch from 0944e05 to 2afe990 Compare November 3, 2016 03:35
@sureshthalamati
Copy link
Contributor Author

Thank you very much for the feed back @gatorsmile . Addressed the review comments.

import org.apache.spark.sql.sources._
import org.apache.spark.sql.types.StructType


Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove this empty line

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I will fix it.

assert(sql("SELECT * FROM mixedCaseCols WHERE Id <=> 2").collect().size == 1)
assert(sql("SELECT * FROM mixedCaseCols WHERE Name LIKE 'fr%'").collect().size == 1)
assert(sql("SELECT * FROM mixedCaseCols WHERE NAME LIKE '%ed'").collect().size == 1)
assert(sql("SELECT * FROM mixedCaseCols WHERE NAME LIKE '%re%'").collect().size == 1)
Copy link
Member

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?

Copy link
Contributor Author

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.

@gatorsmile
Copy link
Member

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?

org.h2.jdbc.JdbcSQLException: Column "ID" not found; SQL statement:
SELECT "Name","Id" FROM TEST."mixedCaseCols" WHERE (Id < 1) [42122-183]

cc @srowen Could you please check it? Any comment? Thanks!

@SparkQA
Copy link

SparkQA commented Nov 3, 2016

Test build #68044 has finished for PR 15662 at commit 2afe990.

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

@sureshthalamati
Copy link
Contributor Author

Thank you for reviewing, @gatorsmile . Updated the PR description , and addressed all the review comments.

@SparkQA
Copy link

SparkQA commented Nov 3, 2016

Test build #68056 has finished for PR 15662 at commit 4e22e3c.

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

@sureshthalamati
Copy link
Contributor Author

@gatorsmile I addressed all the review comments , can you please take a look.

@gatorsmile
Copy link
Member

@srowen Any comment on this?

@@ -172,7 +172,7 @@ class JDBCSuite extends SparkFunSuite
""".stripMargin.replaceAll("\n", " "))

conn.prepareStatement(
"create table test.emp(name TEXT(32) NOT NULL," +
"create table test.emp(\"Name\" TEXT(32) NOT NULL," +
Copy link
Member

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?

@gatorsmile
Copy link
Member

@sureshthalamati Could you resolve the conflict? Thanks!

@sureshthalamati sureshthalamati force-pushed the filter_quoted_cols-SPARK-18141 branch from 4e22e3c to 2178e3f Compare November 28, 2016 23:00
@SparkQA
Copy link

SparkQA commented Nov 29, 2016

Test build #69269 has finished for PR 15662 at commit 2178e3f.

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

@sureshthalamati
Copy link
Contributor Author

Thanks, @gatorsmile . Resolved the conflicts, and also added test case for empty in clause with mixed case column name.

@@ -855,6 +855,8 @@ class JDBCSuite extends SparkFunSuite
assert(sql("SELECT * FROM mixedCaseCols WHERE Name LIKE '%re%'").collect().size == 1)
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)
Copy link
Member

@gatorsmile gatorsmile Nov 30, 2016

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)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks , @gatorsmile . Fixed it.

@gatorsmile
Copy link
Member

LGTM except a minor comment

cc @cloud-fan

@SparkQA
Copy link

SparkQA commented Nov 30, 2016

Test build #69427 has finished for PR 15662 at commit f0d731f.

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

@gatorsmile
Copy link
Member

retest this please

@SparkQA
Copy link

SparkQA commented Dec 1, 2016

Test build #69434 has finished for PR 15662 at commit f0d731f.

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

@cloud-fan
Copy link
Contributor

LGTM

asfgit pushed a commit that referenced this pull request Dec 2, 2016
…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 <suresh.thalamati@gmail.com>

Closes #15662 from sureshthalamati/filter_quoted_cols-SPARK-18141.

(cherry picked from commit 70c5549)
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
@gatorsmile
Copy link
Member

Merging to master/2.1! Thanks!

@asfgit asfgit closed this in 70c5549 Dec 2, 2016
robert3005 pushed a commit to palantir/spark that referenced this pull request Dec 2, 2016
…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 <suresh.thalamati@gmail.com>

Closes apache#15662 from sureshthalamati/filter_quoted_cols-SPARK-18141.
@sureshthalamati
Copy link
Contributor Author

Thank you , @gatorsmile @cloud-fan

robert3005 pushed a commit to palantir/spark that referenced this pull request Dec 15, 2016
…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 <suresh.thalamati@gmail.com>

Closes apache#15662 from sureshthalamati/filter_quoted_cols-SPARK-18141.
uzadude pushed a commit to uzadude/spark that referenced this pull request Jan 27, 2017
…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 <suresh.thalamati@gmail.com>

Closes apache#15662 from sureshthalamati/filter_quoted_cols-SPARK-18141.
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.

4 participants