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 @@ -41,8 +41,6 @@ private[parquet] object ParquetFilters {
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 @@ -52,7 +50,6 @@ 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 @@ -67,8 +64,6 @@ private[parquet] object ParquetFilters {
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 @@ -77,7 +72,6 @@ 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 @@ -90,16 +84,13 @@ private[parquet] object ParquetFilters {
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 @@ -112,16 +103,13 @@ private[parquet] object ParquetFilters {
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 @@ -134,16 +122,13 @@ private[parquet] object ParquetFilters {
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 @@ -156,16 +141,13 @@ private[parquet] object ParquetFilters {
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 @@ -546,21 +546,8 @@ private[parquet] class ParquetSchemaConverter(
private[parquet] object ParquetSchemaConverter {
val SPARK_PARQUET_SCHEMA_NAME = "spark_schema"

// !! HACK ALERT !!
//
// PARQUET-363 & PARQUET-278: parquet-mr 1.8.1 doesn't allow constructing empty GroupType,
// which prevents us to avoid selecting any columns for queries like `SELECT COUNT(*) FROM t`.
// This issue has been fixed in parquet-mr 1.8.2-SNAPSHOT.
//
// To workaround this problem, here we first construct a `MessageType` with a single dummy
// field, and then remove the field to obtain an empty `MessageType`.
//
// TODO Reverts this change after upgrading parquet-mr to 1.8.2+
val EMPTY_MESSAGE = Types
.buildMessage()
.required(PrimitiveType.PrimitiveTypeName.INT32).named("dummy")
.named(ParquetSchemaConverter.SPARK_PARQUET_SCHEMA_NAME)
EMPTY_MESSAGE.getFields.clear()
val EMPTY_MESSAGE: MessageType =
Types.buildMessage().named(ParquetSchemaConverter.SPARK_PARQUET_SCHEMA_NAME)

def checkFieldName(name: String): Unit = {
// ,;{}()\n\t= and space are special characters in Parquet schema
Expand Down