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

Choose a reason for hiding this comment

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

indent.

Copy link
Member Author

Choose a reason for hiding this comment

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

oops..will fix in a follow-up. Thanks.

}
// scalastyle:on caselocale

FileFormatWriter.write(
sparkSession = sparkSession,
plan = plan,
Expand All @@ -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)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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")
Copy link
Member

Choose a reason for hiding this comment

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

Should we set it back?

Use withSQLConf ?

Also do we need to set the case sensitivity conf?

Copy link
Member Author

Choose a reason for hiding this comment

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

This set follows other tests in same suite. Using withSQLConf is good, yes.

The case sensitivity conf has no effect on this, I think it is fine.

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
Expand Down