From 1aec7f83292269757bb4c5e863e1d439cd26dd02 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 16 Jun 2019 23:49:55 +0800 Subject: [PATCH] Fix error when insert partitioned table dynamically where partition name is upper case. --- .../sql/hive/execution/SaveAsHiveFile.scala | 12 +++++++++++- .../sql/hive/execution/HiveQuerySuite.scala | 18 ++++++++++++++++++ 2 files changed, 29 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala index 4be3cd45454c6..234acb7debd49 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala @@ -83,6 +83,16 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { jobId = java.util.UUID.randomUUID().toString, outputPath = outputLocation) + // SPARK-28054: Hive metastore is not case preserving and keeps partition columns + // with lower cased names, Hive will validate the column names in partition spec and + // the partition paths. Besides lowercasing the column names in the partition spec, + // we also need to lowercase the column names in written partition paths. + // scalastyle:off caselocale + val hiveCompatiblePartitionColumns = partitionAttributes.map { attr => + attr.withName(attr.name.toLowerCase) + } + // scalastyle:on caselocale + FileFormatWriter.write( sparkSession = sparkSession, plan = plan, @@ -91,7 +101,7 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { outputSpec = FileFormatWriter.OutputSpec(outputLocation, customPartitionLocations, outputColumns), hadoopConf = hadoopConf, - partitionColumns = partitionAttributes, + partitionColumns = hiveCompatiblePartitionColumns, bucketSpec = None, statsTrackers = Seq(basicWriteJobStatsTracker(hadoopConf)), options = Map.empty) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index df2f693e7147a..13a533ccfea81 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -1188,6 +1188,24 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd } } } + + test("SPARK-28054: Unable to insert partitioned table when partition name is upper case") { + withTable("spark_28054_test") { + sql("set hive.exec.dynamic.partition.mode=nonstrict") + sql("CREATE TABLE spark_28054_test (KEY STRING, VALUE STRING) PARTITIONED BY (DS STRING)") + + sql("INSERT INTO TABLE spark_28054_test PARTITION(DS) SELECT 'k' KEY, 'v' VALUE, '1' DS") + + assertResult(Array(Row("k", "v", "1"))) { + sql("SELECT * from spark_28054_test").collect() + } + + sql("INSERT INTO TABLE spark_28054_test PARTITION(ds) SELECT 'k' key, 'v' value, '2' ds") + assertResult(Array(Row("k", "v", "1"), Row("k", "v", "2"))) { + sql("SELECT * from spark_28054_test").collect() + } + } + } } // for SPARK-2180 test