Repository: spark
Updated Branches:
  refs/heads/branch-2.4 db121a2a1 -> a9f200e11


[SPARK-25832][SQL][BRANCH-2.4] Revert newly added map related functions

## What changes were proposed in this pull request?

- Revert [SPARK-23935][SQL] Adding map_entries function: 
https://github.com/apache/spark/pull/21236
- Revert [SPARK-23937][SQL] Add map_filter SQL function: 
https://github.com/apache/spark/pull/21986
- Revert [SPARK-23940][SQL] Add transform_values SQL function: 
https://github.com/apache/spark/pull/22045
- Revert [SPARK-23939][SQL] Add transform_keys function: 
https://github.com/apache/spark/pull/22013
- Revert [SPARK-23938][SQL] Add map_zip_with function: 
https://github.com/apache/spark/pull/22017
- Revert the changes of map_entries in [SPARK-24331][SPARKR][SQL] Adding 
arrays_overlap, array_repeat, map_entries to SparkR: 
https://github.com/apache/spark/pull/21434/

## How was this patch tested?
The existing tests.

Closes #22827 from gatorsmile/revertMap2.4.

Authored-by: gatorsmile <[email protected]>
Signed-off-by: Wenchen Fan <[email protected]>


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

Branch: refs/heads/branch-2.4
Commit: a9f200e11da3d26158f9f75e48756d47d61bfacb
Parents: db121a2
Author: gatorsmile <[email protected]>
Authored: Fri Oct 26 07:38:55 2018 +0800
Committer: Wenchen Fan <[email protected]>
Committed: Fri Oct 26 07:38:55 2018 +0800

----------------------------------------------------------------------
 R/pkg/NAMESPACE                                 |   1 -
 R/pkg/R/functions.R                             |  15 +-
 R/pkg/R/generics.R                              |   4 -
 R/pkg/tests/fulltests/test_sparkSQL.R           |   7 +-
 python/pyspark/sql/functions.py                 |  20 -
 .../catalyst/analysis/FunctionRegistry.scala    |   5 -
 .../sql/catalyst/analysis/TypeCoercion.scala    |  25 --
 .../expressions/collectionOperations.scala      | 168 --------
 .../expressions/higherOrderFunctions.scala      | 330 --------------
 .../CollectionExpressionsSuite.scala            |  24 --
 .../expressions/HigherOrderFunctionsSuite.scala | 315 --------------
 .../scala/org/apache/spark/sql/functions.scala  |   7 -
 .../sql-tests/inputs/higher-order-functions.sql |  23 -
 .../inputs/typeCoercion/native/mapZipWith.sql   |  78 ----
 .../results/higher-order-functions.sql.out      |  66 +--
 .../typeCoercion/native/mapZipWith.sql.out      | 179 --------
 .../spark/sql/DataFrameFunctionsSuite.scala     | 425 -------------------
 17 files changed, 3 insertions(+), 1689 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/a9f200e1/R/pkg/NAMESPACE
----------------------------------------------------------------------
diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE
index 96ff389..d77c62a 100644
--- a/R/pkg/NAMESPACE
+++ b/R/pkg/NAMESPACE
@@ -313,7 +313,6 @@ exportMethods("%<=>%",
               "lower",
               "lpad",
               "ltrim",
-              "map_entries",
               "map_from_arrays",
               "map_keys",
               "map_values",

http://git-wip-us.apache.org/repos/asf/spark/blob/a9f200e1/R/pkg/R/functions.R
----------------------------------------------------------------------
diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R
index 63bd427..1e70244 100644
--- a/R/pkg/R/functions.R
+++ b/R/pkg/R/functions.R
@@ -219,7 +219,7 @@ NULL
 #' head(select(tmp, sort_array(tmp$v1)))
 #' head(select(tmp, sort_array(tmp$v1, asc = FALSE)))
 #' tmp3 <- mutate(df, v3 = create_map(df$model, df$cyl))
-#' head(select(tmp3, map_entries(tmp3$v3), map_keys(tmp3$v3), 
map_values(tmp3$v3)))
+#' head(select(tmp3, map_keys(tmp3$v3), map_values(tmp3$v3)))
 #' head(select(tmp3, element_at(tmp3$v3, "Valiant")))
 #' tmp4 <- mutate(df, v4 = create_array(df$mpg, df$cyl), v5 = 
create_array(df$cyl, df$hp))
 #' head(select(tmp4, concat(tmp4$v4, tmp4$v5), arrays_overlap(tmp4$v4, 
tmp4$v5)))
@@ -3253,19 +3253,6 @@ setMethod("flatten",
           })
 
 #' @details
-#' \code{map_entries}: Returns an unordered array of all entries in the given 
map.
-#'
-#' @rdname column_collection_functions
-#' @aliases map_entries map_entries,Column-method
-#' @note map_entries since 2.4.0
-setMethod("map_entries",
-          signature(x = "Column"),
-          function(x) {
-            jc <- callJStatic("org.apache.spark.sql.functions", "map_entries", 
x@jc)
-            column(jc)
-         })
-
-#' @details
 #' \code{map_from_arrays}: Creates a new map column. The array in the first 
column is used for
 #' keys. The array in the second column is used for values. All elements in 
the array for key
 #' should not be null.

http://git-wip-us.apache.org/repos/asf/spark/blob/a9f200e1/R/pkg/R/generics.R
----------------------------------------------------------------------
diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R
index 27c1b31..e74f1c8 100644
--- a/R/pkg/R/generics.R
+++ b/R/pkg/R/generics.R
@@ -1078,10 +1078,6 @@ setGeneric("ltrim", function(x, trimString) { 
standardGeneric("ltrim") })
 
 #' @rdname column_collection_functions
 #' @name NULL
-setGeneric("map_entries", function(x) { standardGeneric("map_entries") })
-
-#' @rdname column_collection_functions
-#' @name NULL
 setGeneric("map_from_arrays", function(x, y) { 
standardGeneric("map_from_arrays") })
 
 #' @rdname column_collection_functions

http://git-wip-us.apache.org/repos/asf/spark/blob/a9f200e1/R/pkg/tests/fulltests/test_sparkSQL.R
----------------------------------------------------------------------
diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R 
b/R/pkg/tests/fulltests/test_sparkSQL.R
index ce3922f..0101d0c 100644
--- a/R/pkg/tests/fulltests/test_sparkSQL.R
+++ b/R/pkg/tests/fulltests/test_sparkSQL.R
@@ -1570,13 +1570,8 @@ test_that("column functions", {
   result <- collect(select(df, flatten(df[[1]])))[[1]]
   expect_equal(result, list(list(1L, 2L, 3L, 4L), list(5L, 6L, 7L, 8L)))
 
-  # Test map_entries(), map_keys(), map_values() and element_at()
+  # Test map_keys(), map_values() and element_at()
   df <- createDataFrame(list(list(map = as.environment(list(x = 1, y = 2)))))
-  result <- collect(select(df, map_entries(df$map)))[[1]]
-  expected_entries <-  list(listToStruct(list(key = "x", value = 1)),
-                            listToStruct(list(key = "y", value = 2)))
-  expect_equal(result, list(expected_entries))
-
   result <- collect(select(df, map_keys(df$map)))[[1]]
   expect_equal(result, list(list("x", "y")))
 

http://git-wip-us.apache.org/repos/asf/spark/blob/a9f200e1/python/pyspark/sql/functions.py
----------------------------------------------------------------------
diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py
index 8c54179..785e55e 100644
--- a/python/pyspark/sql/functions.py
+++ b/python/pyspark/sql/functions.py
@@ -2541,26 +2541,6 @@ def map_values(col):
 
 
 @since(2.4)
-def map_entries(col):
-    """
-    Collection function: Returns an unordered array of all entries in the 
given map.
-
-    :param col: name of column or expression
-
-    >>> from pyspark.sql.functions import map_entries
-    >>> df = spark.sql("SELECT map(1, 'a', 2, 'b') as data")
-    >>> df.select(map_entries("data").alias("entries")).show()
-    +----------------+
-    |         entries|
-    +----------------+
-    |[[1, a], [2, b]]|
-    +----------------+
-    """
-    sc = SparkContext._active_spark_context
-    return Column(sc._jvm.functions.map_entries(_to_java_column(col)))
-
-
-@since(2.4)
 def map_from_entries(col):
     """
     Collection function: Returns a map created from the given array of entries.

http://git-wip-us.apache.org/repos/asf/spark/blob/a9f200e1/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
index 695267a..efd760c 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
@@ -414,7 +414,6 @@ object FunctionRegistry {
     expression[MapFromArrays]("map_from_arrays"),
     expression[MapKeys]("map_keys"),
     expression[MapValues]("map_values"),
-    expression[MapEntries]("map_entries"),
     expression[MapFromEntries]("map_from_entries"),
     expression[MapConcat]("map_concat"),
     expression[Size]("size"),
@@ -433,13 +432,9 @@ object FunctionRegistry {
     expression[ArrayRemove]("array_remove"),
     expression[ArrayDistinct]("array_distinct"),
     expression[ArrayTransform]("transform"),
-    expression[MapFilter]("map_filter"),
     expression[ArrayFilter]("filter"),
     expression[ArrayExists]("exists"),
     expression[ArrayAggregate]("aggregate"),
-    expression[TransformValues]("transform_values"),
-    expression[TransformKeys]("transform_keys"),
-    expression[MapZipWith]("map_zip_with"),
     expression[ZipWith]("zip_with"),
 
     CreateStruct.registryEntry,

http://git-wip-us.apache.org/repos/asf/spark/blob/a9f200e1/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
index 72ac80e..1659156 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
@@ -54,7 +54,6 @@ object TypeCoercion {
       BooleanEquality ::
       FunctionArgumentConversion ::
       ConcatCoercion(conf) ::
-      MapZipWithCoercion ::
       EltCoercion(conf) ::
       CaseWhenCoercion ::
       IfCoercion ::
@@ -764,30 +763,6 @@ object TypeCoercion {
   }
 
   /**
-   * Coerces key types of two different [[MapType]] arguments of the 
[[MapZipWith]] expression
-   * to a common type.
-   */
-  object MapZipWithCoercion extends TypeCoercionRule {
-    override protected def coerceTypes(plan: LogicalPlan): LogicalPlan = plan 
resolveExpressions {
-      // Lambda function isn't resolved when the rule is executed.
-      case m @ MapZipWith(left, right, function) if m.arguments.forall(a => 
a.resolved &&
-          MapType.acceptsType(a.dataType)) && 
!m.leftKeyType.sameType(m.rightKeyType) =>
-        findWiderTypeForTwo(m.leftKeyType, m.rightKeyType) match {
-          case Some(finalKeyType) if !Cast.forceNullable(m.leftKeyType, 
finalKeyType) &&
-              !Cast.forceNullable(m.rightKeyType, finalKeyType) =>
-            val newLeft = castIfNotSameType(
-              left,
-              MapType(finalKeyType, m.leftValueType, m.leftValueContainsNull))
-            val newRight = castIfNotSameType(
-              right,
-              MapType(finalKeyType, m.rightValueType, 
m.rightValueContainsNull))
-            MapZipWith(newLeft, newRight, function)
-          case _ => m
-        }
-    }
-  }
-
-  /**
    * Coerces the types of [[Elt]] children to expected ones.
    *
    * If `spark.sql.function.eltOutputAsString` is false and all children types 
are binary,

http://git-wip-us.apache.org/repos/asf/spark/blob/a9f200e1/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 b24d748..0a39b43 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
@@ -341,174 +341,6 @@ case class MapValues(child: Expression)
 }
 
 /**
- * Returns an unordered array of all entries in the given map.
- */
-@ExpressionDescription(
-  usage = "_FUNC_(map) - Returns an unordered array of all entries in the 
given map.",
-  examples = """
-    Examples:
-      > SELECT _FUNC_(map(1, 'a', 2, 'b'));
-       [{"key":1,"value":"a"},{"key":2,"value":"b"}]
-  """,
-  since = "2.4.0")
-case class MapEntries(child: Expression) extends UnaryExpression with 
ExpectsInputTypes {
-
-  override def inputTypes: Seq[AbstractDataType] = Seq(MapType)
-
-  @transient private lazy val childDataType: MapType = 
child.dataType.asInstanceOf[MapType]
-
-  override def dataType: DataType = {
-    ArrayType(
-      StructType(
-        StructField("key", childDataType.keyType, false) ::
-        StructField("value", childDataType.valueType, 
childDataType.valueContainsNull) ::
-        Nil),
-      false)
-  }
-
-  override protected def nullSafeEval(input: Any): Any = {
-    val childMap = input.asInstanceOf[MapData]
-    val keys = childMap.keyArray()
-    val values = childMap.valueArray()
-    val length = childMap.numElements()
-    val resultData = new Array[AnyRef](length)
-    var i = 0
-    while (i < length) {
-      val key = keys.get(i, childDataType.keyType)
-      val value = values.get(i, childDataType.valueType)
-      val row = new GenericInternalRow(Array[Any](key, value))
-      resultData.update(i, row)
-      i += 1
-    }
-    new GenericArrayData(resultData)
-  }
-
-  override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): 
ExprCode = {
-    nullSafeCodeGen(ctx, ev, c => {
-      val arrayData = ctx.freshName("arrayData")
-      val numElements = ctx.freshName("numElements")
-      val keys = ctx.freshName("keys")
-      val values = ctx.freshName("values")
-      val isKeyPrimitive = CodeGenerator.isPrimitiveType(childDataType.keyType)
-      val isValuePrimitive = 
CodeGenerator.isPrimitiveType(childDataType.valueType)
-
-      val wordSize = UnsafeRow.WORD_SIZE
-      val structSize = UnsafeRow.calculateBitSetWidthInBytes(2) + wordSize * 2
-      val (isPrimitive, elementSize) = if (isKeyPrimitive && isValuePrimitive) 
{
-        (true, structSize + wordSize)
-      } else {
-        (false, -1)
-      }
-
-      val allocation =
-        s"""
-           |ArrayData $arrayData = ArrayData.allocateArrayData(
-           |  $elementSize, $numElements, " $prettyName failed.");
-         """.stripMargin
-
-      val code = if (isPrimitive) {
-        val genCodeForPrimitive = genCodeForPrimitiveElements(
-          ctx, arrayData, keys, values, ev.value, numElements, structSize)
-        s"""
-           |if ($arrayData instanceof UnsafeArrayData) {
-           |  $genCodeForPrimitive
-           |} else {
-           |  ${genCodeForAnyElements(ctx, arrayData, keys, values, ev.value, 
numElements)}
-           |}
-         """.stripMargin
-      } else {
-        s"${genCodeForAnyElements(ctx, arrayData, keys, values, ev.value, 
numElements)}"
-      }
-
-      s"""
-         |final int $numElements = $c.numElements();
-         |final ArrayData $keys = $c.keyArray();
-         |final ArrayData $values = $c.valueArray();
-         |$allocation
-         |$code
-       """.stripMargin
-    })
-  }
-
-  private def getKey(varName: String, index: String) =
-    CodeGenerator.getValue(varName, childDataType.keyType, index)
-
-  private def getValue(varName: String, index: String) =
-    CodeGenerator.getValue(varName, childDataType.valueType, index)
-
-  private def genCodeForPrimitiveElements(
-      ctx: CodegenContext,
-      arrayData: String,
-      keys: String,
-      values: String,
-      resultArrayData: String,
-      numElements: String,
-      structSize: Int): String = {
-    val unsafeArrayData = ctx.freshName("unsafeArrayData")
-    val baseObject = ctx.freshName("baseObject")
-    val unsafeRow = ctx.freshName("unsafeRow")
-    val structsOffset = ctx.freshName("structsOffset")
-    val offset = ctx.freshName("offset")
-    val z = ctx.freshName("z")
-    val calculateHeader = "UnsafeArrayData.calculateHeaderPortionInBytes"
-
-    val baseOffset = Platform.BYTE_ARRAY_OFFSET
-    val wordSize = UnsafeRow.WORD_SIZE
-    val structSizeAsLong = s"${structSize}L"
-
-    val setKey = CodeGenerator.setColumn(unsafeRow, childDataType.keyType, 0, 
getKey(keys, z))
-
-    val valueAssignmentChecked = CodeGenerator.createArrayAssignment(
-      unsafeRow, childDataType.valueType, values, "1", z, 
childDataType.valueContainsNull)
-
-    s"""
-       |UnsafeArrayData $unsafeArrayData = (UnsafeArrayData)$arrayData;
-       |Object $baseObject = $unsafeArrayData.getBaseObject();
-       |final int $structsOffset = $calculateHeader($numElements) + 
$numElements * $wordSize;
-       |UnsafeRow $unsafeRow = new UnsafeRow(2);
-       |for (int $z = 0; $z < $numElements; $z++) {
-       |  long $offset = $structsOffset + $z * $structSizeAsLong;
-       |  $unsafeArrayData.setLong($z, ($offset << 32) + $structSizeAsLong);
-       |  $unsafeRow.pointTo($baseObject, $baseOffset + $offset, $structSize);
-       |  $setKey;
-       |  $valueAssignmentChecked
-       |}
-       |$resultArrayData = $arrayData;
-     """.stripMargin
-  }
-
-  private def genCodeForAnyElements(
-      ctx: CodegenContext,
-      arrayData: String,
-      keys: String,
-      values: String,
-      resultArrayData: String,
-      numElements: String): String = {
-    val z = ctx.freshName("z")
-    val isValuePrimitive = 
CodeGenerator.isPrimitiveType(childDataType.valueType)
-    val getValueWithCheck = if (childDataType.valueContainsNull && 
isValuePrimitive) {
-      s"$values.isNullAt($z) ? null : (Object)${getValue(values, z)}"
-    } else {
-      getValue(values, z)
-    }
-
-    val rowClass = classOf[GenericInternalRow].getName
-    val genericArrayDataClass = classOf[GenericArrayData].getName
-    val genericArrayData = ctx.freshName("genericArrayData")
-    val rowObject = s"new $rowClass(new Object[]{${getKey(keys, z)}, 
$getValueWithCheck})"
-    s"""
-       |$genericArrayDataClass $genericArrayData = 
($genericArrayDataClass)$arrayData;
-       |for (int $z = 0; $z < $numElements; $z++) {
-       |  $genericArrayData.update($z, $rowObject);
-       |}
-       |$resultArrayData = $arrayData;
-     """.stripMargin
-  }
-
-  override def prettyName: String = "map_entries"
-}
-
-/**
  * Returns the union of all the given maps.
  */
 @ExpressionDescription(

http://git-wip-us.apache.org/repos/asf/spark/blob/a9f200e1/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 b07d946..32f9753 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
@@ -261,56 +261,6 @@ case class ArrayTransform(
 }
 
 /**
- * Filters entries in a map using the provided function.
- */
-@ExpressionDescription(
-usage = "_FUNC_(expr, func) - Filters entries in a map using the function.",
-examples = """
-    Examples:
-      > SELECT _FUNC_(map(1, 0, 2, 2, 3, -1), (k, v) -> k > v);
-       {1:0,3:-1}
-  """,
-since = "2.4.0")
-case class MapFilter(
-    argument: Expression,
-    function: Expression)
-  extends MapBasedSimpleHigherOrderFunction with CodegenFallback {
-
-  @transient lazy val (keyVar, valueVar) = {
-    val args = function.asInstanceOf[LambdaFunction].arguments
-    (args.head.asInstanceOf[NamedLambdaVariable], 
args.tail.head.asInstanceOf[NamedLambdaVariable])
-  }
-
-  @transient lazy val MapType(keyType, valueType, valueContainsNull) = 
argument.dataType
-
-  override def bind(f: (Expression, Seq[(DataType, Boolean)]) => 
LambdaFunction): MapFilter = {
-    copy(function = f(function, (keyType, false) :: (valueType, 
valueContainsNull) :: Nil))
-  }
-
-  override def nullSafeEval(inputRow: InternalRow, argumentValue: Any): Any = {
-    val m = argumentValue.asInstanceOf[MapData]
-    val f = functionForEval
-    val retKeys = new mutable.ListBuffer[Any]
-    val retValues = new mutable.ListBuffer[Any]
-    m.foreach(keyType, valueType, (k, v) => {
-      keyVar.value.set(k)
-      valueVar.value.set(v)
-      if (f.eval(inputRow).asInstanceOf[Boolean]) {
-        retKeys += k
-        retValues += v
-      }
-    })
-    ArrayBasedMapData(retKeys.toArray, retValues.toArray)
-  }
-
-  override def dataType: DataType = argument.dataType
-
-  override def functionType: AbstractDataType = BooleanType
-
-  override def prettyName: String = "map_filter"
-}
-
-/**
  * Filters the input array using the given lambda function.
  */
 @ExpressionDescription(
@@ -491,286 +441,6 @@ case class ArrayAggregate(
   override def prettyName: String = "aggregate"
 }
 
-/**
- * Transform Keys for every entry of the map by applying the transform_keys 
function.
- * Returns map with transformed key entries
- */
-@ExpressionDescription(
-  usage = "_FUNC_(expr, func) - Transforms elements in a map using the 
function.",
-  examples = """
-    Examples:
-      > SELECT _FUNC_(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) 
-> k + 1);
-       {2:1,3:2,4:3}
-      > SELECT _FUNC_(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) 
-> k + v);
-       {2:1,4:2,6:3}
-  """,
-  since = "2.4.0")
-case class TransformKeys(
-    argument: Expression,
-    function: Expression)
-  extends MapBasedSimpleHigherOrderFunction with CodegenFallback {
-
-  @transient lazy val MapType(keyType, valueType, valueContainsNull) = 
argument.dataType
-
-  override def dataType: DataType = MapType(function.dataType, valueType, 
valueContainsNull)
-
-  override def bind(f: (Expression, Seq[(DataType, Boolean)]) => 
LambdaFunction): TransformKeys = {
-    copy(function = f(function, (keyType, false) :: (valueType, 
valueContainsNull) :: Nil))
-  }
-
-  @transient lazy val LambdaFunction(
-    _, (keyVar: NamedLambdaVariable) :: (valueVar: NamedLambdaVariable) :: 
Nil, _) = function
-
-
-  override def nullSafeEval(inputRow: InternalRow, argumentValue: Any): Any = {
-    val map = argumentValue.asInstanceOf[MapData]
-    val resultKeys = new GenericArrayData(new Array[Any](map.numElements))
-    var i = 0
-    while (i < map.numElements) {
-      keyVar.value.set(map.keyArray().get(i, keyVar.dataType))
-      valueVar.value.set(map.valueArray().get(i, valueVar.dataType))
-      val result = functionForEval.eval(inputRow)
-      if (result == null) {
-        throw new RuntimeException("Cannot use null as map key!")
-      }
-      resultKeys.update(i, result)
-      i += 1
-    }
-    new ArrayBasedMapData(resultKeys, map.valueArray())
-  }
-
-  override def prettyName: String = "transform_keys"
-}
-
-/**
- * Returns a map that applies the function to each value of the map.
- */
-@ExpressionDescription(
-  usage = "_FUNC_(expr, func) - Transforms values in the map using the 
function.",
-  examples = """
-    Examples:
-      > SELECT _FUNC_(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) 
-> v + 1);
-       {1:2,2:3,3:4}
-      > SELECT _FUNC_(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) 
-> k + v);
-       {1:2,2:4,3:6}
-  """,
-  since = "2.4.0")
-case class TransformValues(
-    argument: Expression,
-    function: Expression)
-  extends MapBasedSimpleHigherOrderFunction with CodegenFallback {
-
-  @transient lazy val MapType(keyType, valueType, valueContainsNull) = 
argument.dataType
-
-  override def dataType: DataType = MapType(keyType, function.dataType, 
function.nullable)
-
-  override def bind(f: (Expression, Seq[(DataType, Boolean)]) => 
LambdaFunction)
-  : TransformValues = {
-    copy(function = f(function, (keyType, false) :: (valueType, 
valueContainsNull) :: Nil))
-  }
-
-  @transient lazy val LambdaFunction(
-    _, (keyVar: NamedLambdaVariable) :: (valueVar: NamedLambdaVariable) :: 
Nil, _) = function
-
-  override def nullSafeEval(inputRow: InternalRow, argumentValue: Any): Any = {
-    val map = argumentValue.asInstanceOf[MapData]
-    val resultValues = new GenericArrayData(new Array[Any](map.numElements))
-    var i = 0
-    while (i < map.numElements) {
-      keyVar.value.set(map.keyArray().get(i, keyVar.dataType))
-      valueVar.value.set(map.valueArray().get(i, valueVar.dataType))
-      resultValues.update(i, functionForEval.eval(inputRow))
-      i += 1
-    }
-    new ArrayBasedMapData(map.keyArray(), resultValues)
-  }
-
-  override def prettyName: String = "transform_values"
-}
-
-/**
- * Merges two given maps into a single map by applying function to the pair of 
values with
- * the same key.
- */
-@ExpressionDescription(
-  usage =
-    """
-      _FUNC_(map1, map2, function) - Merges two given maps into a single map 
by applying
-      function to the pair of values with the same key. For keys only 
presented in one map,
-      NULL will be passed as the value for the missing key. If an input map 
contains duplicated
-      keys, only the first entry of the duplicated key is passed into the 
lambda function.
-    """,
-  examples = """
-    Examples:
-      > SELECT _FUNC_(map(1, 'a', 2, 'b'), map(1, 'x', 2, 'y'), (k, v1, v2) -> 
concat(v1, v2));
-       {1:"ax",2:"by"}
-  """,
-  since = "2.4.0")
-case class MapZipWith(left: Expression, right: Expression, function: 
Expression)
-  extends HigherOrderFunction with CodegenFallback {
-
-  def functionForEval: Expression = functionsForEval.head
-
-  @transient lazy val MapType(leftKeyType, leftValueType, 
leftValueContainsNull) = left.dataType
-
-  @transient lazy val MapType(rightKeyType, rightValueType, 
rightValueContainsNull) = right.dataType
-
-  @transient lazy val keyType =
-    TypeCoercion.findCommonTypeDifferentOnlyInNullFlags(leftKeyType, 
rightKeyType).get
-
-  @transient lazy val ordering = TypeUtils.getInterpretedOrdering(keyType)
-
-  override def arguments: Seq[Expression] = left :: right :: Nil
-
-  override def argumentTypes: Seq[AbstractDataType] = MapType :: MapType :: Nil
-
-  override def functions: Seq[Expression] = function :: Nil
-
-  override def functionTypes: Seq[AbstractDataType] = AnyDataType :: Nil
-
-  override def dataType: DataType = MapType(keyType, function.dataType, 
function.nullable)
-
-  override def bind(f: (Expression, Seq[(DataType, Boolean)]) => 
LambdaFunction): MapZipWith = {
-    val arguments = Seq((keyType, false), (leftValueType, true), 
(rightValueType, true))
-    copy(function = f(function, arguments))
-  }
-
-  override def checkArgumentDataTypes(): TypeCheckResult = {
-    super.checkArgumentDataTypes() match {
-      case TypeCheckResult.TypeCheckSuccess =>
-        if (leftKeyType.sameType(rightKeyType)) {
-          TypeUtils.checkForOrderingExpr(leftKeyType, s"function $prettyName")
-        } else {
-          TypeCheckResult.TypeCheckFailure(s"The input to function $prettyName 
should have " +
-            s"been two ${MapType.simpleString}s with compatible key types, but 
the key types are " +
-            s"[${leftKeyType.catalogString}, ${rightKeyType.catalogString}].")
-        }
-      case failure => failure
-    }
-  }
-
-  override def checkInputDataTypes(): TypeCheckResult = 
checkArgumentDataTypes()
-
-  override def eval(input: InternalRow): Any = {
-    val value1 = left.eval(input)
-    if (value1 == null) {
-      null
-    } else {
-      val value2 = right.eval(input)
-      if (value2 == null) {
-        null
-      } else {
-        nullSafeEval(input, value1, value2)
-      }
-    }
-  }
-
-  @transient lazy val LambdaFunction(_, Seq(
-    keyVar: NamedLambdaVariable,
-    value1Var: NamedLambdaVariable,
-    value2Var: NamedLambdaVariable),
-    _) = function
-
-  /**
-   * The function accepts two key arrays and returns a collection of keys with 
indexes
-   * to value arrays. Indexes are represented as an array of two items. This 
is a small
-   * optimization leveraging mutability of arrays.
-   */
-  @transient private lazy val getKeysWithValueIndexes:
-      (ArrayData, ArrayData) => mutable.Iterable[(Any, Array[Option[Int]])] = {
-    if (TypeUtils.typeWithProperEquals(keyType)) {
-      getKeysWithIndexesFast
-    } else {
-      getKeysWithIndexesBruteForce
-    }
-  }
-
-  private def assertSizeOfArrayBuffer(size: Int): Unit = {
-    if (size > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) {
-      throw new RuntimeException(s"Unsuccessful try to zip maps with $size " +
-        s"unique keys due to exceeding the array size limit " +
-        s"${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}.")
-    }
-  }
-
-  private def getKeysWithIndexesFast(keys1: ArrayData, keys2: ArrayData) = {
-    val hashMap = new mutable.LinkedHashMap[Any, Array[Option[Int]]]
-    for((z, array) <- Array((0, keys1), (1, keys2))) {
-      var i = 0
-      while (i < array.numElements()) {
-        val key = array.get(i, keyType)
-        hashMap.get(key) match {
-          case Some(indexes) =>
-            if (indexes(z).isEmpty) {
-              indexes(z) = Some(i)
-            }
-          case None =>
-            val indexes = Array[Option[Int]](None, None)
-            indexes(z) = Some(i)
-            hashMap.put(key, indexes)
-        }
-        i += 1
-      }
-    }
-    hashMap
-  }
-
-  private def getKeysWithIndexesBruteForce(keys1: ArrayData, keys2: ArrayData) 
= {
-    val arrayBuffer = new mutable.ArrayBuffer[(Any, Array[Option[Int]])]
-    for((z, array) <- Array((0, keys1), (1, keys2))) {
-      var i = 0
-      while (i < array.numElements()) {
-        val key = array.get(i, keyType)
-        var found = false
-        var j = 0
-        while (!found && j < arrayBuffer.size) {
-          val (bufferKey, indexes) = arrayBuffer(j)
-          if (ordering.equiv(bufferKey, key)) {
-            found = true
-            if(indexes(z).isEmpty) {
-              indexes(z) = Some(i)
-            }
-          }
-          j += 1
-        }
-        if (!found) {
-          assertSizeOfArrayBuffer(arrayBuffer.size)
-          val indexes = Array[Option[Int]](None, None)
-          indexes(z) = Some(i)
-          arrayBuffer += Tuple2(key, indexes)
-        }
-        i += 1
-      }
-    }
-    arrayBuffer
-  }
-
-  private def nullSafeEval(inputRow: InternalRow, value1: Any, value2: Any): 
Any = {
-    val mapData1 = value1.asInstanceOf[MapData]
-    val mapData2 = value2.asInstanceOf[MapData]
-    val keysWithIndexes = getKeysWithValueIndexes(mapData1.keyArray(), 
mapData2.keyArray())
-    val size = keysWithIndexes.size
-    val keys = new GenericArrayData(new Array[Any](size))
-    val values = new GenericArrayData(new Array[Any](size))
-    val valueData1 = mapData1.valueArray()
-    val valueData2 = mapData2.valueArray()
-    var i = 0
-    for ((key, Array(index1, index2)) <- keysWithIndexes) {
-      val v1 = index1.map(valueData1.get(_, leftValueType)).getOrElse(null)
-      val v2 = index2.map(valueData2.get(_, rightValueType)).getOrElse(null)
-      keyVar.value.set(key)
-      value1Var.value.set(v1)
-      value2Var.value.set(v2)
-      keys.update(i, key)
-      values.update(i, functionForEval.eval(inputRow))
-      i += 1
-    }
-    new ArrayBasedMapData(keys, values)
-  }
-
-  override def prettyName: String = "map_zip_with"
-}
-
 // scalastyle:off line.size.limit
 @ExpressionDescription(
   usage = "_FUNC_(left, right, func) - Merges the two given arrays, 
element-wise, into a single array using function. If one array is shorter, 
nulls are appended at the end to match the length of the longer array, before 
applying function.",

http://git-wip-us.apache.org/repos/asf/spark/blob/a9f200e1/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 c7db4ec..29014a2 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
@@ -83,30 +83,6 @@ class CollectionExpressionsSuite extends SparkFunSuite with 
ExpressionEvalHelper
     checkEvaluation(MapValues(m2), null)
   }
 
-  test("MapEntries") {
-    def r(values: Any*): InternalRow = create_row(values: _*)
-
-    // Primitive-type keys/values
-    val mi0 = Literal.create(Map(1 -> 1, 2 -> null, 3 -> 2), 
MapType(IntegerType, IntegerType))
-    val mi1 = Literal.create(Map[Int, Int](), MapType(IntegerType, 
IntegerType))
-    val mi2 = Literal.create(null, MapType(IntegerType, IntegerType))
-    val mid0 = Literal.create(Map(1 -> 1.1, 2 -> 2.2), MapType(IntegerType, 
DoubleType))
-
-    checkEvaluation(MapEntries(mi0), Seq(r(1, 1), r(2, null), r(3, 2)))
-    checkEvaluation(MapEntries(mi1), Seq.empty)
-    checkEvaluation(MapEntries(mi2), null)
-    checkEvaluation(MapEntries(mid0), Seq(r(1, 1.1), r(2, 2.2)))
-
-    // Non-primitive-type keys/values
-    val ms0 = Literal.create(Map("a" -> "c", "b" -> null), MapType(StringType, 
StringType))
-    val ms1 = Literal.create(Map[Int, Int](), MapType(StringType, StringType))
-    val ms2 = Literal.create(null, MapType(StringType, StringType))
-
-    checkEvaluation(MapEntries(ms0), Seq(r("a", "c"), r("b", null)))
-    checkEvaluation(MapEntries(ms1), Seq.empty)
-    checkEvaluation(MapEntries(ms2), null)
-  }
-
   test("Map Concat") {
     val m0 = Literal.create(Map("a" -> "1", "b" -> "2"), MapType(StringType, 
StringType,
       valueContainsNull = false))

http://git-wip-us.apache.org/repos/asf/spark/blob/a9f200e1/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..a0155ca 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
@@ -88,11 +88,6 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with 
ExpressionEvalHelper
     ArrayFilter(expr, createLambda(et, cn, f)).bind(validateBinding)
   }
 
-  def transformKeys(expr: Expression, f: (Expression, Expression) => 
Expression): Expression = {
-    val MapType(kt, vt, vcn) = expr.dataType
-    TransformKeys(expr, createLambda(kt, false, vt, vcn, 
f)).bind(validateBinding)
-  }
-
   def aggregate(
       expr: Expression,
       zero: Expression,
@@ -115,11 +110,6 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with 
ExpressionEvalHelper
     aggregate(expr, zero, merge, identity)
   }
 
-  def transformValues(expr: Expression, f: (Expression, Expression) => 
Expression): Expression = {
-    val MapType(kt, vt, vcn) = expr.dataType
-    TransformValues(expr, createLambda(kt, false, vt, vcn, 
f)).bind(validateBinding)
-  }
-
   test("ArrayTransform") {
     val ai0 = Literal.create(Seq(1, 2, 3), ArrayType(IntegerType, containsNull 
= false))
     val ai1 = Literal.create(Seq[Integer](1, null, 3), ArrayType(IntegerType, 
containsNull = true))
@@ -161,55 +151,6 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with 
ExpressionEvalHelper
       Seq("[1, 3, 5]", null, "[4, 6]"))
   }
 
-  test("MapFilter") {
-    def mapFilter(expr: Expression, f: (Expression, Expression) => 
Expression): Expression = {
-      val MapType(kt, vt, vcn) = expr.dataType
-      MapFilter(expr, createLambda(kt, false, vt, vcn, 
f)).bind(validateBinding)
-    }
-    val mii0 = Literal.create(Map(1 -> 0, 2 -> 10, 3 -> -1),
-      MapType(IntegerType, IntegerType, valueContainsNull = false))
-    val mii1 = Literal.create(Map(1 -> null, 2 -> 10, 3 -> null),
-      MapType(IntegerType, IntegerType, valueContainsNull = true))
-    val miin = Literal.create(null, MapType(IntegerType, IntegerType, 
valueContainsNull = false))
-
-    val kGreaterThanV: (Expression, Expression) => Expression = (k, v) => k > v
-
-    checkEvaluation(mapFilter(mii0, kGreaterThanV), Map(1 -> 0, 3 -> -1))
-    checkEvaluation(mapFilter(mii1, kGreaterThanV), Map())
-    checkEvaluation(mapFilter(miin, kGreaterThanV), null)
-
-    val valueIsNull: (Expression, Expression) => Expression = (_, v) => 
v.isNull
-
-    checkEvaluation(mapFilter(mii0, valueIsNull), Map())
-    checkEvaluation(mapFilter(mii1, valueIsNull), Map(1 -> null, 3 -> null))
-    checkEvaluation(mapFilter(miin, valueIsNull), null)
-
-    val msi0 = Literal.create(Map("abcdf" -> 5, "abc" -> 10, "" -> 0),
-      MapType(StringType, IntegerType, valueContainsNull = false))
-    val msi1 = Literal.create(Map("abcdf" -> 5, "abc" -> 10, "" -> null),
-      MapType(StringType, IntegerType, valueContainsNull = true))
-    val msin = Literal.create(null, MapType(StringType, IntegerType, 
valueContainsNull = false))
-
-    val isLengthOfKey: (Expression, Expression) => Expression = (k, v) => 
Length(k) === v
-
-    checkEvaluation(mapFilter(msi0, isLengthOfKey), Map("abcdf" -> 5, "" -> 0))
-    checkEvaluation(mapFilter(msi1, isLengthOfKey), Map("abcdf" -> 5))
-    checkEvaluation(mapFilter(msin, isLengthOfKey), null)
-
-    val mia0 = Literal.create(Map(1 -> Seq(0, 1, 2), 2 -> Seq(10), -3 -> 
Seq(-1, 0, -2, 3)),
-      MapType(IntegerType, ArrayType(IntegerType), valueContainsNull = false))
-    val mia1 = Literal.create(Map(1 -> Seq(0, 1, 2), 2 -> null, -3 -> Seq(-1, 
0, -2, 3)),
-      MapType(IntegerType, ArrayType(IntegerType), valueContainsNull = true))
-    val mian = Literal.create(
-      null, MapType(IntegerType, ArrayType(IntegerType), valueContainsNull = 
false))
-
-    val customFunc: (Expression, Expression) => Expression = (k, v) => Size(v) 
+ k > 3
-
-    checkEvaluation(mapFilter(mia0, customFunc), Map(1 -> Seq(0, 1, 2)))
-    checkEvaluation(mapFilter(mia1, customFunc), Map(1 -> Seq(0, 1, 2)))
-    checkEvaluation(mapFilter(mian, customFunc), null)
-  }
-
   test("ArrayFilter") {
     val ai0 = Literal.create(Seq(1, 2, 3), ArrayType(IntegerType, containsNull 
= false))
     val ai1 = Literal.create(Seq[Integer](1, null, 3), ArrayType(IntegerType, 
containsNull = true))
@@ -308,262 +249,6 @@ class HigherOrderFunctionsSuite extends SparkFunSuite 
with ExpressionEvalHelper
       15)
   }
 
-  test("TransformKeys") {
-    val ai0 = Literal.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),
-      MapType(IntegerType, IntegerType, valueContainsNull = true))
-    val ai3 = Literal.create(null, MapType(IntegerType, IntegerType, 
valueContainsNull = false))
-
-    val plusOne: (Expression, Expression) => Expression = (k, v) => k + 1
-    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(transformKeys(ai0, plusOne), plusValue), 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(transformKeys(ai2, plusOne), plusOne), Map(3 -> 1, 4 -> 
null, 5 -> 3))
-    checkEvaluation(transformKeys(ai3, plusOne), null)
-
-    val as0 = Literal.create(
-      Map("a" -> "xy", "bb" -> "yz", "ccc" -> "zx"),
-      MapType(StringType, StringType, valueContainsNull = false))
-    val as1 = Literal.create(
-      Map("a" -> "xy", "bb" -> "yz", "ccc" -> null),
-      MapType(StringType, StringType, valueContainsNull = true))
-    val as2 = Literal.create(null,
-      MapType(StringType, StringType, valueContainsNull = false))
-    val as3 = Literal.create(Map.empty[StringType, StringType],
-      MapType(StringType, StringType, valueContainsNull = true))
-
-    val concatValue: (Expression, Expression) => Expression = (k, v) => 
Concat(Seq(k, v))
-    val convertKeyToKeyLength: (Expression, Expression) => Expression =
-      (k, v) => Length(k) + 1
-
-    checkEvaluation(
-      transformKeys(as0, concatValue), Map("axy" -> "xy", "bbyz" -> "yz", 
"ccczx" -> "zx"))
-    checkEvaluation(
-      transformKeys(transformKeys(as0, concatValue), concatValue),
-      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"))
-    checkEvaluation(transformKeys(as1, convertKeyToKeyLength),
-      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"),
-      MapType(IntegerType, StringType, valueContainsNull = false))
-
-    checkEvaluation(transformKeys(ax0, plusOne), Map(2 -> "x", 3 -> "y", 4 -> 
"z"))
-  }
-
-  test("TransformValues") {
-    val ai0 = Literal.create(
-      Map(1 -> 1, 2 -> 2, 3 -> 3),
-      MapType(IntegerType, IntegerType, valueContainsNull = false))
-    val ai1 = Literal.create(
-      Map(1 -> 1, 2 -> null, 3 -> 3),
-      MapType(IntegerType, IntegerType, valueContainsNull = true))
-    val ai2 = Literal.create(
-      Map.empty[Int, Int],
-      MapType(IntegerType, IntegerType, valueContainsNull = true))
-    val ai3 = Literal.create(null, MapType(IntegerType, IntegerType, 
valueContainsNull = false))
-
-    val plusOne: (Expression, Expression) => Expression = (k, v) => v + 1
-    val valueUpdate: (Expression, Expression) => Expression = (k, v) => k * k
-
-    checkEvaluation(transformValues(ai0, plusOne), Map(1 -> 2, 2 -> 3, 3 -> 4))
-    checkEvaluation(transformValues(ai0, valueUpdate), Map(1 -> 1, 2 -> 4, 3 
-> 9))
-    checkEvaluation(
-      transformValues(transformValues(ai0, plusOne), valueUpdate), Map(1 -> 1, 
2 -> 4, 3 -> 9))
-    checkEvaluation(transformValues(ai1, plusOne), Map(1 -> 2, 2 -> null, 3 -> 
4))
-    checkEvaluation(transformValues(ai1, valueUpdate), Map(1 -> 1, 2 -> 4, 3 
-> 9))
-    checkEvaluation(
-      transformValues(transformValues(ai1, plusOne), valueUpdate), Map(1 -> 1, 
2 -> 4, 3 -> 9))
-    checkEvaluation(transformValues(ai2, plusOne), Map.empty[Int, Int])
-    checkEvaluation(transformValues(ai3, plusOne), null)
-
-    val as0 = Literal.create(
-      Map("a" -> "xy", "bb" -> "yz", "ccc" -> "zx"),
-      MapType(StringType, StringType, valueContainsNull = false))
-    val as1 = Literal.create(
-      Map("a" -> "xy", "bb" -> null, "ccc" -> "zx"),
-      MapType(StringType, StringType, valueContainsNull = true))
-    val as2 = Literal.create(Map.empty[StringType, StringType],
-      MapType(StringType, StringType, valueContainsNull = true))
-    val as3 = Literal.create(null, MapType(StringType, StringType, 
valueContainsNull = true))
-
-    val concatValue: (Expression, Expression) => Expression = (k, v) => 
Concat(Seq(k, v))
-    val valueTypeUpdate: (Expression, Expression) => Expression =
-      (k, v) => Length(v) + 1
-
-    checkEvaluation(
-      transformValues(as0, concatValue), Map("a" -> "axy", "bb" -> "bbyz", 
"ccc" -> "ccczx"))
-    checkEvaluation(transformValues(as0, valueTypeUpdate),
-      Map("a" -> 3, "bb" -> 3, "ccc" -> 3))
-    checkEvaluation(
-      transformValues(transformValues(as0, concatValue), concatValue),
-      Map("a" -> "aaxy", "bb" -> "bbbbyz", "ccc" -> "cccccczx"))
-    checkEvaluation(transformValues(as1, concatValue),
-      Map("a" -> "axy", "bb" -> null, "ccc" -> "ccczx"))
-    checkEvaluation(transformValues(as1, valueTypeUpdate),
-      Map("a" -> 3, "bb" -> null, "ccc" -> 3))
-    checkEvaluation(
-      transformValues(transformValues(as1, concatValue), concatValue),
-      Map("a" -> "aaxy", "bb" -> null, "ccc" -> "cccccczx"))
-    checkEvaluation(transformValues(as2, concatValue), Map.empty[String, 
String])
-    checkEvaluation(transformValues(as2, valueTypeUpdate), Map.empty[String, 
Int])
-    checkEvaluation(
-      transformValues(transformValues(as2, concatValue), valueTypeUpdate),
-      Map.empty[String, Int])
-    checkEvaluation(transformValues(as3, concatValue), null)
-
-    val ax0 = Literal.create(
-      Map(1 -> "x", 2 -> "y", 3 -> "z"),
-      MapType(IntegerType, StringType, valueContainsNull = false))
-
-    checkEvaluation(transformValues(ax0, valueUpdate), Map(1 -> 1, 2 -> 4, 3 
-> 9))
-  }
-
-  test("MapZipWith") {
-    def map_zip_with(
-        left: Expression,
-        right: Expression,
-        f: (Expression, Expression, Expression) => Expression): Expression = {
-      val MapType(kt, vt1, _) = left.dataType
-      val MapType(_, vt2, _) = right.dataType
-      MapZipWith(left, right, createLambda(kt, false, vt1, true, vt2, true, f))
-        .bind(validateBinding)
-    }
-
-    val mii0 = Literal.create(Map(1 -> 10, 2 -> 20, 3 -> 30),
-      MapType(IntegerType, IntegerType, valueContainsNull = false))
-    val mii1 = Literal.create(Map(1 -> -1, 2 -> -2, 4 -> -4),
-      MapType(IntegerType, IntegerType, valueContainsNull = false))
-    val mii2 = Literal.create(Map(1 -> null, 2 -> -2, 3 -> null),
-      MapType(IntegerType, IntegerType, valueContainsNull = true))
-    val mii3 = Literal.create(Map(), MapType(IntegerType, IntegerType, 
valueContainsNull = false))
-    val mii4 = MapFromArrays(
-      Literal.create(Seq(2, 2), ArrayType(IntegerType, false)),
-      Literal.create(Seq(20, 200), ArrayType(IntegerType, false)))
-    val miin = Literal.create(null, MapType(IntegerType, IntegerType, 
valueContainsNull = false))
-
-    val multiplyKeyWithValues: (Expression, Expression, Expression) => 
Expression = {
-      (k, v1, v2) => k * v1 * v2
-    }
-
-    checkEvaluation(
-      map_zip_with(mii0, mii1, multiplyKeyWithValues),
-      Map(1 -> -10, 2 -> -80, 3 -> null, 4 -> null))
-    checkEvaluation(
-      map_zip_with(mii0, mii2, multiplyKeyWithValues),
-      Map(1 -> null, 2 -> -80, 3 -> null))
-    checkEvaluation(
-      map_zip_with(mii0, mii3, multiplyKeyWithValues),
-      Map(1 -> null, 2 -> null, 3 -> null))
-    checkEvaluation(
-      map_zip_with(mii0, mii4, multiplyKeyWithValues),
-      Map(1 -> null, 2 -> 800, 3 -> null))
-    checkEvaluation(
-      map_zip_with(mii4, mii0, multiplyKeyWithValues),
-      Map(2 -> 800, 1 -> null, 3 -> null))
-    checkEvaluation(
-      map_zip_with(mii0, miin, multiplyKeyWithValues),
-      null)
-    assert(map_zip_with(mii0, mii1, multiplyKeyWithValues).dataType ===
-      MapType(IntegerType, IntegerType, valueContainsNull = true))
-
-    val mss0 = Literal.create(Map("a" -> "x", "b" -> "y", "d" -> "z"),
-      MapType(StringType, StringType, valueContainsNull = false))
-    val mss1 = Literal.create(Map("d" -> "b", "b" -> "d"),
-      MapType(StringType, StringType, valueContainsNull = false))
-    val mss2 = Literal.create(Map("c" -> null, "b" -> "t", "a" -> null),
-      MapType(StringType, StringType, valueContainsNull = true))
-    val mss3 = Literal.create(Map(), MapType(StringType, StringType, 
valueContainsNull = false))
-    val mss4 = MapFromArrays(
-      Literal.create(Seq("a", "a"), ArrayType(StringType, false)),
-      Literal.create(Seq("a", "n"), ArrayType(StringType, false)))
-    val mssn = Literal.create(null, MapType(StringType, StringType, 
valueContainsNull = false))
-
-    val concat: (Expression, Expression, Expression) => Expression = {
-      (k, v1, v2) => Concat(Seq(k, v1, v2))
-    }
-
-    checkEvaluation(
-      map_zip_with(mss0, mss1, concat),
-      Map("a" -> null, "b" -> "byd", "d" -> "dzb"))
-    checkEvaluation(
-      map_zip_with(mss1, mss2, concat),
-      Map("d" -> null, "b" -> "bdt", "c" -> null, "a" -> null))
-    checkEvaluation(
-      map_zip_with(mss0, mss3, concat),
-      Map("a" -> null, "b" -> null, "d" -> null))
-    checkEvaluation(
-      map_zip_with(mss0, mss4, concat),
-      Map("a" -> "axa", "b" -> null, "d" -> null))
-    checkEvaluation(
-      map_zip_with(mss4, mss0, concat),
-      Map("a" -> "aax", "b" -> null, "d" -> null))
-    checkEvaluation(
-      map_zip_with(mss0, mssn, concat),
-      null)
-    assert(map_zip_with(mss0, mss1, concat).dataType ===
-      MapType(StringType, StringType, valueContainsNull = true))
-
-    def b(data: Byte*): Array[Byte] = Array[Byte](data: _*)
-
-    val mbb0 = Literal.create(Map(b(1, 2) -> b(4), b(2, 1) -> b(5), b(1, 3) -> 
b(8)),
-      MapType(BinaryType, BinaryType, valueContainsNull = false))
-    val mbb1 = Literal.create(Map(b(2, 1) -> b(7), b(1, 2) -> b(3), b(1, 1) -> 
b(6)),
-      MapType(BinaryType, BinaryType, valueContainsNull = false))
-    val mbb2 = Literal.create(Map(b(1, 3) -> null, b(1, 2) -> b(2), b(2, 1) -> 
null),
-      MapType(BinaryType, BinaryType, valueContainsNull = true))
-    val mbb3 = Literal.create(Map(), MapType(BinaryType, BinaryType, 
valueContainsNull = false))
-    val mbb4 = MapFromArrays(
-      Literal.create(Seq(b(2, 1), b(2, 1)), ArrayType(BinaryType, false)),
-      Literal.create(Seq(b(1), b(9)), ArrayType(BinaryType, false)))
-    val mbbn = Literal.create(null, MapType(BinaryType, BinaryType, 
valueContainsNull = false))
-
-    checkEvaluation(
-      map_zip_with(mbb0, mbb1, concat),
-      Map(b(1, 2) -> b(1, 2, 4, 3), b(2, 1) -> b(2, 1, 5, 7), b(1, 3) -> null, 
b(1, 1) -> null))
-    checkEvaluation(
-      map_zip_with(mbb1, mbb2, concat),
-      Map(b(2, 1) -> null, b(1, 2) -> b(1, 2, 3, 2), b(1, 1) -> null, b(1, 3) 
-> null))
-    checkEvaluation(
-      map_zip_with(mbb0, mbb3, concat),
-      Map(b(1, 2) -> null, b(2, 1) -> null, b(1, 3) -> null))
-    checkEvaluation(
-      map_zip_with(mbb0, mbb4, concat),
-      Map(b(1, 2) -> null, b(2, 1) -> b(2, 1, 5, 1), b(1, 3) -> null))
-    checkEvaluation(
-      map_zip_with(mbb4, mbb0, concat),
-      Map(b(2, 1) -> b(2, 1, 1, 5), b(1, 2) -> null, b(1, 3) -> null))
-    checkEvaluation(
-      map_zip_with(mbb0, mbbn, concat),
-      null)
-  }
-
   test("ZipWith") {
     def zip_with(
         left: Expression,

http://git-wip-us.apache.org/repos/asf/spark/blob/a9f200e1/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
index 6a43ce1..5dedc9d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
@@ -3781,13 +3781,6 @@ object functions {
   def map_values(e: Column): Column = withExpr { MapValues(e.expr) }
 
   /**
-   * Returns an unordered array of all entries in the given map.
-   * @group collection_funcs
-   * @since 2.4.0
-   */
-  def map_entries(e: Column): Column = withExpr { MapEntries(e.expr) }
-
-  /**
    * Returns a map created from the given array of entries.
    * @group collection_funcs
    * @since 2.4.0

http://git-wip-us.apache.org/repos/asf/spark/blob/a9f200e1/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql 
b/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql
index 02ad5e3..5043064 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql
@@ -60,26 +60,3 @@ select zip_with(array('a', 'b', 'c'), array('d', 'e', 'f'), 
(x, y) -> concat(x,
 
 -- Zip with array coalesce
 select zip_with(array('a'), array('d', null, 'f'), (x, y) -> coalesce(x, y)) 
as v;
-
-create or replace temporary view nested as values
-  (1, map(1, 1, 2, 2, 3, 3)),
-  (2, map(4, 4, 5, 5, 6, 6))
-  as t(x, ys);
-
--- Identity Transform Keys in a map
-select transform_keys(ys, (k, v) -> k) as v from nested;
-
--- Transform Keys in a map by adding constant
-select transform_keys(ys, (k, v) -> k + 1) as v from nested;
-
--- Transform Keys in a map using values
-select transform_keys(ys, (k, v) -> k + v) as v from nested;
-
--- Identity Transform values in a map
-select transform_values(ys, (k, v) -> v) as v from nested;
-
--- Transform values in a map by adding constant
-select transform_values(ys, (k, v) -> v + 1) as v from nested;
-
--- Transform values in a map using values
-select transform_values(ys, (k, v) -> k + v) as v from nested;

http://git-wip-us.apache.org/repos/asf/spark/blob/a9f200e1/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/mapZipWith.sql
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/mapZipWith.sql
 
b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/mapZipWith.sql
deleted file mode 100644
index 1727ee7..0000000
--- 
a/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/mapZipWith.sql
+++ /dev/null
@@ -1,78 +0,0 @@
-CREATE TEMPORARY VIEW various_maps AS SELECT * FROM VALUES (
-  map(true, false),
-  map(2Y, 1Y),
-  map(2S, 1S),
-  map(2, 1),
-  map(2L, 1L),
-  map(922337203685477897945456575809789456, 
922337203685477897945456575809789456),
-  map(9.22337203685477897945456575809789456, 
9.22337203685477897945456575809789456),
-  map(2.0D, 1.0D),
-  map(float(2.0), float(1.0)),
-  map(date '2016-03-14', date '2016-03-13'),
-  map(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 
20:54:00.000'),
-  map('true', 'false', '2', '1'),
-  map('2016-03-14', '2016-03-13'),
-  map('2016-11-15 20:54:00.000', '2016-11-12 20:54:00.000'),
-  map('922337203685477897945456575809789456', 'text'),
-  map(array(1L, 2L), array(1L, 2L)), map(array(1, 2), array(1, 2)),
-  map(struct(1S, 2L), struct(1S, 2L)), map(struct(1, 2), struct(1, 2))
-) AS various_maps(
-  boolean_map,
-  tinyint_map,
-  smallint_map,
-  int_map,
-  bigint_map,
-  decimal_map1, decimal_map2,
-  double_map,
-  float_map,
-  date_map,
-  timestamp_map,
-  string_map1, string_map2, string_map3, string_map4,
-  array_map1, array_map2,
-  struct_map1, struct_map2
-);
-
-SELECT map_zip_with(tinyint_map, smallint_map, (k, v1, v2) -> struct(k, v1, 
v2)) m
-FROM various_maps;
-
-SELECT map_zip_with(smallint_map, int_map, (k, v1, v2) -> struct(k, v1, v2)) m
-FROM various_maps;
-
-SELECT map_zip_with(int_map, bigint_map, (k, v1, v2) -> struct(k, v1, v2)) m
-FROM various_maps;
-
-SELECT map_zip_with(double_map, float_map, (k, v1, v2) -> struct(k, v1, v2)) m
-FROM various_maps;
-
-SELECT map_zip_with(decimal_map1, decimal_map2, (k, v1, v2) -> struct(k, v1, 
v2)) m
-FROM various_maps;
-
-SELECT map_zip_with(decimal_map1, int_map, (k, v1, v2) -> struct(k, v1, v2)) m
-FROM various_maps;
-
-SELECT map_zip_with(decimal_map1, double_map, (k, v1, v2) -> struct(k, v1, 
v2)) m
-FROM various_maps;
-
-SELECT map_zip_with(decimal_map2, int_map, (k, v1, v2) -> struct(k, v1, v2)) m
-FROM various_maps;
-
-SELECT map_zip_with(decimal_map2, double_map, (k, v1, v2) -> struct(k, v1, 
v2)) m
-FROM various_maps;
-
-SELECT map_zip_with(string_map1, int_map, (k, v1, v2) -> struct(k, v1, v2)) m
-FROM various_maps;
-
-SELECT map_zip_with(string_map2, date_map, (k, v1, v2) -> struct(k, v1, v2)) m
-FROM various_maps;
-
-SELECT map_zip_with(timestamp_map, string_map3, (k, v1, v2) -> struct(k, v1, 
v2)) m
-FROM various_maps;
-
-SELECT map_zip_with(decimal_map1, string_map4, (k, v1, v2) -> struct(k, v1, 
v2)) m
-FROM various_maps;
-
-SELECT map_zip_with(array_map1, array_map2, (k, v1, v2) -> struct(k, v1, v2)) m
-FROM various_maps;
-
-SELECT map_zip_with(struct_map1, struct_map2, (k, v1, v2) -> struct(k, v1, 
v2)) m
-FROM various_maps;

http://git-wip-us.apache.org/repos/asf/spark/blob/a9f200e1/sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out 
b/sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out
index 32d20d1..cdc509a 100644
--- 
a/sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out
+++ 
b/sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out
@@ -1,5 +1,5 @@
 -- Automatically generated by SQLQueryTestSuite
--- Number of queries: 27
+-- Number of queries: 20
 
 
 -- !query 0
@@ -189,67 +189,3 @@ select zip_with(array('a'), array('d', null, 'f'), (x, y) 
-> coalesce(x, y)) as
 struct<v:array<string>>
 -- !query 19 output
 ["a",null,"f"]
-
-
--- !query 20
-create or replace temporary view nested as values
-  (1, map(1, 1, 2, 2, 3, 3)),
-  (2, map(4, 4, 5, 5, 6, 6))
-  as t(x, ys)
--- !query 20 schema
-struct<>
--- !query 20 output
-
-
--- !query 21
-select transform_keys(ys, (k, v) -> k) as v from nested
--- !query 21 schema
-struct<v:map<int,int>>
--- !query 21 output
-{1:1,2:2,3:3}
-{4:4,5:5,6:6}
-
-
--- !query 22
-select transform_keys(ys, (k, v) -> k + 1) as v from nested
--- !query 22 schema
-struct<v:map<int,int>>
--- !query 22 output
-{2:1,3:2,4:3}
-{5:4,6:5,7:6}
-
-
--- !query 23
-select transform_keys(ys, (k, v) -> k + v) as v from nested
--- !query 23 schema
-struct<v:map<int,int>>
--- !query 23 output
-{10:5,12:6,8:4}
-{2:1,4:2,6:3}
-
-
--- !query 24
-select transform_values(ys, (k, v) -> v) as v from nested
--- !query 24 schema
-struct<v:map<int,int>>
--- !query 24 output
-{1:1,2:2,3:3}
-{4:4,5:5,6:6}
-
-
--- !query 25
-select transform_values(ys, (k, v) -> v + 1) as v from nested
--- !query 25 schema
-struct<v:map<int,int>>
--- !query 25 output
-{1:2,2:3,3:4}
-{4:5,5:6,6:7}
-
-
--- !query 26
-select transform_values(ys, (k, v) -> k + v) as v from nested
--- !query 26 schema
-struct<v:map<int,int>>
--- !query 26 output
-{1:2,2:4,3:6}
-{4:8,5:10,6:12}

http://git-wip-us.apache.org/repos/asf/spark/blob/a9f200e1/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out
 
b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out
deleted file mode 100644
index 3574009..0000000
--- 
a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out
+++ /dev/null
@@ -1,179 +0,0 @@
--- Automatically generated by SQLQueryTestSuite
--- Number of queries: 16
-
-
--- !query 0
-CREATE TEMPORARY VIEW various_maps AS SELECT * FROM VALUES (
-  map(true, false),
-  map(2Y, 1Y),
-  map(2S, 1S),
-  map(2, 1),
-  map(2L, 1L),
-  map(922337203685477897945456575809789456, 
922337203685477897945456575809789456),
-  map(9.22337203685477897945456575809789456, 
9.22337203685477897945456575809789456),
-  map(2.0D, 1.0D),
-  map(float(2.0), float(1.0)),
-  map(date '2016-03-14', date '2016-03-13'),
-  map(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 
20:54:00.000'),
-  map('true', 'false', '2', '1'),
-  map('2016-03-14', '2016-03-13'),
-  map('2016-11-15 20:54:00.000', '2016-11-12 20:54:00.000'),
-  map('922337203685477897945456575809789456', 'text'),
-  map(array(1L, 2L), array(1L, 2L)), map(array(1, 2), array(1, 2)),
-  map(struct(1S, 2L), struct(1S, 2L)), map(struct(1, 2), struct(1, 2))
-) AS various_maps(
-  boolean_map,
-  tinyint_map,
-  smallint_map,
-  int_map,
-  bigint_map,
-  decimal_map1, decimal_map2,
-  double_map,
-  float_map,
-  date_map,
-  timestamp_map,
-  string_map1, string_map2, string_map3, string_map4,
-  array_map1, array_map2,
-  struct_map1, struct_map2
-)
--- !query 0 schema
-struct<>
--- !query 0 output
-
-
-
--- !query 1
-SELECT map_zip_with(tinyint_map, smallint_map, (k, v1, v2) -> struct(k, v1, 
v2)) m
-FROM various_maps
--- !query 1 schema
-struct<m:map<smallint,struct<k:smallint,v1:tinyint,v2:smallint>>>
--- !query 1 output
-{2:{"k":2,"v1":1,"v2":1}}
-
-
--- !query 2
-SELECT map_zip_with(smallint_map, int_map, (k, v1, v2) -> struct(k, v1, v2)) m
-FROM various_maps
--- !query 2 schema
-struct<m:map<int,struct<k:int,v1:smallint,v2:int>>>
--- !query 2 output
-{2:{"k":2,"v1":1,"v2":1}}
-
-
--- !query 3
-SELECT map_zip_with(int_map, bigint_map, (k, v1, v2) -> struct(k, v1, v2)) m
-FROM various_maps
--- !query 3 schema
-struct<m:map<bigint,struct<k:bigint,v1:int,v2:bigint>>>
--- !query 3 output
-{2:{"k":2,"v1":1,"v2":1}}
-
-
--- !query 4
-SELECT map_zip_with(double_map, float_map, (k, v1, v2) -> struct(k, v1, v2)) m
-FROM various_maps
--- !query 4 schema
-struct<m:map<double,struct<k:double,v1:double,v2:float>>>
--- !query 4 output
-{2.0:{"k":2.0,"v1":1.0,"v2":1.0}}
-
-
--- !query 5
-SELECT map_zip_with(decimal_map1, decimal_map2, (k, v1, v2) -> struct(k, v1, 
v2)) m
-FROM various_maps
--- !query 5 schema
-struct<>
--- !query 5 output
-org.apache.spark.sql.AnalysisException
-cannot resolve 'map_zip_with(various_maps.`decimal_map1`, 
various_maps.`decimal_map2`, lambdafunction(named_struct(NamePlaceholder(), 
`k`, NamePlaceholder(), `v1`, NamePlaceholder(), `v2`), `k`, `v1`, `v2`))' due 
to argument data type mismatch: The input to function map_zip_with should have 
been two maps with compatible key types, but the key types are [decimal(36,0), 
decimal(36,35)].; line 1 pos 7
-
-
--- !query 6
-SELECT map_zip_with(decimal_map1, int_map, (k, v1, v2) -> struct(k, v1, v2)) m
-FROM various_maps
--- !query 6 schema
-struct<m:map<decimal(36,0),struct<k:decimal(36,0),v1:decimal(36,0),v2:int>>>
--- !query 6 output
-{2:{"k":2,"v1":null,"v2":1},922337203685477897945456575809789456:{"k":922337203685477897945456575809789456,"v1":922337203685477897945456575809789456,"v2":null}}
-
-
--- !query 7
-SELECT map_zip_with(decimal_map1, double_map, (k, v1, v2) -> struct(k, v1, 
v2)) m
-FROM various_maps
--- !query 7 schema
-struct<m:map<double,struct<k:double,v1:decimal(36,0),v2:double>>>
--- !query 7 output
-{2.0:{"k":2.0,"v1":null,"v2":1.0},9.223372036854779E35:{"k":9.223372036854779E35,"v1":922337203685477897945456575809789456,"v2":null}}
-
-
--- !query 8
-SELECT map_zip_with(decimal_map2, int_map, (k, v1, v2) -> struct(k, v1, v2)) m
-FROM various_maps
--- !query 8 schema
-struct<>
--- !query 8 output
-org.apache.spark.sql.AnalysisException
-cannot resolve 'map_zip_with(various_maps.`decimal_map2`, 
various_maps.`int_map`, lambdafunction(named_struct(NamePlaceholder(), `k`, 
NamePlaceholder(), `v1`, NamePlaceholder(), `v2`), `k`, `v1`, `v2`))' due to 
argument data type mismatch: The input to function map_zip_with should have 
been two maps with compatible key types, but the key types are [decimal(36,35), 
int].; line 1 pos 7
-
-
--- !query 9
-SELECT map_zip_with(decimal_map2, double_map, (k, v1, v2) -> struct(k, v1, 
v2)) m
-FROM various_maps
--- !query 9 schema
-struct<m:map<double,struct<k:double,v1:decimal(36,35),v2:double>>>
--- !query 9 output
-{2.0:{"k":2.0,"v1":null,"v2":1.0},9.223372036854778:{"k":9.223372036854778,"v1":9.22337203685477897945456575809789456,"v2":null}}
-
-
--- !query 10
-SELECT map_zip_with(string_map1, int_map, (k, v1, v2) -> struct(k, v1, v2)) m
-FROM various_maps
--- !query 10 schema
-struct<m:map<string,struct<k:string,v1:string,v2:int>>>
--- !query 10 output
-{"2":{"k":"2","v1":"1","v2":1},"true":{"k":"true","v1":"false","v2":null}}
-
-
--- !query 11
-SELECT map_zip_with(string_map2, date_map, (k, v1, v2) -> struct(k, v1, v2)) m
-FROM various_maps
--- !query 11 schema
-struct<m:map<string,struct<k:string,v1:string,v2:date>>>
--- !query 11 output
-{"2016-03-14":{"k":"2016-03-14","v1":"2016-03-13","v2":2016-03-13}}
-
-
--- !query 12
-SELECT map_zip_with(timestamp_map, string_map3, (k, v1, v2) -> struct(k, v1, 
v2)) m
-FROM various_maps
--- !query 12 schema
-struct<m:map<string,struct<k:string,v1:timestamp,v2:string>>>
--- !query 12 output
-{"2016-11-15 20:54:00":{"k":"2016-11-15 20:54:00","v1":2016-11-12 
20:54:00.0,"v2":null},"2016-11-15 20:54:00.000":{"k":"2016-11-15 
20:54:00.000","v1":null,"v2":"2016-11-12 20:54:00.000"}}
-
-
--- !query 13
-SELECT map_zip_with(decimal_map1, string_map4, (k, v1, v2) -> struct(k, v1, 
v2)) m
-FROM various_maps
--- !query 13 schema
-struct<m:map<string,struct<k:string,v1:decimal(36,0),v2:string>>>
--- !query 13 output
-{"922337203685477897945456575809789456":{"k":"922337203685477897945456575809789456","v1":922337203685477897945456575809789456,"v2":"text"}}
-
-
--- !query 14
-SELECT map_zip_with(array_map1, array_map2, (k, v1, v2) -> struct(k, v1, v2)) m
-FROM various_maps
--- !query 14 schema
-struct<m:map<array<bigint>,struct<k:array<bigint>,v1:array<bigint>,v2:array<int>>>>
--- !query 14 output
-{[1,2]:{"k":[1,2],"v1":[1,2],"v2":[1,2]}}
-
-
--- !query 15
-SELECT map_zip_with(struct_map1, struct_map2, (k, v1, v2) -> struct(k, v1, 
v2)) m
-FROM various_maps
--- !query 15 schema
-struct<m:map<struct<col1:int,col2:bigint>,struct<k:struct<col1:int,col2:bigint>,v1:struct<col1:smallint,col2:bigint>,v2:struct<col1:int,col2:int>>>>
--- !query 15 output
-{{"col1":1,"col2":2}:{"k":{"col1":1,"col2":2},"v1":{"col1":1,"col2":2},"v2":{"col1":1,"col2":2}}}

http://git-wip-us.apache.org/repos/asf/spark/blob/a9f200e1/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
index 60ebc5e..d4f9b90 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
@@ -509,58 +509,6 @@ class DataFrameFunctionsSuite extends QueryTest with 
SharedSQLContext {
     )
   }
 
-  test("map_entries") {
-    // Primitive-type elements
-    val idf = Seq(
-      Map[Int, Int](1 -> 100, 2 -> 200, 3 -> 300),
-      Map[Int, Int](),
-      null
-    ).toDF("m")
-    val iExpected = Seq(
-      Row(Seq(Row(1, 100), Row(2, 200), Row(3, 300))),
-      Row(Seq.empty),
-      Row(null)
-    )
-
-    def testPrimitiveType(): Unit = {
-      checkAnswer(idf.select(map_entries('m)), iExpected)
-      checkAnswer(idf.selectExpr("map_entries(m)"), iExpected)
-      checkAnswer(idf.selectExpr("map_entries(map(1, null, 2, null))"),
-        Seq.fill(iExpected.length)(Row(Seq(Row(1, null), Row(2, null)))))
-    }
-
-    // Test with local relation, the Project will be evaluated without codegen
-    testPrimitiveType()
-    // Test with cached relation, the Project will be evaluated with codegen
-    idf.cache()
-    testPrimitiveType()
-
-    // Non-primitive-type elements
-    val sdf = Seq(
-      Map[String, String]("a" -> "f", "b" -> "o", "c" -> "o"),
-      Map[String, String]("a" -> null, "b" -> null),
-      Map[String, String](),
-      null
-    ).toDF("m")
-    val sExpected = Seq(
-      Row(Seq(Row("a", "f"), Row("b", "o"), Row("c", "o"))),
-      Row(Seq(Row("a", null), Row("b", null))),
-      Row(Seq.empty),
-      Row(null)
-    )
-
-    def testNonPrimitiveType(): Unit = {
-      checkAnswer(sdf.select(map_entries('m)), sExpected)
-      checkAnswer(sdf.selectExpr("map_entries(m)"), sExpected)
-    }
-
-    // Test with local relation, the Project will be evaluated without codegen
-    testNonPrimitiveType()
-    // Test with cached relation, the Project will be evaluated with codegen
-    sdf.cache()
-    testNonPrimitiveType()
-  }
-
   test("map_concat function") {
     val df1 = Seq(
       (Map[Int, Int](1 -> 100, 2 -> 200), Map[Int, Int](3 -> 300, 4 -> 400)),
@@ -2070,57 +2018,6 @@ class DataFrameFunctionsSuite extends QueryTest with 
SharedSQLContext {
     assert(ex3.getMessage.contains("cannot resolve '`a`'"))
   }
 
-  test("map_filter") {
-    val dfInts = Seq(
-      Map(1 -> 10, 2 -> 20, 3 -> 30),
-      Map(1 -> -1, 2 -> -2, 3 -> -3),
-      Map(1 -> 10, 2 -> 5, 3 -> -3)).toDF("m")
-
-    checkAnswer(dfInts.selectExpr(
-      "map_filter(m, (k, v) -> k * 10 = v)", "map_filter(m, (k, v) -> k = 
-v)"),
-      Seq(
-        Row(Map(1 -> 10, 2 -> 20, 3 -> 30), Map()),
-        Row(Map(), Map(1 -> -1, 2 -> -2, 3 -> -3)),
-        Row(Map(1 -> 10), Map(3 -> -3))))
-
-    val dfComplex = Seq(
-      Map(1 -> Seq(Some(1)), 2 -> Seq(Some(1), Some(2)), 3 -> Seq(Some(1), 
Some(2), Some(3))),
-      Map(1 -> null, 2 -> Seq(Some(-2), Some(-2)), 3 -> 
Seq[Option[Int]](None))).toDF("m")
-
-    checkAnswer(dfComplex.selectExpr(
-      "map_filter(m, (k, v) -> k = v[0])", "map_filter(m, (k, v) -> k = 
size(v))"),
-      Seq(
-        Row(Map(1 -> Seq(1)), Map(1 -> Seq(1), 2 -> Seq(1, 2), 3 -> Seq(1, 2, 
3))),
-        Row(Map(), Map(2 -> Seq(-2, -2)))))
-
-    // Invalid use cases
-    val df = Seq(
-      (Map(1 -> "a"), 1),
-      (Map.empty[Int, String], 2),
-      (null, 3)
-    ).toDF("s", "i")
-
-    val ex1 = intercept[AnalysisException] {
-      df.selectExpr("map_filter(s, (x, y, z) -> x + y + z)")
-    }
-    assert(ex1.getMessage.contains("The number of lambda function arguments 
'3' does not match"))
-
-    val ex2 = intercept[AnalysisException] {
-      df.selectExpr("map_filter(s, x -> x)")
-    }
-    assert(ex2.getMessage.contains("The number of lambda function arguments 
'1' does not match"))
-
-    val ex3 = intercept[AnalysisException] {
-      df.selectExpr("map_filter(i, (k, v) -> k > v)")
-    }
-    assert(ex3.getMessage.contains("data type mismatch: argument 1 requires 
map type"))
-
-    val ex4 = intercept[AnalysisException] {
-      df.selectExpr("map_filter(a, (k, v) -> k > v)")
-    }
-    assert(ex4.getMessage.contains("cannot resolve '`a`'"))
-  }
-
   test("filter function - array for primitive type not containing null") {
     val df = Seq(
       Seq(1, 9, 8, 7),
@@ -2449,328 +2346,6 @@ class DataFrameFunctionsSuite extends QueryTest with 
SharedSQLContext {
     assert(ex5.getMessage.contains("cannot resolve '`a`'"))
   }
 
-  test("map_zip_with function - map of primitive types") {
-    val df = Seq(
-      (Map(8 -> 6L, 3 -> 5L, 6 -> 2L), Map[Integer, Integer]((6, 4), (8, 2), 
(3, 2))),
-      (Map(10 -> 6L, 8 -> 3L), Map[Integer, Integer]((8, 4), (4, null))),
-      (Map.empty[Int, Long], Map[Integer, Integer]((5, 1))),
-      (Map(5 -> 1L), null)
-    ).toDF("m1", "m2")
-
-    checkAnswer(df.selectExpr("map_zip_with(m1, m2, (k, v1, v2) -> k == v1 + 
v2)"),
-      Seq(
-        Row(Map(8 -> true, 3 -> false, 6 -> true)),
-        Row(Map(10 -> null, 8 -> false, 4 -> null)),
-        Row(Map(5 -> null)),
-        Row(null)))
-  }
-
-  test("map_zip_with function - map of non-primitive types") {
-    val df = Seq(
-      (Map("z" -> "a", "y" -> "b", "x" -> "c"), Map("x" -> "a", "z" -> "c")),
-      (Map("b" -> "a", "c" -> "d"), Map("c" -> "a", "b" -> null, "d" -> "k")),
-      (Map("a" -> "d"), Map.empty[String, String]),
-      (Map("a" -> "d"), null)
-    ).toDF("m1", "m2")
-
-    checkAnswer(df.selectExpr("map_zip_with(m1, m2, (k, v1, v2) -> (v1, v2))"),
-      Seq(
-        Row(Map("z" -> Row("a", "c"), "y" -> Row("b", null), "x" -> Row("c", 
"a"))),
-        Row(Map("b" -> Row("a", null), "c" -> Row("d", "a"), "d" -> Row(null, 
"k"))),
-        Row(Map("a" -> Row("d", null))),
-        Row(null)))
-  }
-
-  test("map_zip_with function - invalid") {
-    val df = Seq(
-      (Map(1 -> 2), Map(1 -> "a"), Map("a" -> "b"), Map(Map(1 -> 2) -> 2), 1)
-    ).toDF("mii", "mis", "mss", "mmi", "i")
-
-    val ex1 = intercept[AnalysisException] {
-      df.selectExpr("map_zip_with(mii, mis, (x, y) -> x + y)")
-    }
-    assert(ex1.getMessage.contains("The number of lambda function arguments 
'2' does not match"))
-
-    val ex2 = intercept[AnalysisException] {
-      df.selectExpr("map_zip_with(mis, mmi, (x, y, z) -> concat(x, y, z))")
-    }
-    assert(ex2.getMessage.contains("The input to function map_zip_with should 
have " +
-      "been two maps with compatible key types"))
-
-    val ex3 = intercept[AnalysisException] {
-      df.selectExpr("map_zip_with(i, mis, (x, y, z) -> concat(x, y, z))")
-    }
-    assert(ex3.getMessage.contains("type mismatch: argument 1 requires map 
type"))
-
-    val ex4 = intercept[AnalysisException] {
-      df.selectExpr("map_zip_with(mis, i, (x, y, z) -> concat(x, y, z))")
-    }
-    assert(ex4.getMessage.contains("type mismatch: argument 2 requires map 
type"))
-
-    val ex5 = intercept[AnalysisException] {
-      df.selectExpr("map_zip_with(mmi, mmi, (x, y, z) -> x)")
-    }
-    assert(ex5.getMessage.contains("function map_zip_with does not support 
ordering on type map"))
-  }
-
-  test("transform keys function - primitive data types") {
-    val dfExample1 = Seq(
-      Map[Int, Int](1 -> 1, 9 -> 9, 8 -> 8, 7 -> 7)
-    ).toDF("i")
-
-    val dfExample2 = Seq(
-      Map[Int, Double](1 -> 1.0, 2 -> 1.40, 3 -> 1.70)
-    ).toDF("j")
-
-    val dfExample3 = Seq(
-      Map[Int, Boolean](25 -> true, 26 -> false)
-    ).toDF("x")
-
-    val dfExample4 = Seq(
-      Map[Array[Int], Boolean](Array(1, 2) -> false)
-    ).toDF("y")
-
-
-    def testMapOfPrimitiveTypesCombination(): Unit = {
-      checkAnswer(dfExample1.selectExpr("transform_keys(i, (k, v) -> k + v)"),
-        Seq(Row(Map(2 -> 1, 18 -> 9, 16 -> 8, 14 -> 7))))
-
-      checkAnswer(dfExample2.selectExpr("transform_keys(j, " +
-        "(k, v) -> map_from_arrays(ARRAY(1, 2, 3), ARRAY('one', 'two', 
'three'))[k])"),
-        Seq(Row(Map("one" -> 1.0, "two" -> 1.4, "three" -> 1.7))))
-
-      checkAnswer(dfExample2.selectExpr("transform_keys(j, (k, v) -> CAST(v * 
2 AS BIGINT) + k)"),
-        Seq(Row(Map(3 -> 1.0, 4 -> 1.4, 6 -> 1.7))))
-
-      checkAnswer(dfExample2.selectExpr("transform_keys(j, (k, v) -> k + v)"),
-        Seq(Row(Map(2.0 -> 1.0, 3.4 -> 1.4, 4.7 -> 1.7))))
-
-      checkAnswer(dfExample3.selectExpr("transform_keys(x, (k, v) ->  k % 2 = 
0 OR v)"),
-        Seq(Row(Map(true -> true, true -> false))))
-
-      checkAnswer(dfExample3.selectExpr("transform_keys(x, (k, v) -> if(v, 2 * 
k, 3 * k))"),
-        Seq(Row(Map(50 -> true, 78 -> false))))
-
-      checkAnswer(dfExample3.selectExpr("transform_keys(x, (k, v) -> if(v, 2 * 
k, 3 * k))"),
-        Seq(Row(Map(50 -> true, 78 -> false))))
-
-      checkAnswer(dfExample4.selectExpr("transform_keys(y, (k, v) -> 
array_contains(k, 3) AND v)"),
-        Seq(Row(Map(false -> false))))
-    }
-
-    // Test with local relation, the Project will be evaluated without codegen
-    testMapOfPrimitiveTypesCombination()
-    dfExample1.cache()
-    dfExample2.cache()
-    dfExample3.cache()
-    dfExample4.cache()
-    // Test with cached relation, the Project will be evaluated with codegen
-    testMapOfPrimitiveTypesCombination()
-  }
-
-  test("transform keys function - Invalid lambda functions and exceptions") {
-    val dfExample1 = Seq(
-      Map[String, String]("a" -> null)
-    ).toDF("i")
-
-    val dfExample2 = Seq(
-      Seq(1, 2, 3, 4)
-    ).toDF("j")
-
-    val ex1 = intercept[AnalysisException] {
-      dfExample1.selectExpr("transform_keys(i, k -> k)")
-    }
-    assert(ex1.getMessage.contains("The number of lambda function arguments 
'1' does not match"))
-
-    val ex2 = intercept[AnalysisException] {
-      dfExample1.selectExpr("transform_keys(i, (k, v, x) -> k + 1)")
-    }
-    assert(ex2.getMessage.contains(
-      "The number of lambda function arguments '3' does not match"))
-
-    val ex3 = intercept[Exception] {
-      dfExample1.selectExpr("transform_keys(i, (k, v) -> v)").show()
-    }
-    assert(ex3.getMessage.contains("Cannot use null as map key!"))
-
-    val ex4 = intercept[AnalysisException] {
-      dfExample2.selectExpr("transform_keys(j, (k, v) -> k + 1)")
-    }
-    assert(ex4.getMessage.contains(
-      "data type mismatch: argument 1 requires map type"))
-  }
-
-  test("transform values function - test primitive data types") {
-    val dfExample1 = Seq(
-      Map[Int, Int](1 -> 1, 9 -> 9, 8 -> 8, 7 -> 7)
-    ).toDF("i")
-
-    val dfExample2 = Seq(
-      Map[Boolean, String](false -> "abc", true -> "def")
-    ).toDF("x")
-
-    val dfExample3 = Seq(
-      Map[String, Int]("a" -> 1, "b" -> 2, "c" -> 3)
-    ).toDF("y")
-
-    val dfExample4 = Seq(
-      Map[Int, Double](1 -> 1.0, 2 -> 1.40, 3 -> 1.70)
-    ).toDF("z")
-
-    val dfExample5 = Seq(
-      Map[Int, Array[Int]](1 -> Array(1, 2))
-    ).toDF("c")
-
-    def testMapOfPrimitiveTypesCombination(): Unit = {
-      checkAnswer(dfExample1.selectExpr("transform_values(i, (k, v) -> k + 
v)"),
-        Seq(Row(Map(1 -> 2, 9 -> 18, 8 -> 16, 7 -> 14))))
-
-      checkAnswer(dfExample2.selectExpr(
-        "transform_values(x, (k, v) -> if(k, v, CAST(k AS String)))"),
-        Seq(Row(Map(false -> "false", true -> "def"))))
-
-      checkAnswer(dfExample2.selectExpr("transform_values(x, (k, v) -> NOT k 
AND v = 'abc')"),
-        Seq(Row(Map(false -> true, true -> false))))
-
-      checkAnswer(dfExample3.selectExpr("transform_values(y, (k, v) -> v * 
v)"),
-        Seq(Row(Map("a" -> 1, "b" -> 4, "c" -> 9))))
-
-      checkAnswer(dfExample3.selectExpr(
-        "transform_values(y, (k, v) -> k || ':' || CAST(v as String))"),
-        Seq(Row(Map("a" -> "a:1", "b" -> "b:2", "c" -> "c:3"))))
-
-      checkAnswer(
-        dfExample3.selectExpr("transform_values(y, (k, v) -> concat(k, cast(v 
as String)))"),
-        Seq(Row(Map("a" -> "a1", "b" -> "b2", "c" -> "c3"))))
-
-      checkAnswer(
-        dfExample4.selectExpr(
-          "transform_values(" +
-            "z,(k, v) -> map_from_arrays(ARRAY(1, 2, 3), " +
-            "ARRAY('one', 'two', 'three'))[k] || '_' || CAST(v AS String))"),
-        Seq(Row(Map(1 -> "one_1.0", 2 -> "two_1.4", 3 ->"three_1.7"))))
-
-      checkAnswer(
-        dfExample4.selectExpr("transform_values(z, (k, v) -> k-v)"),
-        Seq(Row(Map(1 -> 0.0, 2 -> 0.6000000000000001, 3 -> 1.3))))
-
-      checkAnswer(
-        dfExample5.selectExpr("transform_values(c, (k, v) -> k + 
cardinality(v))"),
-        Seq(Row(Map(1 -> 3))))
-    }
-
-    // Test with local relation, the Project will be evaluated without codegen
-    testMapOfPrimitiveTypesCombination()
-    dfExample1.cache()
-    dfExample2.cache()
-    dfExample3.cache()
-    dfExample4.cache()
-    dfExample5.cache()
-    // Test with cached relation, the Project will be evaluated with codegen
-    testMapOfPrimitiveTypesCombination()
-  }
-
-  test("transform values function - test empty") {
-    val dfExample1 = Seq(
-      Map.empty[Integer, Integer]
-    ).toDF("i")
-
-    val dfExample2 = Seq(
-      Map.empty[BigInt, String]
-    ).toDF("j")
-
-    def testEmpty(): Unit = {
-      checkAnswer(dfExample1.selectExpr("transform_values(i, (k, v) -> NULL)"),
-        Seq(Row(Map.empty[Integer, Integer])))
-
-      checkAnswer(dfExample1.selectExpr("transform_values(i, (k, v) -> k)"),
-        Seq(Row(Map.empty[Integer, Integer])))
-
-      checkAnswer(dfExample1.selectExpr("transform_values(i, (k, v) -> v)"),
-        Seq(Row(Map.empty[Integer, Integer])))
-
-      checkAnswer(dfExample1.selectExpr("transform_values(i, (k, v) -> 0)"),
-        Seq(Row(Map.empty[Integer, Integer])))
-
-      checkAnswer(dfExample1.selectExpr("transform_values(i, (k, v) -> 
'value')"),
-        Seq(Row(Map.empty[Integer, String])))
-
-      checkAnswer(dfExample1.selectExpr("transform_values(i, (k, v) -> true)"),
-        Seq(Row(Map.empty[Integer, Boolean])))
-
-      checkAnswer(dfExample2.selectExpr("transform_values(j, (k, v) -> k + 
cast(v as BIGINT))"),
-        Seq(Row(Map.empty[BigInt, BigInt])))
-    }
-
-    testEmpty()
-    dfExample1.cache()
-    dfExample2.cache()
-    testEmpty()
-  }
-
-  test("transform values function - test null values") {
-    val dfExample1 = Seq(
-      Map[Int, Integer](1 -> 1, 2 -> 2, 3 -> 3, 4 -> 4)
-    ).toDF("a")
-
-    val dfExample2 = Seq(
-      Map[Int, String](1 -> "a", 2 -> "b", 3 -> null)
-    ).toDF("b")
-
-    def testNullValue(): Unit = {
-      checkAnswer(dfExample1.selectExpr("transform_values(a, (k, v) -> null)"),
-        Seq(Row(Map[Int, Integer](1 -> null, 2 -> null, 3 -> null, 4 -> 
null))))
-
-      checkAnswer(dfExample2.selectExpr(
-        "transform_values(b, (k, v) -> IF(v IS NULL, k + 1, k + 2))"),
-        Seq(Row(Map(1 -> 3, 2 -> 4, 3 -> 4))))
-    }
-
-    testNullValue()
-    dfExample1.cache()
-    dfExample2.cache()
-    testNullValue()
-  }
-
-  test("transform values function - test invalid functions") {
-    val dfExample1 = Seq(
-      Map[Int, Int](1 -> 1, 9 -> 9, 8 -> 8, 7 -> 7)
-    ).toDF("i")
-
-    val dfExample2 = Seq(
-      Map[String, String]("a" -> "b")
-    ).toDF("j")
-
-    val dfExample3 = Seq(
-      Seq(1, 2, 3, 4)
-    ).toDF("x")
-
-    def testInvalidLambdaFunctions(): Unit = {
-
-      val ex1 = intercept[AnalysisException] {
-        dfExample1.selectExpr("transform_values(i, k -> k)")
-      }
-      assert(ex1.getMessage.contains("The number of lambda function arguments 
'1' does not match"))
-
-      val ex2 = intercept[AnalysisException] {
-        dfExample2.selectExpr("transform_values(j, (k, v, x) -> k + 1)")
-      }
-      assert(ex2.getMessage.contains("The number of lambda function arguments 
'3' does not match"))
-
-      val ex3 = intercept[AnalysisException] {
-        dfExample3.selectExpr("transform_values(x, (k, v) -> k + 1)")
-      }
-      assert(ex3.getMessage.contains(
-        "data type mismatch: argument 1 requires map type"))
-    }
-
-    testInvalidLambdaFunctions()
-    dfExample1.cache()
-    dfExample2.cache()
-    dfExample3.cache()
-    testInvalidLambdaFunctions()
-  }
-
   test("arrays zip_with function - for primitive types") {
     val df1 = Seq[(Seq[Integer], Seq[Integer])](
       (Seq(9001, 9002, 9003), Seq(4, 5, 6)),


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to