You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by ma...@apache.org on 2014/12/17 06:27:07 UTC

spark git commit: [SPARK-4618][SQL] Make foreign DDL commands options case-insensitive

Repository: spark
Updated Branches:
  refs/heads/master ec5c4279e -> 60698801e


[SPARK-4618][SQL] Make foreign DDL commands options case-insensitive

Using lowercase for ```options``` key to make it case-insensitive, then we should use lower case to get value from parameters.
So flowing cmd work
```
      create temporary table normal_parquet
      USING org.apache.spark.sql.parquet
      OPTIONS (
        PATH '/xxx/data'
      )
```

Author: scwf <wa...@huawei.com>
Author: wangfei <wa...@huawei.com>

Closes #3470 from scwf/ddl-ulcase and squashes the following commits:

ae78509 [scwf] address comments
8f4f585 [wangfei] address comments
3c132ef [scwf] minor fix
a0fc20b [scwf] Merge branch 'master' of https://github.com/apache/spark into ddl-ulcase
4f86401 [scwf] adding CaseInsensitiveMap
e244e8d [wangfei] using lower case in json
e0cb017 [wangfei] make options in-casesensitive


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

Branch: refs/heads/master
Commit: 60698801ebc4c4947cfc7f46762a7cca2ed40452
Parents: ec5c427
Author: scwf <wa...@huawei.com>
Authored: Tue Dec 16 21:26:36 2014 -0800
Committer: Michael Armbrust <mi...@databricks.com>
Committed: Tue Dec 16 21:26:36 2014 -0800

----------------------------------------------------------------------
 .../scala/org/apache/spark/sql/sources/ddl.scala | 19 ++++++++++++++++++-
 .../apache/spark/sql/sources/interfaces.scala    |  6 +++++-
 .../spark/sql/sources/TableScanSuite.scala       |  6 +++---
 3 files changed, 26 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/60698801/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala
index ca510cb..8a66ac3 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala
@@ -100,9 +100,26 @@ private[sql] case class CreateTableUsing(
         }
     }
     val dataSource = clazz.newInstance().asInstanceOf[org.apache.spark.sql.sources.RelationProvider]
-    val relation = dataSource.createRelation(sqlContext, options)
+    val relation = dataSource.createRelation(sqlContext, new CaseInsensitiveMap(options))
 
     sqlContext.baseRelationToSchemaRDD(relation).registerTempTable(tableName)
     Seq.empty
   }
 }
+
+/**
+ * Builds a map in which keys are case insensitive
+ */
+protected class CaseInsensitiveMap(map: Map[String, String]) extends Map[String, String] {
+
+  val baseMap = map.map(kv => kv.copy(_1 = kv._1.toLowerCase))
+
+  override def get(k: String): Option[String] = baseMap.get(k.toLowerCase)
+
+  override def + [B1 >: String](kv: (String, B1)): Map[String, B1] =
+    baseMap + kv.copy(_1 = kv._1.toLowerCase)
+
+  override def iterator: Iterator[(String, String)] = baseMap.iterator
+
+  override def -(key: String): Map[String, String] = baseMap - key.toLowerCase()
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/60698801/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala
index 939b4e1..02eff80 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala
@@ -36,7 +36,11 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, Attribute}
  */
 @DeveloperApi
 trait RelationProvider {
-  /** Returns a new base relation with the given parameters. */
+  /**
+   * Returns a new base relation with the given parameters.
+   * Note: the parameters' keywords are case insensitive and this insensitivity is enforced
+   * by the Map that is passed to the function.
+   */
   def createRelation(sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation
 }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/60698801/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala
index b254b06..3cd7b01 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala
@@ -25,7 +25,7 @@ class SimpleScanSource extends RelationProvider {
   override def createRelation(
       sqlContext: SQLContext,
       parameters: Map[String, String]): BaseRelation = {
-    SimpleScan(parameters("from").toInt, parameters("to").toInt)(sqlContext)
+    SimpleScan(parameters("from").toInt, parameters("TO").toInt)(sqlContext)
   }
 }
 
@@ -47,8 +47,8 @@ class TableScanSuite extends DataSourceTest {
         |CREATE TEMPORARY TABLE oneToTen
         |USING org.apache.spark.sql.sources.SimpleScanSource
         |OPTIONS (
-        |  from '1',
-        |  to '10'
+        |  From '1',
+        |  To '10'
         |)
       """.stripMargin)
   }


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