[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-22 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r204275071
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -175,6 +191,41 @@ abstract class Optimizer(sessionCatalog: 
SessionCatalog)
* Override to provide additional rules for the operator optimization 
batch.
*/
   def extendedOperatorOptimizationRules: Seq[Rule[LogicalPlan]] = Nil
+
+  override def batches: Seq[Batch] = {
+val excludedRulesConf =
+  
SQLConf.get.optimizerExcludedRules.toSeq.flatMap(_.split(",").map(_.trim).filter(_.nonEmpty))
--- End diff --

Any reason not to use `Utils.stringToSeq`?
https://github.com/apache/spark/pull/21764#discussion_r204202735


---

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



[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-22 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r204261317
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -160,6 +160,25 @@ abstract class Optimizer(sessionCatalog: 
SessionCatalog)
   UpdateNullabilityInAttributeReferences)
   }
 
+  def nonExcludableRules: Seq[String] =
+EliminateDistinct.ruleName ::
+  EliminateSubqueryAliases.ruleName ::
+  EliminateView.ruleName ::
+  ReplaceExpressions.ruleName ::
+  ComputeCurrentTime.ruleName ::
+  GetCurrentDatabase(sessionCatalog).ruleName ::
+  RewriteDistinctAggregates.ruleName ::
+  ReplaceDeduplicateWithAggregate.ruleName ::
+  ReplaceIntersectWithSemiJoin.ruleName ::
+  ReplaceExceptWithFilter.ruleName ::
+  ReplaceExceptWithAntiJoin.ruleName ::
+  ReplaceDistinctWithAggregate.ruleName ::
+  PullupCorrelatedPredicates.ruleName ::
+  RewritePredicateSubquery.ruleName ::
+  ColumnPruning.ruleName ::
+  CollapseProject.ruleName ::
+  RemoveRedundantProject.ruleName :: Nil
--- End diff --

remove the last three?


---

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



[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-20 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r204202735
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -175,6 +182,44 @@ abstract class Optimizer(sessionCatalog: 
SessionCatalog)
* Override to provide additional rules for the operator optimization 
batch.
*/
   def extendedOperatorOptimizationRules: Seq[Rule[LogicalPlan]] = Nil
+
+  override def batches: Seq[Batch] = {
+val excludedRules =
+  
SQLConf.get.optimizerExcludedRules.toSeq.flatMap(_.split(",").map(_.trim).filter(!_.isEmpty))
--- End diff --

You can use `Utils.stringToSeq`?


---

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



[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-19 Thread maryannxue
Github user maryannxue commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r203731087
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -127,6 +127,14 @@ object SQLConf {
 }
   }
 
+  val OPTIMIZER_EXCLUDED_RULES = 
buildConf("spark.sql.optimizer.excludedRules")
--- End diff --

+1 on debugging purpose. Still, CacheManager matches the *analyzed* plan 
not the optimized plan.


---

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



[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-19 Thread maryannxue
Github user maryannxue commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r203730778
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerRuleExclusionSuite.scala
 ---
@@ -0,0 +1,84 @@
+/*
+ * 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.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
+import org.apache.spark.sql.internal.SQLConf.OPTIMIZER_EXCLUDED_RULES
+
+
+class OptimizerRuleExclusionSuite extends PlanTest {
--- End diff --

Added :)


---

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



[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-19 Thread maryannxue
Github user maryannxue commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r203730652
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -160,6 +160,13 @@ abstract class Optimizer(sessionCatalog: 
SessionCatalog)
   UpdateNullabilityInAttributeReferences)
   }
 
+  def nonExcludableBatches: Seq[String] =
+"Eliminate Distinct" ::
+  "Finish Analysis" ::
+  "Replace Operators" ::
+  "Pullup Correlated Expressions" ::
+  "RewriteSubquery" :: Nil
--- End diff --

I'll change to rule black list.


---

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



[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-19 Thread maryannxue
Github user maryannxue commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r203730125
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -175,6 +182,44 @@ abstract class Optimizer(sessionCatalog: 
SessionCatalog)
* Override to provide additional rules for the operator optimization 
batch.
*/
   def extendedOperatorOptimizationRules: Seq[Rule[LogicalPlan]] = Nil
+
+  override def batches: Seq[Batch] = {
+val excludedRules =
+  
SQLConf.get.optimizerExcludedRules.toSeq.flatMap(_.split(",").map(_.trim).filter(!_.isEmpty))
--- End diff --

There is an auto-generated field `ruleName` in `Rule`, so we do exact name 
matching (case sensitive).


---

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



[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-18 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r203278576
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -175,6 +182,44 @@ abstract class Optimizer(sessionCatalog: 
SessionCatalog)
* Override to provide additional rules for the operator optimization 
batch.
*/
   def extendedOperatorOptimizationRules: Seq[Rule[LogicalPlan]] = Nil
+
+  override def batches: Seq[Batch] = {
+val excludedRules =
+  
SQLConf.get.optimizerExcludedRules.toSeq.flatMap(_.split(",").map(_.trim).filter(!_.isEmpty))
--- End diff --

Also, you need to handle case-sensitivity.


---

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



[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-18 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r203278443
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -175,6 +182,44 @@ abstract class Optimizer(sessionCatalog: 
SessionCatalog)
* Override to provide additional rules for the operator optimization 
batch.
*/
   def extendedOperatorOptimizationRules: Seq[Rule[LogicalPlan]] = Nil
+
+  override def batches: Seq[Batch] = {
+val excludedRules =
+  
SQLConf.get.optimizerExcludedRules.toSeq.flatMap(_.split(",").map(_.trim).filter(!_.isEmpty))
--- End diff --

nit: `!_.isEmpty` -> `_.nonEmpty`


---

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



[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-18 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r203277729
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -160,6 +160,13 @@ abstract class Optimizer(sessionCatalog: 
SessionCatalog)
   UpdateNullabilityInAttributeReferences)
   }
 
+  def nonExcludableBatches: Seq[String] =
+"Eliminate Distinct" ::
+  "Finish Analysis" ::
+  "Replace Operators" ::
+  "Pullup Correlated Expressions" ::
+  "RewriteSubquery" :: Nil
--- End diff --

We use not rule names but batch names in this black list?


---

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



[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-18 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r203270277
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -127,6 +127,14 @@ object SQLConf {
 }
   }
 
+  val OPTIMIZER_EXCLUDED_RULES = 
buildConf("spark.sql.optimizer.excludedRules")
--- End diff --

Since an optimizer should not change query semantics(results), it should 
work well for the case @dongjoon-hyun described. If this is mainly used for 
debugging uses, I think it would be nice to use this conf on runtime.


---

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



[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-17 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r202903084
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -46,7 +47,23 @@ abstract class Optimizer(sessionCatalog: SessionCatalog)
 
   protected def fixedPoint = FixedPoint(SQLConf.get.optimizerMaxIterations)
 
-  def batches: Seq[Batch] = {
+  protected def postAnalysisBatches: Seq[Batch] = {
+Batch("Eliminate Distinct", Once, EliminateDistinct) ::
+// Technically some of the rules in Finish Analysis are not optimizer 
rules and belong more
+// in the analyzer, because they are needed for correctness (e.g. 
ComputeCurrentTime).
+// However, because we also use the analyzer to canonicalized queries 
(for view definition),
+// we do not eliminate subqueries or compute current time in the 
analyzer.
+Batch("Finish Analysis", Once,
+  EliminateSubqueryAliases,
+  EliminateView,
+  ReplaceExpressions,
+  ComputeCurrentTime,
+  GetCurrentDatabase(sessionCatalog),
+  RewriteDistinctAggregates,
+  ReplaceDeduplicateWithAggregate) :: Nil
+  }
+
+  protected def optimizationBatches: Seq[Batch] = {
--- End diff --

yes. We need to exclude `Batch("Eliminate Distinct")`, `Batch("Finish 
Analysis")`, `Batch("Replace Operators")`, `Batch("Pullup Correlated 
Expressions")`, and ` Batch("RewriteSubquery")` 


---

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



[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-16 Thread maryannxue
Github user maryannxue commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r202786530
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -127,6 +127,14 @@ object SQLConf {
 }
   }
 
+  val OPTIMIZER_EXCLUDED_RULES = 
buildConf("spark.sql.optimizer.excludedRules")
--- End diff --

Are you talking about SQL cache? I don't think optimizer has anything to do 
with SQL cache though, since the logical plans used to match cache entries are 
"analyzed" plans not "optimized" plans.


---

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



[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-16 Thread maryannxue
Github user maryannxue commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r202762054
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -127,6 +127,14 @@ object SQLConf {
 }
   }
 
+  val OPTIMIZER_EXCLUDED_RULES = 
buildConf("spark.sql.optimizer.excludedRules")
+.doc("Configures a list of rules to be disabled in the optimizer, in 
which the rules are " +
+  "specified by their rule names and separated by comma. It is not 
guaranteed that all the " +
+  "rules in this configuration will eventually be excluded, as some 
rules are necessary " +
--- End diff --

Nice suggestion! @gatorsmile's other suggestion was to introduce a 
blacklist, in which case this enumeration of rules that cannot be excluded can 
be made possible. I can do a warning as well.


---

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



[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-16 Thread maryannxue
Github user maryannxue commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r202760924
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -175,6 +179,35 @@ abstract class Optimizer(sessionCatalog: 
SessionCatalog)
* Override to provide additional rules for the operator optimization 
batch.
*/
   def extendedOperatorOptimizationRules: Seq[Rule[LogicalPlan]] = Nil
+
+  override def batches: Seq[Batch] = {
+val excludedRules =
+  
SQLConf.get.optimizerExcludedRules.toSeq.flatMap(_.split(",").map(_.trim).filter(!_.isEmpty))
+val filteredOptimizationBatches = if (excludedRules.isEmpty) {
+  optimizationBatches
+} else {
+  optimizationBatches.flatMap { batch =>
+val filteredRules =
+  batch.rules.filter { rule =>
+val exclude = excludedRules.contains(rule.ruleName)
+if (exclude) {
+  logInfo(s"Optimization rule '${rule.ruleName}' is excluded 
from the optimizer.")
+}
+!exclude
+  }
+if (batch.rules == filteredRules) {
--- End diff --

It is to:
1) avoid unnecessary object creation if all rules have been preserved.
2) avoid empty batches if all rules in the batch have been removed.


---

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



[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-16 Thread maryannxue
Github user maryannxue commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r202759884
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -46,7 +47,23 @@ abstract class Optimizer(sessionCatalog: SessionCatalog)
 
   protected def fixedPoint = FixedPoint(SQLConf.get.optimizerMaxIterations)
 
-  def batches: Seq[Batch] = {
+  protected def postAnalysisBatches: Seq[Batch] = {
+Batch("Eliminate Distinct", Once, EliminateDistinct) ::
+// Technically some of the rules in Finish Analysis are not optimizer 
rules and belong more
+// in the analyzer, because they are needed for correctness (e.g. 
ComputeCurrentTime).
+// However, because we also use the analyzer to canonicalized queries 
(for view definition),
+// we do not eliminate subqueries or compute current time in the 
analyzer.
+Batch("Finish Analysis", Once,
+  EliminateSubqueryAliases,
+  EliminateView,
+  ReplaceExpressions,
+  ComputeCurrentTime,
+  GetCurrentDatabase(sessionCatalog),
+  RewriteDistinctAggregates,
+  ReplaceDeduplicateWithAggregate) :: Nil
+  }
+
+  protected def optimizationBatches: Seq[Batch] = {
--- End diff --

So can I do black list of batches?


---

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



[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-15 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r202556147
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -127,6 +127,14 @@ object SQLConf {
 }
   }
 
+  val OPTIMIZER_EXCLUDED_RULES = 
buildConf("spark.sql.optimizer.excludedRules")
--- End diff --

If we allow this here, this will affect Spark's caching/uncaching plans and 
tables inconsistently. For the purpose of this PR, `StaticSQLConf.scala` would 
be a perfect place for this.


---

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



[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-15 Thread dmateusp
Github user dmateusp commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r202539843
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerRuleExclusionSuite.scala
 ---
@@ -0,0 +1,84 @@
+/*
+ * 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.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
+import org.apache.spark.sql.internal.SQLConf.OPTIMIZER_EXCLUDED_RULES
+
+
+class OptimizerRuleExclusionSuite extends PlanTest {
--- End diff --

Any test case for when a required rule is being passed as a "to be 
excluded" rule ?


---

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



[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-15 Thread dmateusp
Github user dmateusp commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r202539342
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -175,6 +179,35 @@ abstract class Optimizer(sessionCatalog: 
SessionCatalog)
* Override to provide additional rules for the operator optimization 
batch.
*/
   def extendedOperatorOptimizationRules: Seq[Rule[LogicalPlan]] = Nil
+
+  override def batches: Seq[Batch] = {
+val excludedRules =
+  
SQLConf.get.optimizerExcludedRules.toSeq.flatMap(_.split(",").map(_.trim).filter(!_.isEmpty))
+val filteredOptimizationBatches = if (excludedRules.isEmpty) {
+  optimizationBatches
+} else {
+  optimizationBatches.flatMap { batch =>
+val filteredRules =
+  batch.rules.filter { rule =>
+val exclude = excludedRules.contains(rule.ruleName)
+if (exclude) {
+  logInfo(s"Optimization rule '${rule.ruleName}' is excluded 
from the optimizer.")
+}
+!exclude
+  }
+if (batch.rules == filteredRules) {
--- End diff --

My understanding is that it is written that way to allow for logging


---

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



[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-15 Thread dmateusp
Github user dmateusp commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r202539784
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -127,6 +127,14 @@ object SQLConf {
 }
   }
 
+  val OPTIMIZER_EXCLUDED_RULES = 
buildConf("spark.sql.optimizer.excludedRules")
+.doc("Configures a list of rules to be disabled in the optimizer, in 
which the rules are " +
+  "specified by their rule names and separated by comma. It is not 
guaranteed that all the " +
+  "rules in this configuration will eventually be excluded, as some 
rules are necessary " +
--- End diff --

I don't understand the optimizer at a low level (I'd be one of those users 
for which it is a blackbox), would you think it would be feasible to enumerate 
the rules that cannot be excluded ? Maybe even logging a WARNING when 
validating the config parameters if it contains required rules


---

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



[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-15 Thread dmateusp
Github user dmateusp commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r202538924
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -46,7 +47,23 @@ abstract class Optimizer(sessionCatalog: SessionCatalog)
 
   protected def fixedPoint = FixedPoint(SQLConf.get.optimizerMaxIterations)
 
-  def batches: Seq[Batch] = {
+  protected def postAnalysisBatches: Seq[Batch] = {
+Batch("Eliminate Distinct", Once, EliminateDistinct) ::
+// Technically some of the rules in Finish Analysis are not optimizer 
rules and belong more
+// in the analyzer, because they are needed for correctness (e.g. 
ComputeCurrentTime).
+// However, because we also use the analyzer to canonicalized queries 
(for view definition),
--- End diff --

"to canonicalized" -> "to canonicalize" ?


---

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



[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-14 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r202521545
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -175,6 +179,35 @@ abstract class Optimizer(sessionCatalog: 
SessionCatalog)
* Override to provide additional rules for the operator optimization 
batch.
*/
   def extendedOperatorOptimizationRules: Seq[Rule[LogicalPlan]] = Nil
+
+  override def batches: Seq[Batch] = {
+val excludedRules =
+  
SQLConf.get.optimizerExcludedRules.toSeq.flatMap(_.split(",").map(_.trim).filter(!_.isEmpty))
+val filteredOptimizationBatches = if (excludedRules.isEmpty) {
+  optimizationBatches
+} else {
+  optimizationBatches.flatMap { batch =>
+val filteredRules =
+  batch.rules.filter { rule =>
+val exclude = excludedRules.contains(rule.ruleName)
+if (exclude) {
+  logInfo(s"Optimization rule '${rule.ruleName}' is excluded 
from the optimizer.")
+}
+!exclude
+  }
+if (batch.rules == filteredRules) {
--- End diff --

Maybe the `if, else if and else` can be removed? Just return the filtered 
batch?


---

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



[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-14 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r202521451
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -46,7 +47,23 @@ abstract class Optimizer(sessionCatalog: SessionCatalog)
 
   protected def fixedPoint = FixedPoint(SQLConf.get.optimizerMaxIterations)
 
-  def batches: Seq[Batch] = {
+  protected def postAnalysisBatches: Seq[Batch] = {
+Batch("Eliminate Distinct", Once, EliminateDistinct) ::
+// Technically some of the rules in Finish Analysis are not optimizer 
rules and belong more
+// in the analyzer, because they are needed for correctness (e.g. 
ComputeCurrentTime).
+// However, because we also use the analyzer to canonicalized queries 
(for view definition),
+// we do not eliminate subqueries or compute current time in the 
analyzer.
+Batch("Finish Analysis", Once,
+  EliminateSubqueryAliases,
+  EliminateView,
+  ReplaceExpressions,
+  ComputeCurrentTime,
+  GetCurrentDatabase(sessionCatalog),
+  RewriteDistinctAggregates,
+  ReplaceDeduplicateWithAggregate) :: Nil
+  }
+
+  protected def optimizationBatches: Seq[Batch] = {
--- End diff --

In `optimizationBatches`, some rules can't be excluded. Without them, the 
affected queries can't be executed. For example,
```Scala
 Batch("Replace Operators", fixedPoint,
  ReplaceIntersectWithSemiJoin,
  ReplaceExceptWithFilter,
  ReplaceExceptWithAntiJoin,
  ReplaceDistinctWithAggregate)
```

 Can we just introduce a black list?



---

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



[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-13 Thread maryannxue
GitHub user maryannxue opened a pull request:

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

[SPARK-24802] Optimization Rule Exclusion

## What changes were proposed in this pull request?

Since Spark has provided fairly clear interfaces for adding user-defined 
optimization rules, it would be nice to have an easy-to-use interface for 
excluding an optimization rule from the Spark query optimizer as well.

This would make customizing Spark optimizer easier and sometimes could 
debugging issues too.

- Add a new config spark.sql.optimizer.excludedRules, with the value being 
a list of rule names separated by comma.
- Modify the current batches method to remove the excluded rules from the 
default batches. Log the rules that have been excluded.
- Split the existing default batches into "post-analysis batches" and 
"optimization batches" so that only rules in the "optimization batches" can be 
excluded.

## How was this patch tested?

Add a new test suite: OptimizerRuleExclusionSuite

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

$ git pull https://github.com/maryannxue/spark rule-exclusion

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

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


commit eaec2f5f2b4e3193de41655b84a1dc936b0e50a3
Author: maryannxue 
Date:   2018-07-13T21:32:01Z

[SPARK-24802] Optimization Rule Exclusion




---

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