[GitHub] [spark] wangyum commented on a change in pull request #31908: [SPARK-34808][SQL] Removes outer join if it only has DISTINCT on streamed side

2021-06-02 Thread GitBox


wangyum commented on a change in pull request #31908:
URL: https://github.com/apache/spark/pull/31908#discussion_r643781246



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala
##
@@ -165,6 +170,23 @@ object EliminateOuterJoin extends Rule[LogicalPlan] with 
PredicateHelper {
 case f @ Filter(condition, j @ Join(_, _, RightOuter | LeftOuter | 
FullOuter, _, _)) =>
   val newJoinType = buildNewJoinType(f, j)
   if (j.joinType == newJoinType) f else Filter(condition, j.copy(joinType 
= newJoinType))
+
+case a @ Aggregate(_, _, join @ Join(left, _, LeftOuter, _, _))
+if a.isDistinct && a.references.subsetOf(AttributeSet(left.output)) &&
+  !canPlanAsBroadcastHashJoin(join, conf) =>

Review comment:
   https://github.com/apache/spark/pull/32744




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] wangyum commented on a change in pull request #31908: [SPARK-34808][SQL] Removes outer join if it only has DISTINCT on streamed side

2021-06-01 Thread GitBox


wangyum commented on a change in pull request #31908:
URL: https://github.com/apache/spark/pull/31908#discussion_r643628381



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala
##
@@ -165,6 +170,23 @@ object EliminateOuterJoin extends Rule[LogicalPlan] with 
PredicateHelper {
 case f @ Filter(condition, j @ Join(_, _, RightOuter | LeftOuter | 
FullOuter, _, _)) =>
   val newJoinType = buildNewJoinType(f, j)
   if (j.joinType == newJoinType) f else Filter(condition, j.copy(joinType 
= newJoinType))
+
+case a @ Aggregate(_, _, join @ Join(left, _, LeftOuter, _, _))
+if a.isDistinct && a.references.subsetOf(AttributeSet(left.output)) &&
+  !canPlanAsBroadcastHashJoin(join, conf) =>

Review comment:
   The result may be incorrect if always remove the join. For example:
   ```
   0: jdbc:hive2://hdc49-mcc10-01-0510-2005-006-> create table test11.t1 using 
parquet as select id % 3 as a, id as b from range(10);
   +-+--+
   | Result  |
   +-+--+
   +-+--+
   No rows selected (1.611 seconds)
   0: jdbc:hive2://hdc49-mcc10-01-0510-2005-006-> create table test11.t2 using 
parquet as select id % 3 as x, id as y from range(5);
   +-+--+
   | Result  |
   +-+--+
   +-+--+
   No rows selected (1.043 seconds)
   0: jdbc:hive2://hdc49-mcc10-01-0510-2005-006-> select t1.a from t1 left join 
t2 on a = x;
   ++--+
   | a  |
   ++--+
   | 0  |
   | 0  |
   | 1  |
   | 1  |
   | 0  |
   | 0  |
   | 1  |
   | 1  |
   | 2  |
   | 0  |
   | 0  |
   | 1  |
   | 1  |
   | 2  |
   | 0  |
   | 0  |
   | 2  |
   ++--+
   17 rows selected (1.409 seconds)
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] wangyum commented on a change in pull request #31908: [SPARK-34808][SQL] Removes outer join if it only has DISTINCT on streamed side

2021-05-27 Thread GitBox


wangyum commented on a change in pull request #31908:
URL: https://github.com/apache/spark/pull/31908#discussion_r640469134



##
File path: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala
##
@@ -71,4 +74,65 @@ class AggregateOptimizeSuite extends AnalysisTest {
 
 comparePlans(optimized, correctAnswer)
   }
+
+  test("SPARK-34808: Remove left join if it only has distinct on left side") {
+val x = testRelation.subquery('x)
+val y = testRelation.subquery('y)
+val query = Distinct(x.join(y, LeftOuter, Some("x.a".attr === 
"y.a".attr)).select("x.b".attr))
+
+Seq(-1, 1).foreach { autoBroadcastJoinThreshold =>
+  withSQLConf(AUTO_BROADCASTJOIN_THRESHOLD.key -> 
s"$autoBroadcastJoinThreshold") {
+val correctAnswer = if (autoBroadcastJoinThreshold < 0) {
+  x.select("x.b".attr).groupBy("x.b".attr)("x.b".attr)
+} else {
+  Aggregate(query.child.output, query.child.output, query.child)
+}
+comparePlans(Optimize.execute(query.analyze), correctAnswer.analyze)
+  }
+}
+  }
+
+  test("SPARK-34808: Remove right join if it only has distinct on right side") 
{
+val x = testRelation.subquery('x)
+val y = testRelation.subquery('y)
+val query = Distinct(x.join(y, RightOuter, Some("x.a".attr === 
"y.a".attr)).select("y.b".attr))
+
+Seq(-1, 1).foreach { autoBroadcastJoinThreshold =>
+  withSQLConf(AUTO_BROADCASTJOIN_THRESHOLD.key -> 
s"$autoBroadcastJoinThreshold") {
+val correctAnswer = if (autoBroadcastJoinThreshold < 0) {
+  y.select("y.b".attr).groupBy("y.b".attr)("y.b".attr)
+} else {
+  Aggregate(query.child.output, query.child.output, query.child)
+}
+comparePlans(Optimize.execute(query.analyze), correctAnswer.analyze)
+  }
+}
+  }
+
+  test("SPARK-34808: Should not remove left join if select 2 join sides") {
+val x = testRelation.subquery('x)
+val y = testRelation.subquery('y)
+val query = Distinct(x.join(y, RightOuter, Some("x.a".attr === "y.a".attr))
+  .select("x.b".attr, "y.c".attr))
+
+Seq(-1, 1).foreach { autoBroadcastJoinThreshold =>
+  withSQLConf(AUTO_BROADCASTJOIN_THRESHOLD.key -> 
s"$autoBroadcastJoinThreshold") {
+val correctAnswer = Aggregate(query.child.output, query.child.output, 
query.child)
+comparePlans(Optimize.execute(query.analyze), correctAnswer.analyze)
+  }
+}
+  }
+
+  test("SPARK-34808: EliminateOuterJoin must before 
RemoveRepetitionFromGroupExpressions") {

Review comment:
   `RemoveRepetitionFromGroupExpressions` will remove repetition from group 
expressions:
   ```
   === Applying Rule 
org.apache.spark.sql.catalyst.optimizer.RemoveRepetitionFromGroupExpressions ===
   !Aggregate [a#2L, a#2L], [a#2L, a#2L] Aggregate [a#2L], 
[a#2L, a#2L]
+- Project [a#2L, a#2L]  +- Project [a#2L, a#2L]
   +- Join LeftOuter, (a#2L = b#6L) +- Join LeftOuter, 
(a#2L = b#6L)
  :- Project [id#0L AS a#2L]   :- Project 
[id#0L AS a#2L]
  :  +- Range (0, 200, step=1, splits=Some(2)) :  +- Range (0, 
200, step=1, splits=Some(2))
  +- Project [id#4L AS b#6L]   +- Project 
[id#4L AS b#6L]
 +- Range (0, 300, step=1, splits=Some(2))+- Range (0, 
300, step=1, splits=Some(2))
   ```
   
   We can remove this limitation if we refine  `isDistinct`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] wangyum commented on a change in pull request #31908: [SPARK-34808][SQL] Removes outer join if it only has DISTINCT on streamed side

2021-03-29 Thread GitBox


wangyum commented on a change in pull request #31908:
URL: https://github.com/apache/spark/pull/31908#discussion_r603281294



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveUnnecessaryOuterJoin.scala
##
@@ -0,0 +1,40 @@
+/*
+ * 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.AttributeSet
+import org.apache.spark.sql.catalyst.plans.{LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Join, 
LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * Removes outer join if it only has distinct on streamed side.
+ * {{{
+ *   SELECT DISTINCT f1 FROM t1 LEFT JOIN t2 ON t1.id = t2.id  ==>  SELECT 
DISTINCT f1 FROM t1
+ * }}}
+ */
+object RemoveUnnecessaryOuterJoin extends Rule[LogicalPlan] {

Review comment:
   Seems reasonable.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] wangyum commented on a change in pull request #31908: [SPARK-34808][SQL] Removes outer join if it only has DISTINCT on streamed side

2021-03-29 Thread GitBox


wangyum commented on a change in pull request #31908:
URL: https://github.com/apache/spark/pull/31908#discussion_r603280467



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateUnnecessaryOuterJoin.scala
##
@@ -0,0 +1,37 @@
+/*
+ * 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.AttributeSet
+import org.apache.spark.sql.catalyst.plans.{LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Join, 
LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * Removes outer join if it only has distinct on streamed side.
+ */
+object EliminateUnnecessaryOuterJoin extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+case a @ Aggregate(_, _, p @ Project(_, Join(left, _, LeftOuter, _, _)))

Review comment:
   done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] wangyum commented on a change in pull request #31908: [SPARK-34808][SQL] Removes outer join if it only has DISTINCT on streamed side

2021-03-22 Thread GitBox


wangyum commented on a change in pull request #31908:
URL: https://github.com/apache/spark/pull/31908#discussion_r598740161



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateUnnecessaryOuterJoin.scala
##
@@ -0,0 +1,37 @@
+/*
+ * 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.AttributeSet
+import org.apache.spark.sql.catalyst.plans.{LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Join, 
LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * Removes outer join if it only has distinct on streamed side.
+ */
+object EliminateUnnecessaryOuterJoin extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+case a @ Aggregate(_, _, p @ Project(_, Join(left, _, LeftOuter, _, _)))

Review comment:
   Because the supported case must the subset of the join's output(Either 
all on the left side or all on the right side):
   
![image](https://user-images.githubusercontent.com/5399861/112001045-871b5e00-8b59-11eb-9494-10a2cf035b0b.png)
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] wangyum commented on a change in pull request #31908: [SPARK-34808][SQL] Removes outer join if it only has DISTINCT on streamed side

2021-03-22 Thread GitBox


wangyum commented on a change in pull request #31908:
URL: https://github.com/apache/spark/pull/31908#discussion_r598682821



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
##
@@ -190,7 +190,8 @@ abstract class Optimizer(catalogManager: CatalogManager)
   ReplaceDeduplicateWithAggregate) ::
 Batch("Aggregate", fixedPoint,
   RemoveLiteralFromGroupExpressions,
-  RemoveRepetitionFromGroupExpressions) :: Nil ++
+  RemoveRepetitionFromGroupExpressions,
+  EliminateUnnecessaryOuterJoin) :: Nil ++

Review comment:
   ok




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] wangyum commented on a change in pull request #31908: [SPARK-34808][SQL] Removes outer join if it only has DISTINCT on streamed side

2021-03-22 Thread GitBox


wangyum commented on a change in pull request #31908:
URL: https://github.com/apache/spark/pull/31908#discussion_r598672944



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateUnnecessaryOuterJoin.scala
##
@@ -0,0 +1,37 @@
+/*
+ * 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.AttributeSet
+import org.apache.spark.sql.catalyst.plans.{LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Join, 
LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * Removes outer join if it only has distinct on streamed side.
+ */
+object EliminateUnnecessaryOuterJoin extends Rule[LogicalPlan] {

Review comment:
   No difference.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] wangyum commented on a change in pull request #31908: [SPARK-34808][SQL] Removes outer join if it only has DISTINCT on streamed side

2021-03-22 Thread GitBox


wangyum commented on a change in pull request #31908:
URL: https://github.com/apache/spark/pull/31908#discussion_r598570422



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateUnnecessaryOuterJoin.scala
##
@@ -0,0 +1,37 @@
+/*
+ * 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.AttributeSet
+import org.apache.spark.sql.catalyst.plans.{LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Join, 
LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * Removes outer join if it only has distinct on streamed side.
+ */
+object EliminateUnnecessaryOuterJoin extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+case a @ Aggregate(_, _, p @ Project(_, Join(left, _, LeftOuter, _, _)))

Review comment:
   It must has a `Project` between `Aggregate` and `Join` because their 
output is different if we can remove `Join`.
   
   
   For example, we can not remove Join for the following case:
   ```sql
   SELECT DISTINCT * FROM t1 LEFT JOIN t2 ON a = b
   
Aggregate [a#2L, b#6L], [a#2L, b#6L]  
+- Join LeftOuter, (a#2L = b#6L)  
   :- Project [id#0L AS a#2L] 
   :  +- Range (0, 200, step=1, splits=Some(2))   
   +- Project [id#4L AS b#6L] 
  +- Range (0, 300, step=1, splits=Some(2))   
   ```
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] wangyum commented on a change in pull request #31908: [SPARK-34808][SQL] Removes outer join if it only has DISTINCT on streamed side

2021-03-21 Thread GitBox


wangyum commented on a change in pull request #31908:
URL: https://github.com/apache/spark/pull/31908#discussion_r598285144



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
##
@@ -659,6 +659,12 @@ case class Aggregate(
 val nonAgg = 
aggregateExpressions.filter(_.find(_.isInstanceOf[AggregateExpression]).isEmpty)
 getAllValidConstraints(nonAgg)
   }
+
+  // Whether this Aggregate operator is equally the Distinct operator.
+  private[sql] def isEquallyDistinct: Boolean = {

Review comment:
   This method can be used elsewhere. For example:
   
   
https://github.com/apache/spark/blob/e226e2ceea7b648a1864c7ff1ba1e67eee83789b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala#L1809-L1813
 in this pr: https://github.com/apache/spark/pull/31113




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] wangyum commented on a change in pull request #31908: [SPARK-34808][SQL] Removes outer join if it only has DISTINCT on streamed side

2021-03-21 Thread GitBox


wangyum commented on a change in pull request #31908:
URL: https://github.com/apache/spark/pull/31908#discussion_r598285144



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
##
@@ -659,6 +659,12 @@ case class Aggregate(
 val nonAgg = 
aggregateExpressions.filter(_.find(_.isInstanceOf[AggregateExpression]).isEmpty)
 getAllValidConstraints(nonAgg)
   }
+
+  // Whether this Aggregate operator is equally the Distinct operator.
+  private[sql] def isEquallyDistinct: Boolean = {

Review comment:
   This method can be used elsewhere. For example:
   
   
https://github.com/apache/spark/blob/e226e2ceea7b648a1864c7ff1ba1e67eee83789b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala#L1809-L1813




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] wangyum commented on a change in pull request #31908: [SPARK-34808][SQL] Removes outer join if it only has DISTINCT on streamed side

2021-03-21 Thread GitBox


wangyum commented on a change in pull request #31908:
URL: https://github.com/apache/spark/pull/31908#discussion_r598283231



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveOuterJoin.scala
##
@@ -0,0 +1,37 @@
+/*
+ * 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.AttributeSet
+import org.apache.spark.sql.catalyst.plans.{LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Join, 
LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * Removes outer join if it only has distinct on streamed side.
+ */
+object RemoveOuterJoin extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+case a @ Aggregate(_, _, p @ Project(_, Join(left, _, LeftOuter, _, _)))

Review comment:
   For example, we can not remove Join for the following case:
   ```sql
   SELECT DISTINCT * FROM t1 LEFT JOIN t2 ON a = b
   
Aggregate [a#2L, b#6L], [a#2L, b#6L]  
+- Join LeftOuter, (a#2L = b#6L)  
   :- Project [id#0L AS a#2L] 
   :  +- Range (0, 200, step=1, splits=Some(2))   
   +- Project [id#4L AS b#6L] 
  +- Range (0, 300, step=1, splits=Some(2))   
   ```
   
   ```sql
   SELECT DISTINCT b FROM t1 LEFT JOIN t2 ON a = b
   
   Aggregate [b#6L], [b#6L] 
   +- Project [b#6L]
  +- Join LeftOuter, (a#2L = b#6L)  
 :- Project [id#0L AS a#2L] 
 :  +- Range (0, 200, step=1, splits=Some(2))   
 +- Project [id#4L AS b#6L] 
+- Range (0, 300, step=1, splits=Some(2))   
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] wangyum commented on a change in pull request #31908: [SPARK-34808][SQL] Removes outer join if it only has DISTINCT on streamed side

2021-03-21 Thread GitBox


wangyum commented on a change in pull request #31908:
URL: https://github.com/apache/spark/pull/31908#discussion_r598282486



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveOuterJoin.scala
##
@@ -0,0 +1,37 @@
+/*
+ * 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.AttributeSet
+import org.apache.spark.sql.catalyst.plans.{LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Join, 
LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * Removes outer join if it only has distinct on streamed side.
+ */
+object RemoveOuterJoin extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+case a @ Aggregate(_, _, p @ Project(_, Join(left, _, LeftOuter, _, _)))

Review comment:
   It must has a `Project` between `Aggregate` and `Join` because their 
output is different if we can remove `Join`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] wangyum commented on a change in pull request #31908: [SPARK-34808][SQL] Removes outer join if it only has DISTINCT on streamed side

2021-03-20 Thread GitBox


wangyum commented on a change in pull request #31908:
URL: https://github.com/apache/spark/pull/31908#discussion_r598206411



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
##
@@ -1998,6 +1999,20 @@ object RemoveRepetitionFromGroupExpressions extends 
Rule[LogicalPlan] {
   }
 }
 
+/**
+ * Removes outer join if it only has distinct on streamed side.
+ */
+object RemoveOuterJoin extends Rule[LogicalPlan] {

Review comment:
   Done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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