You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by ueshin <gi...@git.apache.org> on 2018/08/02 08:53:02 UTC

[GitHub] spark pull request #21965: [WIP][SPARK-23909][SQL] Add filter function.

GitHub user ueshin opened a pull request:

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

    [WIP][SPARK-23909][SQL] Add filter function.

    ## What changes were proposed in this pull request?
    
    This pr adds `filter` function which filters the input array using the given predicate.
    
    ```sql
    > SELECT filter(array(1, 2, 3), x -> x % 2 == 1);
     array(1, 3)
    ```
    
    ## How was this patch tested?
    
    Added tests.


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

    $ git pull https://github.com/ueshin/apache-spark issues/SPARK-23909/filter

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

    https://github.com/apache/spark/pull/21965.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 #21965
    
----
commit c3bf6a0059a151ba23cf32c842e31ced3b28726c
Author: Takuya UESHIN <ue...@...>
Date:   2018-08-01T03:46:00Z

    Add `ArrayTransform`.

commit fe48510dc3c5d67e849d504212cd17657faa3966
Author: Takuya UESHIN <ue...@...>
Date:   2018-08-02T06:11:52Z

    Add `ArrayFilter`.

----


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #21965: [SPARK-23909][SQL] Add filter function.

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---

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


[GitHub] spark pull request #21965: [SPARK-23909][SQL] Add filter function.

Posted by ueshin <gi...@git.apache.org>.
Github user ueshin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21965#discussion_r207485584
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala ---
    @@ -210,3 +219,54 @@ case class ArrayTransform(
     
       override def prettyName: String = "transform"
     }
    +
    +/**
    + * Filters the input array using the given lambda function.
    + */
    +@ExpressionDescription(
    +  usage = "_FUNC_(expr, func) - Filters the input array using the given predicate.",
    +  examples = """
    +    Examples:
    +      > SELECT _FUNC_(array(1, 2, 3), x -> x % 2 == 1);
    +       array(1, 3)
    +  """,
    +  since = "2.4.0")
    +case class ArrayFilter(
    +    input: Expression,
    +    function: Expression)
    +  extends ArrayBasedHigherOrderFunction with CodegenFallback {
    +
    +  override def nullable: Boolean = input.nullable
    +
    +  override def dataType: DataType = input.dataType
    +
    +  override def expectingFunctionType: AbstractDataType = BooleanType
    +
    +  override def bind(f: (Expression, Seq[(DataType, Boolean)]) => LambdaFunction): ArrayFilter = {
    +    val elem = ArrayBasedHigherOrderFunction.elementArgumentType(input.dataType)
    +    copy(function = f(function, elem :: Nil))
    +  }
    +
    +  @transient lazy val LambdaFunction(_, Seq(elementVar: NamedLambdaVariable), _) = function
    +
    +  override def eval(input: InternalRow): Any = {
    +    val arr = this.input.eval(input).asInstanceOf[ArrayData]
    +    if (arr == null) {
    +      null
    +    } else {
    +      val f = functionForEval
    +      val buffer = new mutable.ArrayBuffer[Any]
    +      var i = 0
    +      while (i < arr.numElements) {
    +        elementVar.value.set(arr.get(i, elementVar.dataType))
    +        if (f.eval(input).asInstanceOf[Boolean]) {
    +          buffer += elementVar.value.get
    +        }
    +        i += 1
    +      }
    +      new GenericArrayData(buffer)
    +    }
    +  }
    +
    +  override def prettyName: String = "filter"
    --- End diff --
    
    Yeah, it might be. How about `array_filter`?


---

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


[GitHub] spark issue #21965: [WIP][SPARK-23909][SQL] Add filter function.

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    **[Test build #93983 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93983/testReport)** for PR 21965 at commit [`fe48510`](https://github.com/apache/spark/commit/fe48510dc3c5d67e849d504212cd17657faa3966).


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    **[Test build #94124 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94124/testReport)** for PR 21965 at commit [`f6aaa90`](https://github.com/apache/spark/commit/f6aaa902b687c1bddf233c65c3739ec4ac407774).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21965: [WIP][SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    **[Test build #94113 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94113/testReport)** for PR 21965 at commit [`ace19dd`](https://github.com/apache/spark/commit/ace19dd7230598350838aa60fc93b32a08642acd).


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1745/
    Test PASSed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    **[Test build #94088 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94088/testReport)** for PR 21965 at commit [`ace19dd`](https://github.com/apache/spark/commit/ace19dd7230598350838aa60fc93b32a08642acd).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class ArrayFilter(`


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/94122/
    Test PASSed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    **[Test build #94128 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94128/testReport)** for PR 21965 at commit [`f6aaa90`](https://github.com/apache/spark/commit/f6aaa902b687c1bddf233c65c3739ec4ac407774).


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/94128/
    Test PASSed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #21965: [SPARK-23909][SQL] Add filter function.

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21965#discussion_r207479758
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala ---
    @@ -210,3 +219,54 @@ case class ArrayTransform(
     
       override def prettyName: String = "transform"
     }
    +
    +/**
    + * Filters the input array using the given lambda function.
    + */
    +@ExpressionDescription(
    +  usage = "_FUNC_(expr, func) - Filters the input array using the given predicate.",
    +  examples = """
    +    Examples:
    +      > SELECT _FUNC_(array(1, 2, 3), x -> x % 2 == 1);
    +       array(1, 3)
    +  """,
    +  since = "2.4.0")
    +case class ArrayFilter(
    +    input: Expression,
    +    function: Expression)
    +  extends ArrayBasedHigherOrderFunction with CodegenFallback {
    +
    +  override def nullable: Boolean = input.nullable
    +
    +  override def dataType: DataType = input.dataType
    +
    +  override def expectingFunctionType: AbstractDataType = BooleanType
    +
    +  override def bind(f: (Expression, Seq[(DataType, Boolean)]) => LambdaFunction): ArrayFilter = {
    +    val elem = ArrayBasedHigherOrderFunction.elementArgumentType(input.dataType)
    +    copy(function = f(function, elem :: Nil))
    +  }
    +
    +  @transient lazy val LambdaFunction(_, Seq(elementVar: NamedLambdaVariable), _) = function
    +
    +  override def eval(input: InternalRow): Any = {
    +    val arr = this.input.eval(input).asInstanceOf[ArrayData]
    +    if (arr == null) {
    +      null
    +    } else {
    +      val f = functionForEval
    +      val buffer = new mutable.ArrayBuffer[Any]
    --- End diff --
    
    I am wondering if we should use the buffer builder with a size hint here? Or, alternatively manage the array ourself.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1737/
    Test PASSed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    **[Test build #94107 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94107/testReport)** for PR 21965 at commit [`ace19dd`](https://github.com/apache/spark/commit/ace19dd7230598350838aa60fc93b32a08642acd).


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    **[Test build #94107 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94107/testReport)** for PR 21965 at commit [`ace19dd`](https://github.com/apache/spark/commit/ace19dd7230598350838aa60fc93b32a08642acd).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class ArrayFilter(`


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1723/
    Test PASSed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    cc @hvanhovell 


---

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


[GitHub] spark issue #21965: [WIP][SPARK-23909][SQL] Add filter function.

Posted by ueshin <gi...@git.apache.org>.
Github user ueshin commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    This is based on #21954. I'll update after it is merged.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add array_filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add array_filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1749/
    Test PASSed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #21965: [WIP][SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1640/
    Test PASSed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    **[Test build #94119 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94119/testReport)** for PR 21965 at commit [`e9af094`](https://github.com/apache/spark/commit/e9af0947044312fdd8da74498f9935784cdc44f2).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/94119/
    Test FAILed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    **[Test build #94122 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94122/testReport)** for PR 21965 at commit [`e79ebbd`](https://github.com/apache/spark/commit/e79ebbdb71d851b7f95bbb5eb7ce02b6c6edb3eb).


---

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


[GitHub] spark issue #21965: [WIP][SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    **[Test build #94113 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94113/testReport)** for PR 21965 at commit [`ace19dd`](https://github.com/apache/spark/commit/ace19dd7230598350838aa60fc93b32a08642acd).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class ArrayFilter(`


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1748/
    Test PASSed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/94088/
    Test FAILed.


---

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


[GitHub] spark pull request #21965: [SPARK-23909][SQL] Add filter function.

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21965#discussion_r207480086
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala ---
    @@ -210,3 +219,54 @@ case class ArrayTransform(
     
       override def prettyName: String = "transform"
     }
    +
    +/**
    + * Filters the input array using the given lambda function.
    + */
    +@ExpressionDescription(
    +  usage = "_FUNC_(expr, func) - Filters the input array using the given predicate.",
    +  examples = """
    +    Examples:
    +      > SELECT _FUNC_(array(1, 2, 3), x -> x % 2 == 1);
    +       array(1, 3)
    +  """,
    +  since = "2.4.0")
    +case class ArrayFilter(
    +    input: Expression,
    +    function: Expression)
    +  extends ArrayBasedHigherOrderFunction with CodegenFallback {
    +
    +  override def nullable: Boolean = input.nullable
    +
    +  override def dataType: DataType = input.dataType
    +
    +  override def expectingFunctionType: AbstractDataType = BooleanType
    +
    +  override def bind(f: (Expression, Seq[(DataType, Boolean)]) => LambdaFunction): ArrayFilter = {
    +    val elem = ArrayBasedHigherOrderFunction.elementArgumentType(input.dataType)
    +    copy(function = f(function, elem :: Nil))
    +  }
    +
    +  @transient lazy val LambdaFunction(_, Seq(elementVar: NamedLambdaVariable), _) = function
    +
    +  override def eval(input: InternalRow): Any = {
    +    val arr = this.input.eval(input).asInstanceOf[ArrayData]
    +    if (arr == null) {
    +      null
    +    } else {
    +      val f = functionForEval
    +      val buffer = new mutable.ArrayBuffer[Any]
    +      var i = 0
    +      while (i < arr.numElements) {
    +        elementVar.value.set(arr.get(i, elementVar.dataType))
    +        if (f.eval(input).asInstanceOf[Boolean]) {
    +          buffer += elementVar.value.get
    +        }
    +        i += 1
    +      }
    +      new GenericArrayData(buffer)
    +    }
    +  }
    +
    +  override def prettyName: String = "filter"
    --- End diff --
    
    Is filter too generic? wdyt?


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    **[Test build #94128 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94128/testReport)** for PR 21965 at commit [`f6aaa90`](https://github.com/apache/spark/commit/f6aaa902b687c1bddf233c65c3739ec4ac407774).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/94107/
    Test FAILed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    **[Test build #94119 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94119/testReport)** for PR 21965 at commit [`e9af094`](https://github.com/apache/spark/commit/e9af0947044312fdd8da74498f9935784cdc44f2).


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add array_filter function.

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    **[Test build #94124 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94124/testReport)** for PR 21965 at commit [`f6aaa90`](https://github.com/apache/spark/commit/f6aaa902b687c1bddf233c65c3739ec4ac407774).


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1753/
    Test PASSed.


---

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


[GitHub] spark issue #21965: [WIP][SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/93983/
    Test PASSed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/94124/
    Test FAILed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by ueshin <gi...@git.apache.org>.
Github user ueshin commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Thanks! merging to master.


---

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


[GitHub] spark issue #21965: [WIP][SPARK-23909][SQL] Add filter function.

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    **[Test build #93983 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93983/testReport)** for PR 21965 at commit [`fe48510`](https://github.com/apache/spark/commit/fe48510dc3c5d67e849d504212cd17657faa3966).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class ArrayFilter(`


---

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


[GitHub] spark pull request #21965: [SPARK-23909][SQL] Add filter function.

Posted by ueshin <gi...@git.apache.org>.
Github user ueshin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21965#discussion_r207485560
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala ---
    @@ -210,3 +219,54 @@ case class ArrayTransform(
     
       override def prettyName: String = "transform"
     }
    +
    +/**
    + * Filters the input array using the given lambda function.
    + */
    +@ExpressionDescription(
    +  usage = "_FUNC_(expr, func) - Filters the input array using the given predicate.",
    +  examples = """
    +    Examples:
    +      > SELECT _FUNC_(array(1, 2, 3), x -> x % 2 == 1);
    +       array(1, 3)
    +  """,
    +  since = "2.4.0")
    +case class ArrayFilter(
    +    input: Expression,
    +    function: Expression)
    +  extends ArrayBasedHigherOrderFunction with CodegenFallback {
    +
    +  override def nullable: Boolean = input.nullable
    +
    +  override def dataType: DataType = input.dataType
    +
    +  override def expectingFunctionType: AbstractDataType = BooleanType
    +
    +  override def bind(f: (Expression, Seq[(DataType, Boolean)]) => LambdaFunction): ArrayFilter = {
    +    val elem = ArrayBasedHigherOrderFunction.elementArgumentType(input.dataType)
    +    copy(function = f(function, elem :: Nil))
    +  }
    +
    +  @transient lazy val LambdaFunction(_, Seq(elementVar: NamedLambdaVariable), _) = function
    +
    +  override def eval(input: InternalRow): Any = {
    +    val arr = this.input.eval(input).asInstanceOf[ArrayData]
    +    if (arr == null) {
    +      null
    +    } else {
    +      val f = functionForEval
    +      val buffer = new mutable.ArrayBuffer[Any]
    --- End diff --
    
    Sounds good. I'll use the original length as a size hint.


---

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


[GitHub] spark issue #21965: [WIP][SPARK-23909][SQL] Add filter function.

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    @ueshin Please rebase it. Thanks!


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/94113/
    Test PASSed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    **[Test build #94088 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94088/testReport)** for PR 21965 at commit [`ace19dd`](https://github.com/apache/spark/commit/ace19dd7230598350838aa60fc93b32a08642acd).


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    retest this please


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1741/
    Test PASSed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    retest this please


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by ueshin <gi...@git.apache.org>.
Github user ueshin commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    Jenkins, retest this please.


---

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


[GitHub] spark issue #21965: [SPARK-23909][SQL] Add filter function.

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21965
  
    **[Test build #94122 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94122/testReport)** for PR 21965 at commit [`e79ebbd`](https://github.com/apache/spark/commit/e79ebbdb71d851b7f95bbb5eb7ce02b6c6edb3eb).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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