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/03/27 05:48:14 UTC

[GitHub] [spark] yaooqinn opened a new pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

yaooqinn opened a new pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043
 
 
   
   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
   -->
   
   ### What changes were proposed in this pull request?
   ```sql
   scala> spark.sql(" select * from values(1), (2) t(key) where key in (select 1 as key where 1=0)").queryExecution
   res15: org.apache.spark.sql.execution.QueryExecution =
   == Parsed Logical Plan ==
   'Project [*]
   +- 'Filter 'key IN (list#39 [])
      :  +- Project [1 AS key#38]
      :     +- Filter (1 = 0)
      :        +- OneRowRelation
      +- 'SubqueryAlias t
         +- 'UnresolvedInlineTable [key], [List(1), List(2)]
   
   == Analyzed Logical Plan ==
   key: int
   Project [key#40]
   +- Filter key#40 IN (list#39 [])
      :  +- Project [1 AS key#38]
      :     +- Filter (1 = 0)
      :        +- OneRowRelation
      +- SubqueryAlias t
         +- LocalRelation [key#40]
   
   == Optimized Logical Plan ==
   Join LeftSemi, (key#40 = key#38)
   :- LocalRelation [key#40]
   +- LocalRelation <empty>, [key#38]
   
   == Physical Plan ==
   *(1) BroadcastHashJoin [key#40], [key#38], LeftSemi, BuildRight
   :- *(1) LocalTableScan [key#40]
   +- Br...
   ```
   
   `LocalRelation <empty> ` should be able to propagate after subqueries are lift up to joins
   
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   optimize query 
   
   ### Does this PR introduce any user-facing change?
   <!--
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If no, write 'No'.
   -->
   
   no
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   -->
   add new tests

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604897966
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605009260
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120481/
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605198318
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604903802
 
 
   **[Test build #120481 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120481/testReport)** for PR 28043 at commit [`2eccef9`](https://github.com/apache/spark/commit/2eccef91f78d7776645b5e4ecbf4af6235878fb9).

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604868387
 
 
   **[Test build #120456 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120456/testReport)** for PR 28043 at commit [`1726355`](https://github.com/apache/spark/commit/17263558c59992c83ff59a44df5ea8080cef79c0).
    * 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


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r400600377
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ##########
 @@ -198,6 +198,8 @@ abstract class Optimizer(catalogManager: CatalogManager)
       CheckCartesianProducts) :+
     Batch("RewriteSubquery", Once,
       RewritePredicateSubquery,
+      ConvertToLocalRelation,
+      PropagateEmptyRelation,
 
 Review comment:
   This fails at `checkBatchIdempotence` of `RuleExecutor`.
   ```
   [info]   2020-03-30 19:00:21.097 - stderr> 20/03/30 19:00:21 INFO TemporaryHiveUDFTest: Running a simple query on the table.
   [info]   2020-03-30 19:00:21.198 - stderr> Exception in thread "main" org.apache.spark.sql.catalyst.errors.package$TreeNodeException:
   [info]   2020-03-30 19:00:21.198 - stderr> Once strategy's idempotence is broken for batch RewriteSubquery
   [info]   2020-03-30 19:00:21.198 - stderr>  Aggregate [count(1) AS count#15L]       Aggregate [count(1) AS count#15L]
   [info]   2020-03-30 19:00:21.198 - stderr>  +- Aggregate [val#5]                    +- Aggregate [val#5]
   [info]   2020-03-30 19:00:21.198 - stderr> !   +- Project [val#5]                      +- LocalRelation [val#5]
   [info]   2020-03-30 19:00:21.198 - stderr> !      +- LocalRelation [key#4, val#5]
   [info]   2020-03-30 19:00:21.198 - stderr>           , tree:
   [info]   2020-03-30 19:00:21.198 - stderr> Aggregate [count(1) AS count#15L]
   [info]   2020-03-30 19:00:21.198 - stderr> +- Aggregate [val#5]
   [info]   2020-03-30 19:00:21.198 - stderr>    +- LocalRelation [val#5]
   ```

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605102737
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399099828
 
 

 ##########
 File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala
 ##########
 @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.plans.{LeftSemi, PlanTest}
 import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan}
 import org.apache.spark.sql.catalyst.rules.RuleExecutor
 
-
 
 Review comment:
   super nit. Shall we avoid this kind of removal?

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


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun closed pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun closed pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043
 
 
   

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


With regards,
Apache Git Services

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


[GitHub] [spark] yaooqinn commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399059590
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
 ##########
 @@ -3494,14 +3494,27 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
       Seq(Row(Map[Int, Int]()), Row(Map(1 -> 2))))
   }
 
-  test("SPARK-31242: clone SparkSession should respect sessionInitWithConfigDefaults") {
-    // Note, only the conf explicitly set in SparkConf(e.g. in SharedSparkSessionBase) would cause
-    // problem before the fix.
-    withSQLConf(SQLConf.CODEGEN_FALLBACK.key -> "true") {
-      val cloned = spark.cloneSession()
-      SparkSession.setActiveSession(cloned)
-      assert(SQLConf.get.getConf(SQLConf.CODEGEN_FALLBACK) === true)
-    }
+  test("Perform propagating empty relation after RewritePredicateSubquery") {
 
 Review comment:
   move or add?

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604884929
 
 
   **[Test build #120475 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120475/testReport)** for PR 28043 at commit [`67f6a31`](https://github.com/apache/spark/commit/67f6a31805bf6e756e60af8125ad1cee9aae2f25).

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


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r400600054
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ##########
 @@ -198,6 +198,8 @@ abstract class Optimizer(catalogManager: CatalogManager)
       CheckCartesianProducts) :+
     Batch("RewriteSubquery", Once,
       RewritePredicateSubquery,
+      ConvertToLocalRelation,
+      PropagateEmptyRelation,
 
 Review comment:
   Although this passes PR Builder, this causes consistent failure after merging for all builds.
   ```
   org.apache.spark.sql.hive.HiveSparkSubmitSuite.temporary Hive UDF: define a UDF and use it
   org.apache.spark.sql.hive.HiveSparkSubmitSuite.permanent Hive UDF: define a UDF and use it
   org.apache.spark.sql.hive.HiveSparkSubmitSuite.permanent Hive UDF: use a already defined permanent function
   ```

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604847753
 
 
   **[Test build #120454 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120454/testReport)** for PR 28043 at commit [`0b93b8a`](https://github.com/apache/spark/commit/0b93b8a3c4e7d681fa54a9b38635aac3474d8827).
    * 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604840868
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25161/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605348803
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120510/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605198328
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25216/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605039312
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399424338
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/EndToEndSQLOptimizerSuite.scala
 ##########
 @@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
+import org.apache.spark.sql.test.SharedSparkSession
+
+class EndToEndSQLOptimizerSuite extends QueryTest with SharedSparkSession {
+
+  test("Perform propagating empty relation after RewritePredicateSubquery") {
+    val df1 = sql(
+      s"""
+         |SELECT *
+         |FROM VALUES(1), (2) t1(key)
+         | WHERE key IN
+         |  (SELECT key FROM VALUES(1) t2(key) WHERE 1=0)
+       """.stripMargin)
+    assert(df1.queryExecution.optimizedPlan.isInstanceOf[LocalRelation])
+    checkAnswer(df1, Nil)
+
+    val df2 = sql(
+      s"""
+         |SELECT *
+         |FROM VALUES(1), (2) t1(key)
+         | WHERE key NOT IN
 
 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605095192
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399424829
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/EndToEndSQLOptimizerSuite.scala
 ##########
 @@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
+import org.apache.spark.sql.test.SharedSparkSession
+
+class EndToEndSQLOptimizerSuite extends QueryTest with SharedSparkSession {
+
+  test("Perform propagating empty relation after RewritePredicateSubquery") {
+    val df1 = sql(
+      s"""
+         |SELECT *
+         |FROM VALUES(1), (2) t1(key)
+         | WHERE key IN
+         |  (SELECT key FROM VALUES(1) t2(key) WHERE 1=0)
+       """.stripMargin)
+    assert(df1.queryExecution.optimizedPlan.isInstanceOf[LocalRelation])
+    checkAnswer(df1, Nil)
+
+    val df2 = sql(
+      s"""
+         |SELECT *
+         |FROM VALUES(1), (2) t1(key)
+         | WHERE key NOT IN
+         |  (SELECT key FROM VALUES(1) t2(key) WHERE 1=0)
+       """.stripMargin)
+
+    assert(df2.queryExecution.optimizedPlan.isInstanceOf[LocalRelation])
+    checkAnswer(df2, Seq(Row(1), Row(2)))
+
+    // Because [[RewriteNonCorrelatedExists]] will rewrite non-correlated exists subqueries to
 
 Review comment:
   Ur, do we need to use `[[..]]` in comment?

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


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-606357793
 
 
   This is reverted via https://github.com/apache/spark/commit/cda2e30e77dfb0d5c20fdd4dd147b329257994c1 .

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605094713
 
 
   **[Test build #120492 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120492/testReport)** for PR 28043 at commit [`6e70768`](https://github.com/apache/spark/commit/6e7076813991a486f3266cb195241bafa0bfc5ce).
    * 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604828553
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399235195
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/EndToEndSQLOptimizerSuite.scala
 ##########
 @@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
+import org.apache.spark.sql.test.SharedSparkSession
+
+class EndToEndSQLOptimizerSuite extends QueryTest with SharedSparkSession {
+
+  test("Perform propagating empty relation after RewritePredicateSubquery") {
+    val df1 = sql(
+      s"""
+         |select *
 
 Review comment:
   nit: better to capitalize the SQL keywords where possible.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605323932
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604900543
 
 
   **[Test build #120479 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120479/testReport)** for PR 28043 at commit [`f36fe44`](https://github.com/apache/spark/commit/f36fe4493cc1423d0fd0adff4accaf20586babe0).

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604847802
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604828553
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604847814
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604897971
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25184/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604904241
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25186/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-606356103
 
 
   Sorry, @yaooqinn . I'll recover master branch first since we are doing RC1. We can talk this more later.

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


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399448588
 
 

 ##########
 File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala
 ##########
 @@ -52,4 +52,13 @@ class RewriteSubquerySuite extends PlanTest {
     comparePlans(optimized, correctAnswer)
   }
 
+  test("Perform propagating empty relation after RewritePredicateSubquery") {
 
 Review comment:
   Got it. I thought the second case, `a PR adds a couple of tests to an existing test class`, but it seems that you disagree.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604998178
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605598064
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605102122
 
 
   **[Test build #120503 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120503/testReport)** for PR 28043 at commit [`6e70768`](https://github.com/apache/spark/commit/6e7076813991a486f3266cb195241bafa0bfc5ce).

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605597937
 
 
   **[Test build #120551 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120551/testReport)** for PR 28043 at commit [`b084a27`](https://github.com/apache/spark/commit/b084a27dec2f68d179a6c245f2845b45232baf63).

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605009250
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605039330
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120479/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605595604
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120546/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604840864
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605595602
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604832641
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25159/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605559203
 
 
   **[Test build #120546 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120546/testReport)** for PR 28043 at commit [`b084a27`](https://github.com/apache/spark/commit/b084a27dec2f68d179a6c245f2845b45232baf63).

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605102741
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25209/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604904235
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605623374
 
 
   **[Test build #120551 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120551/testReport)** for PR 28043 at commit [`b084a27`](https://github.com/apache/spark/commit/b084a27dec2f68d179a6c245f2845b45232baf63).
    * 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605197693
 
 
   **[Test build #120510 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120510/testReport)** for PR 28043 at commit [`832d922`](https://github.com/apache/spark/commit/832d9229c0806b5d49c91fb692e71221781f5c17).

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399059109
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
 ##########
 @@ -3494,14 +3494,27 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
       Seq(Row(Map[Int, Int]()), Row(Map(1 -> 2))))
   }
 
-  test("SPARK-31242: clone SparkSession should respect sessionInitWithConfigDefaults") {
 
 Review comment:
   why remove 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605559305
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399121014
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
 ##########
 @@ -3503,6 +3503,42 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
       assert(SQLConf.get.getConf(SQLConf.CODEGEN_FALLBACK) === true)
     }
   }
+
+  test("Perform propagating empty relation after RewritePredicateSubquery") {
 
 Review comment:
   end-to-end optimizer test is useful, but `SQLQuerySuite` may not be a good place to put it.
   
   How about we create a new test suite `EndToEndSQLOptimizerSuite`?

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605348803
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120510/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605595602
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605595604
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120546/
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604997533
 
 
   **[Test build #120492 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120492/testReport)** for PR 28043 at commit [`6e70768`](https://github.com/apache/spark/commit/6e7076813991a486f3266cb195241bafa0bfc5ce).

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604847161
 
 
   **[Test build #120456 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120456/testReport)** for PR 28043 at commit [`1726355`](https://github.com/apache/spark/commit/17263558c59992c83ff59a44df5ea8080cef79c0).

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604847826
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120452/
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605323942
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120503/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun edited a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun edited a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-606356103
 
 
   Sorry, @yaooqinn . Although this only affects `master` branch, I'll recover master branch first since we are doing RC1. This can block the other PRs which need to be backported to `branch-3.0`. We can talk this more later. I verified that the reverted branch is okay.

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


With regards,
Apache Git Services

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


[GitHub] [spark] yaooqinn commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399059509
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
 ##########
 @@ -3494,14 +3494,27 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
       Seq(Row(Map[Int, Int]()), Row(Map(1 -> 2))))
   }
 
-  test("SPARK-31242: clone SparkSession should respect sessionInitWithConfigDefaults") {
 
 Review comment:
   oh, mistake..

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604868548
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605197693
 
 
   **[Test build #120510 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120510/testReport)** for PR 28043 at commit [`832d922`](https://github.com/apache/spark/commit/832d9229c0806b5d49c91fb692e71221781f5c17).

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604834330
 
 
   **[Test build #120452 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120452/testReport)** for PR 28043 at commit [`25e37bc`](https://github.com/apache/spark/commit/25e37bc8e2a64ebd760cbed300ef4f780a713a1b).

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605020221
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120475/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604847559
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25163/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604847553
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604885402
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25181/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604847802
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399100749
 
 

 ##########
 File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala
 ##########
 @@ -52,4 +51,24 @@ class RewriteSubquerySuite extends PlanTest {
     comparePlans(optimized, correctAnswer)
   }
 
+  object OptimizeWithPropagateEmpty extends RuleExecutor[LogicalPlan] {
 
 Review comment:
   BTW, it seems that we can `object Optimize` instead of making this.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399573396
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/EndToEndSQLOptimizerSuite.scala
 ##########
 @@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
+import org.apache.spark.sql.test.SharedSparkSession
+
+class EndToEndSQLOptimizerSuite extends QueryTest with SharedSparkSession {
 
 Review comment:
   Moving them into `PlannerSuite` looks fine to me.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605598068
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25257/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun edited a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun edited a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-606356103
 
 
   Sorry, @yaooqinn . I'll recover master branch first since we are doing RC1. We can talk this more later. I verified that the reverted branch is okay.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604900543
 
 
   **[Test build #120479 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120479/testReport)** for PR 28043 at commit [`f36fe44`](https://github.com/apache/spark/commit/f36fe4493cc1423d0fd0adff4accaf20586babe0).

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604998178
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399432425
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/EndToEndSQLOptimizerSuite.scala
 ##########
 @@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
+import org.apache.spark.sql.test.SharedSparkSession
+
+class EndToEndSQLOptimizerSuite extends QueryTest with SharedSparkSession {
 
 Review comment:
   Instead of creating a new suite, shall we add the following test into `PlannerSuite.scala`?
   What do you think about that, @cloud-fan and @maropu ?

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605020214
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605559306
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25252/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605039312
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604966468
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120470/
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604867441
 
 
   **[Test build #120470 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120470/testReport)** for PR 28043 at commit [`356f684`](https://github.com/apache/spark/commit/356f684d243a3a0bbeec2204f1121b87754fe5fb).

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604832641
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25159/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399448588
 
 

 ##########
 File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala
 ##########
 @@ -52,4 +52,13 @@ class RewriteSubquerySuite extends PlanTest {
     comparePlans(optimized, correctAnswer)
   }
 
+  test("Perform propagating empty relation after RewritePredicateSubquery") {
 
 Review comment:
   Got 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604865256
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25176/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604867441
 
 
   **[Test build #120470 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120470/testReport)** for PR 28043 at commit [`356f684`](https://github.com/apache/spark/commit/356f684d243a3a0bbeec2204f1121b87754fe5fb).

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605323357
 
 
   **[Test build #120503 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120503/testReport)** for PR 28043 at commit [`6e70768`](https://github.com/apache/spark/commit/6e7076813991a486f3266cb195241bafa0bfc5ce).
    * 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604885396
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605623708
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120551/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] yaooqinn commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r400603176
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ##########
 @@ -198,6 +198,8 @@ abstract class Optimizer(catalogManager: CatalogManager)
       CheckCartesianProducts) :+
     Batch("RewriteSubquery", Once,
       RewritePredicateSubquery,
+      ConvertToLocalRelation,
+      PropagateEmptyRelation,
 
 Review comment:
   I see. We need a separate batch to propagate empty relation after rewriting subquery 

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604828858
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120450/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605323942
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120503/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399059347
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
 ##########
 @@ -3494,14 +3494,27 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
       Seq(Row(Map[Int, Int]()), Row(Map(1 -> 2))))
   }
 
-  test("SPARK-31242: clone SparkSession should respect sessionInitWithConfigDefaults") {
-    // Note, only the conf explicitly set in SparkConf(e.g. in SharedSparkSessionBase) would cause
-    // problem before the fix.
-    withSQLConf(SQLConf.CODEGEN_FALLBACK.key -> "true") {
-      val cloned = spark.cloneSession()
-      SparkSession.setActiveSession(cloned)
-      assert(SQLConf.get.getConf(SQLConf.CODEGEN_FALLBACK) === true)
-    }
+  test("Perform propagating empty relation after RewritePredicateSubquery") {
 
 Review comment:
   can we add a UT in `RewriteSubquerySuite`?

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604868548
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604847810
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120454/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604828858
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120450/
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604868557
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120456/
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605598064
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] yaooqinn commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399110481
 
 

 ##########
 File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala
 ##########
 @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.plans.{LeftSemi, PlanTest}
 import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan}
 import org.apache.spark.sql.catalyst.rules.RuleExecutor
 
-
 
 Review comment:
   thanks. I will obey this from now on

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604865247
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605009250
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604828852
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604847559
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25163/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604847748
 
 
   **[Test build #120451 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120451/testReport)** for PR 28043 at commit [`b8a538e`](https://github.com/apache/spark/commit/b8a538ecef2945bba5083992d4de339d104148ba).
    * 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604828224
 
 
   **[Test build #120450 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120450/testReport)** for PR 28043 at commit [`ba2e6b4`](https://github.com/apache/spark/commit/ba2e6b4ee0924cf4d735c8410211ea123ded7a66).

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605348798
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604847860
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605018646
 
 
   **[Test build #120475 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120475/testReport)** for PR 28043 at commit [`67f6a31`](https://github.com/apache/spark/commit/67f6a31805bf6e756e60af8125ad1cee9aae2f25).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `  case class Optimize(addOn: Rule[LogicalPlan]) extends RuleExecutor[LogicalPlan] `

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605020214
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604904235
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] yaooqinn commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399128316
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
 ##########
 @@ -3503,6 +3503,42 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
       assert(SQLConf.get.getConf(SQLConf.CODEGEN_FALLBACK) === true)
     }
   }
+
+  test("Perform propagating empty relation after RewritePredicateSubquery") {
 
 Review comment:
   sgtm, 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604865256
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25176/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604828558
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25158/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605099513
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399235356
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/EndToEndSQLOptimizerSuite.scala
 ##########
 @@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
+import org.apache.spark.sql.test.SharedSparkSession
+
+class EndToEndSQLOptimizerSuite extends QueryTest with SharedSparkSession {
+
+  test("Perform propagating empty relation after RewritePredicateSubquery") {
+    val df1 = sql(
+      s"""
+         |select *
+         |from values(1), (2) t1(key)
+         | where key in
+         |  (select key from values(1) t2(key) where 1=0)
+       """.stripMargin)
+    assert(df1.queryExecution.optimizedPlan.isInstanceOf[LocalRelation])
+    checkAnswer(df1, Nil)
+
+    val df2 = sql(
+      s"""
+         |select *
+         |from values(1), (2) t1(key)
+         | where key not in
+         |  (select key from values(1) t2(key) where 1=0)
+       """.stripMargin)
+
+    assert(df2.queryExecution.optimizedPlan.isInstanceOf[LocalRelation])
+    checkAnswer(df2, Seq(Row(1), Row(2)))
+
+    // Because [[RewriteNonCorrelatedExists]] will rewrite non-correlated exists subqueries to
+    // scalar expressions early, so this only take effects on correlated exists subqueries
+    val df3 = sql(
+      s"""
+         |select *
+         |from values(1), (2) t1(key)
+         | where exists
+         |  (select key from values(1) t2(key) where t1.key = 1 and 1=0)
+       """.stripMargin)
+
+    assert(df3.queryExecution.optimizedPlan.isInstanceOf[LocalRelation])
+    checkAnswer(df3, Nil)
+  }
+
 
 Review comment:
   nit: remove a blank line above.

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


With regards,
Apache Git Services

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


[GitHub] [spark] yaooqinn commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399447631
 
 

 ##########
 File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala
 ##########
 @@ -52,4 +52,13 @@ class RewriteSubquerySuite extends PlanTest {
     comparePlans(optimized, correctAnswer)
   }
 
+  test("Perform propagating empty relation after RewritePredicateSubquery") {
 
 Review comment:
   >> In practice, usually it is added when a JIRA type is a bug or a PR adds a couple of tests to an existing test class

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605559305
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] yaooqinn commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605597672
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604885402
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25181/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604897966
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605348798
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399573396
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/EndToEndSQLOptimizerSuite.scala
 ##########
 @@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
+import org.apache.spark.sql.test.SharedSparkSession
+
+class EndToEndSQLOptimizerSuite extends QueryTest with SharedSparkSession {
 
 Review comment:
   Looks fine to me.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604904241
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25186/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604828843
 
 
   **[Test build #120450 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120450/testReport)** for PR 28043 at commit [`ba2e6b4`](https://github.com/apache/spark/commit/ba2e6b4ee0924cf4d735c8410211ea123ded7a66).
    * This patch **fails Scala style 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


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399426163
 
 

 ##########
 File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala
 ##########
 @@ -52,4 +52,13 @@ class RewriteSubquerySuite extends PlanTest {
     comparePlans(optimized, correctAnswer)
   }
 
+  test("Perform propagating empty relation after RewritePredicateSubquery") {
 
 Review comment:
   Shall we add `SPARK-31280: ` prefix?
   - https://spark.apache.org/contributing.html

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


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399098985
 
 

 ##########
 File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala
 ##########
 @@ -52,4 +51,24 @@ class RewriteSubquerySuite extends PlanTest {
     comparePlans(optimized, correctAnswer)
   }
 
+  object OptimizeWithPropagateEmpty extends RuleExecutor[LogicalPlan] {
+    val batches =
+      Batch("Column Pruning", FixedPoint(100), ColumnPruning) ::
+        Batch("Rewrite Subquery", FixedPoint(1),
 
 Review comment:
   indentation?

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


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-606350566
 
 
   Sorry guys. I'm reinvestigating this PR.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605597937
 
 
   **[Test build #120551 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120551/testReport)** for PR 28043 at commit [`b084a27`](https://github.com/apache/spark/commit/b084a27dec2f68d179a6c245f2845b45232baf63).

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605598068
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25257/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604832312
 
 
   **[Test build #120451 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120451/testReport)** for PR 28043 at commit [`b8a538e`](https://github.com/apache/spark/commit/b8a538ecef2945bba5083992d4de339d104148ba).

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604965839
 
 
   **[Test build #120470 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120470/testReport)** for PR 28043 at commit [`356f684`](https://github.com/apache/spark/commit/356f684d243a3a0bbeec2204f1121b87754fe5fb).
    * 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


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399423512
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/EndToEndSQLOptimizerSuite.scala
 ##########
 @@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
+import org.apache.spark.sql.test.SharedSparkSession
+
+class EndToEndSQLOptimizerSuite extends QueryTest with SharedSparkSession {
+
+  test("Perform propagating empty relation after RewritePredicateSubquery") {
+    val df1 = sql(
+      s"""
+         |SELECT *
+         |FROM VALUES(1), (2) t1(key)
+         | WHERE key IN
 
 Review comment:
   nit. `| WHERE` -> `|WHERE`.

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


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399457576
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/EndToEndSQLOptimizerSuite.scala
 ##########
 @@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
+import org.apache.spark.sql.test.SharedSparkSession
+
+class EndToEndSQLOptimizerSuite extends QueryTest with SharedSparkSession {
+
+  test("Perform propagating empty relation after RewritePredicateSubquery") {
+    val df1 = sql(
+      s"""
+         |SELECT *
+         |FROM VALUES(1), (2) t1(key)
+         | WHERE key IN
+         |  (SELECT key FROM VALUES(1) t2(key) WHERE 1=0)
+       """.stripMargin)
+    assert(df1.queryExecution.optimizedPlan.isInstanceOf[LocalRelation])
+    checkAnswer(df1, Nil)
+
+    val df2 = sql(
+      s"""
+         |SELECT *
+         |FROM VALUES(1), (2) t1(key)
+         | WHERE key NOT IN
+         |  (SELECT key FROM VALUES(1) t2(key) WHERE 1=0)
+       """.stripMargin)
+
+    assert(df2.queryExecution.optimizedPlan.isInstanceOf[LocalRelation])
+    checkAnswer(df2, Seq(Row(1), Row(2)))
+
+    // Because [[RewriteNonCorrelatedExists]] will rewrite non-correlated exists subqueries to
 
 Review comment:
   Technically, I'm a little against adding a new decoration instance without a reason.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604828224
 
 
   **[Test build #120450 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120450/testReport)** for PR 28043 at commit [`ba2e6b4`](https://github.com/apache/spark/commit/ba2e6b4ee0924cf4d735c8410211ea123ded7a66).

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604834330
 
 
   **[Test build #120452 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120452/testReport)** for PR 28043 at commit [`25e37bc`](https://github.com/apache/spark/commit/25e37bc8e2a64ebd760cbed300ef4f780a713a1b).

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604884929
 
 
   **[Test build #120475 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120475/testReport)** for PR 28043 at commit [`67f6a31`](https://github.com/apache/spark/commit/67f6a31805bf6e756e60af8125ad1cee9aae2f25).

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605020221
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120475/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604828558
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25158/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604840868
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25161/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604840864
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604897971
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25184/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604847863
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120451/
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605623708
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120551/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604865247
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] yaooqinn commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604827206
 
 
   cc @cloud-fan @dongjoon-hyun, 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605102122
 
 
   **[Test build #120503 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120503/testReport)** for PR 28043 at commit [`6e70768`](https://github.com/apache/spark/commit/6e7076813991a486f3266cb195241bafa0bfc5ce).

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605038006
 
 
   **[Test build #120479 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120479/testReport)** for PR 28043 at commit [`f36fe44`](https://github.com/apache/spark/commit/f36fe4493cc1423d0fd0adff4accaf20586babe0).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `class EndToEndSQLOptimizerSuite extends QueryTest with SharedSparkSession `

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604966468
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120470/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605198318
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604868557
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120456/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604847860
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604847553
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] yaooqinn commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399453009
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/EndToEndSQLOptimizerSuite.scala
 ##########
 @@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
+import org.apache.spark.sql.test.SharedSparkSession
+
+class EndToEndSQLOptimizerSuite extends QueryTest with SharedSparkSession {
+
+  test("Perform propagating empty relation after RewritePredicateSubquery") {
+    val df1 = sql(
+      s"""
+         |SELECT *
+         |FROM VALUES(1), (2) t1(key)
+         | WHERE key IN
+         |  (SELECT key FROM VALUES(1) t2(key) WHERE 1=0)
+       """.stripMargin)
+    assert(df1.queryExecution.optimizedPlan.isInstanceOf[LocalRelation])
+    checkAnswer(df1, Nil)
+
+    val df2 = sql(
+      s"""
+         |SELECT *
+         |FROM VALUES(1), (2) t1(key)
+         | WHERE key NOT IN
+         |  (SELECT key FROM VALUES(1) t2(key) WHERE 1=0)
+       """.stripMargin)
+
+    assert(df2.queryExecution.optimizedPlan.isInstanceOf[LocalRelation])
+    checkAnswer(df2, Seq(Row(1), Row(2)))
+
+    // Because [[RewriteNonCorrelatedExists]] will rewrite non-correlated exists subqueries to
 
 Review comment:
   There seems to be plenty of such use cases. Maybe good about the emphasis for a class.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604840512
 
 
   **[Test build #120454 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120454/testReport)** for PR 28043 at commit [`0b93b8a`](https://github.com/apache/spark/commit/0b93b8a3c4e7d681fa54a9b38635aac3474d8827).

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605102741
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25209/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605559306
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25252/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605559203
 
 
   **[Test build #120546 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120546/testReport)** for PR 28043 at commit [`b084a27`](https://github.com/apache/spark/commit/b084a27dec2f68d179a6c245f2845b45232baf63).

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604847810
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120454/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605102737
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399424422
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/EndToEndSQLOptimizerSuite.scala
 ##########
 @@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
+import org.apache.spark.sql.test.SharedSparkSession
+
+class EndToEndSQLOptimizerSuite extends QueryTest with SharedSparkSession {
+
+  test("Perform propagating empty relation after RewritePredicateSubquery") {
+    val df1 = sql(
+      s"""
+         |SELECT *
+         |FROM VALUES(1), (2) t1(key)
+         | WHERE key IN
+         |  (SELECT key FROM VALUES(1) t2(key) WHERE 1=0)
+       """.stripMargin)
+    assert(df1.queryExecution.optimizedPlan.isInstanceOf[LocalRelation])
+    checkAnswer(df1, Nil)
+
+    val df2 = sql(
+      s"""
+         |SELECT *
+         |FROM VALUES(1), (2) t1(key)
+         | WHERE key NOT IN
+         |  (SELECT key FROM VALUES(1) t2(key) WHERE 1=0)
+       """.stripMargin)
+
+    assert(df2.queryExecution.optimizedPlan.isInstanceOf[LocalRelation])
+    checkAnswer(df2, Seq(Row(1), Row(2)))
+
+    // Because [[RewriteNonCorrelatedExists]] will rewrite non-correlated exists subqueries to
+    // scalar expressions early, so this only take effects on correlated exists subqueries
+    val df3 = sql(
+      s"""
+         |SELECT *
+         |FROM VALUES(1), (2) t1(key)
+         | WHERE EXISTS
 
 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605198328
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25216/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604847863
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120451/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604847814
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605008886
 
 
   **[Test build #120481 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120481/testReport)** for PR 28043 at commit [`2eccef9`](https://github.com/apache/spark/commit/2eccef91f78d7776645b5e4ecbf4af6235878fb9).
    * 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604828852
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604840512
 
 
   **[Test build #120454 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120454/testReport)** for PR 28043 at commit [`0b93b8a`](https://github.com/apache/spark/commit/0b93b8a3c4e7d681fa54a9b38635aac3474d8827).

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605095198
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120492/
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604847161
 
 
   **[Test build #120456 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120456/testReport)** for PR 28043 at commit [`1726355`](https://github.com/apache/spark/commit/17263558c59992c83ff59a44df5ea8080cef79c0).

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605348307
 
 
   **[Test build #120510 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120510/testReport)** for PR 28043 at commit [`832d922`](https://github.com/apache/spark/commit/832d9229c0806b5d49c91fb692e71221781f5c17).
    * 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604997533
 
 
   **[Test build #120492 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120492/testReport)** for PR 28043 at commit [`6e70768`](https://github.com/apache/spark/commit/6e7076813991a486f3266cb195241bafa0bfc5ce).

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604832312
 
 
   **[Test build #120451 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120451/testReport)** for PR 28043 at commit [`b8a538e`](https://github.com/apache/spark/commit/b8a538ecef2945bba5083992d4de339d104148ba).

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604966459
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605623706
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604847826
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120452/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604885396
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604832636
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605039330
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120479/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605623706
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604966459
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604903802
 
 
   **[Test build #120481 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120481/testReport)** for PR 28043 at commit [`2eccef9`](https://github.com/apache/spark/commit/2eccef91f78d7776645b5e4ecbf4af6235878fb9).

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604847747
 
 
   **[Test build #120452 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120452/testReport)** for PR 28043 at commit [`25e37bc`](https://github.com/apache/spark/commit/25e37bc8e2a64ebd760cbed300ef4f780a713a1b).
    * 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


With regards,
Apache Git Services

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


[GitHub] [spark] yaooqinn commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#discussion_r399459570
 
 

 ##########
 File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala
 ##########
 @@ -52,4 +52,13 @@ class RewriteSubquerySuite extends PlanTest {
     comparePlans(optimized, correctAnswer)
   }
 
+  test("Perform propagating empty relation after RewritePredicateSubquery") {
 
 Review comment:
   I added the Jira id because I also think it is good to add in this case although it does not exactly fit these rules

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604832636
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605323932
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605095198
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120492/
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604998194
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25197/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605595565
 
 
   **[Test build #120546 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120546/testReport)** for PR 28043 at commit [`b084a27`](https://github.com/apache/spark/commit/b084a27dec2f68d179a6c245f2845b45232baf63).
    * 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605095192
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-604998194
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25197/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28043: [SPARK-31280][SQL] Perform propagating empty relation after RewritePredicateSubquery
URL: https://github.com/apache/spark/pull/28043#issuecomment-605009260
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120481/
   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


With regards,
Apache Git Services

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