You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by viirya <gi...@git.apache.org> on 2018/01/24 07:11:38 UTC

[GitHub] spark pull request #20379: [SPARK-23177][SQL][PySpark][Backport-2.3] Extract...

GitHub user viirya opened a pull request:

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

    [SPARK-23177][SQL][PySpark][Backport-2.3] Extract zero-parameter UDFs from aggregate

    ## What changes were proposed in this pull request?
    
    We extract Python UDFs in logical aggregate which depends on aggregate expression or grouping key in ExtractPythonUDFFromAggregate rule. But Python UDFs which don't depend on above expressions should also be extracted to avoid the issue reported in the JIRA.
    
    A small code snippet to reproduce that issue looks like:
    ```python
    import pyspark.sql.functions as f
    
    df = spark.createDataFrame([(1,2), (3,4)])
    f_udf = f.udf(lambda: str("const_str"))
    df2 = df.distinct().withColumn("a", f_udf())
    df2.show()
    ```
    
    Error exception is raised as:
    ```
    : org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: pythonUDF0#50
            at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:56)
            at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:91)
            at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:90)
            at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267)
            at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267)
            at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
            at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:266)
            at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272)
            at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272)
            at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306)
            at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187)
            at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304)
            at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272)
            at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:256)
            at org.apache.spark.sql.catalyst.expressions.BindReferences$.bindReference(BoundAttribute.scala:90)
            at org.apache.spark.sql.execution.aggregate.HashAggregateExec$$anonfun$38.apply(HashAggregateExec.scala:514)
            at org.apache.spark.sql.execution.aggregate.HashAggregateExec$$anonfun$38.apply(HashAggregateExec.scala:513)
    ```
    
    This exception raises because `HashAggregateExec` tries to bind the aliased Python UDF expression (e.g., `pythonUDF0#50 AS a#44`) to grouping key.
    
    ## How was this patch tested?
    
    Added test.

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

    $ git pull https://github.com/viirya/spark-1 SPARK-23177-backport-2.3

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

    https://github.com/apache/spark/pull/20379.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 #20379
    
----
commit a66b5e0c4b81444974f02c7154111b47a1a5137c
Author: Liang-Chi Hsieh <vi...@...>
Date:   2018-01-24T06:50:53Z

    Extract parameter-less UDFs from aggregate.

----


---

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


[GitHub] spark issue #20379: [SPARK-23177][SQL][PySpark][Backport-2.3] Extract zero-p...

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

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


---

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


[GitHub] spark issue #20379: [SPARK-23177][SQL][PySpark][Backport-2.3] Extract zero-p...

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

    https://github.com/apache/spark/pull/20379
  
    Merged to branch-2.3.
    
    Thank you @viirya.



---

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


[GitHub] spark issue #20379: [SPARK-23177][SQL][PySpark][Backport-2.3] Extract zero-p...

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

    https://github.com/apache/spark/pull/20379
  
    **[Test build #86571 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86571/testReport)** for PR 20379 at commit [`a66b5e0`](https://github.com/apache/spark/commit/a66b5e0c4b81444974f02c7154111b47a1a5137c).
     * This patch **fails due to an unknown error code, -9**.
     * 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 #20379: [SPARK-23177][SQL][PySpark][Backport-2.3] Extract zero-p...

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

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


---

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


[GitHub] spark issue #20379: [SPARK-23177][SQL][PySpark][Backport-2.3] Extract zero-p...

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

    https://github.com/apache/spark/pull/20379
  
    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 #20379: [SPARK-23177][SQL][PySpark][Backport-2.3] Extract zero-p...

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

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


---

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


[GitHub] spark pull request #20379: [SPARK-23177][SQL][PySpark][Backport-2.3] Extract...

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

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


---

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


[GitHub] spark issue #20379: [SPARK-23177][SQL][PySpark][Backport-2.3] Extract zero-p...

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

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


---

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


[GitHub] spark issue #20379: [SPARK-23177][SQL][PySpark][Backport-2.3] Extract zero-p...

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

    https://github.com/apache/spark/pull/20379
  
    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 #20379: [SPARK-23177][SQL][PySpark][Backport-2.3] Extract zero-p...

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

    https://github.com/apache/spark/pull/20379
  
    Thank you @HyukjinKwon 


---

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


[GitHub] spark issue #20379: [SPARK-23177][SQL][PySpark][Backport-2.3] Extract zero-p...

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

    https://github.com/apache/spark/pull/20379
  
    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/181/
    Test PASSed.


---

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


[GitHub] spark issue #20379: [SPARK-23177][SQL][PySpark][Backport-2.3] Extract zero-p...

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

    https://github.com/apache/spark/pull/20379
  
    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 #20379: [SPARK-23177][SQL][PySpark][Backport-2.3] Extract zero-p...

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

    https://github.com/apache/spark/pull/20379
  
    **[Test build #86575 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86575/testReport)** for PR 20379 at commit [`a66b5e0`](https://github.com/apache/spark/commit/a66b5e0c4b81444974f02c7154111b47a1a5137c).
     * 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 #20379: [SPARK-23177][SQL][PySpark][Backport-2.3] Extract zero-p...

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

    https://github.com/apache/spark/pull/20379
  
    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 #20379: [SPARK-23177][SQL][PySpark][Backport-2.3] Extract zero-p...

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

    https://github.com/apache/spark/pull/20379
  
    cc @HyukjinKwon 


---

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


[GitHub] spark issue #20379: [SPARK-23177][SQL][PySpark][Backport-2.3] Extract zero-p...

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

    https://github.com/apache/spark/pull/20379
  
    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/177/
    Test PASSed.


---

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


[GitHub] spark issue #20379: [SPARK-23177][SQL][PySpark][Backport-2.3] Extract zero-p...

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

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


---

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