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 2021/02/27 17:36:06 UTC

[GitHub] [spark] MaxGekk opened a new pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

MaxGekk opened a new pull request #31676:
URL: https://github.com/apache/spark/pull/31676


   ### What changes were proposed in this pull request?
   In the PR, I propose to generate "stable" output attributes per the logical node of the `DESCRIBE TABLE` command.
   
   ### Why are the changes needed?
   This fixes the issue demonstrated by the example:
   ```scala
   val tbl = "testcat.ns1.ns2.tbl"
   sql(s"CREATE TABLE $tbl (c0 INT) USING _")
   val description = sql(s"DESCRIBE TABLE $tbl")
   description.drop("comment")
   ```
   The `drop()` method fails with the error:
   ```
   org.apache.spark.sql.AnalysisException: Resolved attribute(s) col_name#102,data_type#103 missing from col_name#29,data_type#30,comment#31 in operator !Project [col_name#102, data_type#103]. Attribute(s) with the same name appear in the operation: col_name,data_type. Please check if the right attribute(s) are used.;
   !Project [col_name#102, data_type#103]
   +- LocalRelation [col_name#29, data_type#30, comment#31]
   
   	at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.failAnalysis(CheckAnalysis.scala:51)
   	at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.failAnalysis$(CheckAnalysis.scala:50)
   ```
   
   ### Does this PR introduce _any_ user-facing change?
   Yes. After the changes, `drop()`/`add()` works as expected:
   ```scala
   description.drop("comment").show()
   +---------------+---------+
   |       col_name|data_type|
   +---------------+---------+
   |             c0|      int|
   |               |         |
   | # Partitioning|         |
   |Not partitioned|         |
   +---------------+---------+
   ```
   
   ### How was this patch tested?
   1. Run new test:
   ```
   $ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *DataSourceV2SQLSuite"
   ```
   2. Run existing test suite:
   ```
   $ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *CatalogedDDLSuite"
   ```


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

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



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


[GitHub] [spark] SparkQA commented on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


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


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

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


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


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

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



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


[GitHub] [spark] AmplabJenkins commented on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


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


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

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



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


[GitHub] [spark] SparkQA commented on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


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


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

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


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


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

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



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


[GitHub] [spark] cloud-fan edited a comment on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

Posted by GitBox <gi...@apache.org>.
cloud-fan edited a comment on pull request #31676:
URL: https://github.com/apache/spark/pull/31676#issuecomment-787984907


   thanks, merging to master/3.1! (it has many conflicts in branch-3.0 and may not worth to backport).


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

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


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


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

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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##########
@@ -347,7 +347,7 @@ case class DescribeRelation(
     partitionSpec: TablePartitionSpec,
     isExtended: Boolean) extends Command {
   override def children: Seq[LogicalPlan] = Seq(relation)
-  override def output: Seq[Attribute] = DescribeCommandSchema.describeTableAttributes()
+  override val output: Seq[Attribute] = DescribeCommandSchema.describeTableAttributes()

Review comment:
       Shall we follow others like `ShowTables` and put the `output` as a parameter? Then it's more stable and the output won't after copy/transformation.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##########
@@ -347,7 +347,7 @@ case class DescribeRelation(
     partitionSpec: TablePartitionSpec,
     isExtended: Boolean) extends Command {
   override def children: Seq[LogicalPlan] = Seq(relation)
-  override def output: Seq[Attribute] = DescribeCommandSchema.describeTableAttributes()
+  override val output: Seq[Attribute] = DescribeCommandSchema.describeTableAttributes()

Review comment:
       Shall we follow others like `ShowTables` and put the `output` as a parameter? Then it's more stable and the output won't change after copy/transformation.




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

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



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


[GitHub] [spark] SparkQA commented on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


   **[Test build #135593 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135593/testReport)** for PR 31676 at commit [`05667cf`](https://github.com/apache/spark/commit/05667cf4c455abaee9c415bc55a9a442fef341dd).


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

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



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


[GitHub] [spark] SparkQA commented on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


   **[Test build #135546 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135546/testReport)** for PR 31676 at commit [`82b2cee`](https://github.com/apache/spark/commit/82b2cee18ac48dc01d04357f5c22345df2049676).


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

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


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


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

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



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


[GitHub] [spark] SparkQA commented on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


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


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

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



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


[GitHub] [spark] AmplabJenkins commented on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


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


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

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



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


[GitHub] [spark] SparkQA commented on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


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


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

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



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


[GitHub] [spark] SparkQA commented on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


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


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

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



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


[GitHub] [spark] SparkQA commented on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


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


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

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



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


[GitHub] [spark] SparkQA removed a comment on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


   **[Test build #135593 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135593/testReport)** for PR 31676 at commit [`05667cf`](https://github.com/apache/spark/commit/05667cf4c455abaee9c415bc55a9a442fef341dd).


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

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


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


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

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


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


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

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



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


[GitHub] [spark] AngersZhuuuu commented on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


   > Looks like we should fix all other instances too? But could be done separately.
   
   Yea,  have fix some of this. But maybe there are still incorrect instance.
   


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

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



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


[GitHub] [spark] SparkQA removed a comment on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


   **[Test build #135546 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135546/testReport)** for PR 31676 at commit [`82b2cee`](https://github.com/apache/spark/commit/82b2cee18ac48dc01d04357f5c22345df2049676).


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

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



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


[GitHub] [spark] HyukjinKwon commented on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


   Looks like we should fix all other instances too? But could be done separately.


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

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



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


[GitHub] [spark] AmplabJenkins commented on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


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


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

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



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


[GitHub] [spark] AngersZhuuuu edited a comment on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu edited a comment on pull request #31676:
URL: https://github.com/apache/spark/pull/31676#issuecomment-787670462


   > Looks like we should fix all other instances too? But could be done separately.
   
   Yea,  have fix some of this. But maybe there are still incorrect instance.
   https://issues.apache.org/jira/browse/SPARK-34576
   https://issues.apache.org/jira/browse/SPARK-34577
   and `ExternalCommandExecutor` have similar problem too.


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

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



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


[GitHub] [spark] SparkQA commented on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


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


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

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



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


[GitHub] [spark] AmplabJenkins commented on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


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


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

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



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


[GitHub] [spark] cloud-fan commented on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


   also cc @beliefer @AngersZhuuuu who worked on similar issues before.


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

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



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


[GitHub] [spark] AmplabJenkins commented on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


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


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

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



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


[GitHub] [spark] SparkQA removed a comment on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


   **[Test build #135582 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135582/testReport)** for PR 31676 at commit [`dcb97af`](https://github.com/apache/spark/commit/dcb97aff36df4743cd36fb8deed2b804839dd120).


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

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



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


[GitHub] [spark] MaxGekk commented on a change in pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##########
@@ -347,7 +347,7 @@ case class DescribeRelation(
     partitionSpec: TablePartitionSpec,
     isExtended: Boolean) extends Command {
   override def children: Seq[LogicalPlan] = Seq(relation)
-  override def output: Seq[Attribute] = DescribeCommandSchema.describeTableAttributes()
+  override val output: Seq[Attribute] = DescribeCommandSchema.describeTableAttributes()

Review comment:
       > Then it's more stable and the output won't change after copy/transformation.
   
   ok. `val output` will be re-initialized per every `.copy()`. I will make it as a case class parameter.




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

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



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


[GitHub] [spark] SparkQA commented on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


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


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

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



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


[GitHub] [spark] cloud-fan commented on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


   thanks, merging to master/3.1/3.0!


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

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



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


[GitHub] [spark] AngersZhuuuu edited a comment on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu edited a comment on pull request #31676:
URL: https://github.com/apache/spark/pull/31676#issuecomment-787670462


   > Looks like we should fix all other instances too? But could be done separately.
   
   Yea,  have fix some of this. But maybe there are still incorrect instance.
   https://issues.apache.org/jira/browse/SPARK-34576
   https://issues.apache.org/jira/browse/SPARK-34577


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

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



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


[GitHub] [spark] MaxGekk commented on a change in pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##########
@@ -347,7 +347,7 @@ case class DescribeRelation(
     partitionSpec: TablePartitionSpec,
     isExtended: Boolean) extends Command {
   override def children: Seq[LogicalPlan] = Seq(relation)
-  override def output: Seq[Attribute] = DescribeCommandSchema.describeTableAttributes()
+  override val output: Seq[Attribute] = DescribeCommandSchema.describeTableAttributes()

Review comment:
       > Shall we follow others like ShowTables and put the output as a parameter?
   
   Put the `output` as a parameter doesn't solve any problems.
   
   > Then it's more stable ...
   
   I would say "super stable" even it is not necessary, see https://github.com/apache/spark/pull/31675




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

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



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


[GitHub] [spark] AngersZhuuuu edited a comment on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

Posted by GitBox <gi...@apache.org>.
AngersZhuuuu edited a comment on pull request #31676:
URL: https://github.com/apache/spark/pull/31676#issuecomment-787670462


   > Looks like we should fix all other instances too? But could be done separately.
   
   Yea,  have fix some of this. But maybe there are still incorrect instance.
   https://issues.apache.org/jira/browse/SPARK-34576
   https://issues.apache.org/jira/browse/SPARK-34577


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

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



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


[GitHub] [spark] cloud-fan closed pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


   


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

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



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


[GitHub] [spark] SparkQA commented on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


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


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

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



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


[GitHub] [spark] AmplabJenkins commented on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


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


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

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



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


[GitHub] [spark] SparkQA commented on pull request #31676: [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 `DESCRIBE TABLE`

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


   **[Test build #135582 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135582/testReport)** for PR 31676 at commit [`dcb97af`](https://github.com/apache/spark/commit/dcb97aff36df4743cd36fb8deed2b804839dd120).


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

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



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