Repository: spark
Updated Branches:
  refs/heads/branch-2.1 30ce056d8 -> 56865a1e9


[SPARK-19318][SPARK-22041][SPARK-16625][BACKPORT-2.1][SQL] Docker test case 
failure: `: General data types to be mapped to Oracle`

## What changes were proposed in this pull request?

This PR is backport of https://github.com/apache/spark/pull/16891 to Spark 2.1.

## How was this patch tested?

unit tests

Author: Yuming Wang <wgy...@gmail.com>

Closes #19259 from wangyum/SPARK-22041-BACKPORT-2.1.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/56865a1e
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/56865a1e
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/56865a1e

Branch: refs/heads/branch-2.1
Commit: 56865a1e9319f18b83c7b7a10738f270d5b1dc50
Parents: 30ce056
Author: Yuming Wang <wgy...@gmail.com>
Authored: Tue Sep 19 16:55:13 2017 -0700
Committer: gatorsmile <gatorsm...@gmail.com>
Committed: Tue Sep 19 16:55:13 2017 -0700

----------------------------------------------------------------------
 .../spark/sql/jdbc/OracleIntegrationSuite.scala | 40 +++++++++++++++-
 .../spark/sql/catalyst/json/JSONOptions.scala   |  4 +-
 .../sql/catalyst/util/CaseInsensitiveMap.scala  | 31 ++++++++----
 .../sql/execution/datasources/DataSource.scala  |  4 +-
 .../execution/datasources/csv/CSVOptions.scala  |  6 +--
 .../datasources/jdbc/JDBCOptions.scala          | 10 ++--
 .../datasources/parquet/ParquetOptions.scala    |  4 +-
 .../execution/streaming/FileStreamOptions.scala |  4 +-
 .../org/apache/spark/sql/jdbc/JDBCSuite.scala   | 50 +++++++++++++++++++-
 .../apache/spark/sql/jdbc/JDBCWriteSuite.scala  | 13 +++++
 .../spark/sql/hive/HiveExternalCatalog.scala    |  4 +-
 .../apache/spark/sql/hive/orc/OrcOptions.scala  |  4 +-
 12 files changed, 143 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/56865a1e/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala
----------------------------------------------------------------------
diff --git 
a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala
 
b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala
index e111e17..3b773f9 100644
--- 
a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala
+++ 
b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala
@@ -67,10 +67,35 @@ class OracleIntegrationSuite extends 
DockerJDBCIntegrationSuite with SharedSQLCo
     conn.prepareStatement(
       "INSERT INTO numerics VALUES (4, 1.23, 9999999999)").executeUpdate();
     conn.commit();
-  }
 
+    conn.prepareStatement("CREATE TABLE datetime (id NUMBER(10), d DATE, t 
TIMESTAMP)")
+      .executeUpdate()
+    conn.prepareStatement(
+      """INSERT INTO datetime VALUES
+        |(1, {d '1991-11-09'}, {ts '1996-01-01 01:23:45'})
+      """.stripMargin.replaceAll("\n", " ")).executeUpdate()
+    conn.commit()
+
+    sql(
+      s"""
+         |CREATE TEMPORARY VIEW datetime
+         |USING org.apache.spark.sql.jdbc
+         |OPTIONS (url '$jdbcUrl', dbTable 'datetime', 
oracle.jdbc.mapDateToTimestamp 'false')
+      """.stripMargin.replaceAll("\n", " "))
+
+    conn.prepareStatement("CREATE TABLE datetime1 (id NUMBER(10), d DATE, t 
TIMESTAMP)")
+      .executeUpdate()
+    conn.commit()
+
+    sql(
+      s"""
+         |CREATE TEMPORARY VIEW datetime1
+         |USING org.apache.spark.sql.jdbc
+         |OPTIONS (url '$jdbcUrl', dbTable 'datetime1', 
oracle.jdbc.mapDateToTimestamp 'false')
+      """.stripMargin.replaceAll("\n", " "))
+  }
 
-  test("SPARK-16625 : Importing Oracle numeric types") { 
+  test("SPARK-16625 : Importing Oracle numeric types") {
     val df = sqlContext.read.jdbc(jdbcUrl, "numerics", new Properties);
     val rows = df.collect()
     assert(rows.size == 1)
@@ -172,4 +197,15 @@ class OracleIntegrationSuite extends 
DockerJDBCIntegrationSuite with SharedSQLCo
     assert(values.getDate(9).equals(dateVal))
     assert(values.getTimestamp(10).equals(timestampVal))
   }
+
+  test("SPARK-19318: connection property keys should be case-sensitive") {
+    def checkRow(row: Row): Unit = {
+      assert(row.getDecimal(0).compareTo(BigDecimal.valueOf(1)) == 0)
+      assert(row.getDate(1).equals(Date.valueOf("1991-11-09")))
+      assert(row.getTimestamp(2).equals(Timestamp.valueOf("1996-01-01 
01:23:45")))
+    }
+    checkRow(sql("SELECT * FROM datetime where id = 1").head())
+    sql("INSERT INTO TABLE datetime1 SELECT * FROM datetime where id = 1")
+    checkRow(sql("SELECT * FROM datetime1 where id = 1").head())
+  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/56865a1e/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala
index 38e191b..02bd8de 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala
@@ -31,10 +31,10 @@ import 
org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, CompressionCodecs
  * Most of these map directly to Jackson's internal options, specified in 
[[JsonParser.Feature]].
  */
 private[sql] class JSONOptions(
-    @transient private val parameters: CaseInsensitiveMap)
+    @transient private val parameters: CaseInsensitiveMap[String])
   extends Logging with Serializable  {
 
-  def this(parameters: Map[String, String]) = this(new 
CaseInsensitiveMap(parameters))
+  def this(parameters: Map[String, String]) = 
this(CaseInsensitiveMap(parameters))
 
   val samplingRatio =
     parameters.get("samplingRatio").map(_.toDouble).getOrElse(1.0)

http://git-wip-us.apache.org/repos/asf/spark/blob/56865a1e/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala
index a7f7a8a..ba84265 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala
@@ -18,19 +18,34 @@
 package org.apache.spark.sql.catalyst.util
 
 /**
- * Builds a map in which keys are case insensitive
+ * Builds a map in which keys are case insensitive. Input map can be accessed 
for cases where
+ * case-sensitive information is required. The primary constructor is marked 
private to avoid
+ * nested case-insensitive map creation, otherwise the keys in the original 
map will become
+ * case-insensitive in this scenario.
  */
-class CaseInsensitiveMap(map: Map[String, String]) extends Map[String, String]
+class CaseInsensitiveMap[T] private (val originalMap: Map[String, T]) extends 
Map[String, T]
   with Serializable {
 
-  val baseMap = map.map(kv => kv.copy(_1 = kv._1.toLowerCase))
+  val keyLowerCasedMap = originalMap.map(kv => kv.copy(_1 = kv._1.toLowerCase))
 
-  override def get(k: String): Option[String] = baseMap.get(k.toLowerCase)
+  override def get(k: String): Option[T] = keyLowerCasedMap.get(k.toLowerCase)
 
-  override def + [B1 >: String](kv: (String, B1)): Map[String, B1] =
-    baseMap + kv.copy(_1 = kv._1.toLowerCase)
+  override def contains(k: String): Boolean = 
keyLowerCasedMap.contains(k.toLowerCase)
 
-  override def iterator: Iterator[(String, String)] = baseMap.iterator
+  override def +[B1 >: T](kv: (String, B1)): Map[String, B1] = {
+    new CaseInsensitiveMap(originalMap + kv)
+  }
 
-  override def -(key: String): Map[String, String] = baseMap - key.toLowerCase
+  override def iterator: Iterator[(String, T)] = keyLowerCasedMap.iterator
+
+  override def -(key: String): Map[String, T] = {
+    new CaseInsensitiveMap(originalMap.filterKeys(!_.equalsIgnoreCase(key)))
+  }
+}
+
+object CaseInsensitiveMap {
+  def apply[T](params: Map[String, T]): CaseInsensitiveMap[T] = params match {
+    case caseSensitiveMap: CaseInsensitiveMap[T] => caseSensitiveMap
+    case _ => new CaseInsensitiveMap(params)
+  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/56865a1e/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
index af70bf7..5319a33 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
@@ -85,7 +85,7 @@ case class DataSource(
 
   lazy val providingClass: Class[_] = DataSource.lookupDataSource(className)
   lazy val sourceInfo = sourceSchema()
-  private val caseInsensitiveOptions = new CaseInsensitiveMap(options)
+  private val caseInsensitiveOptions = CaseInsensitiveMap(options)
 
   /**
    * Get the schema of the given FileFormat, if provided by 
`userSpecifiedSchema`, or try to infer
@@ -637,7 +637,7 @@ object DataSource {
    * [[CatalogStorageFormat]]. Note that, the `path` option is removed from 
options after this.
    */
   def buildStorageFormatFromOptions(options: Map[String, String]): 
CatalogStorageFormat = {
-    val path = new CaseInsensitiveMap(options).get("path")
+    val path = CaseInsensitiveMap(options).get("path")
     val optionsWithoutPath = options.filterKeys(_.toLowerCase != "path")
     CatalogStorageFormat.empty.copy(locationUri = path, properties = 
optionsWithoutPath)
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/56865a1e/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala
index 21e5030..cba7bb0 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala
@@ -25,10 +25,10 @@ import org.apache.commons.lang3.time.FastDateFormat
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, 
CompressionCodecs, ParseModes}
 
-private[csv] class CSVOptions(@transient private val parameters: 
CaseInsensitiveMap)
+private[csv] class CSVOptions(@transient private val parameters: 
CaseInsensitiveMap[String])
   extends Logging with Serializable {
 
-  def this(parameters: Map[String, String]) = this(new 
CaseInsensitiveMap(parameters))
+  def this(parameters: Map[String, String]) = 
this(CaseInsensitiveMap(parameters))
 
   private def getChar(paramName: String, default: Char): Char = {
     val paramValue = parameters.get(paramName)
@@ -130,7 +130,7 @@ private[csv] class CSVOptions(@transient private val 
parameters: CaseInsensitive
 
 object CSVOptions {
 
-  def apply(): CSVOptions = new CSVOptions(new CaseInsensitiveMap(Map.empty))
+  def apply(): CSVOptions = new CSVOptions(CaseInsensitiveMap(Map.empty))
 
   def apply(paramName: String, paramValue: String): CSVOptions = {
     new CSVOptions(Map(paramName -> paramValue))

http://git-wip-us.apache.org/repos/asf/spark/blob/56865a1e/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala
index d94fa7e..0af5a99 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala
@@ -26,15 +26,15 @@ import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
  * Options for the JDBC data source.
  */
 class JDBCOptions(
-    @transient private val parameters: CaseInsensitiveMap)
+    @transient private val parameters: CaseInsensitiveMap[String])
   extends Serializable {
 
   import JDBCOptions._
 
-  def this(parameters: Map[String, String]) = this(new 
CaseInsensitiveMap(parameters))
+  def this(parameters: Map[String, String]) = 
this(CaseInsensitiveMap(parameters))
 
   def this(url: String, table: String, parameters: Map[String, String]) = {
-    this(new CaseInsensitiveMap(parameters ++ Map(
+    this(CaseInsensitiveMap(parameters ++ Map(
       JDBCOptions.JDBC_URL -> url,
       JDBCOptions.JDBC_TABLE_NAME -> table)))
   }
@@ -44,7 +44,7 @@ class JDBCOptions(
    */
   val asProperties: Properties = {
     val properties = new Properties()
-    parameters.foreach { case (k, v) => properties.setProperty(k, v) }
+    parameters.originalMap.foreach { case (k, v) => properties.setProperty(k, 
v) }
     properties
   }
 
@@ -55,7 +55,7 @@ class JDBCOptions(
    */
   val asConnectionProperties: Properties = {
     val properties = new Properties()
-    parameters.filterKeys(key => !jdbcOptionNames(key.toLowerCase))
+    parameters.originalMap.filterKeys(key => !jdbcOptionNames(key.toLowerCase))
       .foreach { case (k, v) => properties.setProperty(k, v) }
     properties
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/56865a1e/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala
index a81a95d..bdda299 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala
@@ -26,14 +26,14 @@ import org.apache.spark.sql.internal.SQLConf
  * Options for the Parquet data source.
  */
 private[parquet] class ParquetOptions(
-    @transient private val parameters: CaseInsensitiveMap,
+    @transient private val parameters: CaseInsensitiveMap[String],
     @transient private val sqlConf: SQLConf)
   extends Serializable {
 
   import ParquetOptions._
 
   def this(parameters: Map[String, String], sqlConf: SQLConf) =
-    this(new CaseInsensitiveMap(parameters), sqlConf)
+    this(CaseInsensitiveMap(parameters), sqlConf)
 
   /**
    * Compression codec to use. By default use the value specified in SQLConf.

http://git-wip-us.apache.org/repos/asf/spark/blob/56865a1e/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamOptions.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamOptions.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamOptions.scala
index fe64838..e7ba901 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamOptions.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamOptions.scala
@@ -26,9 +26,9 @@ import org.apache.spark.util.Utils
 /**
  * User specified options for file streams.
  */
-class FileStreamOptions(parameters: CaseInsensitiveMap) extends Logging {
+class FileStreamOptions(parameters: CaseInsensitiveMap[String]) extends 
Logging {
 
-  def this(parameters: Map[String, String]) = this(new 
CaseInsensitiveMap(parameters))
+  def this(parameters: Map[String, String]) = 
this(CaseInsensitiveMap(parameters))
 
   val maxFilesPerTrigger: Option[Int] = 
parameters.get("maxFilesPerTrigger").map { str =>
     Try(str.toInt).toOption.filter(_ > 0).getOrElse {

http://git-wip-us.apache.org/repos/asf/spark/blob/56865a1e/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
index 3846329..6bdbc58 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
@@ -913,6 +913,54 @@ class JDBCSuite extends SparkFunSuite
       "dbtable" -> "t1",
       "numPartitions" -> "10")
     assert(new JDBCOptions(parameters).asConnectionProperties.isEmpty)
-    assert(new JDBCOptions(new 
CaseInsensitiveMap(parameters)).asConnectionProperties.isEmpty)
+    assert(new 
JDBCOptions(CaseInsensitiveMap(parameters)).asConnectionProperties.isEmpty)
+  }
+
+  test("SPARK-19318: Connection properties keys should be case-sensitive.") {
+    def testJdbcOptions(options: JDBCOptions): Unit = {
+      // Spark JDBC data source options are case-insensitive
+      assert(options.table == "t1")
+      // When we convert it to properties, it should be case-sensitive.
+      assert(options.asProperties.size == 3)
+      assert(options.asProperties.get("customkey") == null)
+      assert(options.asProperties.get("customKey") == "a-value")
+      assert(options.asConnectionProperties.size == 1)
+      assert(options.asConnectionProperties.get("customkey") == null)
+      assert(options.asConnectionProperties.get("customKey") == "a-value")
+    }
+
+    val parameters = Map("url" -> url, "dbTAblE" -> "t1", "customKey" -> 
"a-value")
+    testJdbcOptions(new JDBCOptions(parameters))
+    testJdbcOptions(new JDBCOptions(CaseInsensitiveMap(parameters)))
+    // test add/remove key-value from the case-insensitive map
+    var modifiedParameters = CaseInsensitiveMap(Map.empty) ++ parameters
+    testJdbcOptions(new JDBCOptions(modifiedParameters))
+    modifiedParameters -= "dbtable"
+    assert(modifiedParameters.get("dbTAblE").isEmpty)
+    modifiedParameters -= "customkey"
+    assert(modifiedParameters.get("customKey").isEmpty)
+    modifiedParameters += ("customKey" -> "a-value")
+    modifiedParameters += ("dbTable" -> "t1")
+    testJdbcOptions(new JDBCOptions(modifiedParameters))
+    assert ((modifiedParameters -- parameters.keys).size == 0)
+  }
+
+  test("SPARK-19318: jdbc data source options should be treated 
case-insensitive.") {
+    val df = spark.read.format("jdbc")
+      .option("Url", urlWithUserAndPass)
+      .option("DbTaBle", "TEST.PEOPLE")
+      .load()
+    assert(df.count() == 3)
+
+    withTempView("people_view") {
+      sql(
+        s"""
+           |CREATE TEMPORARY VIEW people_view
+           |USING org.apache.spark.sql.jdbc
+           |OPTIONS (uRl '$url', DbTaBlE 'TEST.PEOPLE', User 'testUser', 
PassWord 'testPass')
+        """.stripMargin.replaceAll("\n", " "))
+
+      assert(sql("select * from people_view").count() == 3)
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/56865a1e/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala
index e3d3c6c..8283ff5 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala
@@ -312,4 +312,17 @@ class JDBCWriteSuite extends SharedSQLContext with 
BeforeAndAfter {
       .options(properties.asScala)
       .save()
   }
+
+  test("SPARK-19318 temporary view data source option keys should be 
case-insensitive") {
+    withTempView("people_view") {
+      sql(
+        s"""
+           |CREATE TEMPORARY VIEW people_view
+           |USING org.apache.spark.sql.jdbc
+           |OPTIONS (uRl '$url1', DbTaBlE 'TEST.PEOPLE1', User 'testUser', 
PassWord 'testPass')
+        """.stripMargin.replaceAll("\n", " "))
+      sql("INSERT OVERWRITE TABLE PEOPLE_VIEW SELECT * FROM PEOPLE")
+      assert(sql("select * from people_view").count() == 2)
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/56865a1e/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
----------------------------------------------------------------------
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 23777f2..1f5d7e0 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
@@ -473,7 +473,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, 
hadoopConf: Configurat
     // will be updated automatically in Hive metastore by the `alterTable` 
call at the end of this
     // method. Here we only update the path option if the path option already 
exists in storage
     // properties, to avoid adding a unnecessary path option for Hive serde 
tables.
-    val hasPathOption = new 
CaseInsensitiveMap(rawTable.storage.properties).contains("path")
+    val hasPathOption = 
CaseInsensitiveMap(rawTable.storage.properties).contains("path")
     val storageWithNewPath = if (rawTable.tableType == MANAGED && 
hasPathOption) {
       // If it's a managed table with path option and we are renaming it, then 
the path option
       // becomes inaccurate and we need to update it according to the new 
table name.
@@ -491,7 +491,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, 
hadoopConf: Configurat
   }
 
   private def getLocationFromStorageProps(table: CatalogTable): Option[String] 
= {
-    new CaseInsensitiveMap(table.storage.properties).get("path")
+    CaseInsensitiveMap(table.storage.properties).get("path")
   }
 
   private def updateLocationInStorageProps(

http://git-wip-us.apache.org/repos/asf/spark/blob/56865a1e/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala
index ac587ab..ccaa568 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala
@@ -22,12 +22,12 @@ import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
 /**
  * Options for the ORC data source.
  */
-private[orc] class OrcOptions(@transient private val parameters: 
CaseInsensitiveMap)
+private[orc] class OrcOptions(@transient private val parameters: 
CaseInsensitiveMap[String])
   extends Serializable {
 
   import OrcOptions._
 
-  def this(parameters: Map[String, String]) = this(new 
CaseInsensitiveMap(parameters))
+  def this(parameters: Map[String, String]) = 
this(CaseInsensitiveMap(parameters))
 
   /**
    * Compression codec to use. By default snappy compression.


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to