diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala
index 4331841fbffb45142181a469390bdb71e414a337..6f2a11748b1a5e57682e4c37cf0c8f4744082ee3 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala
@@ -108,18 +108,21 @@ object ExternalCatalogUtils {
       partitionColumnNames: Seq[String],
       tablePath: Path): Path = {
     val partitionPathStrings = partitionColumnNames.map { col =>
-      val partitionValue = spec(col)
-      val partitionString = if (partitionValue == null) {
-        DEFAULT_PARTITION_NAME
-      } else {
-        escapePathName(partitionValue)
-      }
-      escapePathName(col) + "=" + partitionString
+      getPartitionPathString(col, spec(col))
     }
     partitionPathStrings.foldLeft(tablePath) { (totalPath, nextPartPath) =>
       new Path(totalPath, nextPartPath)
     }
   }
+
+  def getPartitionPathString(col: String, value: String): String = {
+    val partitionString = if (value == null || value.isEmpty) {
+      DEFAULT_PARTITION_NAME
+    } else {
+      escapePathName(value)
+    }
+    escapePathName(col) + "=" + partitionString
+  }
 }
 
 object CatalogUtils {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
index aa561e57f77f5d7969e86406fc5e666888eda311..051fcaa63c7f8eae1f2a27de22e1b19244ca8d92 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
@@ -111,7 +111,12 @@ case class CatalogTablePartition(
    */
   def toRow(partitionSchema: StructType): InternalRow = {
     InternalRow.fromSeq(partitionSchema.map { field =>
-      Cast(Literal(spec(field.name)), field.dataType).eval()
+      val partValue = if (spec(field.name) == ExternalCatalogUtils.DEFAULT_PARTITION_NAME) {
+        null
+      } else {
+        spec(field.name)
+      }
+      Cast(Literal(partValue), field.dataType).eval()
     })
   }
 }
@@ -158,7 +163,7 @@ case class BucketSpec(
  * @param tracksPartitionsInCatalog whether this table's partition metadata is stored in the
  *                                  catalog. If false, it is inferred automatically based on file
  *                                  structure.
- * @param schemaPresevesCase Whether or not the schema resolved for this table is case-sensitive.
+ * @param schemaPreservesCase Whether or not the schema resolved for this table is case-sensitive.
  *                           When using a Hive Metastore, this flag is set to false if a case-
  *                           sensitive schema was unable to be read from the table properties.
  *                           Used to trigger case-sensitive schema inference at query time, when
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala
index 76161643976aaed783de30dc3d0251241ac86de3..b4aed23218357a468fbbed228833afc6aa632769 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala
@@ -319,7 +319,7 @@ case class FileSourceScanExec(
     val input = ctx.freshName("input")
     ctx.addMutableState("scala.collection.Iterator", input, s"$input = inputs[0];")
     val exprRows = output.zipWithIndex.map{ case (a, i) =>
-      new BoundReference(i, a.dataType, a.nullable)
+      BoundReference(i, a.dataType, a.nullable)
     }
     val row = ctx.freshName("row")
     ctx.INPUT_ROW = row
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala
index a9f79da6358dcf039699a1a28a179e494aca7871..92b22b813312e48dee6c3d75aacfe66ffe7baefe 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala
@@ -285,14 +285,11 @@ object FileFormatWriter extends Logging {
     /** Expressions that given a partition key build a string like: col1=val/col2=val/... */
     private def partitionStringExpression: Seq[Expression] = {
       description.partitionColumns.zipWithIndex.flatMap { case (c, i) =>
-        val escaped = ScalaUDF(
-          ExternalCatalogUtils.escapePathName _,
+        val partitionName = ScalaUDF(
+          ExternalCatalogUtils.getPartitionPathString _,
           StringType,
-          Seq(Cast(c, StringType)),
-          Seq(StringType))
-        val str = If(IsNull(c), Literal(ExternalCatalogUtils.DEFAULT_PARTITION_NAME), escaped)
-        val partitionName = Literal(c.name + "=") :: str :: Nil
-        if (i == 0) partitionName else Literal(Path.SEPARATOR) :: partitionName
+          Seq(Literal(c.name), Cast(c, StringType)))
+        if (i == 0) Seq(partitionName) else Seq(Literal(Path.SEPARATOR), partitionName)
       }
     }
 
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala
index bc290702dc37ff5c6f74b6d3e4f5693bb80ce5a6..ef29ee22e9501d7152a122d43ccd41112db7bdd5 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala
@@ -118,7 +118,7 @@ object PartitioningUtils {
       //   "hdfs://host:9000/invalidPath"
       //   "hdfs://host:9000/path"
       // TODO: Selective case sensitivity.
-      val discoveredBasePaths = optDiscoveredBasePaths.flatMap(x => x).map(_.toString.toLowerCase())
+      val discoveredBasePaths = optDiscoveredBasePaths.flatten.map(_.toString.toLowerCase())
       assert(
         discoveredBasePaths.distinct.size == 1,
         "Conflicting directory structures detected. Suspicious paths:\b" +
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
index cbf146966bcfa16eb0251c535d92b4acd10443ee..2f0feee0efa8c3d6f78cfe279a5a1e8f4b1f8893 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
@@ -968,8 +968,8 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
     val partColNameMap = buildLowerCasePartColNameMap(catalogTable).mapValues(escapePathName)
     val clientPartitionNames =
       client.getPartitionNames(catalogTable, partialSpec.map(lowerCasePartitionSpec))
-    clientPartitionNames.map { partName =>
-      val partSpec = PartitioningUtils.parsePathFragmentAsSeq(partName)
+    clientPartitionNames.map { partitionPath =>
+      val partSpec = PartitioningUtils.parsePathFragmentAsSeq(partitionPath)
       partSpec.map { case (partName, partValue) =>
         partColNameMap(partName.toLowerCase) + "=" + escapePathName(partValue)
       }.mkString("/")
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala
index 3f84cbdb1b09c506fa958aa8ed6c8130822c2098..d98f174f6c553e5e6bdbbb60b991d1c7ed8a0a3c 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.hive
 import java.io.File
 
 import org.apache.spark.metrics.source.HiveCatalogMetrics
-import org.apache.spark.sql.{AnalysisException, QueryTest}
+import org.apache.spark.sql.{AnalysisException, QueryTest, Row}
 import org.apache.spark.sql.hive.test.TestHiveSingleton
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.test.SQLTestUtils
@@ -28,6 +28,7 @@ import org.apache.spark.util.Utils
 
 class PartitionProviderCompatibilitySuite
   extends QueryTest with TestHiveSingleton with SQLTestUtils {
+  import testImplicits._
 
   private def setupPartitionedDatasourceTable(tableName: String, dir: File): Unit = {
     spark.range(5).selectExpr("id as fieldOne", "id as partCol").write
@@ -294,6 +295,28 @@ class PartitionProviderCompatibilitySuite
         }
       }
     }
+
+    test(s"SPARK-19887 partition value is null - partition management $enabled") {
+      withTable("test") {
+        Seq((1, "p", 1), (2, null, 2)).toDF("a", "b", "c")
+          .write.partitionBy("b", "c").saveAsTable("test")
+        checkAnswer(spark.table("test"),
+          Row(1, "p", 1) :: Row(2, null, 2) :: Nil)
+
+        Seq((3, null: String, 3)).toDF("a", "b", "c")
+          .write.mode("append").partitionBy("b", "c").saveAsTable("test")
+        checkAnswer(spark.table("test"),
+          Row(1, "p", 1) :: Row(2, null, 2) :: Row(3, null, 3) :: Nil)
+        // make sure partition pruning also works.
+        checkAnswer(spark.table("test").filter($"b".isNotNull), Row(1, "p", 1))
+
+        // empty string is an invalid partition value and we treat it as null when read back.
+        Seq((4, "", 4)).toDF("a", "b", "c")
+          .write.mode("append").partitionBy("b", "c").saveAsTable("test")
+        checkAnswer(spark.table("test"),
+          Row(1, "p", 1) :: Row(2, null, 2) :: Row(3, null, 3) :: Row(4, null, 4) :: Nil)
+      }
+    }
   }
 
   /**