You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by xwu0226 <gi...@git.apache.org> on 2016/05/07 05:26:22 UTC

[GitHub] spark pull request: [SPARK-14495][SQL] fix resolution failure of h...

GitHub user xwu0226 opened a pull request:

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

    [SPARK-14495][SQL] fix resolution failure of having clause with distinct aggregate function

    #### Symptom:
    In the latest **branch 1.6**, when a `DISTINCT` aggregation function is used in the `HAVING` clause, Analyzer throws `AnalysisException` with a message like following:
    ```
    resolved attribute(s) gid#558,id#559 missing from date#554,id#555 in operator !Expand [List(date#554, null, 0, if ((gid#558 = 1)) id#559 else null),List(date#554, id#555, 1, null)], [date#554,id#561,gid#560,if ((gid = 1)) id else null#562];
    ```
    #### Root cause:
    The problem is that the distinct aggregate in having condition are resolved by the rule `DistinctAggregationRewriter` twice, which messes up the resulted `EXPAND` operator. 
    
    In a `ResolveAggregateFunctions` rule, when resolving ```Filter(havingCondition, _: Aggregate)```, the `havingCondition` is resolved as an `Aggregate` in a nested loop of analyzer rule execution (by invoking `RuleExecutor.execute`). At this nested level of analysis, the rule `DistinctAggregationRewriter` rewrites this distinct aggregate clause to an expanded two-layer aggregation, where the `aggregateExpresssions` of the final `Aggregate` contains the resolved `gid` and the aggregate expression attributes (In the above case, they are  `gid#558, id#559`).  
    
    After completion of the nested analyzer rule execution, the resulted `aggregateExpressions` in the `havingCondition` is pushed down into the underlying `Aggregate` operator. The `DistinctAggregationRewriter` rule is executed again. The `projections` field of `EXPAND` operator is populated with the `aggregateExpressions` of the `havingCondition` mentioned above. However, the attributes (In the above case, they are `gid#558, id#559`) in the projection list of `EXPAND` operator can not be found in the underlying relation.
    
    #### Solution:
    This PR retrofits part of [#11579](https://github.com/apache/spark/pull/11579) that moves the `DistinctAggregationRewriter` to the beginning of Optimizer, so that it guarantees that the rewrite only happens after all the aggregate functions are resolved first. Thus, it avoid resolution failure. 
    This PR also removes the unnecessary SQLConf property `spark.sql.specializeSingleDistinctAggPlanning` due to the above change. @cloud-fan @yhuai 
    
    #### How is the PR change tested
    New [test cases ](https://github.com/xwu0226/spark/blob/f73428f94746d6d074baf6702589545bdbd11cad/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala#L927-L988) are added to drive `DistinctAggregationRewriter` rewrites for multi-distinct aggregations , involving having clause. 
    
    A following up PR will be submitted to add these test cases to master(2.0) branch. 
     

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

    $ git pull https://github.com/xwu0226/spark SPARK-14495_review

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

    https://github.com/apache/spark/pull/12974.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 #12974
    
----
commit c51448d1173739dd592895b0902ab61d66da499d
Author: xin Wu <xi...@us.ibm.com>
Date:   2016-05-05T14:50:37Z

    move DistinctAggregateRewrite rule to optimizer

commit f73428f94746d6d074baf6702589545bdbd11cad
Author: xin Wu <xi...@us.ibm.com>
Date:   2016-05-07T02:23:30Z

    modify testcases and remove property spark.sql.specializeSingleDistinctAggPlanning

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#issuecomment-218394164
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#discussion_r62798600
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala ---
    @@ -930,6 +930,69 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te
             Row(11) :: Nil)
         }
       }
    +
    +  test("SPARK-14495: two distinct aggregation with having clause of one distinct aggregation") {
    --- End diff --
    
    Ok. Thanks! I will update and push once the current test build is done. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL] fix resolution failure of h...

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

    https://github.com/apache/spark/pull/12974#issuecomment-217609077
  
    Can one of the admins verify this patch?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#discussion_r62453423
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala ---
    @@ -123,15 +119,7 @@ case class DistinctAggregationRewriter(conf: CatalystConf) extends Rule[LogicalP
           .filter(_.isDistinct)
           .groupBy(_.aggregateFunction.children.toSet)
     
    -    val shouldRewrite = if (conf.specializeSingleDistinctAggPlanning) {
    --- End diff --
    
    This flag is for the purpose of benchmarking the performance of single distinct aggregation by `DistinctAggregationRewriter`. The default value is false, which means `DistinctAggregationRewriter` will not be used for a single distinct case. I see 2.0 has removed this flag, so i guess the decision has been made. If it is still needed for 1.6, I can add it back, which will involves more change in Optimizer to take the CatalystConf.  Please let me know. Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#issuecomment-218376819
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

Posted by xwu0226 <gi...@git.apache.org>.
Github user xwu0226 commented on the pull request:

    https://github.com/apache/spark/pull/12974#issuecomment-218026769
  
    @davies @cloud-fan  I modified the change to keep the SQLConf property. In order to pass `conf: CatalystConf` to Optimizer, I did similar thing as how 2.0 branch handles Optimizer. Please help take a look. Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#issuecomment-218027135
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#discussion_r62796179
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala ---
    @@ -930,6 +930,69 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te
             Row(11) :: Nil)
         }
       }
    +
    +  test("SPARK-14495: two distinct aggregation with having clause of one distinct aggregation") {
    --- End diff --
    
    these 4 tests look a little verbose to me. The key to trigger this bug is to put distinct aggregate function in having, right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the pull request:

    https://github.com/apache/spark/pull/12974#issuecomment-218395888
  
    Can you close it? Looks like PR not merged to master won't be closed automatically.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#issuecomment-218037038
  
    **[Test build #58183 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/58183/consoleFull)** for PR 12974 at commit [`326eb4b`](https://github.com/apache/spark/commit/326eb4bc8586be8ce51c3d7c39c96de684321e94).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#issuecomment-218025507
  
    **[Test build #58183 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/58183/consoleFull)** for PR 12974 at commit [`326eb4b`](https://github.com/apache/spark/commit/326eb4bc8586be8ce51c3d7c39c96de684321e94).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#issuecomment-218377742
  
    **[Test build #58341 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/58341/consoleFull)** for PR 12974 at commit [`3782cda`](https://github.com/apache/spark/commit/3782cda0cf5bde4d9c8fd7dadc1313eed134e99b).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#issuecomment-217613866
  
    **[Test build #58061 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/58061/consoleFull)** for PR 12974 at commit [`fb550a1`](https://github.com/apache/spark/commit/fb550a14706eb188dda7a1561269acb80a375227).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL] fix resolution failure of h...

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

    https://github.com/apache/spark/pull/12974#issuecomment-217611919
  
    @xwu0226 Because this is for Branch 1.6 only, please update the PR title to 
    ```
    [SPARK-14495][SQL] [1.6] Fix Resolution Failure of Having Clause with Distinct Aggregate Functions
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#discussion_r62789705
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala ---
    @@ -44,10 +44,12 @@ class PlannerSuite extends SharedSQLContext {
             fail(s"Could query play aggregation query $query. Is it an aggregation query?"))
         val aggregations = planned.collect { case n if n.nodeName contains "Aggregate" => n }
     
    -    // For the new aggregation code path, there will be four aggregate operator for
    -    // distinct aggregations.
    +    // For the new aggregation code path, there will be three aggregate operator for
    +    // distinct aggregations. There used to be four aggregate operators because single
    +    // distinct aggregate used to trigger DistinctAggregationRewriter rewrite. Now the
    +    // the rewrite only happens when there are multiple distinct aggregations.
         assert(
    -      aggregations.size == 2 || aggregations.size == 4,
    +      aggregations.size == 2 || aggregations.size == 3,
    --- End diff --
    
    Before my change of moving the `DistinctAggregationRewriter` rule to Optimizer, Analyzer resolved the single distinct case into 2-layered non-distinct Aggregates and an EXPAND as following:
    ```
    Aggregate [value#3], [value#3,(count(if ((gid#48 = 1)) key#49 else null),mode=Complete,isDistinct=false) AS count(key)#47L]
    +- Aggregate [value#3,key#49,gid#48], [value#3,key#49,gid#48]
       +- Expand [ArrayBuffer(value#3, key#2, 1)], [value#3,key#49,gid#48]
          +- LogicalRDD [key#2,value#3], MapPartitionsRDD[3] at beforeAll at BeforeAndAfterAll.scala:187
    ```
    Then, the resulted physical Plan in this testcase had 4 `TungstenAggregate` generated by [aggregate/utils.planAggregateWithoutDistinct](https://github.com/apache/spark/blob/branch-1.6/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/utils.scala#L65-L131). So this testcase checked for 4 aggregations. 
    
    After my change, there is no rewrite for the single distinct aggregate anymore, the analyzed plan looks like this:
    ```
    Aggregate [value#3], [value#3,(count(key#2),mode=Complete,isDistinct=true) AS count(key)#47L]
    +- LogicalRDD [key#2,value#3], MapPartitionsRDD[3] at beforeAll at BeforeAndAfterAll.scala:187
    ```
    Then, the physical plan are resulted in 3 `TungstenAggregate` by [aggregate/utils.planAggregateWithOneDistinct](https://github.com/apache/spark/blob/branch-1.6/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/utils.scala#L160-L288)
    
    Therefore, I need to modify this testcase. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#issuecomment-218027132
  
    **[Test build #58184 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/58184/consoleFull)** for PR 12974 at commit [`e3deb13`](https://github.com/apache/spark/commit/e3deb13f347978dcff22afd15b6f398eefc3deef).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#discussion_r62793285
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala ---
    @@ -44,10 +44,12 @@ class PlannerSuite extends SharedSQLContext {
             fail(s"Could query play aggregation query $query. Is it an aggregation query?"))
         val aggregations = planned.collect { case n if n.nodeName contains "Aggregate" => n }
     
    -    // For the new aggregation code path, there will be four aggregate operator for
    -    // distinct aggregations.
    +    // For the new aggregation code path, there will be three aggregate operator for
    +    // distinct aggregations. There used to be four aggregate operators because single
    +    // distinct aggregate used to trigger DistinctAggregationRewriter rewrite. Now the
    +    // the rewrite only happens when there are multiple distinct aggregations.
         assert(
    -      aggregations.size == 2 || aggregations.size == 4,
    +      aggregations.size == 2 || aggregations.size == 3,
    --- End diff --
    
    The rewrite I mentioned was by Analyzer. In this testcase, the analyzed logical plan, instead of the optimized one, is passed to generate physical plan.  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#issuecomment-217621642
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#discussion_r62786239
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala ---
    @@ -44,10 +44,12 @@ class PlannerSuite extends SharedSQLContext {
             fail(s"Could query play aggregation query $query. Is it an aggregation query?"))
         val aggregations = planned.collect { case n if n.nodeName contains "Aggregate" => n }
     
    -    // For the new aggregation code path, there will be four aggregate operator for
    -    // distinct aggregations.
    +    // For the new aggregation code path, there will be three aggregate operator for
    +    // distinct aggregations. There used to be four aggregate operators because single
    +    // distinct aggregate used to trigger DistinctAggregationRewriter rewrite. Now the
    +    // the rewrite only happens when there are multiple distinct aggregations.
         assert(
    -      aggregations.size == 2 || aggregations.size == 4,
    +      aggregations.size == 2 || aggregations.size == 3,
    --- End diff --
    
    why this change? This PR just moves the rewrite rule to the beginning of optimizer rule right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#discussion_r62797100
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala ---
    @@ -930,6 +930,69 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te
             Row(11) :: Nil)
         }
       }
    +
    +  test("SPARK-14495: two distinct aggregation with having clause of one distinct aggregation") {
    --- End diff --
    
    Yes. Thanks for pointing it out! The original thinking was that because I removed the configuration property, single distinct aggregate case will not get into rewrite in Optimizer. Therefore, I wanted to cover cases for both single distinct and multi-distinct in having clause. I think I can just keep the first one. Any suggestions? Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL] fix resolution failure of h...

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

    https://github.com/apache/spark/pull/12974#issuecomment-217611768
  
    @rxin @cloud-fan Thanks! 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#discussion_r62786196
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala ---
    @@ -560,80 +560,73 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te
       }
     
       test("single distinct column set") {
    -    Seq(true, false).foreach { specializeSingleDistinctAgg =>
    --- End diff --
    
    This config is not removed, looks like we don't need to change this test?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#issuecomment-218393999
  
    **[Test build #58341 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/58341/consoleFull)** for PR 12974 at commit [`3782cda`](https://github.com/apache/spark/commit/3782cda0cf5bde4d9c8fd7dadc1313eed134e99b).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the pull request:

    https://github.com/apache/spark/pull/12974#issuecomment-217613759
  
    So #11579 accidentally fixed this bug in 2.0?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the pull request:

    https://github.com/apache/spark/pull/12974#issuecomment-218395681
  
    merging to 1.6, thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#issuecomment-218376708
  
    **[Test build #58327 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/58327/consoleFull)** for PR 12974 at commit [`e0eeb7d`](https://github.com/apache/spark/commit/e0eeb7d92f397f94698208afa4081a083e835074).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#discussion_r62533945
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala ---
    @@ -123,15 +119,7 @@ case class DistinctAggregationRewriter(conf: CatalystConf) extends Rule[LogicalP
           .filter(_.isDistinct)
           .groupBy(_.aggregateFunction.children.toSet)
     
    -    val shouldRewrite = if (conf.specializeSingleDistinctAggPlanning) {
    --- End diff --
    
    Even it's not that useful, we should not remove it in minor release.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL] fix resolution failure of h...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the pull request:

    https://github.com/apache/spark/pull/12974#issuecomment-217612527
  
    OK to test


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#discussion_r62450742
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala ---
    @@ -123,15 +119,7 @@ case class DistinctAggregationRewriter(conf: CatalystConf) extends Rule[LogicalP
           .filter(_.isDistinct)
           .groupBy(_.aggregateFunction.children.toSet)
     
    -    val shouldRewrite = if (conf.specializeSingleDistinctAggPlanning) {
    --- End diff --
    
    is this flag still useful in 1.6? cc @davies @yhuai 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#issuecomment-218364589
  
    **[Test build #58327 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/58327/consoleFull)** for PR 12974 at commit [`e0eeb7d`](https://github.com/apache/spark/commit/e0eeb7d92f397f94698208afa4081a083e835074).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

Posted by Jiangwlee <gi...@git.apache.org>.
Github user Jiangwlee commented on the pull request:

    https://github.com/apache/spark/pull/12974#issuecomment-218701450
  
    Thanks! I just encountered this issue today.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#discussion_r62547572
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala ---
    @@ -123,15 +119,7 @@ case class DistinctAggregationRewriter(conf: CatalystConf) extends Rule[LogicalP
           .filter(_.isDistinct)
           .groupBy(_.aggregateFunction.children.toSet)
     
    -    val shouldRewrite = if (conf.specializeSingleDistinctAggPlanning) {
    --- End diff --
    
    @davies Thanks for your input! Let me modify Optimizer to add the conf parameter. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the pull request:

    https://github.com/apache/spark/pull/12974#issuecomment-217613767
  
    ok to test


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#issuecomment-218042213
  
    retest this please



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#discussion_r62792092
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala ---
    @@ -44,10 +44,12 @@ class PlannerSuite extends SharedSQLContext {
             fail(s"Could query play aggregation query $query. Is it an aggregation query?"))
         val aggregations = planned.collect { case n if n.nodeName contains "Aggregate" => n }
     
    -    // For the new aggregation code path, there will be four aggregate operator for
    -    // distinct aggregations.
    +    // For the new aggregation code path, there will be three aggregate operator for
    +    // distinct aggregations. There used to be four aggregate operators because single
    +    // distinct aggregate used to trigger DistinctAggregationRewriter rewrite. Now the
    +    // the rewrite only happens when there are multiple distinct aggregations.
         assert(
    -      aggregations.size == 2 || aggregations.size == 4,
    +      aggregations.size == 2 || aggregations.size == 3,
    --- End diff --
    
    Why is there no rewrite after your change? You just moved the rule but didn't change it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

Posted by xwu0226 <gi...@git.apache.org>.
Github user xwu0226 commented on the pull request:

    https://github.com/apache/spark/pull/12974#issuecomment-217617622
  
    @cloud-fan Yes. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#discussion_r62798315
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala ---
    @@ -930,6 +930,69 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te
             Row(11) :: Nil)
         }
       }
    +
    +  test("SPARK-14495: two distinct aggregation with having clause of one distinct aggregation") {
    --- End diff --
    
    SGTM, please also update the test case name, thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

Posted by xwu0226 <gi...@git.apache.org>.
Github user xwu0226 commented on the pull request:

    https://github.com/apache/spark/pull/12974#issuecomment-218493095
  
    @cloud-fan Thank! I will close it. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#discussion_r62788662
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala ---
    @@ -560,80 +560,73 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te
       }
     
       test("single distinct column set") {
    -    Seq(true, false).foreach { specializeSingleDistinctAgg =>
    --- End diff --
    
    yes. you are right. I will add it back. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

Posted by xwu0226 <gi...@git.apache.org>.
Github user xwu0226 commented on the pull request:

    https://github.com/apache/spark/pull/12974#issuecomment-218190486
  
    retest this please 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#issuecomment-218037141
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#issuecomment-217621619
  
    **[Test build #58061 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/58061/consoleFull)** for PR 12974 at commit [`fb550a1`](https://github.com/apache/spark/commit/fb550a14706eb188dda7a1561269acb80a375227).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-14495][SQL][1.6] fix resolution failure...

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

    https://github.com/apache/spark/pull/12974#issuecomment-218026273
  
    **[Test build #58184 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/58184/consoleFull)** for PR 12974 at commit [`e3deb13`](https://github.com/apache/spark/commit/e3deb13f347978dcff22afd15b6f398eefc3deef).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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