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 @@ -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
Expand Down Expand Up @@ -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))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This move looks nicer and also cleaner due to the removal of scalastyle:off caselocale.

}

saveAsHiveFile(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This move looks good. And we should use partitionAttributes below.

FileFormatWriter.write(
sparkSession = sparkSession,
plan = plan,
Expand All @@ -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)
Expand Down