Skip to content

[SPARK-17213][SQL] Disable Parquet filter push-down for string and binary columns due to PARQUET-686 #16106

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

Closed
wants to merge 4 commits into from
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 @@ -40,6 +40,9 @@ private[parquet] object ParquetFilters {
(n: String, v: Any) => FilterApi.eq(floatColumn(n), v.asInstanceOf[java.lang.Float])
case DoubleType =>
(n: String, v: Any) => FilterApi.eq(doubleColumn(n), v.asInstanceOf[java.lang.Double])

// See SPARK-17213: https://issues.apache.org/jira/browse/SPARK-17213
/*
// Binary.fromString and Binary.fromByteArray don't accept null values
case StringType =>
(n: String, v: Any) => FilterApi.eq(
Expand All @@ -49,6 +52,7 @@ private[parquet] object ParquetFilters {
(n: String, v: Any) => FilterApi.eq(
binaryColumn(n),
Option(v).map(b => Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])).orNull)
*/
}

private val makeNotEq: PartialFunction[DataType, (String, Any) => FilterPredicate] = {
Expand All @@ -62,6 +66,9 @@ private[parquet] object ParquetFilters {
(n: String, v: Any) => FilterApi.notEq(floatColumn(n), v.asInstanceOf[java.lang.Float])
case DoubleType =>
(n: String, v: Any) => FilterApi.notEq(doubleColumn(n), v.asInstanceOf[java.lang.Double])

// See SPARK-17213: https://issues.apache.org/jira/browse/SPARK-17213
/*
case StringType =>
(n: String, v: Any) => FilterApi.notEq(
binaryColumn(n),
Expand All @@ -70,6 +77,7 @@ private[parquet] object ParquetFilters {
(n: String, v: Any) => FilterApi.notEq(
binaryColumn(n),
Option(v).map(b => Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])).orNull)
*/
}

private val makeLt: PartialFunction[DataType, (String, Any) => FilterPredicate] = {
Expand All @@ -81,13 +89,17 @@ private[parquet] object ParquetFilters {
(n: String, v: Any) => FilterApi.lt(floatColumn(n), v.asInstanceOf[java.lang.Float])
case DoubleType =>
(n: String, v: Any) => FilterApi.lt(doubleColumn(n), v.asInstanceOf[java.lang.Double])

// See SPARK-17213: https://issues.apache.org/jira/browse/SPARK-17213
/*
case StringType =>
(n: String, v: Any) =>
FilterApi.lt(binaryColumn(n),
Binary.fromString(v.asInstanceOf[String]))
case BinaryType =>
(n: String, v: Any) =>
FilterApi.lt(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]]))
*/
}

private val makeLtEq: PartialFunction[DataType, (String, Any) => FilterPredicate] = {
Expand All @@ -99,13 +111,17 @@ private[parquet] object ParquetFilters {
(n: String, v: Any) => FilterApi.ltEq(floatColumn(n), v.asInstanceOf[java.lang.Float])
case DoubleType =>
(n: String, v: Any) => FilterApi.ltEq(doubleColumn(n), v.asInstanceOf[java.lang.Double])

// See SPARK-17213: https://issues.apache.org/jira/browse/SPARK-17213
/*
case StringType =>
(n: String, v: Any) =>
FilterApi.ltEq(binaryColumn(n),
Binary.fromString(v.asInstanceOf[String]))
case BinaryType =>
(n: String, v: Any) =>
FilterApi.ltEq(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]]))
*/
}

private val makeGt: PartialFunction[DataType, (String, Any) => FilterPredicate] = {
Expand All @@ -117,13 +133,17 @@ private[parquet] object ParquetFilters {
(n: String, v: Any) => FilterApi.gt(floatColumn(n), v.asInstanceOf[java.lang.Float])
case DoubleType =>
(n: String, v: Any) => FilterApi.gt(doubleColumn(n), v.asInstanceOf[java.lang.Double])

// See SPARK-17213: https://issues.apache.org/jira/browse/SPARK-17213
/*
case StringType =>
(n: String, v: Any) =>
FilterApi.gt(binaryColumn(n),
Binary.fromString(v.asInstanceOf[String]))
case BinaryType =>
(n: String, v: Any) =>
FilterApi.gt(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]]))
*/
}

private val makeGtEq: PartialFunction[DataType, (String, Any) => FilterPredicate] = {
Expand All @@ -135,13 +155,17 @@ private[parquet] object ParquetFilters {
(n: String, v: Any) => FilterApi.gtEq(floatColumn(n), v.asInstanceOf[java.lang.Float])
case DoubleType =>
(n: String, v: Any) => FilterApi.gtEq(doubleColumn(n), v.asInstanceOf[java.lang.Double])

// See SPARK-17213: https://issues.apache.org/jira/browse/SPARK-17213
/*
case StringType =>
(n: String, v: Any) =>
FilterApi.gtEq(binaryColumn(n),
Binary.fromString(v.asInstanceOf[String]))
case BinaryType =>
(n: String, v: Any) =>
FilterApi.gtEq(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]]))
*/
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,6 @@ import org.apache.spark.util.{AccumulatorContext, LongAccumulator}
* data type is nullable.
*/
class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContext {

private def checkFilterPredicate(
df: DataFrame,
predicate: Predicate,
Expand Down Expand Up @@ -230,7 +229,8 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex
}
}

test("filter pushdown - string") {
// See SPARK-17213: https://issues.apache.org/jira/browse/SPARK-17213
ignore("filter pushdown - string") {
withParquetDataFrame((1 to 4).map(i => Tuple1(i.toString))) { implicit df =>
checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row])
checkFilterPredicate(
Expand Down Expand Up @@ -258,7 +258,8 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex
}
}

test("filter pushdown - binary") {
// See SPARK-17213: https://issues.apache.org/jira/browse/SPARK-17213
ignore("filter pushdown - binary") {
implicit class IntToBinary(int: Int) {
def b: Array[Byte] = int.toString.getBytes(StandardCharsets.UTF_8)
}
Expand Down Expand Up @@ -558,4 +559,23 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex
}
}
}

test("SPARK-17213: Broken Parquet filter push-down for string columns") {
withTempPath { dir =>
import testImplicits._

val path = dir.getCanonicalPath
// scalastyle:off nonascii
Seq("a", "é").toDF("name").write.parquet(path)
// scalastyle:on nonascii

assert(spark.read.parquet(path).where("name > 'a'").count() == 1)
assert(spark.read.parquet(path).where("name >= 'a'").count() == 2)

// scalastyle:off nonascii
assert(spark.read.parquet(path).where("name < 'é'").count() == 1)
assert(spark.read.parquet(path).where("name <= 'é'").count() == 2)
// scalastyle:on nonascii
}
}
}