spark-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From wenc...@apache.org
Subject spark git commit: [SPARK-26071][SQL] disallow map as map key
Date Mon, 19 Nov 2018 14:42:35 GMT
Repository: spark
Updated Branches:
  refs/heads/master 35c551635 -> 219b037f0


[SPARK-26071][SQL] disallow map as map key

## What changes were proposed in this pull request?

Due to implementation limitation, currently Spark can't compare or do equality check between map types. As a result, map values can't appear in EQUAL or comparison expressions, can't be grouping key, etc.

The more important thing is, map loop up needs to do equality check of the map key, and thus can't support map as map key when looking up values from a map. Thus it's not useful to have map as map key.

This PR proposes to stop users from creating maps using map type as key. The list of expressions that are updated: `CreateMap`, `MapFromArrays`, `MapFromEntries`, `MapConcat`, `TransformKeys`. I manually checked all the places that create `MapType`, and came up with this list.

Note that, maps with map type key still exist, via reading from parquet files, converting from scala/java map, etc. This PR is not to completely forbid map as map key, but to avoid creating it by Spark itself.

Motivation: when I was trying to fix the duplicate key problem, I found it's impossible to do it with map type map key. I think it's reasonable to avoid map type map key for builtin functions.

## How was this patch tested?

updated test

Closes #23045 from cloud-fan/map-key.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>


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

Branch: refs/heads/master
Commit: 219b037f05636a3a7c8116987c319773f4145b63
Parents: 35c5516
Author: Wenchen Fan <wenchen@databricks.com>
Authored: Mon Nov 19 22:42:24 2018 +0800
Committer: Wenchen Fan <wenchen@databricks.com>
Committed: Mon Nov 19 22:42:24 2018 +0800

----------------------------------------------------------------------
 docs/sql-migration-guide-upgrade.md             |   6 +-
 .../expressions/collectionOperations.scala      |  12 +-
 .../expressions/complexTypeCreator.scala        |  14 ++-
 .../expressions/higherOrderFunctions.scala      |   4 +
 .../spark/sql/catalyst/util/TypeUtils.scala     |  10 +-
 .../CollectionExpressionsSuite.scala            | 113 +++++++++++--------
 .../catalyst/expressions/ComplexTypeSuite.scala |  83 +++++++-------
 .../expressions/ExpressionEvalHelper.scala      |  21 +++-
 .../expressions/HigherOrderFunctionsSuite.scala |  41 ++++---
 .../inputs/typeCoercion/native/mapconcat.sql    |   9 +-
 .../typeCoercion/native/mapconcat.sql.out       |  19 ++--
 11 files changed, 203 insertions(+), 129 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/219b037f/docs/sql-migration-guide-upgrade.md
----------------------------------------------------------------------
diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md
index 50458e9..07079d9 100644
--- a/docs/sql-migration-guide-upgrade.md
+++ b/docs/sql-migration-guide-upgrade.md
@@ -17,6 +17,8 @@ displayTitle: Spark SQL Upgrading Guide
 
   - The `ADD JAR` command previously returned a result set with the single value 0. It now returns an empty result set.
 
+  - In Spark version 2.4 and earlier, users can create map values with map type key via built-in function like `CreateMap`, `MapFromArrays`, etc. Since Spark 3.0, it's not allowed to create map values with map type key with these built-in functions. Users can still read map values with map type key from data source or Java/Scala collections, though they are not very useful.
+
 ## Upgrading From Spark SQL 2.3 to 2.4
 
   - In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause `array_contains` function to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below.
@@ -117,7 +119,7 @@ displayTitle: Spark SQL Upgrading Guide
 
   - Since Spark 2.4, Metadata files (e.g. Parquet summary files) and temporary files are not counted as data files when calculating table size during Statistics computation.
 
-  - Since Spark 2.4, empty strings are saved as quoted empty strings `""`. In version 2.3 and earlier, empty strings are equal to `null` values and do not reflect to any characters in saved CSV files. For example, the row of `"a", null, "", 1` was written as `a,,,1`. Since Spark 2.4, the same row is saved as `a,,"",1`. To restore the previous behavior, set the CSV option `emptyValue` to empty (not quoted) string.  
+  - Since Spark 2.4, empty strings are saved as quoted empty strings `""`. In version 2.3 and earlier, empty strings are equal to `null` values and do not reflect to any characters in saved CSV files. For example, the row of `"a", null, "", 1` was written as `a,,,1`. Since Spark 2.4, the same row is saved as `a,,"",1`. To restore the previous behavior, set the CSV option `emptyValue` to empty (not quoted) string.
 
   - Since Spark 2.4, The LOAD DATA command supports wildcard `?` and `*`, which match any one character, and zero or more characters, respectively. Example: `LOAD DATA INPATH '/tmp/folder*/'` or `LOAD DATA INPATH '/tmp/part-?'`. Special Characters like `space` also now work in paths. Example: `LOAD DATA INPATH '/tmp/folder name/'`.
 
@@ -303,7 +305,7 @@ displayTitle: Spark SQL Upgrading Guide
 ## Upgrading From Spark SQL 2.1 to 2.2
 
   - Spark 2.1.1 introduced a new configuration key: `spark.sql.hive.caseSensitiveInferenceMode`. It had a default setting of `NEVER_INFER`, which kept behavior identical to 2.1.0. However, Spark 2.2.0 changes this setting's default value to `INFER_AND_SAVE` to restore compatibility with reading Hive metastore tables whose underlying file schema have mixed-case column names. With the `INFER_AND_SAVE` configuration value, on first access Spark will perform schema inference on any Hive metastore table for which it has not already saved an inferred schema. Note that schema inference can be a very time-consuming operation for tables with thousands of partitions. If compatibility with mixed-case column names is not a concern, you can safely set `spark.sql.hive.caseSensitiveInferenceMode` to `NEVER_INFER` to avoid the initial overhead of schema inference. Note that with the new default `INFER_AND_SAVE` setting, the results of the schema inference are saved as a metastore key for future use
 . Therefore, the initial schema inference occurs only at a table's first access.
-  
+
   - Since Spark 2.2.1 and 2.3.0, the schema is always inferred at runtime when the data source tables have the columns that exist in both partition schema and data schema. The inferred schema does not have the partitioned columns. When reading the table, Spark respects the partition values of these overlapping columns instead of the values stored in the data source files. In 2.2.0 and 2.1.x release, the inferred schema is partitioned but the data of the table is invisible to users (i.e., the result set is empty).
 
   - Since Spark 2.2, view definitions are stored in a different way from prior versions. This may cause Spark unable to read views created by prior versions. In such cases, you need to recreate the views using `ALTER VIEW AS` or `CREATE OR REPLACE VIEW AS` with newer Spark versions.

http://git-wip-us.apache.org/repos/asf/spark/blob/219b037f/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
index 3c26095..4311674 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
@@ -521,13 +521,18 @@ case class MapEntries(child: Expression) extends UnaryExpression with ExpectsInp
 case class MapConcat(children: Seq[Expression]) extends ComplexTypeMergingExpression {
 
   override def checkInputDataTypes(): TypeCheckResult = {
-    var funcName = s"function $prettyName"
+    val funcName = s"function $prettyName"
     if (children.exists(!_.dataType.isInstanceOf[MapType])) {
       TypeCheckResult.TypeCheckFailure(
         s"input to $funcName should all be of type map, but it's " +
           children.map(_.dataType.catalogString).mkString("[", ", ", "]"))
     } else {
-      TypeUtils.checkForSameTypeInputExpr(children.map(_.dataType), funcName)
+      val sameTypeCheck = TypeUtils.checkForSameTypeInputExpr(children.map(_.dataType), funcName)
+      if (sameTypeCheck.isFailure) {
+        sameTypeCheck
+      } else {
+        TypeUtils.checkForMapKeyType(dataType.keyType)
+      }
     }
   }
 
@@ -740,7 +745,8 @@ case class MapFromEntries(child: Expression) extends UnaryExpression {
   @transient override lazy val dataType: MapType = dataTypeDetails.get._1
 
   override def checkInputDataTypes(): TypeCheckResult = dataTypeDetails match {
-    case Some(_) => TypeCheckResult.TypeCheckSuccess
+    case Some((mapType, _, _)) =>
+      TypeUtils.checkForMapKeyType(mapType.keyType)
     case None => TypeCheckResult.TypeCheckFailure(s"'${child.sql}' is of " +
       s"${child.dataType.catalogString} type. $prettyName accepts only arrays of pair structs.")
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/219b037f/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
index 0361372..6b77996 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
@@ -161,11 +161,11 @@ case class CreateMap(children: Seq[Expression]) extends Expression {
         "The given values of function map should all be the same type, but they are " +
           values.map(_.dataType.catalogString).mkString("[", ", ", "]"))
     } else {
-      TypeCheckResult.TypeCheckSuccess
+      TypeUtils.checkForMapKeyType(dataType.keyType)
     }
   }
 
-  override def dataType: DataType = {
+  override def dataType: MapType = {
     MapType(
       keyType = TypeCoercion.findCommonTypeDifferentOnlyInNullFlags(keys.map(_.dataType))
         .getOrElse(StringType),
@@ -224,6 +224,16 @@ case class MapFromArrays(left: Expression, right: Expression)
 
   override def inputTypes: Seq[AbstractDataType] = Seq(ArrayType, ArrayType)
 
+  override def checkInputDataTypes(): TypeCheckResult = {
+    val defaultCheck = super.checkInputDataTypes()
+    if (defaultCheck.isFailure) {
+      defaultCheck
+    } else {
+      val keyType = left.dataType.asInstanceOf[ArrayType].elementType
+      TypeUtils.checkForMapKeyType(keyType)
+    }
+  }
+
   override def dataType: DataType = {
     MapType(
       keyType = left.dataType.asInstanceOf[ArrayType].elementType,

http://git-wip-us.apache.org/repos/asf/spark/blob/219b037f/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala
index 0b698f9..8b31021 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala
@@ -514,6 +514,10 @@ case class TransformKeys(
 
   override def dataType: DataType = MapType(function.dataType, valueType, valueContainsNull)
 
+  override def checkInputDataTypes(): TypeCheckResult = {
+    TypeUtils.checkForMapKeyType(function.dataType)
+  }
+
   override def bind(f: (Expression, Seq[(DataType, Boolean)]) => LambdaFunction): TransformKeys = {
     copy(function = f(function, (keyType, false) :: (valueType, valueContainsNull) :: Nil))
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/219b037f/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala
index 76218b4..2a71fdb 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala
@@ -46,12 +46,20 @@ object TypeUtils {
     if (TypeCoercion.haveSameType(types)) {
       TypeCheckResult.TypeCheckSuccess
     } else {
-      return TypeCheckResult.TypeCheckFailure(
+      TypeCheckResult.TypeCheckFailure(
         s"input to $caller should all be the same type, but it's " +
           types.map(_.catalogString).mkString("[", ", ", "]"))
     }
   }
 
+  def checkForMapKeyType(keyType: DataType): TypeCheckResult = {
+    if (keyType.existsRecursively(_.isInstanceOf[MapType])) {
+      TypeCheckResult.TypeCheckFailure("The key of map cannot be/contain map.")
+    } else {
+      TypeCheckResult.TypeCheckSuccess
+    }
+  }
+
   def getNumeric(t: DataType): Numeric[Any] =
     t.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]]
 

http://git-wip-us.apache.org/repos/asf/spark/blob/219b037f/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala
index 2e0adbb..1415b7d 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala
@@ -25,6 +25,7 @@ import scala.util.Random
 import org.apache.spark.SparkFunSuite
 import org.apache.spark.sql.Row
 import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
 import org.apache.spark.sql.catalyst.util.DateTimeTestUtils
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
@@ -108,32 +109,28 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper
   }
 
   test("Map Concat") {
-    val m0 = Literal.create(Map("a" -> "1", "b" -> "2"), MapType(StringType, StringType,
+    val m0 = Literal.create(create_map("a" -> "1", "b" -> "2"), MapType(StringType, StringType,
       valueContainsNull = false))
-    val m1 = Literal.create(Map("c" -> "3", "a" -> "4"), MapType(StringType, StringType,
+    val m1 = Literal.create(create_map("c" -> "3", "a" -> "4"), MapType(StringType, StringType,
       valueContainsNull = false))
-    val m2 = Literal.create(Map("d" -> "4", "e" -> "5"), MapType(StringType, StringType))
-    val m3 = Literal.create(Map("a" -> "1", "b" -> "2"), MapType(StringType, StringType))
-    val m4 = Literal.create(Map("a" -> null, "c" -> "3"), MapType(StringType, StringType))
-    val m5 = Literal.create(Map("a" -> 1, "b" -> 2), MapType(StringType, IntegerType))
-    val m6 = Literal.create(Map("a" -> null, "c" -> 3), MapType(StringType, IntegerType))
-    val m7 = Literal.create(Map(List(1, 2) -> 1, List(3, 4) -> 2),
+    val m2 = Literal.create(create_map("d" -> "4", "e" -> "5"), MapType(StringType, StringType))
+    val m3 = Literal.create(create_map("a" -> "1", "b" -> "2"), MapType(StringType, StringType))
+    val m4 = Literal.create(create_map("a" -> null, "c" -> "3"), MapType(StringType, StringType))
+    val m5 = Literal.create(create_map("a" -> 1, "b" -> 2), MapType(StringType, IntegerType))
+    val m6 = Literal.create(create_map("a" -> null, "c" -> 3), MapType(StringType, IntegerType))
+    val m7 = Literal.create(create_map(List(1, 2) -> 1, List(3, 4) -> 2),
       MapType(ArrayType(IntegerType), IntegerType))
-    val m8 = Literal.create(Map(List(5, 6) -> 3, List(1, 2) -> 4),
+    val m8 = Literal.create(create_map(List(5, 6) -> 3, List(1, 2) -> 4),
       MapType(ArrayType(IntegerType), IntegerType))
-    val m9 = Literal.create(Map(Map(1 -> 2, 3 -> 4) -> 1, Map(5 -> 6, 7 -> 8) -> 2),
-      MapType(MapType(IntegerType, IntegerType), IntegerType))
-    val m10 = Literal.create(Map(Map(9 -> 10, 11 -> 12) -> 3, Map(1 -> 2, 3 -> 4) -> 4),
-      MapType(MapType(IntegerType, IntegerType), IntegerType))
-    val m11 = Literal.create(Map(1 -> "1", 2 -> "2"), MapType(IntegerType, StringType,
+    val m9 = Literal.create(create_map(1 -> "1", 2 -> "2"), MapType(IntegerType, StringType,
       valueContainsNull = false))
-    val m12 = Literal.create(Map(3 -> "3", 4 -> "4"), MapType(IntegerType, StringType,
+    val m10 = Literal.create(create_map(3 -> "3", 4 -> "4"), MapType(IntegerType, StringType,
       valueContainsNull = false))
-    val m13 = Literal.create(Map(1 -> 2, 3 -> 4),
+    val m11 = Literal.create(create_map(1 -> 2, 3 -> 4),
       MapType(IntegerType, IntegerType, valueContainsNull = false))
-    val m14 = Literal.create(Map(5 -> 6),
+    val m12 = Literal.create(create_map(5 -> 6),
       MapType(IntegerType, IntegerType, valueContainsNull = false))
-    val m15 = Literal.create(Map(7 -> null),
+    val m13 = Literal.create(create_map(7 -> null),
       MapType(IntegerType, IntegerType, valueContainsNull = true))
     val mNull = Literal.create(null, MapType(StringType, StringType))
 
@@ -147,7 +144,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper
 
     // maps with no overlap
     checkEvaluation(MapConcat(Seq(m0, m2)),
-      Map("a" -> "1", "b" -> "2", "d" -> "4", "e" -> "5"))
+      create_map("a" -> "1", "b" -> "2", "d" -> "4", "e" -> "5"))
 
     // 3 maps
     checkEvaluation(MapConcat(Seq(m0, m1, m2)),
@@ -174,7 +171,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper
     )
 
     // keys that are primitive
-    checkEvaluation(MapConcat(Seq(m11, m12)),
+    checkEvaluation(MapConcat(Seq(m9, m10)),
       (
         Array(1, 2, 3, 4), // keys
         Array("1", "2", "3", "4") // values
@@ -189,20 +186,11 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper
       )
     )
 
-    // keys that are maps, with overlap
-    checkEvaluation(MapConcat(Seq(m9, m10)),
-      (
-        Array(Map(1 -> 2, 3 -> 4), Map(5 -> 6, 7 -> 8), Map(9 -> 10, 11 -> 12),
-          Map(1 -> 2, 3 -> 4)), // keys
-        Array(1, 2, 3, 4) // values
-      )
-    )
-
     // both keys and value are primitive and valueContainsNull = false
-    checkEvaluation(MapConcat(Seq(m13, m14)), Map(1 -> 2, 3 -> 4, 5 -> 6))
+    checkEvaluation(MapConcat(Seq(m11, m12)), create_map(1 -> 2, 3 -> 4, 5 -> 6))
 
     // both keys and value are primitive and valueContainsNull = true
-    checkEvaluation(MapConcat(Seq(m13, m15)), Map(1 -> 2, 3 -> 4, 7 -> null))
+    checkEvaluation(MapConcat(Seq(m11, m13)), create_map(1 -> 2, 3 -> 4, 7 -> null))
 
     // null map
     checkEvaluation(MapConcat(Seq(m0, mNull)), null)
@@ -211,7 +199,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper
     checkEvaluation(MapConcat(Seq(mNull)), null)
 
     // single map
-    checkEvaluation(MapConcat(Seq(m0)), Map("a" -> "1", "b" -> "2"))
+    checkEvaluation(MapConcat(Seq(m0)), create_map("a" -> "1", "b" -> "2"))
 
     // no map
     checkEvaluation(MapConcat(Seq.empty), Map.empty)
@@ -245,12 +233,12 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper
     assert(MapConcat(Seq(m1, mNull)).nullable)
 
     val mapConcat = MapConcat(Seq(
-      Literal.create(Map(Seq(1, 2) -> Seq("a", "b")),
+      Literal.create(create_map(Seq(1, 2) -> Seq("a", "b")),
         MapType(
           ArrayType(IntegerType, containsNull = false),
           ArrayType(StringType, containsNull = false),
           valueContainsNull = false)),
-      Literal.create(Map(Seq(3, 4, null) -> Seq("c", "d", null), Seq(6) -> null),
+      Literal.create(create_map(Seq(3, 4, null) -> Seq("c", "d", null), Seq(6) -> null),
         MapType(
           ArrayType(IntegerType, containsNull = true),
           ArrayType(StringType, containsNull = true),
@@ -264,6 +252,18 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper
       Seq(1, 2) -> Seq("a", "b"),
       Seq(3, 4, null) -> Seq("c", "d", null),
       Seq(6) -> null))
+
+    // map key can't be map
+    val mapOfMap = Literal.create(Map(Map(1 -> 2, 3 -> 4) -> 1, Map(5 -> 6, 7 -> 8) -> 2),
+      MapType(MapType(IntegerType, IntegerType), IntegerType))
+    val mapOfMap2 = Literal.create(Map(Map(9 -> 10, 11 -> 12) -> 3, Map(1 -> 2, 3 -> 4) -> 4),
+      MapType(MapType(IntegerType, IntegerType), IntegerType))
+    val map = MapConcat(Seq(mapOfMap, mapOfMap2))
+    map.checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckSuccess => fail("should not allow map as map key")
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("The key of map cannot be/contain map"))
+    }
   }
 
   test("MapFromEntries") {
@@ -274,20 +274,20 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper
           StructField("b", valueType))),
         true)
     }
-    def r(values: Any*): InternalRow = create_row(values: _*)
+    def row(values: Any*): InternalRow = create_row(values: _*)
 
     // Primitive-type keys and values
     val aiType = arrayType(IntegerType, IntegerType)
-    val ai0 = Literal.create(Seq(r(1, 10), r(2, 20), r(3, 20)), aiType)
-    val ai1 = Literal.create(Seq(r(1, null), r(2, 20), r(3, null)), aiType)
+    val ai0 = Literal.create(Seq(row(1, 10), row(2, 20), row(3, 20)), aiType)
+    val ai1 = Literal.create(Seq(row(1, null), row(2, 20), row(3, null)), aiType)
     val ai2 = Literal.create(Seq.empty, aiType)
     val ai3 = Literal.create(null, aiType)
-    val ai4 = Literal.create(Seq(r(1, 10), r(1, 20)), aiType)
-    val ai5 = Literal.create(Seq(r(1, 10), r(null, 20)), aiType)
-    val ai6 = Literal.create(Seq(null, r(2, 20), null), aiType)
+    val ai4 = Literal.create(Seq(row(1, 10), row(1, 20)), aiType)
+    val ai5 = Literal.create(Seq(row(1, 10), row(null, 20)), aiType)
+    val ai6 = Literal.create(Seq(null, row(2, 20), null), aiType)
 
-    checkEvaluation(MapFromEntries(ai0), Map(1 -> 10, 2 -> 20, 3 -> 20))
-    checkEvaluation(MapFromEntries(ai1), Map(1 -> null, 2 -> 20, 3 -> null))
+    checkEvaluation(MapFromEntries(ai0), create_map(1 -> 10, 2 -> 20, 3 -> 20))
+    checkEvaluation(MapFromEntries(ai1), create_map(1 -> null, 2 -> 20, 3 -> null))
     checkEvaluation(MapFromEntries(ai2), Map.empty)
     checkEvaluation(MapFromEntries(ai3), null)
     checkEvaluation(MapKeys(MapFromEntries(ai4)), Seq(1, 1))
@@ -298,23 +298,36 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper
 
     // Non-primitive-type keys and values
     val asType = arrayType(StringType, StringType)
-    val as0 = Literal.create(Seq(r("a", "aa"), r("b", "bb"), r("c", "bb")), asType)
-    val as1 = Literal.create(Seq(r("a", null), r("b", "bb"), r("c", null)), asType)
+    val as0 = Literal.create(Seq(row("a", "aa"), row("b", "bb"), row("c", "bb")), asType)
+    val as1 = Literal.create(Seq(row("a", null), row("b", "bb"), row("c", null)), asType)
     val as2 = Literal.create(Seq.empty, asType)
     val as3 = Literal.create(null, asType)
-    val as4 = Literal.create(Seq(r("a", "aa"), r("a", "bb")), asType)
-    val as5 = Literal.create(Seq(r("a", "aa"), r(null, "bb")), asType)
-    val as6 = Literal.create(Seq(null, r("b", "bb"), null), asType)
+    val as4 = Literal.create(Seq(row("a", "aa"), row("a", "bb")), asType)
+    val as5 = Literal.create(Seq(row("a", "aa"), row(null, "bb")), asType)
+    val as6 = Literal.create(Seq(null, row("b", "bb"), null), asType)
 
-    checkEvaluation(MapFromEntries(as0), Map("a" -> "aa", "b" -> "bb", "c" -> "bb"))
-    checkEvaluation(MapFromEntries(as1), Map("a" -> null, "b" -> "bb", "c" -> null))
+    checkEvaluation(MapFromEntries(as0), create_map("a" -> "aa", "b" -> "bb", "c" -> "bb"))
+    checkEvaluation(MapFromEntries(as1), create_map("a" -> null, "b" -> "bb", "c" -> null))
     checkEvaluation(MapFromEntries(as2), Map.empty)
     checkEvaluation(MapFromEntries(as3), null)
     checkEvaluation(MapKeys(MapFromEntries(as4)), Seq("a", "a"))
+    checkEvaluation(MapFromEntries(as6), null)
+
+    // Map key can't be null
     checkExceptionInExpression[RuntimeException](
       MapFromEntries(as5),
       "The first field from a struct (key) can't be null.")
-    checkEvaluation(MapFromEntries(as6), null)
+
+    // map key can't be map
+    val structOfMap = row(create_map(1 -> 1), 1)
+    val map = MapFromEntries(Literal.create(
+      Seq(structOfMap),
+      arrayType(keyType = MapType(IntegerType, IntegerType), valueType = IntegerType)))
+    map.checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckSuccess => fail("should not allow map as map key")
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("The key of map cannot be/contain map"))
+    }
   }
 
   test("Sort Array") {

http://git-wip-us.apache.org/repos/asf/spark/blob/219b037f/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala
index 77aaf55..d95f42e 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.sql.catalyst.expressions
 
 import org.apache.spark.SparkFunSuite
-import org.apache.spark.sql.catalyst.analysis.UnresolvedExtractValue
+import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, UnresolvedExtractValue}
 import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext
 import org.apache.spark.sql.types._
@@ -158,40 +158,32 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper {
       keys.zip(values).flatMap { case (k, v) => Seq(k, v) }
     }
 
-    def createMap(keys: Seq[Any], values: Seq[Any]): Map[Any, Any] = {
-      // catalyst map is order-sensitive, so we create ListMap here to preserve the elements order.
-      scala.collection.immutable.ListMap(keys.zip(values): _*)
-    }
-
     val intSeq = Seq(5, 10, 15, 20, 25)
     val longSeq = intSeq.map(_.toLong)
     val strSeq = intSeq.map(_.toString)
+
     checkEvaluation(CreateMap(Nil), Map.empty)
     checkEvaluation(
       CreateMap(interlace(intSeq.map(Literal(_)), longSeq.map(Literal(_)))),
-      createMap(intSeq, longSeq))
+      create_map(intSeq, longSeq))
     checkEvaluation(
       CreateMap(interlace(strSeq.map(Literal(_)), longSeq.map(Literal(_)))),
-      createMap(strSeq, longSeq))
+      create_map(strSeq, longSeq))
     checkEvaluation(
       CreateMap(interlace(longSeq.map(Literal(_)), strSeq.map(Literal(_)))),
-      createMap(longSeq, strSeq))
+      create_map(longSeq, strSeq))
 
     val strWithNull = strSeq.drop(1).map(Literal(_)) :+ Literal.create(null, StringType)
     checkEvaluation(
       CreateMap(interlace(intSeq.map(Literal(_)), strWithNull)),
-      createMap(intSeq, strWithNull.map(_.value)))
-    intercept[RuntimeException] {
-      checkEvaluationWithoutCodegen(
-        CreateMap(interlace(strWithNull, intSeq.map(Literal(_)))),
-        null, null)
-    }
-    intercept[RuntimeException] {
-      checkEvaluationWithUnsafeProjection(
-        CreateMap(interlace(strWithNull, intSeq.map(Literal(_)))),
-        null, null)
-    }
+      create_map(intSeq, strWithNull.map(_.value)))
 
+    // Map key can't be null
+    checkExceptionInExpression[RuntimeException](
+      CreateMap(interlace(strWithNull, intSeq.map(Literal(_)))),
+      "Cannot use null as map key")
+
+    // ArrayType map key and value
     val map = CreateMap(Seq(
       Literal.create(intSeq, ArrayType(IntegerType, containsNull = false)),
       Literal.create(strSeq, ArrayType(StringType, containsNull = false)),
@@ -202,15 +194,21 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper {
         ArrayType(IntegerType, containsNull = true),
         ArrayType(StringType, containsNull = true),
         valueContainsNull = false))
-    checkEvaluation(map, createMap(Seq(intSeq, intSeq :+ null), Seq(strSeq, strSeq :+ null)))
+    checkEvaluation(map, create_map(intSeq -> strSeq, (intSeq :+ null) -> (strSeq :+ null)))
+
+    // map key can't be map
+    val map2 = CreateMap(Seq(
+      Literal.create(create_map(1 -> 1), MapType(IntegerType, IntegerType)),
+      Literal(1)
+    ))
+    map2.checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckSuccess => fail("should not allow map as map key")
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("The key of map cannot be/contain map"))
+    }
   }
 
   test("MapFromArrays") {
-    def createMap(keys: Seq[Any], values: Seq[Any]): Map[Any, Any] = {
-      // catalyst map is order-sensitive, so we create ListMap here to preserve the elements order.
-      scala.collection.immutable.ListMap(keys.zip(values): _*)
-    }
-
     val intSeq = Seq(5, 10, 15, 20, 25)
     val longSeq = intSeq.map(_.toLong)
     val strSeq = intSeq.map(_.toString)
@@ -228,24 +226,33 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper {
 
     val nullArray = Literal.create(null, ArrayType(StringType, false))
 
-    checkEvaluation(MapFromArrays(intArray, longArray), createMap(intSeq, longSeq))
-    checkEvaluation(MapFromArrays(intArray, strArray), createMap(intSeq, strSeq))
-    checkEvaluation(MapFromArrays(integerArray, strArray), createMap(integerSeq, strSeq))
+    checkEvaluation(MapFromArrays(intArray, longArray), create_map(intSeq, longSeq))
+    checkEvaluation(MapFromArrays(intArray, strArray), create_map(intSeq, strSeq))
+    checkEvaluation(MapFromArrays(integerArray, strArray), create_map(integerSeq, strSeq))
 
     checkEvaluation(
-      MapFromArrays(strArray, intWithNullArray), createMap(strSeq, intWithNullSeq))
+      MapFromArrays(strArray, intWithNullArray), create_map(strSeq, intWithNullSeq))
     checkEvaluation(
-      MapFromArrays(strArray, longWithNullArray), createMap(strSeq, longWithNullSeq))
+      MapFromArrays(strArray, longWithNullArray), create_map(strSeq, longWithNullSeq))
     checkEvaluation(
-      MapFromArrays(strArray, longWithNullArray), createMap(strSeq, longWithNullSeq))
+      MapFromArrays(strArray, longWithNullArray), create_map(strSeq, longWithNullSeq))
     checkEvaluation(MapFromArrays(nullArray, nullArray), null)
 
-    intercept[RuntimeException] {
-      checkEvaluation(MapFromArrays(intWithNullArray, strArray), null)
-    }
-    intercept[RuntimeException] {
-      checkEvaluation(
-        MapFromArrays(intArray, Literal.create(Seq(1), ArrayType(IntegerType))), null)
+    // Map key can't be null
+    checkExceptionInExpression[RuntimeException](
+      MapFromArrays(intWithNullArray, strArray),
+      "Cannot use null as map key")
+
+    // map key can't be map
+    val arrayOfMap = Seq(create_map(1 -> "a", 2 -> "b"))
+    val map = MapFromArrays(
+      Literal.create(arrayOfMap, ArrayType(MapType(IntegerType, StringType))),
+      Literal.create(Seq(1), ArrayType(IntegerType))
+    )
+    map.checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckSuccess => fail("should not allow map as map key")
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("The key of map cannot be/contain map"))
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/219b037f/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala
index da18475..eb33325 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala
@@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen._
 import org.apache.spark.sql.catalyst.optimizer.SimpleTestOptimizer
 import org.apache.spark.sql.catalyst.plans.PlanTestBase
 import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Project}
-import org.apache.spark.sql.catalyst.util.{ArrayData, MapData}
+import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, MapData}
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 import org.apache.spark.util.Utils
@@ -48,6 +48,25 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa
     InternalRow.fromSeq(values.map(CatalystTypeConverters.convertToCatalyst))
   }
 
+  // Currently MapData just stores the key and value arrays. Its equality is not well implemented,
+  // as the order of the map entries should not matter for equality. This method creates MapData
+  // with the entries ordering preserved, so that we can deterministically test expressions with
+  // map input/output.
+  protected def create_map(entries: (_, _)*): ArrayBasedMapData = {
+    create_map(entries.map(_._1), entries.map(_._2))
+  }
+
+  protected def create_map(keys: Seq[_], values: Seq[_]): ArrayBasedMapData = {
+    assert(keys.length == values.length)
+    val keyArray = CatalystTypeConverters
+      .convertToCatalyst(keys)
+      .asInstanceOf[ArrayData]
+    val valueArray = CatalystTypeConverters
+      .convertToCatalyst(values)
+      .asInstanceOf[ArrayData]
+    new ArrayBasedMapData(keyArray, valueArray)
+  }
+
   private def prepareEvaluation(expression: Expression): Expression = {
     val serializer = new JavaSerializer(new SparkConf()).newInstance
     val resolver = ResolveTimeZone(new SQLConf)

http://git-wip-us.apache.org/repos/asf/spark/blob/219b037f/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala
index e13f4d9..66bf18a 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.sql.catalyst.expressions
 
 import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
 import org.apache.spark.sql.catalyst.util.ArrayBasedMapData
 import org.apache.spark.sql.types._
 
@@ -310,13 +311,13 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper
 
   test("TransformKeys") {
     val ai0 = Literal.create(
-      Map(1 -> 1, 2 -> 2, 3 -> 3, 4 -> 4),
+      create_map(1 -> 1, 2 -> 2, 3 -> 3, 4 -> 4),
       MapType(IntegerType, IntegerType, valueContainsNull = false))
     val ai1 = Literal.create(
       Map.empty[Int, Int],
       MapType(IntegerType, IntegerType, valueContainsNull = true))
     val ai2 = Literal.create(
-      Map(1 -> 1, 2 -> null, 3 -> 3),
+      create_map(1 -> 1, 2 -> null, 3 -> 3),
       MapType(IntegerType, IntegerType, valueContainsNull = true))
     val ai3 = Literal.create(null, MapType(IntegerType, IntegerType, valueContainsNull = false))
 
@@ -324,26 +325,27 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper
     val plusValue: (Expression, Expression) => Expression = (k, v) => k + v
     val modKey: (Expression, Expression) => Expression = (k, v) => k % 3
 
-    checkEvaluation(transformKeys(ai0, plusOne), Map(2 -> 1, 3 -> 2, 4 -> 3, 5 -> 4))
-    checkEvaluation(transformKeys(ai0, plusValue), Map(2 -> 1, 4 -> 2, 6 -> 3, 8 -> 4))
+    checkEvaluation(transformKeys(ai0, plusOne), create_map(2 -> 1, 3 -> 2, 4 -> 3, 5 -> 4))
+    checkEvaluation(transformKeys(ai0, plusValue), create_map(2 -> 1, 4 -> 2, 6 -> 3, 8 -> 4))
     checkEvaluation(
-      transformKeys(transformKeys(ai0, plusOne), plusValue), Map(3 -> 1, 5 -> 2, 7 -> 3, 9 -> 4))
+      transformKeys(transformKeys(ai0, plusOne), plusValue),
+      create_map(3 -> 1, 5 -> 2, 7 -> 3, 9 -> 4))
     checkEvaluation(transformKeys(ai0, modKey),
       ArrayBasedMapData(Array(1, 2, 0, 1), Array(1, 2, 3, 4)))
     checkEvaluation(transformKeys(ai1, plusOne), Map.empty[Int, Int])
     checkEvaluation(transformKeys(ai1, plusOne), Map.empty[Int, Int])
     checkEvaluation(
       transformKeys(transformKeys(ai1, plusOne), plusValue), Map.empty[Int, Int])
-    checkEvaluation(transformKeys(ai2, plusOne), Map(2 -> 1, 3 -> null, 4 -> 3))
+    checkEvaluation(transformKeys(ai2, plusOne), create_map(2 -> 1, 3 -> null, 4 -> 3))
     checkEvaluation(
-      transformKeys(transformKeys(ai2, plusOne), plusOne), Map(3 -> 1, 4 -> null, 5 -> 3))
+      transformKeys(transformKeys(ai2, plusOne), plusOne), create_map(3 -> 1, 4 -> null, 5 -> 3))
     checkEvaluation(transformKeys(ai3, plusOne), null)
 
     val as0 = Literal.create(
-      Map("a" -> "xy", "bb" -> "yz", "ccc" -> "zx"),
+      create_map("a" -> "xy", "bb" -> "yz", "ccc" -> "zx"),
       MapType(StringType, StringType, valueContainsNull = false))
     val as1 = Literal.create(
-      Map("a" -> "xy", "bb" -> "yz", "ccc" -> null),
+      create_map("a" -> "xy", "bb" -> "yz", "ccc" -> null),
       MapType(StringType, StringType, valueContainsNull = true))
     val as2 = Literal.create(null,
       MapType(StringType, StringType, valueContainsNull = false))
@@ -355,26 +357,35 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper
       (k, v) => Length(k) + 1
 
     checkEvaluation(
-      transformKeys(as0, concatValue), Map("axy" -> "xy", "bbyz" -> "yz", "ccczx" -> "zx"))
+      transformKeys(as0, concatValue), create_map("axy" -> "xy", "bbyz" -> "yz", "ccczx" -> "zx"))
     checkEvaluation(
       transformKeys(transformKeys(as0, concatValue), concatValue),
-      Map("axyxy" -> "xy", "bbyzyz" -> "yz", "ccczxzx" -> "zx"))
+      create_map("axyxy" -> "xy", "bbyzyz" -> "yz", "ccczxzx" -> "zx"))
     checkEvaluation(transformKeys(as3, concatValue), Map.empty[String, String])
     checkEvaluation(
       transformKeys(transformKeys(as3, concatValue), convertKeyToKeyLength),
       Map.empty[Int, String])
     checkEvaluation(transformKeys(as0, convertKeyToKeyLength),
-      Map(2 -> "xy", 3 -> "yz", 4 -> "zx"))
+      create_map(2 -> "xy", 3 -> "yz", 4 -> "zx"))
     checkEvaluation(transformKeys(as1, convertKeyToKeyLength),
-      Map(2 -> "xy", 3 -> "yz", 4 -> null))
+      create_map(2 -> "xy", 3 -> "yz", 4 -> null))
     checkEvaluation(transformKeys(as2, convertKeyToKeyLength), null)
     checkEvaluation(transformKeys(as3, convertKeyToKeyLength), Map.empty[Int, String])
 
     val ax0 = Literal.create(
-      Map(1 -> "x", 2 -> "y", 3 -> "z"),
+      create_map(1 -> "x", 2 -> "y", 3 -> "z"),
       MapType(IntegerType, StringType, valueContainsNull = false))
 
-    checkEvaluation(transformKeys(ax0, plusOne), Map(2 -> "x", 3 -> "y", 4 -> "z"))
+    checkEvaluation(transformKeys(ax0, plusOne), create_map(2 -> "x", 3 -> "y", 4 -> "z"))
+
+    // map key can't be map
+    val makeMap: (Expression, Expression) => Expression = (k, v) => CreateMap(Seq(k, v))
+    val map = transformKeys(ai0, makeMap)
+    map.checkInputDataTypes() match {
+      case TypeCheckResult.TypeCheckSuccess => fail("should not allow map as map key")
+      case TypeCheckResult.TypeCheckFailure(msg) =>
+        assert(msg.contains("The key of map cannot be/contain map"))
+    }
   }
 
   test("TransformValues") {

http://git-wip-us.apache.org/repos/asf/spark/blob/219b037f/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/mapconcat.sql
----------------------------------------------------------------------
diff --git a/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/mapconcat.sql b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/mapconcat.sql
index 69da67f..6089502 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/mapconcat.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/mapconcat.sql
@@ -13,7 +13,6 @@ CREATE TEMPORARY VIEW various_maps AS SELECT * FROM VALUES (
   map('a', 'b'), map('c', 'd'),
   map(array('a', 'b'), array('c', 'd')), map(array('e'), array('f')),
   map(struct('a', 1), struct('b', 2)), map(struct('c', 3), struct('d', 4)),
-  map(map('a', 1), map('b', 2)), map(map('c', 3), map('d', 4)),
   map('a', 1), map('c', 2),
   map(1, 'a'), map(2, 'c')
 ) AS various_maps (
@@ -31,7 +30,6 @@ CREATE TEMPORARY VIEW various_maps AS SELECT * FROM VALUES (
   string_map1, string_map2,
   array_map1, array_map2,
   struct_map1, struct_map2,
-  map_map1, map_map2,
   string_int_map1, string_int_map2,
   int_string_map1, int_string_map2
 );
@@ -51,7 +49,6 @@ SELECT
     map_concat(string_map1, string_map2) string_map,
     map_concat(array_map1, array_map2) array_map,
     map_concat(struct_map1, struct_map2) struct_map,
-    map_concat(map_map1, map_map2) map_map,
     map_concat(string_int_map1, string_int_map2) string_int_map,
     map_concat(int_string_map1, int_string_map2) int_string_map
 FROM various_maps;
@@ -71,7 +68,7 @@ FROM various_maps;
 
 -- Concatenate map of incompatible types 1
 SELECT
-    map_concat(tinyint_map1, map_map2) tm_map
+    map_concat(tinyint_map1, array_map1) tm_map
 FROM various_maps;
 
 -- Concatenate map of incompatible types 2
@@ -86,10 +83,10 @@ FROM various_maps;
 
 -- Concatenate map of incompatible types 4
 SELECT
-    map_concat(map_map1, array_map2) ma_map
+    map_concat(struct_map1, array_map2) ma_map
 FROM various_maps;
 
 -- Concatenate map of incompatible types 5
 SELECT
-    map_concat(map_map1, struct_map2) ms_map
+    map_concat(int_map1, array_map2) ms_map
 FROM various_maps;

http://git-wip-us.apache.org/repos/asf/spark/blob/219b037f/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out
----------------------------------------------------------------------
diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out
index efc88e4..79e0086 100644
--- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out
@@ -18,7 +18,6 @@ CREATE TEMPORARY VIEW various_maps AS SELECT * FROM VALUES (
   map('a', 'b'), map('c', 'd'),
   map(array('a', 'b'), array('c', 'd')), map(array('e'), array('f')),
   map(struct('a', 1), struct('b', 2)), map(struct('c', 3), struct('d', 4)),
-  map(map('a', 1), map('b', 2)), map(map('c', 3), map('d', 4)),
   map('a', 1), map('c', 2),
   map(1, 'a'), map(2, 'c')
 ) AS various_maps (
@@ -36,7 +35,6 @@ CREATE TEMPORARY VIEW various_maps AS SELECT * FROM VALUES (
   string_map1, string_map2,
   array_map1, array_map2,
   struct_map1, struct_map2,
-  map_map1, map_map2,
   string_int_map1, string_int_map2,
   int_string_map1, int_string_map2
 )
@@ -61,14 +59,13 @@ SELECT
     map_concat(string_map1, string_map2) string_map,
     map_concat(array_map1, array_map2) array_map,
     map_concat(struct_map1, struct_map2) struct_map,
-    map_concat(map_map1, map_map2) map_map,
     map_concat(string_int_map1, string_int_map2) string_int_map,
     map_concat(int_string_map1, int_string_map2) int_string_map
 FROM various_maps
 -- !query 1 schema
-struct<boolean_map:map<boolean,boolean>,tinyint_map:map<tinyint,tinyint>,smallint_map:map<smallint,smallint>,int_map:map<int,int>,bigint_map:map<bigint,bigint>,decimal_map:map<decimal(19,0),decimal(19,0)>,float_map:map<float,float>,double_map:map<double,double>,date_map:map<date,date>,timestamp_map:map<timestamp,timestamp>,string_map:map<string,string>,array_map:map<array<string>,array<string>>,struct_map:map<struct<col1:string,col2:int>,struct<col1:string,col2:int>>,map_map:map<map<string,int>,map<string,int>>,string_int_map:map<string,int>,int_string_map:map<int,string>>
+struct<boolean_map:map<boolean,boolean>,tinyint_map:map<tinyint,tinyint>,smallint_map:map<smallint,smallint>,int_map:map<int,int>,bigint_map:map<bigint,bigint>,decimal_map:map<decimal(19,0),decimal(19,0)>,float_map:map<float,float>,double_map:map<double,double>,date_map:map<date,date>,timestamp_map:map<timestamp,timestamp>,string_map:map<string,string>,array_map:map<array<string>,array<string>>,struct_map:map<struct<col1:string,col2:int>,struct<col1:string,col2:int>>,string_int_map:map<string,int>,int_string_map:map<int,string>>
 -- !query 1 output
-{false:true,true:false}	{1:2,3:4}	{1:2,3:4}	{4:6,7:8}	{6:7,8:9}	{9223372036854775808:9223372036854775809,9223372036854775809:9223372036854775808}	{1.0:2.0,3.0:4.0}	{1.0:2.0,3.0:4.0}	{2016-03-12:2016-03-11,2016-03-14:2016-03-13}	{2016-11-11 20:54:00.0:2016-11-09 20:54:00.0,2016-11-15 20:54:00.0:2016-11-12 20:54:00.0}	{"a":"b","c":"d"}	{["a","b"]:["c","d"],["e"]:["f"]}	{{"col1":"a","col2":1}:{"col1":"b","col2":2},{"col1":"c","col2":3}:{"col1":"d","col2":4}}	{{"a":1}:{"b":2},{"c":3}:{"d":4}}	{"a":1,"c":2}	{1:"a",2:"c"}
+{false:true,true:false}	{1:2,3:4}	{1:2,3:4}	{4:6,7:8}	{6:7,8:9}	{9223372036854775808:9223372036854775809,9223372036854775809:9223372036854775808}	{1.0:2.0,3.0:4.0}	{1.0:2.0,3.0:4.0}	{2016-03-12:2016-03-11,2016-03-14:2016-03-13}	{2016-11-11 20:54:00.0:2016-11-09 20:54:00.0,2016-11-15 20:54:00.0:2016-11-12 20:54:00.0}	{"a":"b","c":"d"}	{["a","b"]:["c","d"],["e"]:["f"]}	{{"col1":"a","col2":1}:{"col1":"b","col2":2},{"col1":"c","col2":3}:{"col1":"d","col2":4}}	{"a":1,"c":2}	{1:"a",2:"c"}
 
 
 -- !query 2
@@ -91,13 +88,13 @@ struct<ts_map:map<smallint,smallint>,si_map:map<int,int>,ib_map:map<bigint,bigin
 
 -- !query 3
 SELECT
-    map_concat(tinyint_map1, map_map2) tm_map
+    map_concat(tinyint_map1, array_map1) tm_map
 FROM various_maps
 -- !query 3 schema
 struct<>
 -- !query 3 output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'map_concat(various_maps.`tinyint_map1`, various_maps.`map_map2`)' due to data type mismatch: input to function map_concat should all be the same type, but it's [map<tinyint,tinyint>, map<map<string,int>,map<string,int>>]; line 2 pos 4
+cannot resolve 'map_concat(various_maps.`tinyint_map1`, various_maps.`array_map1`)' due to data type mismatch: input to function map_concat should all be the same type, but it's [map<tinyint,tinyint>, map<array<string>,array<string>>]; line 2 pos 4
 
 
 -- !query 4
@@ -124,21 +121,21 @@ cannot resolve 'map_concat(various_maps.`int_map1`, various_maps.`struct_map2`)'
 
 -- !query 6
 SELECT
-    map_concat(map_map1, array_map2) ma_map
+    map_concat(struct_map1, array_map2) ma_map
 FROM various_maps
 -- !query 6 schema
 struct<>
 -- !query 6 output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'map_concat(various_maps.`map_map1`, various_maps.`array_map2`)' due to data type mismatch: input to function map_concat should all be the same type, but it's [map<map<string,int>,map<string,int>>, map<array<string>,array<string>>]; line 2 pos 4
+cannot resolve 'map_concat(various_maps.`struct_map1`, various_maps.`array_map2`)' due to data type mismatch: input to function map_concat should all be the same type, but it's [map<struct<col1:string,col2:int>,struct<col1:string,col2:int>>, map<array<string>,array<string>>]; line 2 pos 4
 
 
 -- !query 7
 SELECT
-    map_concat(map_map1, struct_map2) ms_map
+    map_concat(int_map1, array_map2) ms_map
 FROM various_maps
 -- !query 7 schema
 struct<>
 -- !query 7 output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'map_concat(various_maps.`map_map1`, various_maps.`struct_map2`)' due to data type mismatch: input to function map_concat should all be the same type, but it's [map<map<string,int>,map<string,int>>, map<struct<col1:string,col2:int>,struct<col1:string,col2:int>>]; line 2 pos 4
+cannot resolve 'map_concat(various_maps.`int_map1`, various_maps.`array_map2`)' due to data type mismatch: input to function map_concat should all be the same type, but it's [map<int,int>, map<array<string>,array<string>>]; line 2 pos 4


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


Mime
View raw message