[GitHub] spark pull request #15187: [SPARK-17616][SQL] Support a single distinct aggr...

2016-09-22 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15187: [SPARK-17616][SQL] Support a single distinct aggr...

2016-09-22 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/15187#discussion_r80115663
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregatesSuite.scala
 ---
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.SimpleCatalystConf
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, 
EmptyFunctionRegistry}
+import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, 
SessionCatalog}
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.{If, Literal}
+import org.apache.spark.sql.catalyst.expressions.aggregate.{CollectSet, 
Count}
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Expand, 
LocalRelation, LogicalPlan}
+import org.apache.spark.sql.types.{IntegerType, StringType}
+
+class RewriteDistinctAggregatesSuite extends PlanTest {
+  val conf = SimpleCatalystConf(caseSensitiveAnalysis = false, 
groupByOrdinal = false)
+  val catalog = new SessionCatalog(new InMemoryCatalog, 
EmptyFunctionRegistry, conf)
+  val analyzer = new Analyzer(catalog, conf)
+
+  val nullInt = Literal(null, IntegerType)
+  val nullString = Literal(null, StringType)
+  val testRelation = LocalRelation('a.string, 'b.string, 'c.string, 
'd.string, 'e.int)
+
+  private def checkRewrite(rewrite: LogicalPlan): Unit = rewrite match {
+case Aggregate(_, _, Aggregate(_, _, _: Expand)) =>
+case _ => fail(s"Plan is not rewritten:\n$rewrite")
+  }
+
+  test("single distinct group") {
+val input = testRelation
+  .groupBy('a)(countDistinct('e))
+  .analyze
+val rewrite = RewriteDistinctAggregates(input)
+comparePlans(input, rewrite)
+  }
+
+  test("single distinct group with non-partial aggregates") {
+val input = testRelation
+  .groupBy('a, 'd)(
+countDistinct('e, 'c).as('agg1),
+CollectSet('b).toAggregateExpression().as('agg2))
+  .analyze
+checkRewrite(RewriteDistinctAggregates(input))
+  }
+
+  test("multiple distinct groups") {
+val input = testRelation
+  .groupBy('a)(countDistinct('b, 'c), countDistinct('d), sum('e))
+  .analyze
+checkRewrite(RewriteDistinctAggregates(input))
--- End diff --

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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15187: [SPARK-17616][SQL] Support a single distinct aggr...

2016-09-22 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/15187#discussion_r80115646
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregatesSuite.scala
 ---
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.SimpleCatalystConf
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, 
EmptyFunctionRegistry}
+import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, 
SessionCatalog}
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.{If, Literal}
+import org.apache.spark.sql.catalyst.expressions.aggregate.{CollectSet, 
Count}
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Expand, 
LocalRelation, LogicalPlan}
+import org.apache.spark.sql.types.{IntegerType, StringType}
+
+class RewriteDistinctAggregatesSuite extends PlanTest {
+  val conf = SimpleCatalystConf(caseSensitiveAnalysis = false, 
groupByOrdinal = false)
+  val catalog = new SessionCatalog(new InMemoryCatalog, 
EmptyFunctionRegistry, conf)
+  val analyzer = new Analyzer(catalog, conf)
+
+  val nullInt = Literal(null, IntegerType)
+  val nullString = Literal(null, StringType)
+  val testRelation = LocalRelation('a.string, 'b.string, 'c.string, 
'd.string, 'e.int)
+
+  private def checkRewrite(rewrite: LogicalPlan): Unit = rewrite match {
+case Aggregate(_, _, Aggregate(_, _, _: Expand)) =>
+case _ => fail(s"Plan is not rewritten:\n$rewrite")
+  }
+
+  test("single distinct group") {
+val input = testRelation
+  .groupBy('a)(countDistinct('e))
+  .analyze
+val rewrite = RewriteDistinctAggregates(input)
+comparePlans(input, rewrite)
+  }
--- End diff --

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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15187: [SPARK-17616][SQL] Support a single distinct aggr...

2016-09-22 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/15187#discussion_r80115529
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregatesSuite.scala
 ---
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.SimpleCatalystConf
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, 
EmptyFunctionRegistry}
+import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, 
SessionCatalog}
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.{If, Literal}
+import org.apache.spark.sql.catalyst.expressions.aggregate.{CollectSet, 
Count}
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Expand, 
LocalRelation, LogicalPlan}
+import org.apache.spark.sql.types.{IntegerType, StringType}
+
+class RewriteDistinctAggregatesSuite extends PlanTest {
+  val conf = SimpleCatalystConf(caseSensitiveAnalysis = false, 
groupByOrdinal = false)
+  val catalog = new SessionCatalog(new InMemoryCatalog, 
EmptyFunctionRegistry, conf)
+  val analyzer = new Analyzer(catalog, conf)
+
+  val nullInt = Literal(null, IntegerType)
+  val nullString = Literal(null, StringType)
+  val testRelation = LocalRelation('a.string, 'b.string, 'c.string, 
'd.string, 'e.int)
+
+  private def checkRewrite(rewrite: LogicalPlan): Unit = rewrite match {
+case Aggregate(_, _, Aggregate(_, _, _: Expand)) =>
+case _ => fail(s"Plan is not rewritten:\n$rewrite")
+  }
+
+  test("single distinct group") {
+val input = testRelation
+  .groupBy('a)(countDistinct('e))
+  .analyze
+val rewrite = RewriteDistinctAggregates(input)
+comparePlans(input, rewrite)
+  }
+
+  test("single distinct group with non-partial aggregates") {
+val input = testRelation
+  .groupBy('a, 'd)(
+countDistinct('e, 'c).as('agg1),
+CollectSet('b).toAggregateExpression().as('agg2))
+  .analyze
+checkRewrite(RewriteDistinctAggregates(input))
+  }
+
+  test("multiple distinct groups") {
+val input = testRelation
+  .groupBy('a)(countDistinct('b, 'c), countDistinct('d), sum('e))
+  .analyze
+checkRewrite(RewriteDistinctAggregates(input))
+  }
+
+  test("multiple distinct groups without non-distinct aggregates") {
--- End diff --

I actually mean that the test only contains distinct aggregates.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15187: [SPARK-17616][SQL] Support a single distinct aggr...

2016-09-22 Thread srinathshankar
Github user srinathshankar commented on a diff in the pull request:

https://github.com/apache/spark/pull/15187#discussion_r80112793
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregatesSuite.scala
 ---
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.SimpleCatalystConf
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, 
EmptyFunctionRegistry}
+import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, 
SessionCatalog}
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.{If, Literal}
+import org.apache.spark.sql.catalyst.expressions.aggregate.{CollectSet, 
Count}
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Expand, 
LocalRelation, LogicalPlan}
+import org.apache.spark.sql.types.{IntegerType, StringType}
+
+class RewriteDistinctAggregatesSuite extends PlanTest {
+  val conf = SimpleCatalystConf(caseSensitiveAnalysis = false, 
groupByOrdinal = false)
+  val catalog = new SessionCatalog(new InMemoryCatalog, 
EmptyFunctionRegistry, conf)
+  val analyzer = new Analyzer(catalog, conf)
+
+  val nullInt = Literal(null, IntegerType)
+  val nullString = Literal(null, StringType)
+  val testRelation = LocalRelation('a.string, 'b.string, 'c.string, 
'd.string, 'e.int)
+
+  private def checkRewrite(rewrite: LogicalPlan): Unit = rewrite match {
+case Aggregate(_, _, Aggregate(_, _, _: Expand)) =>
+case _ => fail(s"Plan is not rewritten:\n$rewrite")
+  }
+
+  test("single distinct group") {
+val input = testRelation
+  .groupBy('a)(countDistinct('e))
+  .analyze
+val rewrite = RewriteDistinctAggregates(input)
+comparePlans(input, rewrite)
+  }
+
+  test("single distinct group with non-partial aggregates") {
+val input = testRelation
+  .groupBy('a, 'd)(
+countDistinct('e, 'c).as('agg1),
+CollectSet('b).toAggregateExpression().as('agg2))
+  .analyze
+checkRewrite(RewriteDistinctAggregates(input))
+  }
+
+  test("multiple distinct groups") {
+val input = testRelation
+  .groupBy('a)(countDistinct('b, 'c), countDistinct('d), sum('e))
+  .analyze
+checkRewrite(RewriteDistinctAggregates(input))
--- End diff --

Could you also add a test with partials, and one without partials here ? 
(part of the same 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15187: [SPARK-17616][SQL] Support a single distinct aggr...

2016-09-22 Thread srinathshankar
Github user srinathshankar commented on a diff in the pull request:

https://github.com/apache/spark/pull/15187#discussion_r80112666
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregatesSuite.scala
 ---
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.SimpleCatalystConf
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, 
EmptyFunctionRegistry}
+import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, 
SessionCatalog}
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.{If, Literal}
+import org.apache.spark.sql.catalyst.expressions.aggregate.{CollectSet, 
Count}
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Expand, 
LocalRelation, LogicalPlan}
+import org.apache.spark.sql.types.{IntegerType, StringType}
+
+class RewriteDistinctAggregatesSuite extends PlanTest {
+  val conf = SimpleCatalystConf(caseSensitiveAnalysis = false, 
groupByOrdinal = false)
+  val catalog = new SessionCatalog(new InMemoryCatalog, 
EmptyFunctionRegistry, conf)
+  val analyzer = new Analyzer(catalog, conf)
+
+  val nullInt = Literal(null, IntegerType)
+  val nullString = Literal(null, StringType)
+  val testRelation = LocalRelation('a.string, 'b.string, 'c.string, 
'd.string, 'e.int)
+
+  private def checkRewrite(rewrite: LogicalPlan): Unit = rewrite match {
+case Aggregate(_, _, Aggregate(_, _, _: Expand)) =>
+case _ => fail(s"Plan is not rewritten:\n$rewrite")
+  }
+
+  test("single distinct group") {
+val input = testRelation
+  .groupBy('a)(countDistinct('e))
+  .analyze
+val rewrite = RewriteDistinctAggregates(input)
+comparePlans(input, rewrite)
+  }
--- End diff --

Could you also add single distinct group with aggregates that have partial


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15187: [SPARK-17616][SQL] Support a single distinct aggr...

2016-09-22 Thread srinathshankar
Github user srinathshankar commented on a diff in the pull request:

https://github.com/apache/spark/pull/15187#discussion_r80109506
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregatesSuite.scala
 ---
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.SimpleCatalystConf
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, 
EmptyFunctionRegistry}
+import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, 
SessionCatalog}
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.{If, Literal}
+import org.apache.spark.sql.catalyst.expressions.aggregate.{CollectSet, 
Count}
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Expand, 
LocalRelation, LogicalPlan}
+import org.apache.spark.sql.types.{IntegerType, StringType}
+
+class RewriteDistinctAggregatesSuite extends PlanTest {
+  val conf = SimpleCatalystConf(caseSensitiveAnalysis = false, 
groupByOrdinal = false)
+  val catalog = new SessionCatalog(new InMemoryCatalog, 
EmptyFunctionRegistry, conf)
+  val analyzer = new Analyzer(catalog, conf)
+
+  val nullInt = Literal(null, IntegerType)
+  val nullString = Literal(null, StringType)
+  val testRelation = LocalRelation('a.string, 'b.string, 'c.string, 
'd.string, 'e.int)
+
+  private def checkRewrite(rewrite: LogicalPlan): Unit = rewrite match {
+case Aggregate(_, _, Aggregate(_, _, _: Expand)) =>
+case _ => fail(s"Plan is not rewritten:\n$rewrite")
+  }
+
+  test("single distinct group") {
+val input = testRelation
+  .groupBy('a)(countDistinct('e))
+  .analyze
+val rewrite = RewriteDistinctAggregates(input)
+comparePlans(input, rewrite)
+  }
+
+  test("single distinct group with non-partial aggregates") {
+val input = testRelation
+  .groupBy('a, 'd)(
+countDistinct('e, 'c).as('agg1),
+CollectSet('b).toAggregateExpression().as('agg2))
+  .analyze
+checkRewrite(RewriteDistinctAggregates(input))
+  }
+
+  test("multiple distinct groups") {
+val input = testRelation
+  .groupBy('a)(countDistinct('b, 'c), countDistinct('d), sum('e))
+  .analyze
+checkRewrite(RewriteDistinctAggregates(input))
+  }
+
+  test("multiple distinct groups without non-distinct aggregates") {
--- End diff --

Do you mean non-partial aggregates here ?


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15187: [SPARK-17616][SQL] Support a single distinct aggr...

2016-09-21 Thread hvanhovell
GitHub user hvanhovell opened a pull request:

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

[SPARK-17616][SQL] Support a single distinct aggregate combined with a 
non-partial aggregate

## What changes were proposed in this pull request?
We currently cannot execute an aggregate that contains a single distinct 
aggregate function and an one or more non-partially plannable aggregate 
functions, for example:
```sql
select   grp, 
 collect_list(col1),
 count(distinct col2)
from tbl_a
group by 1
```
This is a regression from Spark 1.6. This is caused by the fact that the 
single distinct aggregation code path assumes that all aggregates can be 
planned in two phases (is partially aggregatable). This PR works around this 
issue by triggering the `RewriteDistinctAggregates` in such cases (this is 
similar to the approach taken in 1.6).

## How was this patch tested?
Created `RewriteDistinctAggregatesSuite` which checks if the aggregates 
with distinct aggregate functions get rewritten into two `Aggregates` and an 
`Expand`. Added a regression test to `DataFrameAggregateSuite`.

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

$ git pull https://github.com/hvanhovell/spark SPARK-17616

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

https://github.com/apache/spark/pull/15187.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 #15187


commit 4a9ffaacac03070aa64baeb0a0e3b00d40865491
Author: Herman van Hovell 
Date:   2016-09-21T22:11:56Z

Add case to RewriteDistinctAggregates to rewrite a single distinct 
aggregate combined with a non-partial aggregate.




---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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