Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -174,6 +174,7 @@ private[sql] object JDBCRDD extends Logging {
case stringValue: String => s"'${escapeSql(stringValue)}'"
case timestampValue: Timestamp => "'" + timestampValue + "'"
case dateValue: Date => "'" + dateValue + "'"
case arrayValue: Array[Object] => arrayValue.map(compileValue).mkString(", ")
case _ => value
}

Expand All @@ -186,13 +187,19 @@ private[sql] object JDBCRDD extends Logging {
*/
private def compileFilter(f: Filter): String = f match {
case EqualTo(attr, value) => s"$attr = ${compileValue(value)}"
case Not(EqualTo(attr, value)) => s"$attr != ${compileValue(value)}"
case Not(f) => s"(NOT (${compileFilter(f)}))"
case LessThan(attr, value) => s"$attr < ${compileValue(value)}"
case GreaterThan(attr, value) => s"$attr > ${compileValue(value)}"
case LessThanOrEqual(attr, value) => s"$attr <= ${compileValue(value)}"
case GreaterThanOrEqual(attr, value) => s"$attr >= ${compileValue(value)}"
case StringStartsWith(attr, value) => s"${attr} LIKE '${value}%'"
case StringEndsWith(attr, value) => s"${attr} LIKE '%${value}'"
case StringContains(attr, value) => s"${attr} LIKE '%${value}%'"
case IsNull(attr) => s"$attr IS NULL"
case IsNotNull(attr) => s"$attr IS NOT NULL"
case In(attr, value) => s"$attr IN (${compileValue(value)})"
case Or(f1, f2) => s"(${compileFilter(f1)}) OR (${compileFilter(f2)})"
case And(f1, f2) => s"(${compileFilter(f1)}) AND (${compileFilter(f2)})"
case _ => null
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import org.scalatest.BeforeAndAfter
import org.scalatest.PrivateMethodTester

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.Row
import org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD
import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.sql.types._
Expand Down Expand Up @@ -186,8 +187,26 @@ class JDBCSuite extends SparkFunSuite
assert(stripSparkFilter(sql("SELECT * FROM foobar WHERE NAME = 'fred'")).collect().size == 1)
assert(stripSparkFilter(sql("SELECT * FROM foobar WHERE NAME > 'fred'")).collect().size == 2)
assert(stripSparkFilter(sql("SELECT * FROM foobar WHERE NAME != 'fred'")).collect().size == 2)
assert(stripSparkFilter(sql("SELECT * FROM foobar WHERE NAME IN ('mary', 'fred')"))
.collect().size == 2)
Copy link
Member

Choose a reason for hiding this comment

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

Should we compare using === instead of == here and elsewhere?

Copy link
Member Author

Choose a reason for hiding this comment

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

I have no strong opinion on this.... is it better to fix them? ISTM collection types, e.g., set, need === comparisons.

Copy link
Contributor

Choose a reason for hiding this comment

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

Does the newer version of scalatest library we use already uses macro so == and === are the same? Can you confirm? Anyway it's not that big of a deal here.

Copy link
Member

Choose a reason for hiding this comment

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

Ah yes, now Assertions trait provides assert macro and the trait mixes in TripleEquals trait so we don't need to change == to ===.

assert(stripSparkFilter(sql("SELECT * FROM foobar WHERE NAME NOT IN ('fred')"))
.collect().size === 2)
assert(stripSparkFilter(sql("SELECT * FROM foobar WHERE THEID = 1 OR NAME = 'mary'"))
.collect().size == 2)
assert(stripSparkFilter(sql("SELECT * FROM foobar WHERE THEID = 1 OR NAME = 'mary' "
+ "AND THEID = 2")).collect().size == 2)
assert(stripSparkFilter(sql("SELECT * FROM foobar WHERE NAME LIKE 'fr%'")).collect().size == 1)
assert(stripSparkFilter(sql("SELECT * FROM foobar WHERE NAME LIKE '%ed'")).collect().size == 1)
assert(stripSparkFilter(sql("SELECT * FROM foobar WHERE NAME LIKE '%re%'")).collect().size == 1)
assert(stripSparkFilter(sql("SELECT * FROM nulltypes WHERE A IS NULL")).collect().size == 1)
assert(stripSparkFilter(sql("SELECT * FROM nulltypes WHERE A IS NOT NULL")).collect().size == 0)

// This is a test to reflect discussion in SPARK-12218.
// The older versions of spark have this kind of bugs in parquet data source.
val df1 = sql("SELECT * FROM foobar WHERE NOT (THEID != 2 AND NAME != 'mary')")
Copy link
Member

Choose a reason for hiding this comment

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

The two sub-conditions are both ok to be pushed down. So this doesn't actually test against the nested AND issue in SPARK-12218. See #19776

Btw, the two sub-conditions are filtered out the same rows. This doesn't reflect the issue too.

Copy link
Member Author

Choose a reason for hiding this comment

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

yea, I think ok to drop this test.

val df2 = sql("SELECT * FROM foobar WHERE NOT (THEID != 2) OR NOT (NAME != 'mary')")
assert(df1.collect.toSet === Set(Row("mary", 2)))
assert(df2.collect.toSet === Set(Row("mary", 2)))
}

test("SELECT * WHERE (quoted strings)") {
Expand Down Expand Up @@ -437,14 +456,21 @@ class JDBCSuite extends SparkFunSuite
val compileFilter = PrivateMethod[String]('compileFilter)
def doCompileFilter(f: Filter): String = JDBCRDD invokePrivate compileFilter(f)
assert(doCompileFilter(EqualTo("col0", 3)) === "col0 = 3")
assert(doCompileFilter(Not(EqualTo("col1", "abc"))) === "col1 != 'abc'")
assert(doCompileFilter(Not(EqualTo("col1", "abc"))) === "(NOT (col1 = 'abc'))")
assert(doCompileFilter(And(EqualTo("col0", 0), EqualTo("col1", "def")))
=== "(col0 = 0) AND (col1 = 'def')")
assert(doCompileFilter(Or(EqualTo("col0", 2), EqualTo("col1", "ghi")))
=== "(col0 = 2) OR (col1 = 'ghi')")
assert(doCompileFilter(LessThan("col0", 5)) === "col0 < 5")
assert(doCompileFilter(LessThan("col3",
Timestamp.valueOf("1995-11-21 00:00:00.0"))) === "col3 < '1995-11-21 00:00:00.0'")
assert(doCompileFilter(LessThan("col4", Date.valueOf("1983-08-04"))) === "col4 < '1983-08-04'")
assert(doCompileFilter(LessThanOrEqual("col0", 5)) === "col0 <= 5")
assert(doCompileFilter(GreaterThan("col0", 3)) === "col0 > 3")
assert(doCompileFilter(GreaterThanOrEqual("col0", 3)) === "col0 >= 3")
assert(doCompileFilter(In("col1", Array("jkl"))) === "col1 IN ('jkl')")
assert(doCompileFilter(Not(In("col1", Array("mno", "pqr"))))
=== "(NOT (col1 IN ('mno', 'pqr')))")
assert(doCompileFilter(IsNull("col1")) === "col1 IS NULL")
assert(doCompileFilter(IsNotNull("col1")) === "col1 IS NOT NULL")
}
Expand Down