[GitHub] spark pull request #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-30 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69175695
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala
 ---
@@ -0,0 +1,72 @@
+/*
+ * 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.expressions._
+import 
org.apache.spark.sql.catalyst.expressions.aggregate.AggregateFunction
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+/**
+ * Collapse plans consisting empty local relations generated by 
[[PruneFilters]].
--- End diff --

Since the optimizer's name, `PropagateEmptyRelation`, is intuitive and 
nice, I tried to write something like `Propagate ...` for this heading line. 
But I feel the current description is more descriptive for the behavior of this 
class 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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-30 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69172328
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlanSuite.scala
 ---
@@ -0,0 +1,173 @@
+/*
+ * 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.Row
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+
+class CollapseEmptyPlanSuite extends PlanTest {
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("CollapseEmptyPlan", Once,
+CombineUnions,
+ReplaceDistinctWithAggregate,
+ReplaceExceptWithAntiJoin,
+ReplaceIntersectWithSemiJoin,
+PushDownPredicate,
+PruneFilters,
+CollapseEmptyPlan) :: Nil
+  }
+
+  object OptimizeWithoutCollapseEmptyPlan extends 
RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("CollapseEmptyPlan", Once,
+CombineUnions,
+ReplaceDistinctWithAggregate,
+ReplaceExceptWithAntiJoin,
+ReplaceIntersectWithSemiJoin,
+PushDownPredicate,
+PruneFilters) :: Nil
+  }
+
+  val testRelation1 = LocalRelation.fromExternalRows(Seq('a.int), data = 
Seq(Row(1)))
+  val testRelation2 = LocalRelation.fromExternalRows(Seq('b.int), data = 
Seq(Row(1)))
+  val testRelation3 = LocalRelation.fromExternalRows(Seq('c.int), data = 
Seq(Row(1)))
--- End diff --

Yep. I'll remove this.


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-30 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69171100
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlan.scala
 ---
@@ -0,0 +1,72 @@
+/*
+ * 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.expressions._
+import 
org.apache.spark.sql.catalyst.expressions.aggregate.AggregateFunction
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+/**
+ * Collapse plans consisting empty local relations generated by 
[[PruneFilters]].
+ * 1. Binary(or Higher)-node Logical Plans
+ *- Union with all empty children.
+ *- Join with one or two empty children (including Intersect/Except).
+ * 2. Unary-node Logical Plans
+ *- Project/Filter/Sample/Join/Limit/Repartition with all empty 
children.
+ *- Aggregate with all empty children and without DeclarativeAggregate 
expressions like COUNT.
--- End diff --

Ooops. :)


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-30 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69171076
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlan.scala
 ---
@@ -0,0 +1,72 @@
+/*
+ * 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.expressions._
+import 
org.apache.spark.sql.catalyst.expressions.aggregate.AggregateFunction
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+/**
+ * Collapse plans consisting empty local relations generated by 
[[PruneFilters]].
+ * 1. Binary(or Higher)-node Logical Plans
+ *- Union with all empty children.
+ *- Join with one or two empty children (including Intersect/Except).
+ * 2. Unary-node Logical Plans
+ *- Project/Filter/Sample/Join/Limit/Repartition with all empty 
children.
+ *- Aggregate with all empty children and without DeclarativeAggregate 
expressions like COUNT.
+ *- Generate(Explode) with all empty children. Others like Hive UDTF 
may return results.
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
+  private def isEmptyLocalRelation(plan: LogicalPlan): Boolean = plan 
match {
+case p: LocalRelation => p.data.isEmpty
+case _ => false
+  }
+
+  private def containsAggregateExpression(e: Expression): Boolean = {
+e.collectFirst { case _: AggregateFunction => () }.isDefined
+  }
+
+  private def empty(plan: LogicalPlan) = LocalRelation(plan.output, data = 
Seq.empty)
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case p: Union if p.children.nonEmpty && 
p.children.forall(isEmptyLocalRelation) =>
--- End diff --

Oh, thank you!


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-30 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69141440
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlanSuite.scala
 ---
@@ -0,0 +1,173 @@
+/*
+ * 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.Row
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+
+class CollapseEmptyPlanSuite extends PlanTest {
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("CollapseEmptyPlan", Once,
+CombineUnions,
+ReplaceDistinctWithAggregate,
+ReplaceExceptWithAntiJoin,
+ReplaceIntersectWithSemiJoin,
+PushDownPredicate,
+PruneFilters,
+CollapseEmptyPlan) :: Nil
+  }
+
+  object OptimizeWithoutCollapseEmptyPlan extends 
RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("CollapseEmptyPlan", Once,
+CombineUnions,
+ReplaceDistinctWithAggregate,
+ReplaceExceptWithAntiJoin,
+ReplaceIntersectWithSemiJoin,
+PushDownPredicate,
+PruneFilters) :: Nil
+  }
+
+  val testRelation1 = LocalRelation.fromExternalRows(Seq('a.int), data = 
Seq(Row(1)))
+  val testRelation2 = LocalRelation.fromExternalRows(Seq('b.int), data = 
Seq(Row(1)))
+  val testRelation3 = LocalRelation.fromExternalRows(Seq('c.int), data = 
Seq(Row(1)))
--- End diff --

seems this is never used?


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-30 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69141347
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlanSuite.scala
 ---
@@ -0,0 +1,173 @@
+/*
+ * 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.Row
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+
+class CollapseEmptyPlanSuite extends PlanTest {
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("CollapseEmptyPlan", Once,
+CombineUnions,
+ReplaceDistinctWithAggregate,
+ReplaceExceptWithAntiJoin,
+ReplaceIntersectWithSemiJoin,
+PushDownPredicate,
+PruneFilters,
+CollapseEmptyPlan) :: Nil
+  }
+
+  object OptimizeWithoutCollapseEmptyPlan extends 
RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("CollapseEmptyPlan", Once,
+CombineUnions,
+ReplaceDistinctWithAggregate,
+ReplaceExceptWithAntiJoin,
+ReplaceIntersectWithSemiJoin,
+PushDownPredicate,
+PruneFilters) :: Nil
+  }
+
+  val testRelation1 = LocalRelation.fromExternalRows(Seq('a.int), data = 
Seq(Row(1)))
+  val testRelation2 = LocalRelation.fromExternalRows(Seq('b.int), data = 
Seq(Row(1)))
+  val testRelation3 = LocalRelation.fromExternalRows(Seq('c.int), data = 
Seq(Row(1)))
+
+  test("Binary Logical Plans - Collapse empty union") {
+val query = testRelation1
+  .where(false)
+  .union(testRelation2.where(false))
+
+val optimized = Optimize.execute(query.analyze)
+val correctAnswer = LocalRelation('a.int)
+
+comparePlans(optimized, correctAnswer)
+  }
+
+  test("Binary Logical Plans - Collapse joins") {
--- End diff --

ok then it's fine


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-30 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69140767
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlanSuite.scala
 ---
@@ -0,0 +1,173 @@
+/*
+ * 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.Row
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+
+class CollapseEmptyPlanSuite extends PlanTest {
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("CollapseEmptyPlan", Once,
+CombineUnions,
+ReplaceDistinctWithAggregate,
+ReplaceExceptWithAntiJoin,
+ReplaceIntersectWithSemiJoin,
+PushDownPredicate,
+PruneFilters,
+CollapseEmptyPlan) :: Nil
+  }
+
+  object OptimizeWithoutCollapseEmptyPlan extends 
RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("CollapseEmptyPlan", Once,
+CombineUnions,
+ReplaceDistinctWithAggregate,
+ReplaceExceptWithAntiJoin,
+ReplaceIntersectWithSemiJoin,
+PushDownPredicate,
+PruneFilters) :: Nil
+  }
+
+  val testRelation1 = LocalRelation.fromExternalRows(Seq('a.int), data = 
Seq(Row(1)))
+  val testRelation2 = LocalRelation.fromExternalRows(Seq('b.int), data = 
Seq(Row(1)))
+  val testRelation3 = LocalRelation.fromExternalRows(Seq('c.int), data = 
Seq(Row(1)))
+
+  test("Binary Logical Plans - Collapse empty union") {
+val query = testRelation1
+  .where(false)
+  .union(testRelation2.where(false))
+
+val optimized = Optimize.execute(query.analyze)
+val correctAnswer = LocalRelation('a.int)
+
+comparePlans(optimized, correctAnswer)
+  }
+
+  test("Binary Logical Plans - Collapse joins") {
--- End diff --

Well, I guess the verbosity is necessary since it guarantees that we don't 
do wrong empty relation propagation for certain types of joins (e.g. outer join 
with only one empty child).


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-30 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69140206
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlan.scala
 ---
@@ -0,0 +1,70 @@
+/*
+ * 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.expressions._
+import 
org.apache.spark.sql.catalyst.expressions.aggregate.DeclarativeAggregate
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+/**
+ * Collapse plans consisting empty local relations generated by 
[[PruneFilters]].
+ * 1. Binary(or Higher)-node Logical Plans
+ *- Union with all empty children.
+ *- Join with one or two empty children (including Intersect/Except).
+ * 2. Unary-node Logical Plans
+ *- Project/Filter/Sample/Join/Limit/Repartition with all empty 
children.
+ *- Aggregate with all empty children and without DeclarativeAggregate 
expressions like COUNT.
+ *- Generate(Explode) with all empty children. Others like Hive UDTF 
may return results.
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
+  private def isEmptyLocalRelation(plan: LogicalPlan): Boolean = plan 
match {
+case p: LocalRelation => p.data.isEmpty
+case _ => false
+  }
+
+  private def isDeclarativeAggregate(e: Expression): Boolean = e match {
+case _: DeclarativeAggregate => true
+case _: LeafExpression => false
+case other => other.children.forall(isDeclarativeAggregate)
+  }
+
+  private def empty(plan: LogicalPlan) = LocalRelation(plan.output, data = 
Seq.empty)
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case p: Union if p.children.nonEmpty && 
p.children.forall(isEmptyLocalRelation) =>
+  empty(p)
+
+case p @ Join(_, _, joinType, _) if 
p.children.exists(isEmptyLocalRelation) => joinType match {
+  case Inner => empty(p)
+  case LeftOuter | LeftSemi | LeftAnti if isEmptyLocalRelation(p.left) 
=> empty(p)
+  case RightOuter if isEmptyLocalRelation(p.right) => empty(p)
+  case FullOuter if p.children.forall(isEmptyLocalRelation) => empty(p)
+  case _ => p
+}
--- End diff --

Sorry that I didn't notice the comment above. I think that should be enough.


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-30 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69135765
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlanSuite.scala
 ---
@@ -0,0 +1,173 @@
+/*
+ * 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.Row
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+
+class CollapseEmptyPlanSuite extends PlanTest {
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("CollapseEmptyPlan", Once,
+CombineUnions,
+ReplaceDistinctWithAggregate,
+ReplaceExceptWithAntiJoin,
+ReplaceIntersectWithSemiJoin,
+PushDownPredicate,
+PruneFilters,
+CollapseEmptyPlan) :: Nil
+  }
+
+  object OptimizeWithoutCollapseEmptyPlan extends 
RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("CollapseEmptyPlan", Once,
+CombineUnions,
+ReplaceDistinctWithAggregate,
+ReplaceExceptWithAntiJoin,
+ReplaceIntersectWithSemiJoin,
+PushDownPredicate,
+PruneFilters) :: Nil
+  }
+
+  val testRelation1 = LocalRelation.fromExternalRows(Seq('a.int), data = 
Seq(Row(1)))
+  val testRelation2 = LocalRelation.fromExternalRows(Seq('b.int), data = 
Seq(Row(1)))
+  val testRelation3 = LocalRelation.fromExternalRows(Seq('c.int), data = 
Seq(Row(1)))
+
+  test("Binary Logical Plans - Collapse empty union") {
+val query = testRelation1
+  .where(false)
+  .union(testRelation2.where(false))
+
+val optimized = Optimize.execute(query.analyze)
+val correctAnswer = LocalRelation('a.int)
+
+comparePlans(optimized, correctAnswer)
+  }
+
+  test("Binary Logical Plans - Collapse joins") {
--- End diff --

This test is too verbose... Can you try to simplify it? We don't need to 
cover all cases but some typical ones


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-30 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69135496
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlan.scala
 ---
@@ -0,0 +1,72 @@
+/*
+ * 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.expressions._
+import 
org.apache.spark.sql.catalyst.expressions.aggregate.AggregateFunction
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+/**
+ * Collapse plans consisting empty local relations generated by 
[[PruneFilters]].
+ * 1. Binary(or Higher)-node Logical Plans
+ *- Union with all empty children.
+ *- Join with one or two empty children (including Intersect/Except).
+ * 2. Unary-node Logical Plans
+ *- Project/Filter/Sample/Join/Limit/Repartition with all empty 
children.
+ *- Aggregate with all empty children and without DeclarativeAggregate 
expressions like COUNT.
+ *- Generate(Explode) with all empty children. Others like Hive UDTF 
may return results.
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
--- End diff --

how about `PropagateEmptyRelation`


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-30 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69135425
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlan.scala
 ---
@@ -0,0 +1,72 @@
+/*
+ * 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.expressions._
+import 
org.apache.spark.sql.catalyst.expressions.aggregate.AggregateFunction
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+/**
+ * Collapse plans consisting empty local relations generated by 
[[PruneFilters]].
+ * 1. Binary(or Higher)-node Logical Plans
+ *- Union with all empty children.
+ *- Join with one or two empty children (including Intersect/Except).
+ * 2. Unary-node Logical Plans
+ *- Project/Filter/Sample/Join/Limit/Repartition with all empty 
children.
+ *- Aggregate with all empty children and without DeclarativeAggregate 
expressions like COUNT.
--- End diff --

`DeclarativeAggregate` -> aggregate function?


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-30 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69131609
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlan.scala
 ---
@@ -0,0 +1,72 @@
+/*
+ * 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.expressions._
+import 
org.apache.spark.sql.catalyst.expressions.aggregate.AggregateFunction
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+/**
+ * Collapse plans consisting empty local relations generated by 
[[PruneFilters]].
+ * 1. Binary(or Higher)-node Logical Plans
+ *- Union with all empty children.
+ *- Join with one or two empty children (including Intersect/Except).
+ * 2. Unary-node Logical Plans
+ *- Project/Filter/Sample/Join/Limit/Repartition with all empty 
children.
+ *- Aggregate with all empty children and without DeclarativeAggregate 
expressions like COUNT.
+ *- Generate(Explode) with all empty children. Others like Hive UDTF 
may return results.
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
+  private def isEmptyLocalRelation(plan: LogicalPlan): Boolean = plan 
match {
+case p: LocalRelation => p.data.isEmpty
+case _ => false
+  }
+
+  private def containsAggregateExpression(e: Expression): Boolean = {
+e.collectFirst { case _: AggregateFunction => () }.isDefined
+  }
+
+  private def empty(plan: LogicalPlan) = LocalRelation(plan.output, data = 
Seq.empty)
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case p: Union if p.children.nonEmpty && 
p.children.forall(isEmptyLocalRelation) =>
--- End diff --

`Union.resolved` already guarantee `children.length > 1`


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-30 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69111733
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlanSuite.scala
 ---
@@ -0,0 +1,173 @@
+/*
+ * 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.Row
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+
+class CollapseEmptyPlanSuite extends PlanTest {
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("CollapseEmptyPlan", Once,
+CombineUnions,
+ReplaceDistinctWithAggregate,
+ReplaceExceptWithAntiJoin,
+ReplaceIntersectWithSemiJoin,
+PushDownPredicate,
+PruneFilters,
+CollapseEmptyPlan) :: Nil
+  }
+
+  object OptimizeWithoutCollapseEmptyPlan extends 
RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("CollapseEmptyPlan", Once,
+CombineUnions,
+ReplaceDistinctWithAggregate,
+ReplaceExceptWithAntiJoin,
+ReplaceIntersectWithSemiJoin,
+PushDownPredicate,
+PruneFilters) :: Nil
+  }
+
+  val testRelation1 = LocalRelation.fromExternalRows(Seq('a.int), data = 
Seq(Row(1)))
+  val testRelation2 = LocalRelation.fromExternalRows(Seq('b.int), data = 
Seq(Row(1)))
+  val testRelation3 = LocalRelation.fromExternalRows(Seq('c.int), data = 
Seq(Row(1)))
+
+  test("Binary(or Higher) Logical Plans - Collapse empty union") {
--- End diff --

Yep. I removed.


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-30 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69111430
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlan.scala
 ---
@@ -0,0 +1,70 @@
+/*
+ * 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.expressions._
+import 
org.apache.spark.sql.catalyst.expressions.aggregate.DeclarativeAggregate
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+/**
+ * Collapse plans consisting empty local relations generated by 
[[PruneFilters]].
+ * 1. Binary(or Higher)-node Logical Plans
+ *- Union with all empty children.
+ *- Join with one or two empty children (including Intersect/Except).
+ * 2. Unary-node Logical Plans
+ *- Project/Filter/Sample/Join/Limit/Repartition with all empty 
children.
+ *- Aggregate with all empty children and without DeclarativeAggregate 
expressions like COUNT.
+ *- Generate(Explode) with all empty children. Others like Hive UDTF 
may return results.
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
+  private def isEmptyLocalRelation(plan: LogicalPlan): Boolean = plan 
match {
+case p: LocalRelation => p.data.isEmpty
+case _ => false
+  }
+
+  private def isDeclarativeAggregate(e: Expression): Boolean = e match {
+case _: DeclarativeAggregate => true
+case _: LeafExpression => false
+case other => other.children.forall(isDeclarativeAggregate)
+  }
+
+  private def empty(plan: LogicalPlan) = LocalRelation(plan.output, data = 
Seq.empty)
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case p: Union if p.children.nonEmpty && 
p.children.forall(isEmptyLocalRelation) =>
+  empty(p)
+
+case p @ Join(_, _, joinType, _) if 
p.children.exists(isEmptyLocalRelation) => joinType match {
+  case Inner => empty(p)
+  case LeftOuter | LeftSemi | LeftAnti if isEmptyLocalRelation(p.left) 
=> empty(p)
+  case RightOuter if isEmptyLocalRelation(p.right) => empty(p)
+  case FullOuter if p.children.forall(isEmptyLocalRelation) => empty(p)
+  case _ => p
+}
+
+case p: UnaryNode if p.children.nonEmpty && 
p.children.forall(isEmptyLocalRelation) => p match {
+  case _: Project | _: Filter | _: Sample | _: Sort | _: GlobalLimit | 
_: LocalLimit |
+   _: Repartition | _: RepartitionByExpression => empty(p)
+  case Aggregate(_, ae, _) if !ae.exists(isDeclarativeAggregate) => 
empty(p)
+  case Generate(_ : Explode, _, _, _, _, _) => empty(p)
--- End diff --

Sure!


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-30 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69111287
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlanSuite.scala
 ---
@@ -0,0 +1,173 @@
+/*
+ * 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.Row
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+
+class CollapseEmptyPlanSuite extends PlanTest {
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("CollapseEmptyPlan", Once,
+CombineUnions,
+ReplaceDistinctWithAggregate,
+ReplaceExceptWithAntiJoin,
+ReplaceIntersectWithSemiJoin,
+PushDownPredicate,
+PruneFilters,
+CollapseEmptyPlan) :: Nil
+  }
+
+  object OptimizeWithoutCollapseEmptyPlan extends 
RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("CollapseEmptyPlan", Once,
+CombineUnions,
+ReplaceDistinctWithAggregate,
+ReplaceExceptWithAntiJoin,
+ReplaceIntersectWithSemiJoin,
+PushDownPredicate,
+PruneFilters) :: Nil
+  }
+
+  val testRelation1 = LocalRelation.fromExternalRows(Seq('a.int), data = 
Seq(Row(1)))
+  val testRelation2 = LocalRelation.fromExternalRows(Seq('b.int), data = 
Seq(Row(1)))
+  val testRelation3 = LocalRelation.fromExternalRows(Seq('c.int), data = 
Seq(Row(1)))
+
+  test("Binary(or Higher) Logical Plans - Collapse empty union") {
--- End diff --

Just here or for all? Actually, I added that for `Union`.


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-30 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69111075
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlanSuite.scala
 ---
@@ -0,0 +1,173 @@
+/*
+ * 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.Row
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+
+class CollapseEmptyPlanSuite extends PlanTest {
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("CollapseEmptyPlan", Once,
+CombineUnions,
+ReplaceDistinctWithAggregate,
+ReplaceExceptWithAntiJoin,
+ReplaceIntersectWithSemiJoin,
+PushDownPredicate,
+PruneFilters,
+CollapseEmptyPlan) :: Nil
+  }
+
+  object OptimizeWithoutCollapseEmptyPlan extends 
RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("CollapseEmptyPlan", Once,
+CombineUnions,
+ReplaceDistinctWithAggregate,
+ReplaceExceptWithAntiJoin,
+ReplaceIntersectWithSemiJoin,
+PushDownPredicate,
+PruneFilters) :: Nil
+  }
+
+  val testRelation1 = LocalRelation.fromExternalRows(Seq('a.int), data = 
Seq(Row(1)))
+  val testRelation2 = LocalRelation.fromExternalRows(Seq('b.int), data = 
Seq(Row(1)))
+  val testRelation3 = LocalRelation.fromExternalRows(Seq('c.int), data = 
Seq(Row(1)))
+
+  test("Binary(or Higher) Logical Plans - Collapse empty union") {
--- End diff --

Remove "(or Higher)".


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-30 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69110745
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlan.scala
 ---
@@ -0,0 +1,70 @@
+/*
+ * 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.expressions._
+import 
org.apache.spark.sql.catalyst.expressions.aggregate.DeclarativeAggregate
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+/**
+ * Collapse plans consisting empty local relations generated by 
[[PruneFilters]].
+ * 1. Binary(or Higher)-node Logical Plans
+ *- Union with all empty children.
+ *- Join with one or two empty children (including Intersect/Except).
+ * 2. Unary-node Logical Plans
+ *- Project/Filter/Sample/Join/Limit/Repartition with all empty 
children.
+ *- Aggregate with all empty children and without DeclarativeAggregate 
expressions like COUNT.
+ *- Generate(Explode) with all empty children. Others like Hive UDTF 
may return results.
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
+  private def isEmptyLocalRelation(plan: LogicalPlan): Boolean = plan 
match {
+case p: LocalRelation => p.data.isEmpty
+case _ => false
+  }
+
+  private def isDeclarativeAggregate(e: Expression): Boolean = e match {
+case _: DeclarativeAggregate => true
+case _: LeafExpression => false
+case other => other.children.forall(isDeclarativeAggregate)
+  }
+
+  private def empty(plan: LogicalPlan) = LocalRelation(plan.output, data = 
Seq.empty)
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case p: Union if p.children.nonEmpty && 
p.children.forall(isEmptyLocalRelation) =>
+  empty(p)
+
+case p @ Join(_, _, joinType, _) if 
p.children.exists(isEmptyLocalRelation) => joinType match {
+  case Inner => empty(p)
+  case LeftOuter | LeftSemi | LeftAnti if isEmptyLocalRelation(p.left) 
=> empty(p)
+  case RightOuter if isEmptyLocalRelation(p.right) => empty(p)
+  case FullOuter if p.children.forall(isEmptyLocalRelation) => empty(p)
+  case _ => p
+}
+
+case p: UnaryNode if p.children.nonEmpty && 
p.children.forall(isEmptyLocalRelation) => p match {
+  case _: Project | _: Filter | _: Sample | _: Sort | _: GlobalLimit | 
_: LocalLimit |
+   _: Repartition | _: RepartitionByExpression => empty(p)
+  case Aggregate(_, ae, _) if !ae.exists(isDeclarativeAggregate) => 
empty(p)
+  case Generate(_ : Explode, _, _, _, _, _) => empty(p)
--- End diff --

Let's add comment here to explain why we must special case `Aggregate` and 
`Generate`.


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-30 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69110423
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlan.scala
 ---
@@ -0,0 +1,70 @@
+/*
+ * 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.expressions._
+import 
org.apache.spark.sql.catalyst.expressions.aggregate.DeclarativeAggregate
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+/**
+ * Collapse plans consisting empty local relations generated by 
[[PruneFilters]].
+ * 1. Binary(or Higher)-node Logical Plans
+ *- Union with all empty children.
+ *- Join with one or two empty children (including Intersect/Except).
+ * 2. Unary-node Logical Plans
+ *- Project/Filter/Sample/Join/Limit/Repartition with all empty 
children.
+ *- Aggregate with all empty children and without DeclarativeAggregate 
expressions like COUNT.
+ *- Generate(Explode) with all empty children. Others like Hive UDTF 
may return results.
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
+  private def isEmptyLocalRelation(plan: LogicalPlan): Boolean = plan 
match {
+case p: LocalRelation => p.data.isEmpty
+case _ => false
+  }
+
+  private def isDeclarativeAggregate(e: Expression): Boolean = e match {
+case _: DeclarativeAggregate => true
+case _: LeafExpression => false
+case other => other.children.forall(isDeclarativeAggregate)
+  }
+
+  private def empty(plan: LogicalPlan) = LocalRelation(plan.output, data = 
Seq.empty)
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case p: Union if p.children.nonEmpty && 
p.children.forall(isEmptyLocalRelation) =>
+  empty(p)
+
+case p @ Join(_, _, joinType, _) if 
p.children.exists(isEmptyLocalRelation) => joinType match {
+  case Inner => empty(p)
+  case LeftOuter | LeftSemi | LeftAnti if isEmptyLocalRelation(p.left) 
=> empty(p)
+  case RightOuter if isEmptyLocalRelation(p.right) => empty(p)
+  case FullOuter if p.children.forall(isEmptyLocalRelation) => empty(p)
+  case _ => p
+}
--- End diff --

Yep. I added line 30 
https://github.com/apache/spark/pull/13906/files#diff-315910d950ea08479990c40570fbd216R30
 currently.

But, I will add comment, too. Maybe beyond `LeftSemi` at line 56?


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-30 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69110054
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlan.scala
 ---
@@ -0,0 +1,70 @@
+/*
+ * 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.expressions._
+import 
org.apache.spark.sql.catalyst.expressions.aggregate.DeclarativeAggregate
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+/**
+ * Collapse plans consisting empty local relations generated by 
[[PruneFilters]].
+ * 1. Binary(or Higher)-node Logical Plans
+ *- Union with all empty children.
+ *- Join with one or two empty children (including Intersect/Except).
+ * 2. Unary-node Logical Plans
+ *- Project/Filter/Sample/Join/Limit/Repartition with all empty 
children.
+ *- Aggregate with all empty children and without DeclarativeAggregate 
expressions like COUNT.
+ *- Generate(Explode) with all empty children. Others like Hive UDTF 
may return results.
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
+  private def isEmptyLocalRelation(plan: LogicalPlan): Boolean = plan 
match {
+case p: LocalRelation => p.data.isEmpty
+case _ => false
+  }
+
+  private def isDeclarativeAggregate(e: Expression): Boolean = e match {
+case _: DeclarativeAggregate => true
+case _: LeafExpression => false
+case other => other.children.forall(isDeclarativeAggregate)
+  }
--- End diff --

Thanks, again!


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-30 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69109977
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlan.scala
 ---
@@ -0,0 +1,70 @@
+/*
+ * 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.expressions._
+import 
org.apache.spark.sql.catalyst.expressions.aggregate.DeclarativeAggregate
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+/**
+ * Collapse plans consisting empty local relations generated by 
[[PruneFilters]].
+ * 1. Binary(or Higher)-node Logical Plans
+ *- Union with all empty children.
+ *- Join with one or two empty children (including Intersect/Except).
+ * 2. Unary-node Logical Plans
+ *- Project/Filter/Sample/Join/Limit/Repartition with all empty 
children.
+ *- Aggregate with all empty children and without DeclarativeAggregate 
expressions like COUNT.
+ *- Generate(Explode) with all empty children. Others like Hive UDTF 
may return results.
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
+  private def isEmptyLocalRelation(plan: LogicalPlan): Boolean = plan 
match {
+case p: LocalRelation => p.data.isEmpty
+case _ => false
+  }
+
+  private def isDeclarativeAggregate(e: Expression): Boolean = e match {
+case _: DeclarativeAggregate => true
--- End diff --

Oh. I forgot your comment about this. I'll fix like that.


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-30 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69109823
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlan.scala
 ---
@@ -0,0 +1,70 @@
+/*
+ * 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.expressions._
+import 
org.apache.spark.sql.catalyst.expressions.aggregate.DeclarativeAggregate
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+/**
+ * Collapse plans consisting empty local relations generated by 
[[PruneFilters]].
+ * 1. Binary(or Higher)-node Logical Plans
+ *- Union with all empty children.
+ *- Join with one or two empty children (including Intersect/Except).
+ * 2. Unary-node Logical Plans
+ *- Project/Filter/Sample/Join/Limit/Repartition with all empty 
children.
+ *- Aggregate with all empty children and without DeclarativeAggregate 
expressions like COUNT.
+ *- Generate(Explode) with all empty children. Others like Hive UDTF 
may return results.
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
+  private def isEmptyLocalRelation(plan: LogicalPlan): Boolean = plan 
match {
+case p: LocalRelation => p.data.isEmpty
+case _ => false
+  }
+
+  private def isDeclarativeAggregate(e: Expression): Boolean = e match {
+case _: DeclarativeAggregate => true
+case _: LeafExpression => false
+case other => other.children.forall(isDeclarativeAggregate)
+  }
+
+  private def empty(plan: LogicalPlan) = LocalRelation(plan.output, data = 
Seq.empty)
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case p: Union if p.children.nonEmpty && 
p.children.forall(isEmptyLocalRelation) =>
+  empty(p)
+
+case p @ Join(_, _, joinType, _) if 
p.children.exists(isEmptyLocalRelation) => joinType match {
+  case Inner => empty(p)
+  case LeftOuter | LeftSemi | LeftAnti if isEmptyLocalRelation(p.left) 
=> empty(p)
+  case RightOuter if isEmptyLocalRelation(p.right) => empty(p)
+  case FullOuter if p.children.forall(isEmptyLocalRelation) => empty(p)
+  case _ => p
+}
--- End diff --

Could you please comment that `Intersect` is also covered here? I didn't 
realized that we've already translated `Intersect` using joins at first.


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-30 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69109611
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlan.scala
 ---
@@ -0,0 +1,70 @@
+/*
+ * 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.expressions._
+import 
org.apache.spark.sql.catalyst.expressions.aggregate.DeclarativeAggregate
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+/**
+ * Collapse plans consisting empty local relations generated by 
[[PruneFilters]].
+ * 1. Binary(or Higher)-node Logical Plans
+ *- Union with all empty children.
+ *- Join with one or two empty children (including Intersect/Except).
+ * 2. Unary-node Logical Plans
+ *- Project/Filter/Sample/Join/Limit/Repartition with all empty 
children.
+ *- Aggregate with all empty children and without DeclarativeAggregate 
expressions like COUNT.
+ *- Generate(Explode) with all empty children. Others like Hive UDTF 
may return results.
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
+  private def isEmptyLocalRelation(plan: LogicalPlan): Boolean = plan 
match {
+case p: LocalRelation => p.data.isEmpty
+case _ => false
+  }
+
+  private def isDeclarativeAggregate(e: Expression): Boolean = e match {
+case _: DeclarativeAggregate => true
+case _: LeafExpression => false
+case other => other.children.forall(isDeclarativeAggregate)
+  }
--- End diff --

I'd replace this method with:

```scala
def containsAggregateExpression(e: Expression): Boolean = {
  e.collectFirst { case _: AggregateFunction => () }.isDefined
}
```


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-30 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69109340
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlan.scala
 ---
@@ -0,0 +1,70 @@
+/*
+ * 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.expressions._
+import 
org.apache.spark.sql.catalyst.expressions.aggregate.DeclarativeAggregate
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+/**
+ * Collapse plans consisting empty local relations generated by 
[[PruneFilters]].
+ * 1. Binary(or Higher)-node Logical Plans
+ *- Union with all empty children.
+ *- Join with one or two empty children (including Intersect/Except).
+ * 2. Unary-node Logical Plans
+ *- Project/Filter/Sample/Join/Limit/Repartition with all empty 
children.
+ *- Aggregate with all empty children and without DeclarativeAggregate 
expressions like COUNT.
+ *- Generate(Explode) with all empty children. Others like Hive UDTF 
may return results.
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
+  private def isEmptyLocalRelation(plan: LogicalPlan): Boolean = plan 
match {
+case p: LocalRelation => p.data.isEmpty
+case _ => false
+  }
+
+  private def isDeclarativeAggregate(e: Expression): Boolean = e match {
+case _: DeclarativeAggregate => true
--- End diff --

Why `DeclarativeAggregate` rather than `AggregateFunction`? 
`AggregateFunction` also covers `ImperativeAggregate` like `ScalaUDAF`, which 
should also be covered 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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-30 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69089371
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlan.scala
 ---
@@ -0,0 +1,49 @@
+/*
+ * 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.expressions._
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+/**
+ * Collapse plans consisting empty local relations generated by 
[[PruneFilters]].
+ * 1. InnerJoin with one or two empty children.
+ * 2. Project/Generate/Filter/Sample/Join/Limit/Union/Repartition with all 
empty children.
+ * 3. Aggregate with all empty children and grpExprs containing all 
aggExprs.
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
+  private def isEmptyLocalRelation(plan: LogicalPlan): Boolean =
+plan.isInstanceOf[LocalRelation] && 
plan.asInstanceOf[LocalRelation].data.isEmpty
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case p @ Join(_, _, Inner, _) if 
p.children.exists(isEmptyLocalRelation) =>
+  LocalRelation(p.output, data = Seq.empty)
+
+case p: LogicalPlan if p.children.nonEmpty && 
p.children.forall(isEmptyLocalRelation) =>
+  p match {
+case _: Project | _: Generate | _: Filter | _: Sample | _: Join |
+ _: Sort | _: GlobalLimit | _: LocalLimit | _: Union | _: 
Repartition =>
+  LocalRelation(p.output, data = Seq.empty)
+case Aggregate(ge, ae, _) if ae.forall(ge.contains(_)) =>
--- End diff --

Yea. The following predicate should work:

```scala
ae.forall(_.collectFirst { case _: AggregateExpression => () }.isEmpty)
```

(But probably put it into a separate method though.)


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-29 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69065541
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlan.scala
 ---
@@ -0,0 +1,49 @@
+/*
+ * 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.expressions._
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+/**
+ * Collapse plans consisting empty local relations generated by 
[[PruneFilters]].
+ * 1. InnerJoin with one or two empty children.
+ * 2. Project/Generate/Filter/Sample/Join/Limit/Union/Repartition with all 
empty children.
+ * 3. Aggregate with all empty children and grpExprs containing all 
aggExprs.
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
+  private def isEmptyLocalRelation(plan: LogicalPlan): Boolean =
+plan.isInstanceOf[LocalRelation] && 
plan.asInstanceOf[LocalRelation].data.isEmpty
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case p @ Join(_, _, Inner, _) if 
p.children.exists(isEmptyLocalRelation) =>
--- End diff --

Yea, we can also add `Intersect`.


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-29 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69065425
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlan.scala
 ---
@@ -0,0 +1,49 @@
+/*
+ * 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.expressions._
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+/**
+ * Collapse plans consisting empty local relations generated by 
[[PruneFilters]].
+ * 1. InnerJoin with one or two empty children.
+ * 2. Project/Generate/Filter/Sample/Join/Limit/Union/Repartition with all 
empty children.
+ * 3. Aggregate with all empty children and grpExprs containing all 
aggExprs.
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
+  private def isEmptyLocalRelation(plan: LogicalPlan): Boolean =
+plan.isInstanceOf[LocalRelation] && 
plan.asInstanceOf[LocalRelation].data.isEmpty
--- End diff --

```scala
plan match {
  case p: LocalRelation => p.data.isEmpty
  case _ => false
}
```


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-29 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69065025
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlan.scala
 ---
@@ -0,0 +1,49 @@
+/*
+ * 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.expressions._
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+/**
+ * Collapse plans consisting empty local relations generated by 
[[PruneFilters]].
+ * 1. InnerJoin with one or two empty children.
+ * 2. Project/Generate/Filter/Sample/Join/Limit/Union/Repartition with all 
empty children.
+ * 3. Aggregate with all empty children and grpExprs containing all 
aggExprs.
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
+  private def isEmptyLocalRelation(plan: LogicalPlan): Boolean =
+plan.isInstanceOf[LocalRelation] && 
plan.asInstanceOf[LocalRelation].data.isEmpty
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case p @ Join(_, _, Inner, _) if 
p.children.exists(isEmptyLocalRelation) =>
--- End diff --

I think this rule is very useful, we can avoid scanning one join side if 
the other side is empty


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-29 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69064885
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlan.scala
 ---
@@ -0,0 +1,49 @@
+/*
+ * 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.expressions._
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+/**
+ * Collapse plans consisting empty local relations generated by 
[[PruneFilters]].
+ * 1. InnerJoin with one or two empty children.
+ * 2. Project/Generate/Filter/Sample/Join/Limit/Union/Repartition with all 
empty children.
+ * 3. Aggregate with all empty children and grpExprs containing all 
aggExprs.
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
+  private def isEmptyLocalRelation(plan: LogicalPlan): Boolean =
+plan.isInstanceOf[LocalRelation] && 
plan.asInstanceOf[LocalRelation].data.isEmpty
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case p @ Join(_, _, Inner, _) if 
p.children.exists(isEmptyLocalRelation) =>
+  LocalRelation(p.output, data = Seq.empty)
+
+case p: LogicalPlan if p.children.nonEmpty && 
p.children.forall(isEmptyLocalRelation) =>
+  p match {
+case _: Project | _: Generate | _: Filter | _: Sample | _: Join |
+ _: Sort | _: GlobalLimit | _: LocalLimit | _: Union | _: 
Repartition =>
+  LocalRelation(p.output, data = Seq.empty)
+case Aggregate(ge, ae, _) if ae.forall(ge.contains(_)) =>
--- End diff --

what exactly are we checking here? it looks to me that we can do empty 
relation propagate if aggregate list has no aggregate function, e.g. `select 
col + 1 from tbl group by col` should also work.


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-29 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r69064054
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlan.scala
 ---
@@ -0,0 +1,49 @@
+/*
+ * 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.expressions._
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+/**
+ * Collapse plans consisting empty local relations generated by 
[[PruneFilters]].
+ * 1. InnerJoin with one or two empty children.
+ * 2. Project/Generate/Filter/Sample/Join/Limit/Union/Repartition with all 
empty children.
+ * 3. Aggregate with all empty children and grpExprs containing all 
aggExprs.
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
+  private def isEmptyLocalRelation(plan: LogicalPlan): Boolean =
+plan.isInstanceOf[LocalRelation] && 
plan.asInstanceOf[LocalRelation].data.isEmpty
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case p @ Join(_, _, Inner, _) if 
p.children.exists(isEmptyLocalRelation) =>
+  LocalRelation(p.output, data = Seq.empty)
+
+case p: LogicalPlan if p.children.nonEmpty && 
p.children.forall(isEmptyLocalRelation) =>
+  p match {
+case _: Project | _: Generate | _: Filter | _: Sample | _: Join |
--- End diff --

Actually `Generate` can't be included here. Our `Generate` also support 
Hive style UDTF, which has a weird semantics: for a UDTF `f`, after all rows 
being processed, `f.close()` will be called, and *more rows can be generated* 
within `f.close()`. This means a UDTF may generate one or more rows even if the 
underlying input is empty.

See [here][1] and PR #5338 for more details.

[1]: https://github.com/apache/spark/pull/5383/files


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-28 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r68839609
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -1053,6 +1055,49 @@ object PruneFilters extends Rule[LogicalPlan] with 
PredicateHelper {
 }
 
 /**
+ * Collapse plans consisting all empty local relations generated by 
[[PruneFilters]].
+ * Note that the ObjectProducer/Consumer and direct aggregations are the 
exceptions.
+ * {{{
+ *   SELECT a, b FROM t WHERE 1=0 GROUP BY a, b ORDER BY a, b ==> empty 
result
+ *   SELECT SUM(a) FROM t WHERE 1=0 GROUP BY a HAVING COUNT(*)>1 ORDER BY 
a (Not optimized)
+ * }}}
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
--- End diff --

yup


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-28 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r68838472
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -1053,6 +1055,49 @@ object PruneFilters extends Rule[LogicalPlan] with 
PredicateHelper {
 }
 
 /**
+ * Collapse plans consisting all empty local relations generated by 
[[PruneFilters]].
+ * Note that the ObjectProducer/Consumer and direct aggregations are the 
exceptions.
+ * {{{
+ *   SELECT a, b FROM t WHERE 1=0 GROUP BY a, b ORDER BY a, b ==> empty 
result
+ *   SELECT SUM(a) FROM t WHERE 1=0 GROUP BY a HAVING COUNT(*)>1 ORDER BY 
a (Not optimized)
+ * }}}
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
--- End diff --

Just `CollapseEmptyPlan.scala` is okay?


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-28 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r68838412
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -1053,6 +1055,49 @@ object PruneFilters extends Rule[LogicalPlan] with 
PredicateHelper {
 }
 
 /**
+ * Collapse plans consisting all empty local relations generated by 
[[PruneFilters]].
+ * Note that the ObjectProducer/Consumer and direct aggregations are the 
exceptions.
+ * {{{
+ *   SELECT a, b FROM t WHERE 1=0 GROUP BY a, b ORDER BY a, b ==> empty 
result
+ *   SELECT SUM(a) FROM t WHERE 1=0 GROUP BY a HAVING COUNT(*)>1 ORDER BY 
a (Not optimized)
+ * }}}
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
--- End diff --

Sure. What name is suitable for this optimizer?


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-28 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r68838170
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -1053,6 +1055,49 @@ object PruneFilters extends Rule[LogicalPlan] with 
PredicateHelper {
 }
 
 /**
+ * Collapse plans consisting all empty local relations generated by 
[[PruneFilters]].
+ * Note that the ObjectProducer/Consumer and direct aggregations are the 
exceptions.
+ * {{{
+ *   SELECT a, b FROM t WHERE 1=0 GROUP BY a, b ORDER BY a, b ==> empty 
result
+ *   SELECT SUM(a) FROM t WHERE 1=0 GROUP BY a HAVING COUNT(*)>1 ORDER BY 
a (Not optimized)
+ * }}}
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
+  private def isEmptyLocalRelation(plan: LogicalPlan): Boolean =
+plan.isInstanceOf[LocalRelation] && 
plan.asInstanceOf[LocalRelation].data.isEmpty
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case x if x.isInstanceOf[ObjectProducer] || 
x.isInstanceOf[ObjectConsumer] => x
+
+// Case 1: If groupingExpressions contains all aggregation 
expressions, the result is empty.
+case a @ Aggregate(ge, ae, child) if isEmptyLocalRelation(child) && 
ae.forall(ge.contains(_)) =>
+  LocalRelation(a.output, data = Seq.empty)
+
+// Case 2: General aggregations can generate non-empty results.
+case a: Aggregate => a
+
+// Case 3: The following plans having only empty relations return 
empty results.
+case p: LogicalPlan if p.children.nonEmpty && 
p.children.forall(isEmptyLocalRelation) =>
+  p match {
+case _: Project | _: Generate | _: Filter | _: Sample | _: Join |
+ _: Sort | _: GlobalLimit | _: LocalLimit |
+ _: Distinct | _: Except | _: Union |
+ _: Repartition =>
+  LocalRelation(p.output, data = Seq.empty)
+case _ => p
+  }
+
+// Case 4: The following plans having at least one empty relation 
return empty results.
+case p: LogicalPlan if p.children.exists(isEmptyLocalRelation) =>
+  p match {
+case Join(_, _, Inner, _) | _: Intersect =>
--- End diff --

Sure!


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-28 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r68837197
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -1053,6 +1055,49 @@ object PruneFilters extends Rule[LogicalPlan] with 
PredicateHelper {
 }
 
 /**
+ * Collapse plans consisting all empty local relations generated by 
[[PruneFilters]].
+ * Note that the ObjectProducer/Consumer and direct aggregations are the 
exceptions.
+ * {{{
+ *   SELECT a, b FROM t WHERE 1=0 GROUP BY a, b ORDER BY a, b ==> empty 
result
+ *   SELECT SUM(a) FROM t WHERE 1=0 GROUP BY a HAVING COUNT(*)>1 ORDER BY 
a (Not optimized)
+ * }}}
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
--- End diff --

can you actually move this into a separate file? the optimizer is becoming 
too large and I want to break it apart soon. No point adding new things in this 
file.



---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-28 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r68836963
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -1053,6 +1055,49 @@ object PruneFilters extends Rule[LogicalPlan] with 
PredicateHelper {
 }
 
 /**
+ * Collapse plans consisting all empty local relations generated by 
[[PruneFilters]].
+ * Note that the ObjectProducer/Consumer and direct aggregations are the 
exceptions.
+ * {{{
+ *   SELECT a, b FROM t WHERE 1=0 GROUP BY a, b ORDER BY a, b ==> empty 
result
+ *   SELECT SUM(a) FROM t WHERE 1=0 GROUP BY a HAVING COUNT(*)>1 ORDER BY 
a (Not optimized)
+ * }}}
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
+  private def isEmptyLocalRelation(plan: LogicalPlan): Boolean =
+plan.isInstanceOf[LocalRelation] && 
plan.asInstanceOf[LocalRelation].data.isEmpty
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case x if x.isInstanceOf[ObjectProducer] || 
x.isInstanceOf[ObjectConsumer] => x
+
+// Case 1: If groupingExpressions contains all aggregation 
expressions, the result is empty.
+case a @ Aggregate(ge, ae, child) if isEmptyLocalRelation(child) && 
ae.forall(ge.contains(_)) =>
+  LocalRelation(a.output, data = Seq.empty)
+
+// Case 2: General aggregations can generate non-empty results.
+case a: Aggregate => a
+
+// Case 3: The following plans having only empty relations return 
empty results.
+case p: LogicalPlan if p.children.nonEmpty && 
p.children.forall(isEmptyLocalRelation) =>
+  p match {
+case _: Project | _: Generate | _: Filter | _: Sample | _: Join |
+ _: Sort | _: GlobalLimit | _: LocalLimit |
+ _: Distinct | _: Except | _: Union |
+ _: Repartition =>
+  LocalRelation(p.output, data = Seq.empty)
+case _ => p
+  }
+
+// Case 4: The following plans having at least one empty relation 
return empty results.
+case p: LogicalPlan if p.children.exists(isEmptyLocalRelation) =>
+  p match {
+case Join(_, _, Inner, _) | _: Intersect =>
--- End diff --

can we move this out rather than doing a two level nesting


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-28 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r68703430
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlanSuite.scala
 ---
@@ -0,0 +1,133 @@
+/*
+ * 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.Row
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.plans.{LeftAnti, PlanTest}
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+
+class CollapseEmptyPlanSuite extends PlanTest {
--- End diff --

I'll update to have more.


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-28 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r68701978
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -1053,6 +1055,41 @@ object PruneFilters extends Rule[LogicalPlan] with 
PredicateHelper {
 }
 
 /**
+ * Collapse plans consisting all empty local relations generated by 
[[PruneFilters]].
+ * Note that the ObjectProducer/Consumer and direct aggregations are the 
exceptions.
+ * {{{
+ *   SELECT a, b FROM t WHERE 1=0 GROUP BY a, b ORDER BY a, b ==> empty 
result
+ *   SELECT SUM(a) FROM t WHERE 1=0 GROUP BY a HAVING COUNT(*)>1 ORDER BY 
a (Not optimized)
+ * }}}
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
+  private def isEmptyLocalRelation(plan: LogicalPlan): Boolean =
+plan.isInstanceOf[LocalRelation] && 
plan.asInstanceOf[LocalRelation].data.isEmpty
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case x if x.isInstanceOf[ObjectProducer] || 
x.isInstanceOf[ObjectConsumer] => x
+
+// Case 1: If groupingExpressions contains all aggregation 
expressions, the result is empty.
+case a @ Aggregate(ge, ae, child) if isEmptyLocalRelation(child) && 
ae.forall(ge.contains(_)) =>
+  LocalRelation(a.output, data = Seq.empty)
+
+// Case 2: General aggregations can generate non-empty results.
+case a: Aggregate => a
+
+// Case 3: The following non-leaf plans having only empty relations 
return empty results.
+case p: LogicalPlan if p.children.nonEmpty && 
p.children.forall(isEmptyLocalRelation) =>
+  p match {
+case _: Project | _: Generate | _: Filter | _: Sample | _: Join |
--- End diff --

Yep, right! I'll add them explicitly.


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-28 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r68701768
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -1053,6 +1055,41 @@ object PruneFilters extends Rule[LogicalPlan] with 
PredicateHelper {
 }
 
 /**
+ * Collapse plans consisting all empty local relations generated by 
[[PruneFilters]].
+ * Note that the ObjectProducer/Consumer and direct aggregations are the 
exceptions.
+ * {{{
+ *   SELECT a, b FROM t WHERE 1=0 GROUP BY a, b ORDER BY a, b ==> empty 
result
+ *   SELECT SUM(a) FROM t WHERE 1=0 GROUP BY a HAVING COUNT(*)>1 ORDER BY 
a (Not optimized)
+ * }}}
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
+  private def isEmptyLocalRelation(plan: LogicalPlan): Boolean =
+plan.isInstanceOf[LocalRelation] && 
plan.asInstanceOf[LocalRelation].data.isEmpty
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case x if x.isInstanceOf[ObjectProducer] || 
x.isInstanceOf[ObjectConsumer] => x
+
+// Case 1: If groupingExpressions contains all aggregation 
expressions, the result is empty.
+case a @ Aggregate(ge, ae, child) if isEmptyLocalRelation(child) && 
ae.forall(ge.contains(_)) =>
+  LocalRelation(a.output, data = Seq.empty)
+
+// Case 2: General aggregations can generate non-empty results.
+case a: Aggregate => a
+
+// Case 3: The following non-leaf plans having only empty relations 
return empty results.
+case p: LogicalPlan if p.children.nonEmpty && 
p.children.forall(isEmptyLocalRelation) =>
+  p match {
+case _: Project | _: Generate | _: Filter | _: Sample | _: Join |
--- End diff --

actually for intersect you only need one child to be empty

for join if it is inner join you just need one child to be empty too


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-28 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r68701793
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlanSuite.scala
 ---
@@ -0,0 +1,133 @@
+/*
+ * 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.Row
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.plans.{LeftAnti, PlanTest}
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+
+class CollapseEmptyPlanSuite extends PlanTest {
--- End diff --

Ur, any other scenario except the existing followings?
- test("one non-empty local relation")
- test("one non-empty and one empty local relations")
- test("aggregating expressions on empty 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 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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-28 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r68701620
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseEmptyPlanSuite.scala
 ---
@@ -0,0 +1,133 @@
+/*
+ * 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.Row
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.plans.{LeftAnti, PlanTest}
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+
+class CollapseEmptyPlanSuite extends PlanTest {
--- End diff --

you should test something that shouldn't have been converted too


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-25 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r68495438
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -1053,6 +1055,34 @@ object PruneFilters extends Rule[LogicalPlan] with 
PredicateHelper {
 }
 
 /**
+ * Collapse plans consisting all empty local relations generated by 
[[PruneFilters]].
+ * Note that the ObjectProducer/Consumer and direct aggregations are the 
exceptions.
+ * {{{
+ *   SELECT a, b FROM t WHERE 1=0 GROUP BY a, b ORDER BY a, b ==> empty 
result
+ *   SELECT SUM(a) FROM t WHERE 1=0 GROUP BY a HAVING COUNT(*)>1 ORDER BY 
a (Not optimized)
+ * }}}
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
+  private def isEmptyLocalRelation(plan: LogicalPlan): Boolean =
+plan.isInstanceOf[LocalRelation] && 
plan.asInstanceOf[LocalRelation].data.isEmpty
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case x if x.isInstanceOf[ObjectProducer] || 
x.isInstanceOf[ObjectConsumer] => x
+
+// Case 1: If groupingExpressions contains all aggregation 
expressions, the result is empty.
+case a @ Aggregate(ge, ae, child) if isEmptyLocalRelation(child) && 
ae.forall(ge.contains(_)) =>
--- End diff --

Ur, at the first, I thought you meant line 1080 for `case p: LogicalPlan`,


https://github.com/apache/spark/pull/13906/files#diff-a636a87d8843eeccca90140be91d4fafR1080
 .

Did I understand your advice correctly?


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-25 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r68495422
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -1053,6 +1055,34 @@ object PruneFilters extends Rule[LogicalPlan] with 
PredicateHelper {
 }
 
 /**
+ * Collapse plans consisting all empty local relations generated by 
[[PruneFilters]].
+ * Note that the ObjectProducer/Consumer and direct aggregations are the 
exceptions.
+ * {{{
+ *   SELECT a, b FROM t WHERE 1=0 GROUP BY a, b ORDER BY a, b ==> empty 
result
+ *   SELECT SUM(a) FROM t WHERE 1=0 GROUP BY a HAVING COUNT(*)>1 ORDER BY 
a (Not optimized)
+ * }}}
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
+  private def isEmptyLocalRelation(plan: LogicalPlan): Boolean =
+plan.isInstanceOf[LocalRelation] && 
plan.asInstanceOf[LocalRelation].data.isEmpty
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case x if x.isInstanceOf[ObjectProducer] || 
x.isInstanceOf[ObjectConsumer] => x
+
+// Case 1: If groupingExpressions contains all aggregation 
expressions, the result is empty.
+case a @ Aggregate(ge, ae, child) if isEmptyLocalRelation(child) && 
ae.forall(ge.contains(_)) =>
--- End diff --

Thank you for review, @rxin .
I see. I will update this PR into whitelist approach.


---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-25 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/13906#discussion_r68495048
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -1053,6 +1055,34 @@ object PruneFilters extends Rule[LogicalPlan] with 
PredicateHelper {
 }
 
 /**
+ * Collapse plans consisting all empty local relations generated by 
[[PruneFilters]].
+ * Note that the ObjectProducer/Consumer and direct aggregations are the 
exceptions.
+ * {{{
+ *   SELECT a, b FROM t WHERE 1=0 GROUP BY a, b ORDER BY a, b ==> empty 
result
+ *   SELECT SUM(a) FROM t WHERE 1=0 GROUP BY a HAVING COUNT(*)>1 ORDER BY 
a (Not optimized)
+ * }}}
+ */
+object CollapseEmptyPlan extends Rule[LogicalPlan] with PredicateHelper {
+  private def isEmptyLocalRelation(plan: LogicalPlan): Boolean =
+plan.isInstanceOf[LocalRelation] && 
plan.asInstanceOf[LocalRelation].data.isEmpty
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case x if x.isInstanceOf[ObjectProducer] || 
x.isInstanceOf[ObjectConsumer] => x
+
+// Case 1: If groupingExpressions contains all aggregation 
expressions, the result is empty.
+case a @ Aggregate(ge, ae, child) if isEmptyLocalRelation(child) && 
ae.forall(ge.contains(_)) =>
--- End diff --

this kind of blacklisting approach is too risky -- if we were to introduce 
a new logical node in the future, most likely we will forget to update this 
rule.



---
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 #13906: [SPARK-16208][SQL] Add `CollapseEmptyPlan` optimi...

2016-06-25 Thread dongjoon-hyun
GitHub user dongjoon-hyun opened a pull request:

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

[SPARK-16208][SQL] Add `CollapseEmptyPlan` optimizer

## What changes were proposed in this pull request?

This PR adds a new logical optimizer, `CollapseEmptyPlan`, to collapse a 
logical plans consisting of only empty LocalRelations. The only exceptional 
logical plan is aggregation. For aggregation plan, only simple cases are 
consider for this optimization.

**Before**
```scala
scala> sql("select a from values (1,2) T(a,b) where 1=0 group by a,b having 
a>1 order by a,b").explain
== Physical Plan ==
*Project [a#11]
+- *Sort [a#11 ASC, b#12 ASC], true, 0
   +- Exchange rangepartitioning(a#11 ASC, b#12 ASC, 200)
  +- *HashAggregate(keys=[a#11, b#12], functions=[])
 +- Exchange hashpartitioning(a#11, b#12, 200)
+- *HashAggregate(keys=[a#11, b#12], functions=[])
   +- LocalTableScan , [a#11, b#12]
```

**After**
```scala
scala> sql("select a from values (1,2) T(a,b) where 1=0 group by a,b having 
a>1 order by a,b").explain
== Physical Plan ==
LocalTableScan , [a#0]
```

## How was this patch tested?

Pass the Jenkins tests (including a new testsuite).

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

$ git pull https://github.com/dongjoon-hyun/spark SPARK-16208

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

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


commit 7ddf449d39f22090bc8aa157fae12c79ba00928e
Author: Dongjoon Hyun 
Date:   2016-06-25T09:18:44Z

[SPARK-16208][SQL] Add `CollapseEmptyPlan` optimizer




---
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