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 @@ -1017,9 +1017,9 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
predicates: Seq[Expression]): Seq[CatalogTablePartition] = withClient {
val rawTable = getRawTable(db, table)
val catalogTable = restoreTableMetadata(rawTable)
val partitionColumnNames = catalogTable.partitionColumnNames.toSet
val partitionColumnNames = catalogTable.partitionColumnNames.map(_.toLowerCase).toSet
val nonPartitionPruningPredicates = predicates.filterNot {
_.references.map(_.name).toSet.subsetOf(partitionColumnNames)
_.references.map(_.name.toLowerCase).toSet.subsetOf(partitionColumnNames)
}

if (nonPartitionPruningPredicates.nonEmpty) {
Expand All @@ -1037,7 +1037,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
val boundPredicate =
InterpretedPredicate.create(predicates.reduce(And).transform {
case att: AttributeReference =>
val index = partitionSchema.indexWhere(_.name == att.name)
val index = partitionSchema.indexWhere(_.name.toLowerCase == att.name.toLowerCase)
BoundReference(index, partitionSchema(index).dataType, nullable = true)
})
clientPrunedPartitions.filter { p => boundPredicate(p.toRow(partitionSchema)) }
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1236,6 +1236,12 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd
}
}
}

test("SPARK-19490 case sensitivity when filtering hive partition columns") {
sql("CREATE TABLE partition_test (key Int) partitioned by (date string)")
sql("INSERT INTO TABLE partition_test partition(date='20170101') select 1 from src limit 1")
sql("SELECT * FROM partition_test where DATE = '20170101'")
}
}

// for SPARK-2180 test
Expand Down