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