Skip to content

Commit a0ce845

Browse files
committed
[SPARK-19887][SQL] dynamic partition keys can be null or empty string
When dynamic partition value is null or empty string, we should write the data to a directory like `a=__HIVE_DEFAULT_PARTITION__`, when we read the data back, we should respect this special directory name and treat it as null. This is the same behavior of impala, see https://issues.apache.org/jira/browse/IMPALA-252 new regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #17277 from cloud-fan/partition. (cherry picked from commit dacc382) Signed-off-by: Wenchen Fan <wenchen@databricks.com>
1 parent 4545782 commit a0ce845

File tree

7 files changed

+49
-21
lines changed

7 files changed

+49
-21
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala

Lines changed: 10 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -108,18 +108,21 @@ object ExternalCatalogUtils {
108108
partitionColumnNames: Seq[String],
109109
tablePath: Path): Path = {
110110
val partitionPathStrings = partitionColumnNames.map { col =>
111-
val partitionValue = spec(col)
112-
val partitionString = if (partitionValue == null) {
113-
DEFAULT_PARTITION_NAME
114-
} else {
115-
escapePathName(partitionValue)
116-
}
117-
escapePathName(col) + "=" + partitionString
111+
getPartitionPathString(col, spec(col))
118112
}
119113
partitionPathStrings.foldLeft(tablePath) { (totalPath, nextPartPath) =>
120114
new Path(totalPath, nextPartPath)
121115
}
122116
}
117+
118+
def getPartitionPathString(col: String, value: String): String = {
119+
val partitionString = if (value == null || value.isEmpty) {
120+
DEFAULT_PARTITION_NAME
121+
} else {
122+
escapePathName(value)
123+
}
124+
escapePathName(col) + "=" + partitionString
125+
}
123126
}
124127

125128
object CatalogUtils {

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala

Lines changed: 7 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -111,7 +111,12 @@ case class CatalogTablePartition(
111111
*/
112112
def toRow(partitionSchema: StructType): InternalRow = {
113113
InternalRow.fromSeq(partitionSchema.map { field =>
114-
Cast(Literal(spec(field.name)), field.dataType).eval()
114+
val partValue = if (spec(field.name) == ExternalCatalogUtils.DEFAULT_PARTITION_NAME) {
115+
null
116+
} else {
117+
spec(field.name)
118+
}
119+
Cast(Literal(partValue), field.dataType).eval()
115120
})
116121
}
117122
}
@@ -158,7 +163,7 @@ case class BucketSpec(
158163
* @param tracksPartitionsInCatalog whether this table's partition metadata is stored in the
159164
* catalog. If false, it is inferred automatically based on file
160165
* structure.
161-
* @param schemaPresevesCase Whether or not the schema resolved for this table is case-sensitive.
166+
* @param schemaPreservesCase Whether or not the schema resolved for this table is case-sensitive.
162167
* When using a Hive Metastore, this flag is set to false if a case-
163168
* sensitive schema was unable to be read from the table properties.
164169
* Used to trigger case-sensitive schema inference at query time, when

sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -319,7 +319,7 @@ case class FileSourceScanExec(
319319
val input = ctx.freshName("input")
320320
ctx.addMutableState("scala.collection.Iterator", input, s"$input = inputs[0];")
321321
val exprRows = output.zipWithIndex.map{ case (a, i) =>
322-
new BoundReference(i, a.dataType, a.nullable)
322+
BoundReference(i, a.dataType, a.nullable)
323323
}
324324
val row = ctx.freshName("row")
325325
ctx.INPUT_ROW = row

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala

Lines changed: 4 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -285,14 +285,11 @@ object FileFormatWriter extends Logging {
285285
/** Expressions that given a partition key build a string like: col1=val/col2=val/... */
286286
private def partitionStringExpression: Seq[Expression] = {
287287
description.partitionColumns.zipWithIndex.flatMap { case (c, i) =>
288-
val escaped = ScalaUDF(
289-
ExternalCatalogUtils.escapePathName _,
288+
val partitionName = ScalaUDF(
289+
ExternalCatalogUtils.getPartitionPathString _,
290290
StringType,
291-
Seq(Cast(c, StringType)),
292-
Seq(StringType))
293-
val str = If(IsNull(c), Literal(ExternalCatalogUtils.DEFAULT_PARTITION_NAME), escaped)
294-
val partitionName = Literal(c.name + "=") :: str :: Nil
295-
if (i == 0) partitionName else Literal(Path.SEPARATOR) :: partitionName
291+
Seq(Literal(c.name), Cast(c, StringType)))
292+
if (i == 0) Seq(partitionName) else Seq(Literal(Path.SEPARATOR), partitionName)
296293
}
297294
}
298295

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -118,7 +118,7 @@ object PartitioningUtils {
118118
// "hdfs://host:9000/invalidPath"
119119
// "hdfs://host:9000/path"
120120
// TODO: Selective case sensitivity.
121-
val discoveredBasePaths = optDiscoveredBasePaths.flatMap(x => x).map(_.toString.toLowerCase())
121+
val discoveredBasePaths = optDiscoveredBasePaths.flatten.map(_.toString.toLowerCase())
122122
assert(
123123
discoveredBasePaths.distinct.size == 1,
124124
"Conflicting directory structures detected. Suspicious paths:\b" +

sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -968,8 +968,8 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
968968
val partColNameMap = buildLowerCasePartColNameMap(catalogTable).mapValues(escapePathName)
969969
val clientPartitionNames =
970970
client.getPartitionNames(catalogTable, partialSpec.map(lowerCasePartitionSpec))
971-
clientPartitionNames.map { partName =>
972-
val partSpec = PartitioningUtils.parsePathFragmentAsSeq(partName)
971+
clientPartitionNames.map { partitionPath =>
972+
val partSpec = PartitioningUtils.parsePathFragmentAsSeq(partitionPath)
973973
partSpec.map { case (partName, partValue) =>
974974
partColNameMap(partName.toLowerCase) + "=" + escapePathName(partValue)
975975
}.mkString("/")

sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala

Lines changed: 24 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,14 +20,15 @@ package org.apache.spark.sql.hive
2020
import java.io.File
2121

2222
import org.apache.spark.metrics.source.HiveCatalogMetrics
23-
import org.apache.spark.sql.{AnalysisException, QueryTest}
23+
import org.apache.spark.sql.{AnalysisException, QueryTest, Row}
2424
import org.apache.spark.sql.hive.test.TestHiveSingleton
2525
import org.apache.spark.sql.internal.SQLConf
2626
import org.apache.spark.sql.test.SQLTestUtils
2727
import org.apache.spark.util.Utils
2828

2929
class PartitionProviderCompatibilitySuite
3030
extends QueryTest with TestHiveSingleton with SQLTestUtils {
31+
import testImplicits._
3132

3233
private def setupPartitionedDatasourceTable(tableName: String, dir: File): Unit = {
3334
spark.range(5).selectExpr("id as fieldOne", "id as partCol").write
@@ -294,6 +295,28 @@ class PartitionProviderCompatibilitySuite
294295
}
295296
}
296297
}
298+
299+
test(s"SPARK-19887 partition value is null - partition management $enabled") {
300+
withTable("test") {
301+
Seq((1, "p", 1), (2, null, 2)).toDF("a", "b", "c")
302+
.write.partitionBy("b", "c").saveAsTable("test")
303+
checkAnswer(spark.table("test"),
304+
Row(1, "p", 1) :: Row(2, null, 2) :: Nil)
305+
306+
Seq((3, null: String, 3)).toDF("a", "b", "c")
307+
.write.mode("append").partitionBy("b", "c").saveAsTable("test")
308+
checkAnswer(spark.table("test"),
309+
Row(1, "p", 1) :: Row(2, null, 2) :: Row(3, null, 3) :: Nil)
310+
// make sure partition pruning also works.
311+
checkAnswer(spark.table("test").filter($"b".isNotNull), Row(1, "p", 1))
312+
313+
// empty string is an invalid partition value and we treat it as null when read back.
314+
Seq((4, "", 4)).toDF("a", "b", "c")
315+
.write.mode("append").partitionBy("b", "c").saveAsTable("test")
316+
checkAnswer(spark.table("test"),
317+
Row(1, "p", 1) :: Row(2, null, 2) :: Row(3, null, 3) :: Row(4, null, 4) :: Nil)
318+
}
319+
}
297320
}
298321

299322
/**

0 commit comments

Comments
 (0)