diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 0ed464dad91b1..ee1734b1f232c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive.execution +import java.util.Locale + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.ql.ErrorMsg @@ -186,10 +188,15 @@ case class InsertIntoHiveTable( } val partitionAttributes = partitionColumnNames.takeRight(numDynamicPartitions).map { name => - query.resolve(name :: Nil, sparkSession.sessionState.analyzer.resolver).getOrElse { + val attr = query.resolve(name :: Nil, sparkSession.sessionState.analyzer.resolver).getOrElse { throw new AnalysisException( s"Unable to resolve $name given [${query.output.map(_.name).mkString(", ")}]") }.asInstanceOf[Attribute] + // SPARK-28054: Hive metastore is not case preserving and keeps partition columns + // with lower cased names. Hive will validate the column names in the partition directories + // during `loadDynamicPartitions`. Spark needs to write partition directories with lower-cased + // column names in order to make `loadDynamicPartitions` work. + attr.withName(name.toLowerCase(Locale.ROOT)) } saveAsHiveFile( 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 62d3bad63bf58..4be3cd45454c6 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,16 +83,6 @@ 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, @@ -101,7 +91,7 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { outputSpec = FileFormatWriter.OutputSpec(outputLocation, customPartitionLocations, outputColumns), hadoopConf = hadoopConf, - partitionColumns = hiveCompatiblePartitionColumns, + partitionColumns = partitionAttributes, bucketSpec = None, statsTrackers = Seq(basicWriteJobStatsTracker(hadoopConf)), options = Map.empty)