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/02/19 06:27:13 UTC

[GitHub] [spark] wangyum opened a new pull request #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

wangyum opened a new pull request #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632
 
 
   ### What changes were proposed in this pull request?
   This PR fix a special case about infer additional constraints. How to reproduce this issue:
   ```scala
   scala> spark.range(20).selectExpr("id as a", "id as b", "id as c").write.saveAsTable("t1")
   
   scala> spark.sql("select count(*) from t1 where a = b and b = c and (c = 3 or c = 13)").explain(false)
   == Physical Plan ==
   *(2) HashAggregate(keys=[], functions=[count(1)])
   +- Exchange SinglePartition, true, [id=#76]
      +- *(1) HashAggregate(keys=[], functions=[partial_count(1)])
         +- *(1) Project
            +- *(1) Filter (((((((isnotnull(c#36L) AND ((b#35L = 3) OR (b#35L = 13))) AND isnotnull(b#35L)) AND (a#34L = c#36L)) AND isnotnull(a#34L)) AND (a#34L = b#35L)) AND (b#35L = c#36L)) AND ((c#36L = 3) OR (c#36L = 13)))
               +- *(1) ColumnarToRow
                  +- FileScan parquet default.t1[a#34L,b#35L,c#36L] Batched: true, DataFilters: [isnotnull(c#36L), ((b#35L = 3) OR (b#35L = 13)), isnotnull(b#35L), (a#34L = c#36L), isnotnull(a#..., Format: Parquet, Location: InMemoryFileIndex[file:/Users/yumwang/Downloads/spark-3.0.0-preview2-bin-hadoop2.7/spark-warehous..., PartitionFilters: [], PushedFilters: [IsNotNull(c), Or(EqualTo(b,3),EqualTo(b,13)), IsNotNull(b), IsNotNull(a), Or(EqualTo(c,3),EqualT..., ReadSchema: struct<a:bigint,b:bigint,c:bigint>
   ```
   
   We can infer more constraints: `(a#34L = 3) OR (a#34L = 13)`.
   
   
   ### Why are the changes needed?
   
   Improve query performance.
   
   ### Does this PR introduce any user-facing change?
   No.
   
   
   ### How was this patch tested?
   Unit 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 commented on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588585494
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23444/
   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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588278036
 
 
   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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588585486
 
 
   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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588086862
 
 
   **[Test build #118663 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118663/testReport)** for PR 27632 at commit [`2fe8253`](https://github.com/apache/spark/commit/2fe825302bb6c9252ee135f9febfe339b9af74be).
    * 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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-590163946
 
 
   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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588058965
 
 
   **[Test build #118663 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118663/testReport)** for PR 27632 at commit [`2fe8253`](https://github.com/apache/spark/commit/2fe825302bb6c9252ee135f9febfe339b9af74be).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588059392
 
 
   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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-590163950
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23590/
   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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-590207186
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23604/
   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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-590170275
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118842/
   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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588087038
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118663/
   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] wangyum commented on a change in pull request #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
wangyum commented on a change in pull request #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#discussion_r384385297
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
 ##########
 @@ -3404,6 +3404,15 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
       """.stripMargin)
     checkAnswer(df, Row(Row(1, 2)) :: Nil)
   }
+
+  test("SPARK-30872: Constraints inferred from inferred attributes") {
 
 Review comment:
   It will throws `TreeNodeException`:
   ```
   [info] - SPARK-30872: Constraints inferred from inferred attributes *** FAILED *** (146 milliseconds)
   [info]   org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Once strategy's idempotence is broken for batch Infer Filters
   [info]  Aggregate [count(1) AS count(1)#19182L]                                                                                                                                                                                                                        Aggregate [count(1) AS count(1)#19182L]
   [info]  +- Project                                                                                                                                                                                                                                                     +- Project
   [info] !   +- Filter ((((((a#19179L = c#19181L) AND isnotnull(b#19180L)) AND isnotnull(c#19181L)) AND ((b#19180L = 3) OR (b#19180L = 13))) AND isnotnull(a#19179L)) AND (((a#19179L = b#19180L) AND (b#19180L = c#19181L)) AND ((c#19181L = 3) OR (c#19181L = 13))))      +- Filter (((a#19179L = 3) OR (a#19179L = 13)) AND ((((((a#19179L = c#19181L) AND isnotnull(b#19180L)) AND isnotnull(c#19181L)) AND ((b#19180L = 3) OR (b#19180L = 13))) AND isnotnull(a#19179L)) AND (((a#19179L = b#19180L) AND (b#19180L = c#19181L)) AND ((c#19181L = 3) OR (c#19181L = 13)))))
   [info]        +- Relation[a#19179L,b#19180L,c#19181L] parquet                                                                                                                                                                                                                +- Relation[a#19179L,b#19180L,c#19181L] parquet
   [info]           , tree:
   [info] Aggregate [count(1) AS count(1)#19182L]
   [info] +- Project
   [info]    +- Filter (((a#19179L = 3) OR (a#19179L = 13)) AND ((((((a#19179L = c#19181L) AND isnotnull(b#19180L)) AND isnotnull(c#19181L)) AND ((b#19180L = 3) OR (b#19180L = 13))) AND isnotnull(a#19179L)) AND (((a#19179L = b#19180L) AND (b#19180L = c#19181L)) AND ((c#19181L = 3) OR (c#19181L = 13)))))
   [info]       +- Relation[a#19179L,b#19180L,c#19181L] parquet
   [info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.checkBatchIdempotence(RuleExecutor.scala:100)
   [info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:187)
   [info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:132)
   [info]   at scala.collection.immutable.List.foreach(List.scala:392)
   [info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:132)
   [info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:111)
   [info]   at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:88)
   [info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:111)
   [info]   at org.apache.spark.sql.execution.QueryExecution.$anonfun$optimizedPlan$1(QueryExecution.scala:82)
   [info]   at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
   [info]   at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:119)
   [info]   at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:762)
   [info]   at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:119)
   [info]   at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:82)
   [info]   at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:79)
   [info]   at org.apache.spark.sql.QueryTest.assertEmptyMissingInput(QueryTest.scala:231)
   [info]   at org.apache.spark.sql.QueryTest.checkAnswer(QueryTest.scala:154)
   [info]   at org.apache.spark.sql.SQLQuerySuite.$anonfun$new$746(SQLQuerySuite.scala:3413)
   [info]   at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
   
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588278139
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118668/
   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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-590206747
 
 
   **[Test build #118855 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118855/testReport)** for PR 27632 at commit [`884ba16`](https://github.com/apache/spark/commit/884ba16ae17c07aa5d7d674e592b6afb17d75b11).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-590170273
 
 
   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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-590164847
 
 
   **[Test build #118842 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118842/testReport)** for PR 27632 at commit [`884ba16`](https://github.com/apache/spark/commit/884ba16ae17c07aa5d7d674e592b6afb17d75b11).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588059392
 
 
   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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-590301963
 
 
   **[Test build #118855 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118855/testReport)** for PR 27632 at commit [`884ba16`](https://github.com/apache/spark/commit/884ba16ae17c07aa5d7d674e592b6afb17d75b11).
    * 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 commented on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588661667
 
 
   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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-591307792
 
 
   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] wangyum commented on a change in pull request #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
wangyum commented on a change in pull request #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#discussion_r384385297
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
 ##########
 @@ -3404,6 +3404,15 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
       """.stripMargin)
     checkAnswer(df, Row(Row(1, 2)) :: Nil)
   }
+
+  test("SPARK-30872: Constraints inferred from inferred attributes") {
 
 Review comment:
   It will throw `TreeNodeException: Once strategy's idempotence is broken for batch Infer Filters` before this PR:
   ```
   [info] - SPARK-30872: Constraints inferred from inferred attributes *** FAILED *** (146 milliseconds)
   [info]   org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Once strategy's idempotence is broken for batch Infer Filters
   [info]  Aggregate [count(1) AS count(1)#19182L]                                                                                                                                                                                                                        Aggregate [count(1) AS count(1)#19182L]
   [info]  +- Project                                                                                                                                                                                                                                                     +- Project
   [info] !   +- Filter ((((((a#19179L = c#19181L) AND isnotnull(b#19180L)) AND isnotnull(c#19181L)) AND ((b#19180L = 3) OR (b#19180L = 13))) AND isnotnull(a#19179L)) AND (((a#19179L = b#19180L) AND (b#19180L = c#19181L)) AND ((c#19181L = 3) OR (c#19181L = 13))))      +- Filter (((a#19179L = 3) OR (a#19179L = 13)) AND ((((((a#19179L = c#19181L) AND isnotnull(b#19180L)) AND isnotnull(c#19181L)) AND ((b#19180L = 3) OR (b#19180L = 13))) AND isnotnull(a#19179L)) AND (((a#19179L = b#19180L) AND (b#19180L = c#19181L)) AND ((c#19181L = 3) OR (c#19181L = 13)))))
   [info]        +- Relation[a#19179L,b#19180L,c#19181L] parquet                                                                                                                                                                                                                +- Relation[a#19179L,b#19180L,c#19181L] parquet
   [info]           , tree:
   [info] Aggregate [count(1) AS count(1)#19182L]
   [info] +- Project
   [info]    +- Filter (((a#19179L = 3) OR (a#19179L = 13)) AND ((((((a#19179L = c#19181L) AND isnotnull(b#19180L)) AND isnotnull(c#19181L)) AND ((b#19180L = 3) OR (b#19180L = 13))) AND isnotnull(a#19179L)) AND (((a#19179L = b#19180L) AND (b#19180L = c#19181L)) AND ((c#19181L = 3) OR (c#19181L = 13)))))
   [info]       +- Relation[a#19179L,b#19180L,c#19181L] parquet
   [info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.checkBatchIdempotence(RuleExecutor.scala:100)
   [info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:187)
   [info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:132)
   [info]   at scala.collection.immutable.List.foreach(List.scala:392)
   [info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:132)
   [info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:111)
   [info]   at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:88)
   [info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:111)
   [info]   at org.apache.spark.sql.execution.QueryExecution.$anonfun$optimizedPlan$1(QueryExecution.scala:82)
   [info]   at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
   [info]   at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:119)
   [info]   at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:762)
   [info]   at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:119)
   [info]   at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:82)
   [info]   at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:79)
   [info]   at org.apache.spark.sql.QueryTest.assertEmptyMissingInput(QueryTest.scala:231)
   [info]   at org.apache.spark.sql.QueryTest.checkAnswer(QueryTest.scala:154)
   [info]   at org.apache.spark.sql.SQLQuerySuite.$anonfun$new$746(SQLQuerySuite.scala:3413)
   [info]   at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
   
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588094503
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23420/
   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] wangyum commented on a change in pull request #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
wangyum commented on a change in pull request #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#discussion_r381888828
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala
 ##########
 @@ -322,4 +322,13 @@ class DataFrameJoinSuite extends QueryTest
       }
     }
   }
+
+  test("SPARK-30872: Constraints inferred from inferred attributes") {
+    withTable("t1") {
+      spark.range(20).selectExpr("id as a", "id as b", "id as c").write.saveAsTable("t1")
+      checkAnswer(
+        spark.sql("select count(*) from t1 where a = b and b = c and (c = 3 or c = 13)"),
+        Row(2) :: Nil)
+    }
+  }
 
 Review comment:
   It will throw `TreeNodeException: Once strategy's idempotence is broken for batch Infer Filters` before this PR:
   ```java
   [info] - SPARK-30872: Constraints inferred from inferred attributes *** FAILED *** (222 milliseconds)
   [info]   org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Once strategy's idempotence is broken for batch Infer Filters
   [info]  Aggregate [count(1) AS count(1)#5842L]                                                                                                                                                                                                            Aggregate [count(1) AS count(1)#5842L]
   [info]  +- Project                                                                                                                                                                                                                                        +- Project
   [info] !   +- Filter ((((((a#5839L = c#5841L) AND isnotnull(b#5840L)) AND isnotnull(c#5841L)) AND ((b#5840L = 3) OR (b#5840L = 13))) AND isnotnull(a#5839L)) AND (((a#5839L = b#5840L) AND (b#5840L = c#5841L)) AND ((c#5841L = 3) OR (c#5841L = 13))))      +- Filter (((a#5839L = 3) OR (a#5839L = 13)) AND ((((((a#5839L = c#5841L) AND isnotnull(b#5840L)) AND isnotnull(c#5841L)) AND ((b#5840L = 3) OR (b#5840L = 13))) AND isnotnull(a#5839L)) AND (((a#5839L = b#5840L) AND (b#5840L = c#5841L)) AND ((c#5841L = 3) OR (c#5841L = 13)))))
   [info]        +- Relation[a#5839L,b#5840L,c#5841L] parquet                                                                                                                                                                                                      +- Relation[a#5839L,b#5840L,c#5841L] parquet
   [info]           , tree:
   [info] Aggregate [count(1) AS count(1)#5842L]
   [info] +- Project
   [info]    +- Filter (((a#5839L = 3) OR (a#5839L = 13)) AND ((((((a#5839L = c#5841L) AND isnotnull(b#5840L)) AND isnotnull(c#5841L)) AND ((b#5840L = 3) OR (b#5840L = 13))) AND isnotnull(a#5839L)) AND (((a#5839L = b#5840L) AND (b#5840L = c#5841L)) AND ((c#5841L = 3) OR (c#5841L = 13)))))
   [info]       +- Relation[a#5839L,b#5840L,c#5841L] parquet
   [info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.checkBatchIdempotence(RuleExecutor.scala:100)
   [info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:187)
   [info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:132)
   [info]   at scala.collection.immutable.List.foreach(List.scala:392)
   [info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:132)
   [info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:111)
   [info]   at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:88)
   [info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:111)
   [info]   at org.apache.spark.sql.execution.QueryExecution.$anonfun$optimizedPlan$1(QueryExecution.scala:82)
   [info]   at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
   [info]   at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:119)
   [info]   at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:762)
   [info]   at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:119)
   [info]   at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:82)
   [info]   at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:79)
   [info]   at org.apache.spark.sql.QueryTest.assertEmptyMissingInput(QueryTest.scala:231)
   [info]   at org.apache.spark.sql.QueryTest.checkAnswer(QueryTest.scala:154)
   [info]   at org.apache.spark.sql.DataFrameJoinSuite.$anonfun$new$51(DataFrameJoinSuite.scala:332)
   [info]   at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
   [info]   at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
   [info]   at org.apache.spark.sql.test.SQLTestUtilsBase.withTable(SQLTestUtils.scala:290)
   [info]   at org.apache.spark.sql.test.SQLTestUtilsBase.withTable$(SQLTestUtils.scala:288)
   [info]   at org.apache.spark.sql.DataFrameJoinSuite.withTable(DataFrameJoinSuite.scala:29)
   [info]   at org.apache.spark.sql.DataFrameJoinSuite.$anonfun$new$50(DataFrameJoinSuite.scala:328)
   [info]   at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
   [info]   at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
   [info]   at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
   [info]   at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
   [info]   at org.scalatest.Transformer.apply(Transformer.scala:22)
   [info]   at org.scalatest.Transformer.apply(Transformer.scala:20)
   [info]   at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:186)
   [info]   at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:151)
   [info]   at org.scalatest.FunSuiteLike.invokeWithFixture$1(FunSuiteLike.scala:184)
   [info]   at org.scalatest.FunSuiteLike.$anonfun$runTest$1(FunSuiteLike.scala:196)
   [info]   at org.scalatest.SuperEngine.runTestImpl(Engine.scala:286)
   [info]   at org.scalatest.FunSuiteLike.runTest(FunSuiteLike.scala:196)
   [info]   at org.scalatest.FunSuiteLike.runTest$(FunSuiteLike.scala:178)
   [info]   at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:58)
   [info]   at org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:221)
   [info]   at org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:214)
   [info]   at org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:58)
   [info]   at org.scalatest.FunSuiteLike.$anonfun$runTests$1(FunSuiteLike.scala:229)
   [info]   at org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:393)
   [info]   at scala.collection.immutable.List.foreach(List.scala:392)
   [info]   at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:381)
   [info]   at org.scalatest.SuperEngine.runTestsInBranch(Engine.scala:376)
   [info]   at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:458)
   [info]   at org.scalatest.FunSuiteLike.runTests(FunSuiteLike.scala:229)
   [info]   at org.scalatest.FunSuiteLike.runTests$(FunSuiteLike.scala:228)
   [info]   at org.scalatest.FunSuite.runTests(FunSuite.scala:1560)
   [info]   at org.scalatest.Suite.run(Suite.scala:1124)
   [info]   at org.scalatest.Suite.run$(Suite.scala:1106)
   [info]   at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1560)
   [info]   at org.scalatest.FunSuiteLike.$anonfun$run$1(FunSuiteLike.scala:233)
   [info]   at org.scalatest.SuperEngine.runImpl(Engine.scala:518)
   [info]   at org.scalatest.FunSuiteLike.run(FunSuiteLike.scala:233)
   [info]   at org.scalatest.FunSuiteLike.run$(FunSuiteLike.scala:232)
   [info]   at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:58)
   [info]   at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:213)
   [info]   at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
   [info]   at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
   [info]   at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:58)
   [info]   at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:317)
   [info]   at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:510)
   [info]   at sbt.ForkMain$Run$2.call(ForkMain.java:296)
   [info]   at sbt.ForkMain$Run$2.call(ForkMain.java:286)
   [info]   at java.util.concurrent.FutureTask.run(FutureTask.java:266)
   [info]   at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
   [info]   at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
   [info]   at java.lang.Thread.run(Thread.java:748)
   
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-591431752
 
 
   **[Test build #118963 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118963/testReport)** for PR 27632 at commit [`bf7b8e5`](https://github.com/apache/spark/commit/bf7b8e561029791daa17ffb90aac9e970f002cf9).
    * 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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588278036
 
 
   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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-590302956
 
 
   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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-590170223
 
 
   **[Test build #118842 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118842/testReport)** for PR 27632 at commit [`884ba16`](https://github.com/apache/spark/commit/884ba16ae17c07aa5d7d674e592b6afb17d75b11).
    * 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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-591432526
 
 
   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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588661698
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118693/
   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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588087029
 
 
   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] wangyum commented on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
wangyum commented on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588875904
 
 
   cc @cloud-fan

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588278139
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118668/
   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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-590207178
 
 
   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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588059400
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23415/
   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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588658774
 
 
   **[Test build #118693 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118693/testReport)** for PR 27632 at commit [`780fc35`](https://github.com/apache/spark/commit/780fc3531d39c9a0ef1143b8c49472fe22db9875).
    * 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 commented on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-591307792
 
 
   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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588585494
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23444/
   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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588059400
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23415/
   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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588093565
 
 
   **[Test build #118668 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118668/testReport)** for PR 27632 at commit [`2fe8253`](https://github.com/apache/spark/commit/2fe825302bb6c9252ee135f9febfe339b9af74be).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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] wangyum commented on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
wangyum commented on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588091186
 
 
   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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-590302967
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118855/
   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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-590206747
 
 
   **[Test build #118855 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118855/testReport)** for PR 27632 at commit [`884ba16`](https://github.com/apache/spark/commit/884ba16ae17c07aa5d7d674e592b6afb17d75b11).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-590302956
 
 
   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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588585486
 
 
   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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588094469
 
 
   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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-591432526
 
 
   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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-590207178
 
 
   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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588087038
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118663/
   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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-591432533
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118963/
   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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-591310414
 
 
   **[Test build #118963 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118963/testReport)** for PR 27632 at commit [`bf7b8e5`](https://github.com/apache/spark/commit/bf7b8e561029791daa17ffb90aac9e970f002cf9).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-590163946
 
 
   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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588094469
 
 
   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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588661667
 
 
   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] wangyum commented on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
wangyum commented on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-590205229
 
 
   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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-590207186
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23604/
   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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588087029
 
 
   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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-590164847
 
 
   **[Test build #118842 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118842/testReport)** for PR 27632 at commit [`884ba16`](https://github.com/apache/spark/commit/884ba16ae17c07aa5d7d674e592b6afb17d75b11).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588585126
 
 
   **[Test build #118693 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118693/testReport)** for PR 27632 at commit [`780fc35`](https://github.com/apache/spark/commit/780fc3531d39c9a0ef1143b8c49472fe22db9875).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588094503
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23420/
   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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-591432533
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118963/
   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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-590163950
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23590/
   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] wangyum commented on a change in pull request #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
wangyum commented on a change in pull request #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#discussion_r381892261
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala
 ##########
 @@ -75,7 +85,7 @@ trait ConstraintHelper {
         inferredConstraints ++= replaceConstraints(predicates - eq, l, r)
       case _ => // No inference
     }
-    inferredConstraints -- constraints
+    (inferredConstraints -- constraints).filterNot(i => constraints.exists(_.semanticEquals(i)))
 
 Review comment:
   If the `constraint` contains `a = b`. This change is to filter out `b = a`.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-590170275
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118842/
   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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-590302967
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118855/
   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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588058965
 
 
   **[Test build #118663 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118663/testReport)** for PR 27632 at commit [`2fe8253`](https://github.com/apache/spark/commit/2fe825302bb6c9252ee135f9febfe339b9af74be).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588661698
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118693/
   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 #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27632: [WIP][SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-588585126
 
 
   **[Test build #118693 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118693/testReport)** for PR 27632 at commit [`780fc35`](https://github.com/apache/spark/commit/780fc3531d39c9a0ef1143b8c49472fe22db9875).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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] wangyum commented on a change in pull request #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
wangyum commented on a change in pull request #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#discussion_r381888828
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala
 ##########
 @@ -322,4 +322,13 @@ class DataFrameJoinSuite extends QueryTest
       }
     }
   }
+
+  test("SPARK-30872: Constraints inferred from inferred attributes") {
+    withTable("t1") {
+      spark.range(20).selectExpr("id as a", "id as b", "id as c").write.saveAsTable("t1")
+      checkAnswer(
+        spark.sql("select count(*) from t1 where a = b and b = c and (c = 3 or c = 13)"),
+        Row(2) :: Nil)
+    }
+  }
 
 Review comment:
   It will throw `TreeNodeException: Once strategy's idempotence is broken for batch Infer Filters` before this PR:
   ```java
   [info] - SPARK-30872: Constraints inferred from inferred attributes *** FAILED *** (222 milliseconds)
   [info]   org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Once strategy's idempotence is broken for batch Infer Filters
   [info]  Aggregate [count(1) AS count(1)#5842L]                                                                                                                                                                                                            Aggregate [count(1) AS count(1)#5842L]
   [info]  +- Project                                                                                                                                                                                                                                        +- Project
   [info] !   +- Filter ((((((a#5839L = c#5841L) AND isnotnull(b#5840L)) AND isnotnull(c#5841L)) AND ((b#5840L = 3) OR (b#5840L = 13))) AND isnotnull(a#5839L)) AND (((a#5839L = b#5840L) AND (b#5840L = c#5841L)) AND ((c#5841L = 3) OR (c#5841L = 13))))      +- Filter (((a#5839L = 3) OR (a#5839L = 13)) AND ((((((a#5839L = c#5841L) AND isnotnull(b#5840L)) AND isnotnull(c#5841L)) AND ((b#5840L = 3) OR (b#5840L = 13))) AND isnotnull(a#5839L)) AND (((a#5839L = b#5840L) AND (b#5840L = c#5841L)) AND ((c#5841L = 3) OR (c#5841L = 13)))))
   [info]        +- Relation[a#5839L,b#5840L,c#5841L] parquet                                                                                                                                                                                                      +- Relation[a#5839L,b#5840L,c#5841L] parquet
   [info]           , tree:
   [info] Aggregate [count(1) AS count(1)#5842L]
   [info] +- Project
   [info]    +- Filter (((a#5839L = 3) OR (a#5839L = 13)) AND ((((((a#5839L = c#5841L) AND isnotnull(b#5840L)) AND isnotnull(c#5841L)) AND ((b#5840L = 3) OR (b#5840L = 13))) AND isnotnull(a#5839L)) AND (((a#5839L = b#5840L) AND (b#5840L = c#5841L)) AND ((c#5841L = 3) OR (c#5841L = 13)))))
   [info]       +- Relation[a#5839L,b#5840L,c#5841L] parquet
   [info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.checkBatchIdempotence(RuleExecutor.scala:100)
   [info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:187)
   [info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:132)
   [info]   at scala.collection.immutable.List.foreach(List.scala:392)
   [info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:132)
   [info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:111)
   [info]   at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:88)
   [info]   at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:111)
   [info]   at org.apache.spark.sql.execution.QueryExecution.$anonfun$optimizedPlan$1(QueryExecution.scala:82)
   [info]   at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
   [info]   at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:119)
   [info]   at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:762)
   [info]   at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:119)
   [info]   at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:82)
   [info]   at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:79)
   [info]   at org.apache.spark.sql.QueryTest.assertEmptyMissingInput(QueryTest.scala:231)
   [info]   at org.apache.spark.sql.QueryTest.checkAnswer(QueryTest.scala:154)
   [info]   at org.apache.spark.sql.DataFrameJoinSuite.$anonfun$new$51(DataFrameJoinSuite.scala:332)
   [info]   at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
   [info]   at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
   [info]   at org.apache.spark.sql.test.SQLTestUtilsBase.withTable(SQLTestUtils.scala:290)
   [info]   at org.apache.spark.sql.test.SQLTestUtilsBase.withTable$(SQLTestUtils.scala:288)
   [info]   at org.apache.spark.sql.DataFrameJoinSuite.withTable(DataFrameJoinSuite.scala:29)
   [info]   at org.apache.spark.sql.DataFrameJoinSuite.$anonfun$new$50(DataFrameJoinSuite.scala:328)
   [info]   at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
   [info]   at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
   [info]   at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
   [info]   at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
   [info]   at org.scalatest.Transformer.apply(Transformer.scala:22)
   [info]   at org.scalatest.Transformer.apply(Transformer.scala:20)
   [info]   at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:186)
   [info]   at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:151)
   [info]   at org.scalatest.FunSuiteLike.invokeWithFixture$1(FunSuiteLike.scala:184)
   [info]   at org.scalatest.FunSuiteLike.$anonfun$runTest$1(FunSuiteLike.scala:196)
   [info]   at org.scalatest.SuperEngine.runTestImpl(Engine.scala:286)
   [info]   at org.scalatest.FunSuiteLike.runTest(FunSuiteLike.scala:196)
   [info]   at org.scalatest.FunSuiteLike.runTest$(FunSuiteLike.scala:178)
   [info]   at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:58)
   [info]   at org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:221)
   [info]   at org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:214)
   [info]   at org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:58)
   [info]   at org.scalatest.FunSuiteLike.$anonfun$runTests$1(FunSuiteLike.scala:229)
   [info]   at org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:393)
   [info]   at scala.collection.immutable.List.foreach(List.scala:392)
   [info]   at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:381)
   [info]   at org.scalatest.SuperEngine.runTestsInBranch(Engine.scala:376)
   [info]   at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:458)
   [info]   at org.scalatest.FunSuiteLike.runTests(FunSuiteLike.scala:229)
   [info]   at org.scalatest.FunSuiteLike.runTests$(FunSuiteLike.scala:228)
   [info]   at org.scalatest.FunSuite.runTests(FunSuite.scala:1560)
   [info]   at org.scalatest.Suite.run(Suite.scala:1124)
   [info]   at org.scalatest.Suite.run$(Suite.scala:1106)
   [info]   at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1560)
   [info]   at org.scalatest.FunSuiteLike.$anonfun$run$1(FunSuiteLike.scala:233)
   [info]   at org.scalatest.SuperEngine.runImpl(Engine.scala:518)
   [info]   at org.scalatest.FunSuiteLike.run(FunSuiteLike.scala:233)
   [info]   at org.scalatest.FunSuiteLike.run$(FunSuiteLike.scala:232)
   [info]   at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:58)
   [info]   at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:213)
   [info]   at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
   [info]   at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
   [info]   at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:58)
   [info]   at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:317)
   [info]   at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:510)
   [info]   at sbt.ForkMain$Run$2.call(ForkMain.java:296)
   [info]   at sbt.ForkMain$Run$2.call(ForkMain.java:286)
   [info]   at java.util.concurrent.FutureTask.run(FutureTask.java:266)
   [info]   at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
   [info]   at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
   [info]   at java.lang.Thread.run(Thread.java:748)
   
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-590170273
 
 
   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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-591307796
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23710/
   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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-591310414
 
 
   **[Test build #118963 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118963/testReport)** for PR 27632 at commit [`bf7b8e5`](https://github.com/apache/spark/commit/bf7b8e561029791daa17ffb90aac9e970f002cf9).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27632: [SPARK-30872][SQL] Constraints inferred from inferred attributes
URL: https://github.com/apache/spark/pull/27632#issuecomment-591307796
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23710/
   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