[GitHub] spark pull request #22031: [TODO][SPARK-23932][SQL] Higher order function zi...

2018-08-14 Thread mn-mikke
Github user mn-mikke commented on a diff in the pull request:

https://github.com/apache/spark/pull/22031#discussion_r209920407
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala
 ---
@@ -442,3 +442,91 @@ case class ArrayAggregate(
 
   override def prettyName: String = "aggregate"
 }
+
+// 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.",
+  examples = """
+Examples:
+  > SELECT _FUNC_(array(1, 2, 3), array('a', 'b', 'c'), (x, y) -> (y, 
x));
+   array(('a', 1), ('b', 3), ('c', 5))
+  > SELECT _FUNC_(array(1, 2), array(3, 4), (x, y) -> x + y));
+   array(4, 6)
+  > SELECT _FUNC_(array('a', 'b', 'c'), array('d', 'e', 'f'), (x, y) 
-> concat(x, y));
+   array('ad', 'be', 'cf')
+  """,
+  since = "2.4.0")
+// scalastyle:on line.size.limit
+case class ArraysZipWith(
+left: Expression,
+right: Expression,
+function: Expression)
+  extends HigherOrderFunction with CodegenFallback with ExpectsInputTypes {
+
+  override def inputs: Seq[Expression] = List(left, right)
+
+  override def functions: Seq[Expression] = List(function)
+
+  def expectingFunctionType: AbstractDataType = AnyDataType
+  @transient lazy val functionForEval: Expression = functionsForEval.head
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(ArrayType, 
ArrayType, expectingFunctionType)
+
+  override def nullable: Boolean = inputs.exists(_.nullable)
+
+  override def dataType: ArrayType = ArrayType(function.dataType, 
function.nullable)
+
+  override def bind(f: (Expression, Seq[(DataType, Boolean)]) => 
LambdaFunction): ArraysZipWith = {
+val (leftElementType, leftContainsNull) = left.dataType match {
--- End diff --

This comment is not valid anymore. The method has been removed by 
[#22075](https://github.com/apache/spark/pull/22075).


---

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



[GitHub] spark pull request #22031: [TODO][SPARK-23932][SQL] Higher order function zi...

2018-08-07 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/22031#discussion_r208444793
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala
 ---
@@ -442,3 +442,91 @@ case class ArrayAggregate(
 
   override def prettyName: String = "aggregate"
 }
+
+// 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.",
+  examples = """
+Examples:
+  > SELECT _FUNC_(array(1, 2, 3), array('a', 'b', 'c'), (x, y) -> (y, 
x));
+   array(('a', 1), ('b', 3), ('c', 5))
+  > SELECT _FUNC_(array(1, 2), array(3, 4), (x, y) -> x + y));
+   array(4, 6)
+  > SELECT _FUNC_(array('a', 'b', 'c'), array('d', 'e', 'f'), (x, y) 
-> concat(x, y));
+   array('ad', 'be', 'cf')
+  """,
+  since = "2.4.0")
+// scalastyle:on line.size.limit
+case class ArraysZipWith(
+left: Expression,
+right: Expression,
+function: Expression)
+  extends HigherOrderFunction with CodegenFallback with ExpectsInputTypes {
+
+  override def inputs: Seq[Expression] = List(left, right)
+
+  override def functions: Seq[Expression] = List(function)
+
+  def expectingFunctionType: AbstractDataType = AnyDataType
--- End diff --

We don't need to define this?


---

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



[GitHub] spark pull request #22031: [TODO][SPARK-23932][SQL] Higher order function zi...

2018-08-07 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/22031#discussion_r208444629
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala ---
@@ -2117,6 +2117,65 @@ class DataFrameFunctionsSuite extends QueryTest with 
SharedSQLContext {
 assert(ex4.getMessage.contains("data type mismatch: argument 3 
requires int type"))
   }
 
+  test("zip_with function - arrays for primitive type not containing 
null") {
+val df1 = Seq(
+  (Seq(9001, 9002, 9003), Seq(4, 5, 6)),
+  (Seq(1, 2), Seq(3, 4)),
+  (Seq.empty[Int], Seq.empty[Int]),
+  (null, null)
+).toDF("val1", "val2")
+val df2 = Seq(
+  (Seq(1, 2, 3), Seq("a", "b", "c")),
+  (Seq(1, 2, 3), Seq("a", "b"))
+).toDF("val1", "val2")
+
+def testArrayOfPrimitiveTypeNotContainsNull(): Unit = {
+  val expectedValue1 = Seq(
+Row(Seq(9005, 9007, 9009)),
+Row(Seq(4, 6)),
+Row(Seq.empty),
+Row(null))
+  checkAnswer(df1.selectExpr("zip_with(val1, val2, (x, y) -> x + y)"), 
expectedValue1)
+
+  val expectedValue2 = Seq(
+Row(Seq(Row("a", 1), Row("b", 2), Row("c", 3))),
+Row(Seq(Row("a", 1), Row("b", 2), Row(null, 3
+  checkAnswer(df2.selectExpr("zip_with(val1, val2, (x, y) -> (y, 
x))"), expectedValue2)
+}
+
+// Test with local relation, the Project will be evaluated without 
codegen
+testArrayOfPrimitiveTypeNotContainsNull()
+// Test with cached relation, the Project will be evaluated with 
codegen
+df1.cache()
+df2.cache()
+testArrayOfPrimitiveTypeNotContainsNull()
+  }
+
+  test("zip_with function - arrays for primitive type containing null") {
+val df1 = Seq[(Seq[Integer], Seq[Integer])](
+  (Seq(9001, null, 9003), Seq(4, 5, 6)),
+  (Seq(1, null, 2, 4), Seq(3, 4)),
+  (Seq.empty, Seq.empty),
+  (null, null)
+).toDF("val1", "val2")
+
+def testArrayOfPrimitiveTypeContainsNull(): Unit = {
+  val expectedValue1 = Seq(
+Row(Seq(9005, null, 9009)),
+Row(Seq(4, null, null, null)),
+Row(Seq.empty),
+Row(null))
+  checkAnswer(df1.selectExpr("zip_with(val1, val2, (x, y) -> x + y)"), 
expectedValue1)
+}
+
+// Test with local relation, the Project will be evaluated without 
codegen
+testArrayOfPrimitiveTypeContainsNull()
+// Test with cached relation, the Project will be evaluated with 
codegen
+df1.cache()
+testArrayOfPrimitiveTypeContainsNull()
+  }
+
+
--- End diff --

Can you add a test for invalid cases?


---

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



[GitHub] spark pull request #22031: [TODO][SPARK-23932][SQL] Higher order function zi...

2018-08-07 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/22031#discussion_r208445048
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala
 ---
@@ -442,3 +442,91 @@ case class ArrayAggregate(
 
   override def prettyName: String = "aggregate"
 }
+
+// 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.",
+  examples = """
+Examples:
+  > SELECT _FUNC_(array(1, 2, 3), array('a', 'b', 'c'), (x, y) -> (y, 
x));
+   array(('a', 1), ('b', 3), ('c', 5))
+  > SELECT _FUNC_(array(1, 2), array(3, 4), (x, y) -> x + y));
+   array(4, 6)
+  > SELECT _FUNC_(array('a', 'b', 'c'), array('d', 'e', 'f'), (x, y) 
-> concat(x, y));
+   array('ad', 'be', 'cf')
+  """,
+  since = "2.4.0")
+// scalastyle:on line.size.limit
+case class ArraysZipWith(
+left: Expression,
+right: Expression,
+function: Expression)
+  extends HigherOrderFunction with CodegenFallback with ExpectsInputTypes {
+
+  override def inputs: Seq[Expression] = List(left, right)
+
+  override def functions: Seq[Expression] = List(function)
+
+  def expectingFunctionType: AbstractDataType = AnyDataType
+  @transient lazy val functionForEval: Expression = functionsForEval.head
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(ArrayType, 
ArrayType, expectingFunctionType)
+
+  override def nullable: Boolean = inputs.exists(_.nullable)
+
+  override def dataType: ArrayType = ArrayType(function.dataType, 
function.nullable)
+
+  override def bind(f: (Expression, Seq[(DataType, Boolean)]) => 
LambdaFunction): ArraysZipWith = {
+val (leftElementType, leftContainsNull) = left.dataType match {
+  case ArrayType(elementType, containsNull) => (elementType, 
containsNull)
+  case _ =>
+val ArrayType(elementType, containsNull) = 
ArrayType.defaultConcreteType
+(elementType, containsNull)
+}
+val (rightElementType, rightContainsNull) = right.dataType match {
+  case ArrayType(elementType, containsNull) => (elementType, 
containsNull)
+  case _ =>
+val ArrayType(elementType, containsNull) = 
ArrayType.defaultConcreteType
+(elementType, containsNull)
+}
+copy(function = f(function,
+  (leftElementType, leftContainsNull) :: (rightElementType, 
rightContainsNull) :: Nil))
+  }
+
+  @transient lazy val (arr1Var, arr2Var) = {
+val LambdaFunction(_,
+  (arr1Var: NamedLambdaVariable):: (arr2Var: NamedLambdaVariable) :: 
Nil, _) = function
+(arr1Var, arr2Var)
+  }
--- End diff --

nit: the following should work:

```scala
@transient lazy val LambdaFunction(_,
  Seq(leftElemVar: NamedLambdaVariable, rightElemVar: NamedLambdaVariable), 
_) = function
```


---

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



[GitHub] spark pull request #22031: [TODO][SPARK-23932][SQL] Higher order function zi...

2018-08-07 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/22031#discussion_r208445494
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala ---
@@ -2117,6 +2117,65 @@ class DataFrameFunctionsSuite extends QueryTest with 
SharedSQLContext {
 assert(ex4.getMessage.contains("data type mismatch: argument 3 
requires int type"))
   }
 
+  test("zip_with function - arrays for primitive type not containing 
null") {
+val df1 = Seq(
+  (Seq(9001, 9002, 9003), Seq(4, 5, 6)),
+  (Seq(1, 2), Seq(3, 4)),
+  (Seq.empty[Int], Seq.empty[Int]),
+  (null, null)
+).toDF("val1", "val2")
+val df2 = Seq(
+  (Seq(1, 2, 3), Seq("a", "b", "c")),
+  (Seq(1, 2, 3), Seq("a", "b"))
+).toDF("val1", "val2")
+
+def testArrayOfPrimitiveTypeNotContainsNull(): Unit = {
+  val expectedValue1 = Seq(
+Row(Seq(9005, 9007, 9009)),
+Row(Seq(4, 6)),
+Row(Seq.empty),
+Row(null))
+  checkAnswer(df1.selectExpr("zip_with(val1, val2, (x, y) -> x + y)"), 
expectedValue1)
+
+  val expectedValue2 = Seq(
+Row(Seq(Row("a", 1), Row("b", 2), Row("c", 3))),
+Row(Seq(Row("a", 1), Row("b", 2), Row(null, 3
+  checkAnswer(df2.selectExpr("zip_with(val1, val2, (x, y) -> (y, 
x))"), expectedValue2)
+}
+
+// Test with local relation, the Project will be evaluated without 
codegen
+testArrayOfPrimitiveTypeNotContainsNull()
+// Test with cached relation, the Project will be evaluated with 
codegen
+df1.cache()
+df2.cache()
+testArrayOfPrimitiveTypeNotContainsNull()
+  }
+
+  test("zip_with function - arrays for primitive type containing null") {
+val df1 = Seq[(Seq[Integer], Seq[Integer])](
+  (Seq(9001, null, 9003), Seq(4, 5, 6)),
+  (Seq(1, null, 2, 4), Seq(3, 4)),
+  (Seq.empty, Seq.empty),
+  (null, null)
+).toDF("val1", "val2")
+
+def testArrayOfPrimitiveTypeContainsNull(): Unit = {
+  val expectedValue1 = Seq(
+Row(Seq(9005, null, 9009)),
+Row(Seq(4, null, null, null)),
+Row(Seq.empty),
+Row(null))
+  checkAnswer(df1.selectExpr("zip_with(val1, val2, (x, y) -> x + y)"), 
expectedValue1)
+}
+
+// Test with local relation, the Project will be evaluated without 
codegen
+testArrayOfPrimitiveTypeContainsNull()
+// Test with cached relation, the Project will be evaluated with 
codegen
+df1.cache()
+testArrayOfPrimitiveTypeContainsNull()
+  }
+
+
--- End diff --

Also can you add tests to `HigherOrderFunctionsSuite` to check more 
explicit patterns?


---

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



[GitHub] spark pull request #22031: [TODO][SPARK-23932][SQL] Higher order function zi...

2018-08-07 Thread mn-mikke
Github user mn-mikke commented on a diff in the pull request:

https://github.com/apache/spark/pull/22031#discussion_r208403145
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala
 ---
@@ -442,3 +442,91 @@ case class ArrayAggregate(
 
   override def prettyName: String = "aggregate"
 }
+
+// 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.",
+  examples = """
+Examples:
+  > SELECT _FUNC_(array(1, 2, 3), array('a', 'b', 'c'), (x, y) -> (y, 
x));
+   array(('a', 1), ('b', 3), ('c', 5))
+  > SELECT _FUNC_(array(1, 2), array(3, 4), (x, y) -> x + y));
+   array(4, 6)
+  > SELECT _FUNC_(array('a', 'b', 'c'), array('d', 'e', 'f'), (x, y) 
-> concat(x, y));
+   array('ad', 'be', 'cf')
+  """,
+  since = "2.4.0")
+// scalastyle:on line.size.limit
+case class ArraysZipWith(
+left: Expression,
+right: Expression,
+function: Expression)
+  extends HigherOrderFunction with CodegenFallback with ExpectsInputTypes {
+
+  override def inputs: Seq[Expression] = List(left, right)
+
+  override def functions: Seq[Expression] = List(function)
+
+  def expectingFunctionType: AbstractDataType = AnyDataType
+  @transient lazy val functionForEval: Expression = functionsForEval.head
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(ArrayType, 
ArrayType, expectingFunctionType)
+
+  override def nullable: Boolean = inputs.exists(_.nullable)
+
+  override def dataType: ArrayType = ArrayType(function.dataType, 
function.nullable)
+
+  override def bind(f: (Expression, Seq[(DataType, Boolean)]) => 
LambdaFunction): ArraysZipWith = {
+val (leftElementType, leftContainsNull) = left.dataType match {
+  case ArrayType(elementType, containsNull) => (elementType, 
containsNull)
+  case _ =>
+val ArrayType(elementType, containsNull) = 
ArrayType.defaultConcreteType
+(elementType, containsNull)
+}
+val (rightElementType, rightContainsNull) = right.dataType match {
+  case ArrayType(elementType, containsNull) => (elementType, 
containsNull)
+  case _ =>
+val ArrayType(elementType, containsNull) = 
ArrayType.defaultConcreteType
+(elementType, containsNull)
+}
+copy(function = f(function,
+  (leftElementType, leftContainsNull) :: (rightElementType, 
rightContainsNull) :: Nil))
--- End diff --

If you want to support different size of input arrays (The jira ticket 
says: _"Both arrays must be the same length."_), what about the scenario when 
one array is empty and the second has elements? Shouldn't we use ```true``` 
instead of  ```leftContainsNull``` and ```rightContainsNull```?


---

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



[GitHub] spark pull request #22031: [TODO][SPARK-23932][SQL] Higher order function zi...

2018-08-07 Thread mn-mikke
Github user mn-mikke commented on a diff in the pull request:

https://github.com/apache/spark/pull/22031#discussion_r208398313
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala
 ---
@@ -442,3 +442,91 @@ case class ArrayAggregate(
 
   override def prettyName: String = "aggregate"
 }
+
+// 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.",
+  examples = """
+Examples:
+  > SELECT _FUNC_(array(1, 2, 3), array('a', 'b', 'c'), (x, y) -> (y, 
x));
+   array(('a', 1), ('b', 3), ('c', 5))
+  > SELECT _FUNC_(array(1, 2), array(3, 4), (x, y) -> x + y));
+   array(4, 6)
+  > SELECT _FUNC_(array('a', 'b', 'c'), array('d', 'e', 'f'), (x, y) 
-> concat(x, y));
+   array('ad', 'be', 'cf')
+  """,
+  since = "2.4.0")
+// scalastyle:on line.size.limit
+case class ArraysZipWith(
+left: Expression,
+right: Expression,
+function: Expression)
+  extends HigherOrderFunction with CodegenFallback with ExpectsInputTypes {
+
+  override def inputs: Seq[Expression] = List(left, right)
+
+  override def functions: Seq[Expression] = List(function)
+
+  def expectingFunctionType: AbstractDataType = AnyDataType
+  @transient lazy val functionForEval: Expression = functionsForEval.head
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(ArrayType, 
ArrayType, expectingFunctionType)
+
+  override def nullable: Boolean = inputs.exists(_.nullable)
+
+  override def dataType: ArrayType = ArrayType(function.dataType, 
function.nullable)
+
+  override def bind(f: (Expression, Seq[(DataType, Boolean)]) => 
LambdaFunction): ArraysZipWith = {
+val (leftElementType, leftContainsNull) = left.dataType match {
--- End diff --

You can utilize ```HigherOrderFunction.arrayArgumentType```.


---

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



[GitHub] spark pull request #22031: [TODO][SPARK-23932][SQL] Higher order function zi...

2018-08-07 Thread mn-mikke
Github user mn-mikke commented on a diff in the pull request:

https://github.com/apache/spark/pull/22031#discussion_r208399620
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala
 ---
@@ -442,3 +442,91 @@ case class ArrayAggregate(
 
   override def prettyName: String = "aggregate"
 }
+
+// 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.",
+  examples = """
+Examples:
+  > SELECT _FUNC_(array(1, 2, 3), array('a', 'b', 'c'), (x, y) -> (y, 
x));
+   array(('a', 1), ('b', 3), ('c', 5))
+  > SELECT _FUNC_(array(1, 2), array(3, 4), (x, y) -> x + y));
+   array(4, 6)
+  > SELECT _FUNC_(array('a', 'b', 'c'), array('d', 'e', 'f'), (x, y) 
-> concat(x, y));
+   array('ad', 'be', 'cf')
+  """,
+  since = "2.4.0")
+// scalastyle:on line.size.limit
+case class ArraysZipWith(
+left: Expression,
+right: Expression,
+function: Expression)
+  extends HigherOrderFunction with CodegenFallback with ExpectsInputTypes {
+
+  override def inputs: Seq[Expression] = List(left, right)
+
+  override def functions: Seq[Expression] = List(function)
+
+  def expectingFunctionType: AbstractDataType = AnyDataType
+  @transient lazy val functionForEval: Expression = functionsForEval.head
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(ArrayType, 
ArrayType, expectingFunctionType)
+
+  override def nullable: Boolean = inputs.exists(_.nullable)
+
+  override def dataType: ArrayType = ArrayType(function.dataType, 
function.nullable)
+
+  override def bind(f: (Expression, Seq[(DataType, Boolean)]) => 
LambdaFunction): ArraysZipWith = {
+val (leftElementType, leftContainsNull) = left.dataType match {
+  case ArrayType(elementType, containsNull) => (elementType, 
containsNull)
+  case _ =>
+val ArrayType(elementType, containsNull) = 
ArrayType.defaultConcreteType
+(elementType, containsNull)
+}
+val (rightElementType, rightContainsNull) = right.dataType match {
+  case ArrayType(elementType, containsNull) => (elementType, 
containsNull)
+  case _ =>
+val ArrayType(elementType, containsNull) = 
ArrayType.defaultConcreteType
+(elementType, containsNull)
+}
+copy(function = f(function,
+  (leftElementType, leftContainsNull) :: (rightElementType, 
rightContainsNull) :: Nil))
+  }
+
+  @transient lazy val (arr1Var, arr2Var) = {
+val LambdaFunction(_,
+  (arr1Var: NamedLambdaVariable):: (arr2Var: NamedLambdaVariable) :: 
Nil, _) = function
+(arr1Var, arr2Var)
+  }
+
+  override def eval(input: InternalRow): Any = {
+val leftArr = left.eval(input).asInstanceOf[ArrayData]
+val rightArr = right.eval(input).asInstanceOf[ArrayData]
+
+if (leftArr == null || rightArr == null) {
--- End diff --

If ```leftArr``` is ```null```, ```right``` doesn't have to be evaluated.


---

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



[GitHub] spark pull request #22031: [TODO][SPARK-23932][SQL] Higher order function zi...

2018-08-07 Thread crafty-coder
Github user crafty-coder commented on a diff in the pull request:

https://github.com/apache/spark/pull/22031#discussion_r208387111
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala
 ---
@@ -442,3 +442,93 @@ case class ArrayAggregate(
 
   override def prettyName: String = "aggregate"
 }
+
+/**
+ * Transform elements in an array using the transform function. This is 
similar to
+ * a `map` in functional programming.
+ */
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+  usage = "_FUNC_(expr, 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.",
+  examples = """
+Examples:
+  > SELECT _FUNC_(array(1, 2, 3), x -> x + 1);
--- End diff --

The examples are not accurate.

You could something like:

```
 > SELECT _FUNC_(array(1, 2, 3), array('a', 'b', 'c'), (x, y) -> (y, x));   

  array(('a', 1), ('b', 3), ('c', 5))   

 > SELECT _FUNC_(array(1, 2), array(3, 4), (x, y) -> x + y));   

  array(4, 6)   

 > SELECT _FUNC_(array('a', 'b', 'c'), array('d', 'e', 'f'), (x, y) -> 
concat(x, y));   
  array('ad', 'be', 'cf')   

```


---

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



[GitHub] spark pull request #22031: [TODO][SPARK-23932][SQL] Higher order function zi...

2018-08-07 Thread techaddict
GitHub user techaddict opened a pull request:

https://github.com/apache/spark/pull/22031

[TODO][SPARK-23932][SQL] Higher order function zip_with

## What changes were proposed in this pull request?
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:
```
SELECT zip_with(ARRAY[1, 3, 5], ARRAY['a', 'b', 'c'], (x, y) -> (y, 
x)); -- [ROW('a', 1), ROW('b', 3), ROW('c', 5)]
SELECT zip_with(ARRAY[1, 2], ARRAY[3, 4], (x, y) -> x + y); -- [4, 6]
SELECT zip_with(ARRAY['a', 'b', 'c'], ARRAY['d', 'e', 'f'], (x, y) -> 
concat(x, y)); -- ['ad', 'be', 'cf']
SELECT zip_with(ARRAY['a'], ARRAY['d', null, 'f'], (x, y) -> 
coalesce(x, y)); -- ['a', null, 'f']
```
## How was this patch tested?
Added tests

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/techaddict/spark SPARK-23932

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/22031.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #22031


commit 03d19cee425be90a61b60163ff9d6740716d45a6
Author: Sandeep Singh 
Date:   2018-08-03T04:15:00Z

.

commit 6f91777de93121d668ff11e7701f449bb4c96337
Author: Sandeep Singh 
Date:   2018-08-04T22:00:38Z

fix description




---

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