You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/10/20 04:13:08 UTC

[GitHub] [spark] leanken opened a new pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

leanken opened a new pull request #30097:
URL: https://github.com/apache/spark/pull/30097


   ### What changes were proposed in this pull request?
   
   Since Issue [SPARK-33139](https://issues.apache.org/jira/browse/SPARK-33139) has been done, and SQLConf.get and SparkSession.active are more reliable. We are trying to refine the existing code usage of passing SQLConf and SparkSession into sub-class of Rule[QueryPlan].
   
   In this PR.
   
   * remove SQLConf from ctor-parameter of all sub-class of Rule[QueryPlan].
   * using SQLConf.get to replace the original SQLConf instance.
   * remove SparkSession from ctor-parameter of all sub-class of Rule[QueryPlan].
   * using SparkSession.active to replace the original SparkSession instance.
   
   ### Why are the changes needed?
   
   Code refine.
   
   
   ### Does this PR introduce any user-facing change?
   No.
   
   
   ### How was this patch tested?
   
   Existing test
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713472274


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] leanken commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r510548134



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
##########
@@ -28,7 +28,8 @@ import org.apache.spark.sql.types.{StructField, StructType}
 /**
  * An analyzer rule that replaces [[UnresolvedInlineTable]] with [[LocalRelation]].
  */
-case class ResolveInlineTables(conf: SQLConf) extends Rule[LogicalPlan] with CastSupport {
+object ResolveInlineTables extends Rule[LogicalPlan] with CastSupport {
+  override def conf: SQLConf = SQLConf.get

Review comment:
       If it's final and could not be override, and in its subclass the usage difference would be conf.XXX and SQLConf.get.XXX, I don't think we should bother to add such interface.  ^_^




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] maropu commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r509817059



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -988,11 +988,11 @@ class Analyzer(
       case view @ View(desc, _, child) if !child.resolved =>
         // Resolve all the UnresolvedRelations and Views in the child.
         val newChild = AnalysisContext.withAnalysisContext(desc.viewCatalogAndNamespace) {
-          if (AnalysisContext.get.nestedViewDepth > conf.maxNestedViewDepth) {
+          if (AnalysisContext.get.nestedViewDepth > SQLConf.get.maxNestedViewDepth) {
             view.failAnalysis(s"The depth of view ${desc.identifier} exceeds the maximum " +
-              s"view resolution depth (${conf.maxNestedViewDepth}). Analysis is aborted to " +
-              s"avoid errors. Increase the value of ${SQLConf.MAX_NESTED_VIEW_DEPTH.key} to work " +
-              "around this.")
+              s"view resolution depth (${SQLConf.get.maxNestedViewDepth}). " +
+              s"Analysis is aborted to avoid errors. " +

Review comment:
       nit: remove `s`

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -182,7 +182,7 @@ class Analyzer(
 

Review comment:
       Could we remove `conf: SQLConf` in the constructor of `Analyzer`?

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
##########
@@ -28,7 +28,8 @@ import org.apache.spark.sql.types.{StructField, StructType}
 /**
  * An analyzer rule that replaces [[UnresolvedInlineTable]] with [[LocalRelation]].
  */
-case class ResolveInlineTables(conf: SQLConf) extends Rule[LogicalPlan] with CastSupport {
+object ResolveInlineTables extends Rule[LogicalPlan] with CastSupport {
+  override def conf: SQLConf = SQLConf.get

Review comment:
       Instead, could we define `final def conf: SQLConf = SQLConf.get` in the base class `Rule` if this is a common pattern?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713481350


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34697/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713665217


   **[Test build #130097 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130097/testReport)** for PR 30097 at commit [`83c1577`](https://github.com/apache/spark/commit/83c1577d5ab3b6fd24d0409bcbd6a8df591da96b).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r511779442



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
##########
@@ -771,22 +771,23 @@ class AnalysisSuite extends AnalysisTest with Matchers {
     // RuleExecutor only throw exception or log warning when the rule is supposed to run
     // more than once.
     val maxIterations = 2
-    val conf = new SQLConf().copy(SQLConf.ANALYZER_MAX_ITERATIONS -> maxIterations)

Review comment:
       do we have to update this test?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-717075840


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/34923/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r511786254



##########
File path: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala
##########
@@ -29,22 +29,9 @@ import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.internal.SQLConf
 
-/**
- * Prune hive table partitions using partition filters on [[HiveTableRelation]]. The pruned
- * partitions will be kept in [[HiveTableRelation.prunedPartitions]], and the statistics of
- * the hive table relation will be updated based on pruned partitions.
- *
- * This rule is executed in optimization phase, so the statistics can be updated before physical
- * planning, which is useful for some spark strategy, eg.
- * [[org.apache.spark.sql.execution.SparkStrategies.JoinSelection]].
- *
- * TODO: merge this with PruneFileSourcePartitions after we completely make hive as a data source.
- */
-private[sql] class PruneHiveTablePartitions(session: SparkSession)

Review comment:
       we can replace [[]] with backticks
   ```
   ... using partition filters on `HiveTableRelation`.
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-717181173






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r521932049



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala
##########
@@ -73,9 +73,9 @@ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] {
 /**
  * Preprocess [[CreateTable]], to do some normalization and checking.
  */
-case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[LogicalPlan] {
+object PreprocessTableCreation extends Rule[LogicalPlan] {

Review comment:
       ditto




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713359636






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715879838






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715889562






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716601776


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34890/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715882824






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-712602038


   **[Test build #130031 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130031/testReport)** for PR 30097 at commit [`4792ebd`](https://github.com/apache/spark/commit/4792ebd71d20739ec8345ba2edd0e8d7b28dfd48).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `  class RemoveAllHints extends Rule[LogicalPlan] `
     * `  class DisableHints extends RemoveAllHints `


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] viirya commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r511693300



##########
File path: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala
##########
@@ -29,22 +29,9 @@ import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.internal.SQLConf
 
-/**
- * Prune hive table partitions using partition filters on [[HiveTableRelation]]. The pruned
- * partitions will be kept in [[HiveTableRelation.prunedPartitions]], and the statistics of
- * the hive table relation will be updated based on pruned partitions.
- *
- * This rule is executed in optimization phase, so the statistics can be updated before physical
- * planning, which is useful for some spark strategy, eg.
- * [[org.apache.spark.sql.execution.SparkStrategies.JoinSelection]].
- *
- * TODO: merge this with PruneFileSourcePartitions after we completely make hive as a data source.
- */
-private[sql] class PruneHiveTablePartitions(session: SparkSession)

Review comment:
       Why remove these comments?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715895020






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-712578947


   **[Test build #130031 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130031/testReport)** for PR 30097 at commit [`4792ebd`](https://github.com/apache/spark/commit/4792ebd71d20739ec8345ba2edd0e8d7b28dfd48).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713585191






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713733065


   **[Test build #130097 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130097/testReport)** for PR 30097 at commit [`83c1577`](https://github.com/apache/spark/commit/83c1577d5ab3b6fd24d0409bcbd6a8df591da96b).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713709594






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713328281


   **[Test build #130073 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130073/testReport)** for PR 30097 at commit [`c681752`](https://github.com/apache/spark/commit/c681752d8f0fec5a318126026e27ca2747b114d9).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-712578947


   **[Test build #130031 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130031/testReport)** for PR 30097 at commit [`4792ebd`](https://github.com/apache/spark/commit/4792ebd71d20739ec8345ba2edd0e8d7b28dfd48).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-712774879






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] leanken commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r511733414



##########
File path: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala
##########
@@ -29,22 +29,9 @@ import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.internal.SQLConf
 
-/**
- * Prune hive table partitions using partition filters on [[HiveTableRelation]]. The pruned
- * partitions will be kept in [[HiveTableRelation.prunedPartitions]], and the statistics of
- * the hive table relation will be updated based on pruned partitions.
- *
- * This rule is executed in optimization phase, so the statistics can be updated before physical
- * planning, which is useful for some spark strategy, eg.
- * [[org.apache.spark.sql.execution.SparkStrategies.JoinSelection]].
- *
- * TODO: merge this with PruneFileSourcePartitions after we completely make hive as a data source.
- */
-private[sql] class PruneHiveTablePartitions(session: SparkSession)

Review comment:
       So, how do I fix this issue?
   1. remove the comment
   2. remove the comment with [[]] link
   3. add a empty class definition

##########
File path: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala
##########
@@ -29,22 +29,9 @@ import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.internal.SQLConf
 
-/**
- * Prune hive table partitions using partition filters on [[HiveTableRelation]]. The pruned
- * partitions will be kept in [[HiveTableRelation.prunedPartitions]], and the statistics of
- * the hive table relation will be updated based on pruned partitions.
- *
- * This rule is executed in optimization phase, so the statistics can be updated before physical
- * planning, which is useful for some spark strategy, eg.
- * [[org.apache.spark.sql.execution.SparkStrategies.JoinSelection]].
- *
- * TODO: merge this with PruneFileSourcePartitions after we completely make hive as a data source.
- */
-private[sql] class PruneHiveTablePartitions(session: SparkSession)

Review comment:
       So, how do I fix this issue?
   1. remove the comment
   2. remove the comment with [[]] link
   3. add an empty class definition




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715889557


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34833/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] leanken commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r511710003



##########
File path: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala
##########
@@ -29,22 +29,9 @@ import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.internal.SQLConf
 
-/**
- * Prune hive table partitions using partition filters on [[HiveTableRelation]]. The pruned
- * partitions will be kept in [[HiveTableRelation.prunedPartitions]], and the statistics of
- * the hive table relation will be updated based on pruned partitions.
- *
- * This rule is executed in optimization phase, so the statistics can be updated before physical
- * planning, which is useful for some spark strategy, eg.
- * [[org.apache.spark.sql.execution.SparkStrategies.JoinSelection]].
- *
- * TODO: merge this with PruneFileSourcePartitions after we completely make hive as a data source.
- */
-private[sql] class PruneHiveTablePartitions(session: SparkSession)

Review comment:
       I will upload the error log later to see if you can help me on this issue.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-712711912


   **[Test build #130041 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130041/testReport)** for PR 30097 at commit [`90644da`](https://github.com/apache/spark/commit/90644da25e8ac0efda1fa4c7b829c865cb9ef6ac).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713455425


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34693/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713709576


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34706/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-714170190


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716958410


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34913/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716669794


   **[Test build #130283 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130283/testReport)** for PR 30097 at commit [`f4663d8`](https://github.com/apache/spark/commit/f4663d849aca5a9f540aab578452aaa137e3f673).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713493015


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34698/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713509030


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34698/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713692244


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34706/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713328281


   **[Test build #130073 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130073/testReport)** for PR 30097 at commit [`c681752`](https://github.com/apache/spark/commit/c681752d8f0fec5a318126026e27ca2747b114d9).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-717036263


   **[Test build #130321 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130321/testReport)** for PR 30097 at commit [`7612695`](https://github.com/apache/spark/commit/7612695c78456155a95ad4f7d54ef70e53f88921).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r511780792



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinalsSuite.scala
##########
@@ -36,31 +37,40 @@ class SubstituteUnresolvedOrdinalsSuite extends AnalysisTest {
     // Tests order by ordinal, apply single rule.
     val plan = testRelation2.orderBy(Literal(1).asc, Literal(2).asc)
     comparePlans(
-      new SubstituteUnresolvedOrdinals(conf).apply(plan),
+      SubstituteUnresolvedOrdinals.apply(plan),
       testRelation2.orderBy(UnresolvedOrdinal(1).asc, UnresolvedOrdinal(2).asc))
 
     // Tests order by ordinal, do full analysis
     checkAnalysis(plan, testRelation2.orderBy(a.asc, b.asc))
 
     // order by ordinal can be turned off by config
+    var newPlan: LogicalPlan = null
+    withSQLConf(SQLConf.ORDER_BY_ORDINAL.key -> "false") {
+      newPlan = SubstituteUnresolvedOrdinals.apply(plan)
+    }
+
     comparePlans(
-      new SubstituteUnresolvedOrdinals(conf.copy(SQLConf.ORDER_BY_ORDINAL -> false)).apply(plan),
+      newPlan,
       testRelation2.orderBy(Literal(1).asc, Literal(2).asc))
   }
 
   test("group by ordinal") {
     // Tests group by ordinal, apply single rule.
     val plan2 = testRelation2.groupBy(Literal(1), Literal(2))('a, 'b)
     comparePlans(
-      new SubstituteUnresolvedOrdinals(conf).apply(plan2),
+      SubstituteUnresolvedOrdinals.apply(plan2),
       testRelation2.groupBy(UnresolvedOrdinal(1), UnresolvedOrdinal(2))('a, 'b))
 
     // Tests group by ordinal, do full analysis
     checkAnalysis(plan2, testRelation2.groupBy(a, b)(a, b))
 
     // group by ordinal can be turned off by config
+    var newPlan2: LogicalPlan = null
+    withSQLConf(SQLConf.GROUP_BY_ORDINAL.key -> "false") {
+      newPlan2 = SubstituteUnresolvedOrdinals.apply(plan2)
+    }
     comparePlans(
-      new SubstituteUnresolvedOrdinals(conf.copy(SQLConf.GROUP_BY_ORDINAL -> false)).apply(plan2),

Review comment:
       ditto




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713359636






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r511786935



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
##########
@@ -53,16 +53,18 @@ trait AnalysisTest extends PlanTest {
   }
 
   protected def getAnalyzer(caseSensitive: Boolean) = {
-    if (caseSensitive) caseSensitiveAnalyzer else caseInsensitiveAnalyzer
+    makeAnalyzer(caseSensitive)
   }
 
   protected def checkAnalysis(
       inputPlan: LogicalPlan,
       expectedPlan: LogicalPlan,
       caseSensitive: Boolean = true): Unit = {
-    val analyzer = getAnalyzer(caseSensitive)
-    val actualPlan = analyzer.executeAndCheck(inputPlan, new QueryPlanningTracker)
-    comparePlans(actualPlan, expectedPlan)
+    withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) {

Review comment:
       let's not change the test framework if not necessary. It's OK to test with 2 analyzer instance.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r511781964



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala
##########
@@ -439,7 +438,7 @@ case class RowToColumnarExec(child: SparkPlan) extends RowToColumnarTransition {
   )
 
   override def doExecuteColumnar(): RDD[ColumnarBatch] = {
-    val enableOffHeapColumnVector = sqlContext.conf.offHeapColumnVectorEnabled

Review comment:
       The previous code is better, as `sqlContext` is kind of "materialized", which is the one that was used to create the plan node. `conf` always gets the active one.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715880935


   **[Test build #130233 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130233/testReport)** for PR 30097 at commit [`7fc0716`](https://github.com/apache/spark/commit/7fc07165277fa9aa10dfec07c4de9ac360d20d87).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713464978


   **[Test build #130089 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130089/testReport)** for PR 30097 at commit [`a4822fd`](https://github.com/apache/spark/commit/a4822fd3b797e980ef8d99912e71cb9453d7ba06).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r512004978



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala
##########
@@ -35,6 +35,7 @@ class DataSourceAnalysisSuite extends SparkFunSuite with BeforeAndAfterAll {
 
   override def beforeAll(): Unit = {
     super.beforeAll()
+    SQLConf.get.unsetConf(SQLConf.CASE_SENSITIVE)

Review comment:
       we should do it in `aferAll`, to avoid this test suite changing the global conf object.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-714170202


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/34721/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715880935


   **[Test build #130233 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130233/testReport)** for PR 30097 at commit [`7fc0716`](https://github.com/apache/spark/commit/7fc07165277fa9aa10dfec07c4de9ac360d20d87).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r511780716



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinalsSuite.scala
##########
@@ -36,31 +37,40 @@ class SubstituteUnresolvedOrdinalsSuite extends AnalysisTest {
     // Tests order by ordinal, apply single rule.
     val plan = testRelation2.orderBy(Literal(1).asc, Literal(2).asc)
     comparePlans(
-      new SubstituteUnresolvedOrdinals(conf).apply(plan),
+      SubstituteUnresolvedOrdinals.apply(plan),
       testRelation2.orderBy(UnresolvedOrdinal(1).asc, UnresolvedOrdinal(2).asc))
 
     // Tests order by ordinal, do full analysis
     checkAnalysis(plan, testRelation2.orderBy(a.asc, b.asc))
 
     // order by ordinal can be turned off by config
+    var newPlan: LogicalPlan = null
+    withSQLConf(SQLConf.ORDER_BY_ORDINAL.key -> "false") {

Review comment:
       nit:
   ```
   withSQLConf(SQLConf.ORDER_BY_ORDINAL.key -> "false") {
     comparePlans(SubstituteUnresolvedOrdinals.apply(plan), ...)
   }
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715879841


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/34832/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713492249


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34697/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] leanken commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r511708600



##########
File path: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala
##########
@@ -29,22 +29,9 @@ import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.internal.SQLConf
 
-/**
- * Prune hive table partitions using partition filters on [[HiveTableRelation]]. The pruned
- * partitions will be kept in [[HiveTableRelation.prunedPartitions]], and the statistics of
- * the hive table relation will be updated based on pruned partitions.
- *
- * This rule is executed in optimization phase, so the statistics can be updated before physical
- * planning, which is useful for some spark strategy, eg.
- * [[org.apache.spark.sql.execution.SparkStrategies.JoinSelection]].
- *
- * TODO: merge this with PruneFileSourcePartitions after we completely make hive as a data source.
- */
-private[sql] class PruneHiveTablePartitions(session: SparkSession)

Review comment:
        -Phadoop-3.2 -Phive-2.3 -Phadoop-cloud -Pkinesis-asl -Pspark-ganglia-lgpl -Pyarn -Pkubernetes -Phive -Pmesos -Phive-thriftserver unidoc




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-717216009


   thanks, merging to master!


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-712711912


   **[Test build #130041 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130041/testReport)** for PR 30097 at commit [`90644da`](https://github.com/apache/spark/commit/90644da25e8ac0efda1fa4c7b829c865cb9ef6ac).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-712602328


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716562140


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34885/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r512000955



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala
##########
@@ -32,7 +32,8 @@ import org.apache.spark.sql.internal.SQLConf.{CASE_SENSITIVE, ORDER_BY_ORDINAL}
 import org.apache.spark.sql.types.IntegerType
 
 class EliminateSortsSuite extends PlanTest {
-  override val conf = new SQLConf().copy(CASE_SENSITIVE -> true, ORDER_BY_ORDINAL -> false)
+  SQLConf.get.setConf(CASE_SENSITIVE, true)
+  SQLConf.get.setConf(ORDER_BY_ORDINAL, false)

Review comment:
       ditto




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r521933186



##########
File path: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
##########
@@ -41,7 +41,7 @@ import org.apache.spark.sql.internal.{HiveSerDe, SQLConf}
  * Determine the database, serde/format and schema of the Hive serde table, according to the storage
  * properties.
  */
-class ResolveHiveSerdeTable(session: SparkSession) extends Rule[LogicalPlan] {
+object ResolveHiveSerdeTable extends Rule[LogicalPlan] {

Review comment:
       ditto, this access the catalog: `val dbName = t.identifier.database.getOrElse(session.catalog.currentDatabase)`




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r521933657



##########
File path: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala
##########
@@ -40,10 +39,10 @@ import org.apache.spark.sql.internal.SQLConf
  *
  * TODO: merge this with PruneFileSourcePartitions after we completely make hive as a data source.
  */
-private[sql] class PruneHiveTablePartitions(session: SparkSession)
-  extends Rule[LogicalPlan] with CastSupport with PredicateHelper {
+private[sql] class PruneHiveTablePartitions
 
-  override val conf: SQLConf = session.sessionState.conf
+private[sql] object PruneHiveTablePartitions

Review comment:
       ditto




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713558537






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713434115






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716037905






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715882824


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713430370


   **[Test build #130084 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130084/testReport)** for PR 30097 at commit [`874f7f6`](https://github.com/apache/spark/commit/874f7f6deae02bda0312e02106a31a044d346091).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715940709






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716490495


   **[Test build #130283 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130283/testReport)** for PR 30097 at commit [`f4663d8`](https://github.com/apache/spark/commit/f4663d849aca5a9f540aab578452aaa137e3f673).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713548511






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715879029


   **[Test build #130232 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130232/testReport)** for PR 30097 at commit [`c46be89`](https://github.com/apache/spark/commit/c46be89d8bc9185270a6241312dd043ed910526d).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-717075832


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713492268






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-714244069






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] leanken commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r511733174



##########
File path: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala
##########
@@ -29,22 +29,9 @@ import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.internal.SQLConf
 
-/**
- * Prune hive table partitions using partition filters on [[HiveTableRelation]]. The pruned
- * partitions will be kept in [[HiveTableRelation.prunedPartitions]], and the statistics of
- * the hive table relation will be updated based on pruned partitions.
- *
- * This rule is executed in optimization phase, so the statistics can be updated before physical
- * planning, which is useful for some spark strategy, eg.
- * [[org.apache.spark.sql.execution.SparkStrategies.JoinSelection]].
- *
- * TODO: merge this with PruneFileSourcePartitions after we completely make hive as a data source.
- */
-private[sql] class PruneHiveTablePartitions(session: SparkSession)

Review comment:
       I was trying to add a empty class definition before object, the unidoc build passed.
   
   ```
   private[sql] class PruneHiveTablePartitions
   
   /**
    * Prune hive table partitions using partition filters on [[HiveTableRelation]]. The pruned
    * partitions will be kept in [[HiveTableRelation.prunedPartitions]], and the statistics of
    * the hive table relation will be updated based on pruned partitions.
    *
    * This rule is executed in optimization phase, so the statistics can be updated before physical
    * planning, which is useful for some spark strategy, eg.
    * [[org.apache.spark.sql.execution.SparkStrategies.JoinSelection]].
    *
    * TODO: merge this with PruneFileSourcePartitions after we completely make hive as a data source.
    */
   private[sql] object PruneHiveTablePartitions
     extends Rule[LogicalPlan] with CastSupport with PredicateHelper {
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-712598500


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34637/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715918066






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715879029


   **[Test build #130232 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130232/testReport)** for PR 30097 at commit [`c46be89`](https://github.com/apache/spark/commit/c46be89d8bc9185270a6241312dd043ed910526d).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713464978


   **[Test build #130089 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130089/testReport)** for PR 30097 at commit [`a4822fd`](https://github.com/apache/spark/commit/a4822fd3b797e980ef8d99912e71cb9453d7ba06).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715879580


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715940731


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/34835/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715922894


   **[Test build #130235 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130235/testReport)** for PR 30097 at commit [`2b263f3`](https://github.com/apache/spark/commit/2b263f33d206eaa47ba39739a5e0f3be49e6e1d2).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713434088


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34690/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-714133475


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34721/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713350867


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/130073/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-717181173






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r521931459



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
##########
@@ -243,16 +243,16 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast
  * TODO: we should remove the special handling for hive tables after completely making hive as a
  * data source.
  */
-class FindDataSourceTable(sparkSession: SparkSession) extends Rule[LogicalPlan] {

Review comment:
       I think this rule should still keep the session parameter. The rule should only be dynamic about getting configs, but the session here is used to get the catalog and pass to DS V1.
   
   In practice, this doesn't matter as we always run rules with setting the desired active session. But it's better to be consistent. There are other rules taking catalog as a parameter, e.g. `RelationConversions`




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716966266






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713492268






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713502554


   **[Test build #130088 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130088/testReport)** for PR 30097 at commit [`5872a20`](https://github.com/apache/spark/commit/5872a20de5321264496d4ee8c46bdaf86886c06d).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716945800


   **[Test build #130311 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130311/testReport)** for PR 30097 at commit [`f8e103b`](https://github.com/apache/spark/commit/f8e103bcdef514c27627daea59bfafd00f22dcd9).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r510671693



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -182,7 +182,7 @@ class Analyzer(
 

Review comment:
       I think it's fine to keep it. The main goal is to let rules use `SQLConf.get`

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -182,7 +182,7 @@ class Analyzer(
 
   private def executeSameContext(plan: LogicalPlan): LogicalPlan = super.execute(plan)
 
-  def resolver: Resolver = conf.resolver
+  def resolver: Resolver = SQLConf.get.resolver

Review comment:
       unnecessary change.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716743114


   **[Test build #130289 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130289/testReport)** for PR 30097 at commit [`cd3329f`](https://github.com/apache/spark/commit/cd3329fadfb7bfcfae97b945d4ac6b8b346d38db).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713536246


   **[Test build #130094 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130094/testReport)** for PR 30097 at commit [`47526de`](https://github.com/apache/spark/commit/47526de256a1e7c52e80d0841869d89d63729499).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713503035


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/130088/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715918066






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r512008622



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala
##########
@@ -196,6 +202,7 @@ class DataSourceAnalysisSuite extends SparkFunSuite with BeforeAndAfterAll {
     }
 
     test(s"Static partition and dynamic partition (caseSensitive: $caseSensitive)") {
+      SQLConf.get.setConf(SQLConf.CASE_SENSITIVE, caseSensitive)

Review comment:
       Then we don't need to change `beforeAll/afterAll`, and it's one line diff for each test.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-717036263


   **[Test build #130321 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130321/testReport)** for PR 30097 at commit [`7612695`](https://github.com/apache/spark/commit/7612695c78456155a95ad4f7d54ef70e53f88921).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715882828


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/130233/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716575599






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715895016


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34834/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713566037


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34703/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715940709


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716037685


   **[Test build #130235 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130235/testReport)** for PR 30097 at commit [`2b263f3`](https://github.com/apache/spark/commit/2b263f33d206eaa47ba39739a5e0f3be49e6e1d2).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715886332


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34833/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713509056






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716538184


   **[Test build #130289 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130289/testReport)** for PR 30097 at commit [`cd3329f`](https://github.com/apache/spark/commit/cd3329fadfb7bfcfae97b945d4ac6b8b346d38db).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-714073513


   **[Test build #130112 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130112/testReport)** for PR 30097 at commit [`6e521aa`](https://github.com/apache/spark/commit/6e521aaf352fb11267f5ee04f05bee7580de132a).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715917808


   **[Test build #130234 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130234/testReport)** for PR 30097 at commit [`066a141`](https://github.com/apache/spark/commit/066a14122481902e09b3a83e5976323b2855479f).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan closed pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #30097:
URL: https://github.com/apache/spark/pull/30097


   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r512008197



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala
##########
@@ -196,6 +202,7 @@ class DataSourceAnalysisSuite extends SparkFunSuite with BeforeAndAfterAll {
     }
 
     test(s"Static partition and dynamic partition (caseSensitive: $caseSensitive)") {
+      SQLConf.get.setConf(SQLConf.CASE_SENSITIVE, caseSensitive)

Review comment:
       I have a better idea. We can create a `test` method in this suite:
   ```
   def test(caseSensitive: Boolean, testName: String)(func: => Unit) = {
     test(s"$testName (caseSensitive: $caseSensitive)") {
       withSQLConf(case sen....) {
         func
       }
     }
   }
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713350861






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713430370


   **[Test build #130084 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130084/testReport)** for PR 30097 at commit [`874f7f6`](https://github.com/apache/spark/commit/874f7f6deae02bda0312e02106a31a044d346091).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-712738785


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34648/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-712774688


   **[Test build #130041 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130041/testReport)** for PR 30097 at commit [`90644da`](https://github.com/apache/spark/commit/90644da25e8ac0efda1fa4c7b829c865cb9ef6ac).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713472283


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/34693/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713388472






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-717034660


   **[Test build #130311 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130311/testReport)** for PR 30097 at commit [`f8e103b`](https://github.com/apache/spark/commit/f8e103bcdef514c27627daea59bfafd00f22dcd9).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] leanken commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r510549621



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -182,7 +182,7 @@ class Analyzer(
 

Review comment:
       > Could we remove `conf: SQLConf` in the constructor of `Analyzer`?
   
   My original idea is to remove this parameter, but it's causing much more changing on the Test code. @cloud-fan , what's your thought?
   
   
   
   




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713472249


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34693/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] leanken commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r511784684



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
##########
@@ -53,16 +53,18 @@ trait AnalysisTest extends PlanTest {
   }
 
   protected def getAnalyzer(caseSensitive: Boolean) = {
-    if (caseSensitive) caseSensitiveAnalyzer else caseInsensitiveAnalyzer
+    makeAnalyzer(caseSensitive)

Review comment:
       yes, it's duplicated, since caseSensitiveAnalyzer and caseInsensitiveAnalyzer are already change from val to def. will revert.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] leanken commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r511785233



##########
File path: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala
##########
@@ -29,22 +29,9 @@ import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.internal.SQLConf
 
-/**
- * Prune hive table partitions using partition filters on [[HiveTableRelation]]. The pruned
- * partitions will be kept in [[HiveTableRelation.prunedPartitions]], and the statistics of
- * the hive table relation will be updated based on pruned partitions.
- *
- * This rule is executed in optimization phase, so the statistics can be updated before physical
- * planning, which is useful for some spark strategy, eg.
- * [[org.apache.spark.sql.execution.SparkStrategies.JoinSelection]].
- *
- * TODO: merge this with PruneFileSourcePartitions after we completely make hive as a data source.
- */
-private[sql] class PruneHiveTablePartitions(session: SparkSession)

Review comment:
       @cloud-fan FYI. need your suggestion too.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-712751582






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713434115






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-712751559


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34648/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-712598524


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/34637/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713472274






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r511778077



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -255,19 +255,19 @@ class Analyzer(
       GlobalAggregates ::
       ResolveAggregateFunctions ::
       TimeWindowing ::
-      ResolveInlineTables(conf) ::
+      ResolveInlineTables ::
       ResolveHigherOrderFunctions(v1SessionCatalog) ::
-      ResolveLambdaVariables(conf) ::
-      ResolveTimeZone(conf) ::
+      ResolveLambdaVariables ::
+      ResolveTimeZone ::
       ResolveRandomSeed ::
       ResolveBinaryArithmetic ::
       ResolveUnion ::
-      TypeCoercion.typeCoercionRules(conf) ++
+      TypeCoercion.typeCoercionRules ++
       extendedResolutionRules : _*),
     Batch("Post-Hoc Resolution", Once, postHocResolutionRules: _*),
     Batch("Normalize Alter Table", Once, ResolveAlterTableChanges),
     Batch("Remove Unresolved Hints", Once,
-      new ResolveHints.RemoveAllHints(conf)),
+      new ResolveHints.RemoveAllHints),

Review comment:
       can this be `object`?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716538184


   **[Test build #130289 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130289/testReport)** for PR 30097 at commit [`cd3329f`](https://github.com/apache/spark/commit/cd3329fadfb7bfcfae97b945d4ac6b8b346d38db).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713548518


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/130094/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] leanken commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
leanken commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716259587


   @cloud-fan @maropu test passed, could you please have a look? thanks ^_^


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-712589515


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34637/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716575599






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r511779789



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
##########
@@ -53,16 +53,18 @@ trait AnalysisTest extends PlanTest {
   }
 
   protected def getAnalyzer(caseSensitive: Boolean) = {
-    if (caseSensitive) caseSensitiveAnalyzer else caseInsensitiveAnalyzer
+    makeAnalyzer(caseSensitive)

Review comment:
       why do we need to change it?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716672014






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] leanken commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r512385606



##########
File path: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala
##########
@@ -27,7 +27,8 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation
 import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project}
 import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
-import org.apache.spark.sql.internal.SQLConf
+
+private[sql] class PruneHiveTablePartitions

Review comment:
       done

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala
##########
@@ -196,6 +202,7 @@ class DataSourceAnalysisSuite extends SparkFunSuite with BeforeAndAfterAll {
     }
 
     test(s"Static partition and dynamic partition (caseSensitive: $caseSensitive)") {
+      SQLConf.get.setConf(SQLConf.CASE_SENSITIVE, caseSensitive)

Review comment:
       done

##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala
##########
@@ -32,7 +32,8 @@ import org.apache.spark.sql.internal.SQLConf.{CASE_SENSITIVE, ORDER_BY_ORDINAL}
 import org.apache.spark.sql.types.IntegerType
 
 class EliminateSortsSuite extends PlanTest {
-  override val conf = new SQLConf().copy(CASE_SENSITIVE -> true, ORDER_BY_ORDINAL -> false)
+  SQLConf.get.setConf(CASE_SENSITIVE, true)
+  SQLConf.get.setConf(ORDER_BY_ORDINAL, false)

Review comment:
       done

##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala
##########
@@ -29,7 +29,8 @@ import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.internal.SQLConf.{CASE_SENSITIVE, GROUP_BY_ORDINAL}
 
 class AggregateOptimizeSuite extends PlanTest {
-  override val conf = new SQLConf().copy(CASE_SENSITIVE -> false, GROUP_BY_ORDINAL -> false)
+  SQLConf.get.setConf(CASE_SENSITIVE, false)
+  SQLConf.get.setConf(GROUP_BY_ORDINAL, false)

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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713548453


   **[Test build #130094 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130094/testReport)** for PR 30097 at commit [`47526de`](https://github.com/apache/spark/commit/47526de256a1e7c52e80d0841869d89d63729499).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713420251


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34690/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713733245


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/130097/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-714244069






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715879577


   **[Test build #130232 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130232/testReport)** for PR 30097 at commit [`c46be89`](https://github.com/apache/spark/commit/c46be89d8bc9185270a6241312dd043ed910526d).
    * This patch **fails to build**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715879580






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-717063665


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34923/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713548511


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-717075832






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715892694


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34834/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713359606


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34682/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713509068


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/34698/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716616040






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716616040






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-712774889


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/130041/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713503027






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713558537






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713460144


   **[Test build #130088 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130088/testReport)** for PR 30097 at commit [`5872a20`](https://github.com/apache/spark/commit/5872a20de5321264496d4ee8c46bdaf86886c06d).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713376891


   **[Test build #130081 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130081/testReport)** for PR 30097 at commit [`76f9670`](https://github.com/apache/spark/commit/76f96708f36eddf3306aae6a0619cb13318a1f03).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713509056


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] leanken commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r511782477



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala
##########
@@ -439,7 +438,7 @@ case class RowToColumnarExec(child: SparkPlan) extends RowToColumnarTransition {
   )
 
   override def doExecuteColumnar(): RDD[ColumnarBatch] = {
-    val enableOffHeapColumnVector = sqlContext.conf.offHeapColumnVectorEnabled

Review comment:
       OK. will revert.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-712598520






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716744287






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713585166


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34703/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715889562


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715889564


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/34833/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716672014






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716966266






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715940687


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34835/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] leanken commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r510548134



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
##########
@@ -28,7 +28,8 @@ import org.apache.spark.sql.types.{StructField, StructType}
 /**
  * An analyzer rule that replaces [[UnresolvedInlineTable]] with [[LocalRelation]].
  */
-case class ResolveInlineTables(conf: SQLConf) extends Rule[LogicalPlan] with CastSupport {
+object ResolveInlineTables extends Rule[LogicalPlan] with CastSupport {
+  override def conf: SQLConf = SQLConf.get

Review comment:
       If it's final could not be override, and in its subclass the usage difference would be conf.XXX and SQLConf.get.XXX, I don't think we should bother to add such interface.  ^_^




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] leanken commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r511708238



##########
File path: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala
##########
@@ -29,22 +29,9 @@ import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.internal.SQLConf
 
-/**
- * Prune hive table partitions using partition filters on [[HiveTableRelation]]. The pruned
- * partitions will be kept in [[HiveTableRelation.prunedPartitions]], and the statistics of
- * the hive table relation will be updated based on pruned partitions.
- *
- * This rule is executed in optimization phase, so the statistics can be updated before physical
- * planning, which is useful for some spark strategy, eg.
- * [[org.apache.spark.sql.execution.SparkStrategies.JoinSelection]].
- *
- * TODO: merge this with PruneFileSourcePartitions after we completely make hive as a data source.
- */
-private[sql] class PruneHiveTablePartitions(session: SparkSession)

Review comment:
       When i change from class to object, the building task java unidoc failed。Do you know how to fix this issue??




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715887837


   **[Test build #130234 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130234/testReport)** for PR 30097 at commit [`066a141`](https://github.com/apache/spark/commit/066a14122481902e09b3a83e5976323b2855479f).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713733231






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713665217


   **[Test build #130097 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130097/testReport)** for PR 30097 at commit [`83c1577`](https://github.com/apache/spark/commit/83c1577d5ab3b6fd24d0409bcbd6a8df591da96b).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] leanken edited a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
leanken edited a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-717035399


   retest this please


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715879581


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/130232/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716744287






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713350861


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-712602328






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716575575


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34885/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713350628


   **[Test build #130073 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130073/testReport)** for PR 30097 at commit [`c681752`](https://github.com/apache/spark/commit/c681752d8f0fec5a318126026e27ca2747b114d9).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r511785672



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala
##########
@@ -51,21 +51,26 @@ class DataSourceAnalysisSuite extends SparkFunSuite with BeforeAndAfterAll {
   }
 
   Seq(true, false).foreach { caseSensitive =>
-    val conf = new SQLConf().copy(SQLConf.CASE_SENSITIVE -> caseSensitive)
     def cast(e: Expression, dt: DataType): Expression = {
-      conf.storeAssignmentPolicy match {
+      SQLConf.get.storeAssignmentPolicy match {
         case StoreAssignmentPolicy.ANSI | StoreAssignmentPolicy.STRICT =>
-          AnsiCast(e, dt, Option(conf.sessionLocalTimeZone))
+          AnsiCast(e, dt, Option(SQLConf.get.sessionLocalTimeZone))
         case _ =>
-          Cast(e, dt, Option(conf.sessionLocalTimeZone))
+          Cast(e, dt, Option(SQLConf.get.sessionLocalTimeZone))
       }
     }
-    val rule = DataSourceAnalysis(conf)
+
+    def rule(caseSensitive: Boolean): DataSourceAnalysis.type =
+    {
+      SQLConf.get.setConf(SQLConf.CASE_SENSITIVE, caseSensitive)

Review comment:
       we can set conf at the beginning
   ```
   Seq(true, false).foreach { caseSensitive =>
     SQLConf.get.set...
     ...
     val rule = DataSourceAnalysis
   }
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716966253


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34913/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] leanken commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r511711363



##########
File path: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala
##########
@@ -29,22 +29,9 @@ import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.internal.SQLConf
 
-/**
- * Prune hive table partitions using partition filters on [[HiveTableRelation]]. The pruned
- * partitions will be kept in [[HiveTableRelation.prunedPartitions]], and the statistics of
- * the hive table relation will be updated based on pruned partitions.
- *
- * This rule is executed in optimization phase, so the statistics can be updated before physical
- * planning, which is useful for some spark strategy, eg.
- * [[org.apache.spark.sql.execution.SparkStrategies.JoinSelection]].
- *
- * TODO: merge this with PruneFileSourcePartitions after we completely make hive as a data source.
- */
-private[sql] class PruneHiveTablePartitions(session: SparkSession)

Review comment:
       error message as following
   
   ```
   [error] /Users/linxuewei/Code/github/leanken-spark/sql/hive/target/java/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.java:3:1:  错误: 找不到引用
   [error]  * Prune hive table partitions using partition filters on {@link HiveTableRelation}. The pruned
   [error]                                                                  ^
   [error] /Users/linxuewei/Code/github/leanken-spark/sql/hive/target/java/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.java:4:1:  错误: 找不到引用
   [error]  * partitions will be kept in {@link HiveTableRelation.prunedPartitions}, and the statistics of
   [error]                                      ^
   [error] /Users/linxuewei/Code/github/leanken-spark/sql/hive/target/java/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.java:9:1:  错误: 找不到引用
   [error]  * {@link org.apache.spark.sql.execution.SparkStrategies.JoinSelection}.
   [error]           ^
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715928128


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34835/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] leanken commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
leanken commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-717035399


   retest this, please


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715922894


   **[Test build #130235 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130235/testReport)** for PR 30097 at commit [`2b263f3`](https://github.com/apache/spark/commit/2b263f33d206eaa47ba39739a5e0f3be49e6e1d2).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713558018


   **[Test build #130089 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130089/testReport)** for PR 30097 at commit [`a4822fd`](https://github.com/apache/spark/commit/a4822fd3b797e980ef8d99912e71cb9453d7ba06).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713388486


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/130081/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715879838


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r512000413



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala
##########
@@ -29,7 +29,8 @@ import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.internal.SQLConf.{CASE_SENSITIVE, GROUP_BY_ORDINAL}
 
 class AggregateOptimizeSuite extends PlanTest {
-  override val conf = new SQLConf().copy(CASE_SENSITIVE -> false, GROUP_BY_ORDINAL -> false)
+  SQLConf.get.setConf(CASE_SENSITIVE, false)
+  SQLConf.get.setConf(GROUP_BY_ORDINAL, false)

Review comment:
       Can we follow https://github.com/apache/spark/pull/30097/files#diff-6f79a980de03203104cfacc6f50c302b843e3b60f1d48942bded6706c75a02b7R128 , instead of updating the global conf object?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716945800


   **[Test build #130311 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130311/testReport)** for PR 30097 at commit [`f8e103b`](https://github.com/apache/spark/commit/f8e103bcdef514c27627daea59bfafd00f22dcd9).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-717035349






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713349260


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34682/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713376891


   **[Test build #130081 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130081/testReport)** for PR 30097 at commit [`76f9670`](https://github.com/apache/spark/commit/76f96708f36eddf3306aae6a0619cb13318a1f03).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713442224






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-717075813


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34923/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-714073513


   **[Test build #130112 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130112/testReport)** for PR 30097 at commit [`6e521aa`](https://github.com/apache/spark/commit/6e521aaf352fb11267f5ee04f05bee7580de132a).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713388432


   **[Test build #130081 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130081/testReport)** for PR 30097 at commit [`76f9670`](https://github.com/apache/spark/commit/76f96708f36eddf3306aae6a0619cb13318a1f03).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-712774879


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-717035334






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r512009195



##########
File path: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala
##########
@@ -27,7 +27,8 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation
 import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project}
 import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
-import org.apache.spark.sql.internal.SQLConf
+
+private[sql] class PruneHiveTablePartitions

Review comment:
       Why can't we put the comment as classdoc?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715887837


   **[Test build #130234 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130234/testReport)** for PR 30097 at commit [`066a141`](https://github.com/apache/spark/commit/066a14122481902e09b3a83e5976323b2855479f).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715882814


   **[Test build #130233 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130233/testReport)** for PR 30097 at commit [`7fc0716`](https://github.com/apache/spark/commit/7fc07165277fa9aa10dfec07c4de9ac360d20d87).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-712602334


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/130031/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713460144


   **[Test build #130088 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130088/testReport)** for PR 30097 at commit [`5872a20`](https://github.com/apache/spark/commit/5872a20de5321264496d4ee8c46bdaf86886c06d).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713388472


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713709594






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716616007


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34890/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-714243299


   **[Test build #130112 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130112/testReport)** for PR 30097 at commit [`6e521aa`](https://github.com/apache/spark/commit/6e521aaf352fb11267f5ee04f05bee7580de132a).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713733231


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-712751582






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716490495


   **[Test build #130283 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130283/testReport)** for PR 30097 at commit [`f4663d8`](https://github.com/apache/spark/commit/f4663d849aca5a9f540aab578452aaa137e3f673).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-715895020






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-717180229


   **[Test build #130321 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130321/testReport)** for PR 30097 at commit [`7612695`](https://github.com/apache/spark/commit/7612695c78456155a95ad4f7d54ef70e53f88921).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-712598520


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] leanken commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r511710832



##########
File path: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala
##########
@@ -29,22 +29,9 @@ import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.internal.SQLConf
 
-/**
- * Prune hive table partitions using partition filters on [[HiveTableRelation]]. The pruned
- * partitions will be kept in [[HiveTableRelation.prunedPartitions]], and the statistics of
- * the hive table relation will be updated based on pruned partitions.
- *
- * This rule is executed in optimization phase, so the statistics can be updated before physical
- * planning, which is useful for some spark strategy, eg.
- * [[org.apache.spark.sql.execution.SparkStrategies.JoinSelection]].
- *
- * TODO: merge this with PruneFileSourcePartitions after we completely make hive as a data source.
- */
-private[sql] class PruneHiveTablePartitions(session: SparkSession)

Review comment:
       https://github.com/akka/akka-http/issues/1895
   something might be relative??




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-716037905






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-714170190






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-714170186


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34721/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713585191






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30097:
URL: https://github.com/apache/spark/pull/30097#issuecomment-713536246


   **[Test build #130094 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130094/testReport)** for PR 30097 at commit [`47526de`](https://github.com/apache/spark/commit/47526de256a1e7c52e80d0841869d89d63729499).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] leanken commented on a change in pull request #30097: [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]

Posted by GitBox <gi...@apache.org>.
leanken commented on a change in pull request #30097:
URL: https://github.com/apache/spark/pull/30097#discussion_r511781341



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
##########
@@ -771,22 +771,23 @@ class AnalysisSuite extends AnalysisTest with Matchers {
     // RuleExecutor only throw exception or log warning when the rule is supposed to run
     // more than once.
     val maxIterations = 2
-    val conf = new SQLConf().copy(SQLConf.ANALYZER_MAX_ITERATIONS -> maxIterations)

Review comment:
       after tested, no need to change here anymore. will update.




----------------------------------------------------------------
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:
users@infra.apache.org



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