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 @@ -20,13 +20,13 @@ package org.apache.spark.sql.execution.datasources
import org.apache.spark.{Logging, TaskContext}
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.rdd.{MapPartitionsRDD, RDD, UnionRDD}
import org.apache.spark.sql.catalyst.{InternalRow, expressions}
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, expressions}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.planning.PhysicalOperation
import org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.sources._
import org.apache.spark.sql.types.{StringType, StructType}
import org.apache.spark.sql.types.{TimestampType, DateType, StringType, StructType}
import org.apache.spark.sql.{SaveMode, Strategy, execution, sources, _}
import org.apache.spark.unsafe.types.UTF8String
import org.apache.spark.util.{SerializableConfiguration, Utils}
Expand Down Expand Up @@ -343,31 +343,57 @@ private[sql] object DataSourceStrategy extends Strategy with Logging {
* and convert them.
*/
protected[sql] def selectFilters(filters: Seq[Expression]) = {
import CatalystTypeConverters._

def translate(predicate: Expression): Option[Filter] = predicate match {
case expressions.EqualTo(a: Attribute, Literal(v, _)) =>
Some(sources.EqualTo(a.name, v))
case expressions.EqualTo(Literal(v, _), a: Attribute) =>
Some(sources.EqualTo(a.name, v))
case expressions.EqualTo(Cast(a: Attribute, _), l: Literal) =>
Some(sources.EqualTo(a.name, convertToScala(Cast(l, a.dataType).eval(), a.dataType)))
case expressions.EqualTo(l: Literal, Cast(a: Attribute, _)) =>
Some(sources.EqualTo(a.name, convertToScala(Cast(l, a.dataType).eval(), a.dataType)))
Copy link
Contributor

Choose a reason for hiding this comment

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

is this always safe? or could you for example cast long -> int and truncate?

Copy link
Member Author

Choose a reason for hiding this comment

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

yes, you are right, downcast here just truncate the origin literal value and have a wrong pushed down filter.

Copy link
Contributor

Choose a reason for hiding this comment

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

so should we revert this?

/cc @liancheng

Copy link
Member Author

Choose a reason for hiding this comment

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

Hi, @marmbrus , I've think this twice and may be I could do this?

For HadoopFsRelations, they all assume pushed down column and value are of same type, I think the only safe way is not pushed down these casted filters at all.
For JDBCRelation, since the value itself is converted into a constructed string where clause and pushed to the underlying database, I think it's safe to just pushed to uncasted value down?

Copy link
Contributor

Choose a reason for hiding this comment

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

Hmm, depends on what implicit casting rules various databases do. Can you investigate more? I would not want us to generate queries that fail to analayze.

In the mean time I think we should revert this from the release branch as pushing down wrong filters is worse than not pushing down filters.

Copy link
Member Author

Choose a reason for hiding this comment

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

ok, sorry for the wrong fix, should I make a revert PR?

Copy link
Contributor

Choose a reason for hiding this comment

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

Its okay. I'm glad we caught it. Please do and open a blocker JIRA targeted at 1.5 so we don't miss merging it.

Copy link
Contributor

Choose a reason for hiding this comment

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

It's reverted by #8157. I think we can add casts over literals only when the casts don't introduce truncation?

I reopened SPARK-9182. It's not a regression introduced in 1.5, do we need a blocker JIRA for it?

Copy link
Contributor

Choose a reason for hiding this comment

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

No, given the possible trickiness here I think we should bump the fix to 1.6.


case expressions.GreaterThan(a: Attribute, Literal(v, _)) =>
Some(sources.GreaterThan(a.name, v))
case expressions.GreaterThan(Literal(v, _), a: Attribute) =>
Some(sources.LessThan(a.name, v))
case expressions.GreaterThan(Cast(a: Attribute, _), l: Literal) =>
Some(sources.GreaterThan(a.name, convertToScala(Cast(l, a.dataType).eval(), a.dataType)))
case expressions.GreaterThan(l: Literal, Cast(a: Attribute, _)) =>
Some(sources.LessThan(a.name, convertToScala(Cast(l, a.dataType).eval(), a.dataType)))

case expressions.LessThan(a: Attribute, Literal(v, _)) =>
Some(sources.LessThan(a.name, v))
case expressions.LessThan(Literal(v, _), a: Attribute) =>
Some(sources.GreaterThan(a.name, v))
case expressions.LessThan(Cast(a: Attribute, _), l: Literal) =>
Some(sources.LessThan(a.name, convertToScala(Cast(l, a.dataType).eval(), a.dataType)))
case expressions.LessThan(l: Literal, Cast(a: Attribute, _)) =>
Some(sources.GreaterThan(a.name, convertToScala(Cast(l, a.dataType).eval(), a.dataType)))

case expressions.GreaterThanOrEqual(a: Attribute, Literal(v, _)) =>
Some(sources.GreaterThanOrEqual(a.name, v))
case expressions.GreaterThanOrEqual(Literal(v, _), a: Attribute) =>
Some(sources.LessThanOrEqual(a.name, v))
case expressions.GreaterThanOrEqual(Cast(a: Attribute, _), l: Literal) =>
Some(sources.GreaterThanOrEqual(a.name,
convertToScala(Cast(l, a.dataType).eval(), a.dataType)))
case expressions.GreaterThanOrEqual(l: Literal, Cast(a: Attribute, _)) =>
Some(sources.LessThanOrEqual(a.name,
convertToScala(Cast(l, a.dataType).eval(), a.dataType)))

case expressions.LessThanOrEqual(a: Attribute, Literal(v, _)) =>
Some(sources.LessThanOrEqual(a.name, v))
case expressions.LessThanOrEqual(Literal(v, _), a: Attribute) =>
Some(sources.GreaterThanOrEqual(a.name, v))
case expressions.LessThanOrEqual(Cast(a: Attribute, _), l: Literal) =>
Some(sources.LessThanOrEqual(a.name,
convertToScala(Cast(l, a.dataType).eval(), a.dataType)))
case expressions.LessThanOrEqual(l: Literal, Cast(a: Attribute, _)) =>
Some(sources.GreaterThanOrEqual(a.name,
convertToScala(Cast(l, a.dataType).eval(), a.dataType)))

case expressions.InSet(a: Attribute, set) =>
Some(sources.In(a.name, set.toArray))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -284,7 +284,7 @@ private[sql] class JDBCRDD(
/**
* `filters`, but as a WHERE clause suitable for injection into a SQL query.
*/
private val filterWhereClause: String = {
val filterWhereClause: String = {
val filterStrings = filters map compileFilter filter (_ != null)
if (filterStrings.size > 0) {
val sb = new StringBuilder("WHERE ")
Expand Down
34 changes: 34 additions & 0 deletions sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,8 @@ import org.h2.jdbc.JdbcSQLException
import org.scalatest.BeforeAndAfter

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD
import org.apache.spark.sql.execution.PhysicalRDD
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils

Expand Down Expand Up @@ -148,6 +150,18 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter {
|OPTIONS (url '$url', dbtable 'TEST.FLTTYPES', user 'testUser', password 'testPass')
""".stripMargin.replaceAll("\n", " "))

conn.prepareStatement("create table test.decimals (a DECIMAL(7, 2), b DECIMAL(4, 0))").
executeUpdate()
conn.prepareStatement("insert into test.decimals values (12345.67, 1234)").executeUpdate()
conn.prepareStatement("insert into test.decimals values (34567.89, 1428)").executeUpdate()
conn.commit()
sql(
s"""
|CREATE TEMPORARY TABLE decimals
|USING org.apache.spark.sql.jdbc
|OPTIONS (url '$url', dbtable 'TEST.DECIMALS', user 'testUser', password 'testPass')
""".stripMargin.replaceAll("\n", " "))

conn.prepareStatement(
s"""
|create table test.nulltypes (a INT, b BOOLEAN, c TINYINT, d BINARY(20), e VARCHAR(20),
Expand Down Expand Up @@ -445,4 +459,24 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter {
assert(agg.getCatalystType(1, "", 1, null) === Some(StringType))
}

test("SPARK-9182: filters are not passed through to jdbc source") {
def checkPushedFilter(query: String, filterStr: String): Unit = {
val rddOpt = sql(query).queryExecution.executedPlan.collectFirst {
case PhysicalRDD(_, rdd: JDBCRDD, _) => rdd
}
assert(rddOpt.isDefined)
val pushedFilterStr = rddOpt.get.filterWhereClause
assert(pushedFilterStr.contains(filterStr),
s"Expected to push [$filterStr], actually we pushed [$pushedFilterStr]")
}

checkPushedFilter("select * from foobar where NAME = 'fred'", "NAME = 'fred'")
checkPushedFilter("select * from inttypes where A > '15'", "A > 15")
checkPushedFilter("select * from inttypes where C <= 20", "C <= 20")
checkPushedFilter("select * from decimals where A > 1000", "A > 1000.00")
checkPushedFilter("select * from decimals where A > 1000 AND A < 2000",
"A > 1000.00 AND A < 2000.00")
checkPushedFilter("select * from decimals where A = 2000 AND B > 20", "A = 2000.00 AND B > 20")
checkPushedFilter("select * from timetypes where B > '1998-09-10'", "B > 1998-09-10")
}
}