You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by cloud-fan <gi...@git.apache.org> on 2016/07/21 06:34:42 UTC

[GitHub] spark pull request #14297: [SPARK-16660][SQL] CreateViewCommand should not t...

GitHub user cloud-fan opened a pull request:

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

    [SPARK-16660][SQL] CreateViewCommand should not take CatalogTable

    ## What changes were proposed in this pull request?
    
    `CreateViewCommand` only needs some information of a `CatalogTable`, but not all of them. We have some tricks(e.g. we need to check the table type is `VIEW`, we need to make `CatalogColumn.dataType` nullable) to allow it to take a `CatalogTable`.
    This PR cleans it up and only pass in necessary information to `CreateViewCommand`. 
    
    
    ## How was this patch tested?
    
    existing tests.


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

    $ git pull https://github.com/cloud-fan/spark minor2

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

    https://github.com/apache/spark/pull/14297.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #14297
    
----
commit 26312b91109823de199a6dad031572a5d50f6d3c
Author: Wenchen Fan <we...@databricks.com>
Date:   2016-07-21T06:29:09Z

    CreateViewCommand should not take CatalogTable

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/62701/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by liancheng <gi...@git.apache.org>.
Github user liancheng commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    LGTM except for one minor issue.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14297: [SPARK-16660][SQL] CreateViewCommand should not t...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14297#discussion_r71820977
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala ---
    @@ -88,23 +96,23 @@ case class CreateViewCommand(
         qe.assertAnalyzed()
         val analyzedPlan = qe.analyzed
     
    -    if (tableDesc.schema != Nil && tableDesc.schema.length != analyzedPlan.output.length) {
    +    if (userSpecifiedColumns.nonEmpty &&
    --- End diff --
    
    then? if `userSpecifiedColumns.isEmpty`, what should we check? And it's wrong to not have this check, users don't need to always specify view columns.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14297: [SPARK-16660][SQL] CreateViewCommand should not t...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14297#discussion_r72131816
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala ---
    @@ -44,7 +50,11 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
      *                 unless they are specified with full qualified table name with database prefix.
      */
     case class CreateViewCommand(
    -    tableDesc: CatalogTable,
    +    name: TableIdentifier,
    +    userSpecifiedColumns: Seq[(String, Option[String])],
    +    comment: Option[String],
    +    properties: Map[String, String],
    +    originalText: Option[String],
         child: LogicalPlan,
    --- End diff --
    
    I see. It is different from `CREATE TABLE AS SELECT`. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    **[Test build #62661 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62661/consoleFull)** for PR 14297 at commit [`26312b9`](https://github.com/apache/spark/commit/26312b91109823de199a6dad031572a5d50f6d3c).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    retest this please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14297: [SPARK-16660][SQL] CreateViewCommand should not t...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14297#discussion_r71822672
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala ---
    @@ -88,23 +96,23 @@ case class CreateViewCommand(
         qe.assertAnalyzed()
         val analyzedPlan = qe.analyzed
     
    -    if (tableDesc.schema != Nil && tableDesc.schema.length != analyzedPlan.output.length) {
    +    if (userSpecifiedColumns.nonEmpty &&
    --- End diff --
    
    : ) Yeah, we should check it!  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14297: [SPARK-16660][SQL] CreateViewCommand should not t...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14297#discussion_r71763014
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala ---
    @@ -88,23 +96,23 @@ case class CreateViewCommand(
         qe.assertAnalyzed()
         val analyzedPlan = qe.analyzed
     
    -    if (tableDesc.schema != Nil && tableDesc.schema.length != analyzedPlan.output.length) {
    +    if (userSpecifiedColumns.nonEmpty &&
    --- End diff --
    
    We do not need to check `userSpecifiedColumns.nonEmpty`, right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14297: [SPARK-16660][SQL] CreateViewCommand should not t...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14297#discussion_r71818729
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala ---
    @@ -88,23 +96,23 @@ case class CreateViewCommand(
         qe.assertAnalyzed()
         val analyzedPlan = qe.analyzed
     
    -    if (tableDesc.schema != Nil && tableDesc.schema.length != analyzedPlan.output.length) {
    +    if (userSpecifiedColumns.nonEmpty &&
    +        userSpecifiedColumns.length != analyzedPlan.output.length) {
           throw new AnalysisException(s"The number of columns produced by the SELECT clause " +
             s"(num: `${analyzedPlan.output.length}`) does not match the number of column names " +
    -        s"specified by CREATE VIEW (num: `${tableDesc.schema.length}`).")
    +        s"specified by CREATE VIEW (num: `${userSpecifiedColumns.length}`).")
         }
         val sessionState = sparkSession.sessionState
     
         if (isTemporary) {
    -      createTemporaryView(tableDesc.identifier, sparkSession, analyzedPlan)
    +      createTemporaryView(sparkSession, analyzedPlan)
         } else {
           // Adds default database for permanent table if it doesn't exist, so that tableExists()
           // only check permanent tables.
    -      val database = tableDesc.identifier.database.getOrElse(
    -        sessionState.catalog.getCurrentDatabase)
    -      val tableIdentifier = tableDesc.identifier.copy(database = Option(database))
    +      val database = name.database.getOrElse(sessionState.catalog.getCurrentDatabase)
    +      val qualifiedName = name.copy(database = Option(database))
     
    -      if (sessionState.catalog.tableExists(tableIdentifier)) {
    +      if (sessionState.catalog.tableExists(qualifiedName)) {
    --- End diff --
    
    Below is what I got in Hive.
    ```
    hive> CREATE TABLE tab1 (id int);
    OK
    Time taken: 0.196 seconds
    hive> CREATE OR REPLACE VIEW tab1 AS SELECT * FROM t1;
    FAILED: SemanticException [Error 10218]: Existing table is not a view
     The following is an existing table, not a view: default.tab1
    ```
    
    I found that `LOAD TABLE` also misses the table type checking. Issue a strange runtime error.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    **[Test build #62663 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62663/consoleFull)** for PR 14297 at commit [`7e516ff`](https://github.com/apache/spark/commit/7e516ff1b63b6c932607e856dd6e1b438256d5d1).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    **[Test build #62701 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62701/consoleFull)** for PR 14297 at commit [`5072959`](https://github.com/apache/spark/commit/50729594c1956d185bde4c2b41891ce1567cfd5a).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    **[Test build #62701 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62701/consoleFull)** for PR 14297 at commit [`5072959`](https://github.com/apache/spark/commit/50729594c1956d185bde4c2b41891ce1567cfd5a).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/62786/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14297: [SPARK-16660][SQL] CreateViewCommand should not t...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14297#discussion_r71767033
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala ---
    @@ -44,7 +50,11 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
      *                 unless they are specified with full qualified table name with database prefix.
      */
     case class CreateViewCommand(
    -    tableDesc: CatalogTable,
    +    name: TableIdentifier,
    +    userSpecifiedColumns: Seq[(String, Option[String])],
    +    comment: Option[String],
    +    properties: Map[String, String],
    +    originalText: Option[String],
         child: LogicalPlan,
    --- End diff --
    
    If we do an explain on this command, the physical plan of this `CREATE VIEW` will return a logicla plan of `child`. Even worse, it could return a parsed plan that has not been analyzed. Do you think we should add a logical plan node for `CreateViewCommand`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14297: [SPARK-16660][SQL] CreateViewCommand should not t...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14297#discussion_r72104591
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala ---
    @@ -44,7 +50,11 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
      *                 unless they are specified with full qualified table name with database prefix.
      */
     case class CreateViewCommand(
    -    tableDesc: CatalogTable,
    +    name: TableIdentifier,
    +    userSpecifiedColumns: Seq[(String, Option[String])],
    +    comment: Option[String],
    +    properties: Map[String, String],
    +    originalText: Option[String],
         child: LogicalPlan,
         allowExisting: Boolean,
         replace: Boolean,
    --- End diff --
    
    btw, we can rename `allowExisting` to `ignoreIfExists`. I am not sure we need to `SaveMode` (a public API) to catalyst. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/62818/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    **[Test build #62671 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62671/consoleFull)** for PR 14297 at commit [`3e22096`](https://github.com/apache/spark/commit/3e220961449a906518567863215d214c451697a0).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    **[Test build #62661 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62661/consoleFull)** for PR 14297 at commit [`26312b9`](https://github.com/apache/spark/commit/26312b91109823de199a6dad031572a5d50f6d3c).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/62775/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14297: [SPARK-16660][SQL] CreateViewCommand should not t...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    **[Test build #62818 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62818/consoleFull)** for PR 14297 at commit [`7b78e20`](https://github.com/apache/spark/commit/7b78e20958744563915591eac53413fe8fafc494).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14297: [SPARK-16660][SQL] CreateViewCommand should not t...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14297#discussion_r71821115
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala ---
    @@ -88,23 +96,23 @@ case class CreateViewCommand(
         qe.assertAnalyzed()
         val analyzedPlan = qe.analyzed
     
    -    if (tableDesc.schema != Nil && tableDesc.schema.length != analyzedPlan.output.length) {
    +    if (userSpecifiedColumns.nonEmpty &&
    +        userSpecifiedColumns.length != analyzedPlan.output.length) {
           throw new AnalysisException(s"The number of columns produced by the SELECT clause " +
             s"(num: `${analyzedPlan.output.length}`) does not match the number of column names " +
    -        s"specified by CREATE VIEW (num: `${tableDesc.schema.length}`).")
    +        s"specified by CREATE VIEW (num: `${userSpecifiedColumns.length}`).")
         }
         val sessionState = sparkSession.sessionState
     
         if (isTemporary) {
    -      createTemporaryView(tableDesc.identifier, sparkSession, analyzedPlan)
    +      createTemporaryView(sparkSession, analyzedPlan)
         } else {
           // Adds default database for permanent table if it doesn't exist, so that tableExists()
           // only check permanent tables.
    -      val database = tableDesc.identifier.database.getOrElse(
    -        sessionState.catalog.getCurrentDatabase)
    -      val tableIdentifier = tableDesc.identifier.copy(database = Option(database))
    +      val database = name.database.getOrElse(sessionState.catalog.getCurrentDatabase)
    +      val qualifiedName = name.copy(database = Option(database))
     
    -      if (sessionState.catalog.tableExists(tableIdentifier)) {
    +      if (sessionState.catalog.tableExists(qualifiedName)) {
    --- End diff --
    
    ah i see, it's an existing issue, feel free to open a JIRA for it :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    **[Test build #62818 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62818/consoleFull)** for PR 14297 at commit [`7b78e20`](https://github.com/apache/spark/commit/7b78e20958744563915591eac53413fe8fafc494).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14297: [SPARK-16660][SQL] CreateViewCommand should not t...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14297#discussion_r71818274
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala ---
    @@ -44,7 +50,11 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
      *                 unless they are specified with full qualified table name with database prefix.
      */
     case class CreateViewCommand(
    -    tableDesc: CatalogTable,
    +    name: TableIdentifier,
    +    userSpecifiedColumns: Seq[(String, Option[String])],
    +    comment: Option[String],
    +    properties: Map[String, String],
    +    originalText: Option[String],
         child: LogicalPlan,
    --- End diff --
    
    nope. This is an existing issue. I just found it. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14297: [SPARK-16660][SQL] CreateViewCommand should not t...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14297#discussion_r71762438
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -1271,23 +1272,23 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder {
           ctx: ParserRuleContext,
           name: TableIdentifierContext,
           comment: Option[String],
    -      schema: Seq[CatalogColumn],
    +      userSpecifiedColumns: Seq[(String, Option[String])],
           query: QueryContext,
           properties: Map[String, String],
           allowExist: Boolean,
           replace: Boolean,
           isTemporary: Boolean): LogicalPlan = {
    -    val sql = Option(source(query))
    -    val tableDesc = CatalogTable(
    -      identifier = visitTableIdentifier(name),
    -      tableType = CatalogTableType.VIEW,
    -      schema = schema,
    -      storage = CatalogStorageFormat.empty,
    -      properties = properties,
    -      viewOriginalText = sql,
    -      viewText = sql,
    -      comment = comment)
    -    CreateViewCommand(tableDesc, plan(query), allowExist, replace, isTemporary)
    +    val originalText = source(query)
    +    CreateViewCommand(
    +      visitTableIdentifier(name),
    +      userSpecifiedColumns,
    +      comment,
    +      properties,
    +      Some(originalText),
    +      plan(query),
    +      allowExist,
    +      replace,
    +      isTemporary)
    --- End diff --
    
    The last three input parameters are `Boolean`. The compiler will not complain it if we mix the order. Can we explicitly specify the parameter names here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14297: [SPARK-16660][SQL] CreateViewCommand should not t...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14297#discussion_r71816553
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala ---
    @@ -88,23 +96,23 @@ case class CreateViewCommand(
         qe.assertAnalyzed()
         val analyzedPlan = qe.analyzed
     
    -    if (tableDesc.schema != Nil && tableDesc.schema.length != analyzedPlan.output.length) {
    +    if (userSpecifiedColumns.nonEmpty &&
    --- End diff --
    
    why? The previous code also checks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    cc @yhuai @liancheng


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    **[Test build #62786 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62786/consoleFull)** for PR 14297 at commit [`a5cf2b3`](https://github.com/apache/spark/commit/a5cf2b3261bd974b215ceaf82d88181a2f87f446).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    **[Test build #62786 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62786/consoleFull)** for PR 14297 at commit [`a5cf2b3`](https://github.com/apache/spark/commit/a5cf2b3261bd974b215ceaf82d88181a2f87f446).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14297: [SPARK-16660][SQL] CreateViewCommand should not t...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14297#discussion_r71764709
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala ---
    @@ -88,23 +96,23 @@ case class CreateViewCommand(
         qe.assertAnalyzed()
         val analyzedPlan = qe.analyzed
     
    -    if (tableDesc.schema != Nil && tableDesc.schema.length != analyzedPlan.output.length) {
    +    if (userSpecifiedColumns.nonEmpty &&
    +        userSpecifiedColumns.length != analyzedPlan.output.length) {
           throw new AnalysisException(s"The number of columns produced by the SELECT clause " +
             s"(num: `${analyzedPlan.output.length}`) does not match the number of column names " +
    -        s"specified by CREATE VIEW (num: `${tableDesc.schema.length}`).")
    +        s"specified by CREATE VIEW (num: `${userSpecifiedColumns.length}`).")
         }
         val sessionState = sparkSession.sessionState
     
         if (isTemporary) {
    -      createTemporaryView(tableDesc.identifier, sparkSession, analyzedPlan)
    +      createTemporaryView(sparkSession, analyzedPlan)
         } else {
           // Adds default database for permanent table if it doesn't exist, so that tableExists()
           // only check permanent tables.
    -      val database = tableDesc.identifier.database.getOrElse(
    -        sessionState.catalog.getCurrentDatabase)
    -      val tableIdentifier = tableDesc.identifier.copy(database = Option(database))
    +      val database = name.database.getOrElse(sessionState.catalog.getCurrentDatabase)
    +      val qualifiedName = name.copy(database = Option(database))
     
    -      if (sessionState.catalog.tableExists(tableIdentifier)) {
    +      if (sessionState.catalog.tableExists(qualifiedName)) {
    --- End diff --
    
    The existing code sounds like a bug. It does not check whether the existing table is a table or a view. Will fix it in a separate PR. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    **[Test build #62663 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62663/consoleFull)** for PR 14297 at commit [`7e516ff`](https://github.com/apache/spark/commit/7e516ff1b63b6c932607e856dd6e1b438256d5d1).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/62671/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14297: [SPARK-16660][SQL] CreateViewCommand should not t...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14297#discussion_r71816619
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala ---
    @@ -88,23 +96,23 @@ case class CreateViewCommand(
         qe.assertAnalyzed()
         val analyzedPlan = qe.analyzed
     
    -    if (tableDesc.schema != Nil && tableDesc.schema.length != analyzedPlan.output.length) {
    +    if (userSpecifiedColumns.nonEmpty &&
    +        userSpecifiedColumns.length != analyzedPlan.output.length) {
           throw new AnalysisException(s"The number of columns produced by the SELECT clause " +
             s"(num: `${analyzedPlan.output.length}`) does not match the number of column names " +
    -        s"specified by CREATE VIEW (num: `${tableDesc.schema.length}`).")
    +        s"specified by CREATE VIEW (num: `${userSpecifiedColumns.length}`).")
         }
         val sessionState = sparkSession.sessionState
     
         if (isTemporary) {
    -      createTemporaryView(tableDesc.identifier, sparkSession, analyzedPlan)
    +      createTemporaryView(sparkSession, analyzedPlan)
         } else {
           // Adds default database for permanent table if it doesn't exist, so that tableExists()
           // only check permanent tables.
    -      val database = tableDesc.identifier.database.getOrElse(
    -        sessionState.catalog.getCurrentDatabase)
    -      val tableIdentifier = tableDesc.identifier.copy(database = Option(database))
    +      val database = name.database.getOrElse(sessionState.catalog.getCurrentDatabase)
    +      val qualifiedName = name.copy(database = Option(database))
     
    -      if (sessionState.catalog.tableExists(tableIdentifier)) {
    +      if (sessionState.catalog.tableExists(qualifiedName)) {
    --- End diff --
    
    see this comment: https://github.com/apache/spark/pull/14297/files#diff-cfec39cf8accabd227bd325f0a0a5f30R110


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/62663/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14297: [SPARK-16660][SQL] CreateViewCommand should not t...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14297#discussion_r71761490
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -2438,12 +2441,15 @@ class Dataset[T] private[sql](
        * @since 2.0.0
        */
       def createOrReplaceTempView(viewName: String): Unit = withPlan {
    -    val tableDesc = CatalogTable(
    -      identifier = sparkSession.sessionState.sqlParser.parseTableIdentifier(viewName),
    -      tableType = CatalogTableType.VIEW,
    -      schema = Seq.empty[CatalogColumn],
    -      storage = CatalogStorageFormat.empty)
    -    CreateViewCommand(tableDesc, logicalPlan, allowExisting = false, replace = true,
    +    CreateViewCommand(
    +      name = sparkSession.sessionState.sqlParser.parseTableIdentifier(viewName),
    +      userSpecifiedColumns = Nil,
    +      comment = None,
    +      properties = Map.empty,
    +      originalText = None,
    +      child = logicalPlan,
    +      allowExisting = false,
    +      replace = true,
    --- End diff --
    
    `createOrReplaceTempView` and `createTempView` share all the input values, except this `replace`. Should we create a private help function to avoid duplicate codes?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14297: [SPARK-16660][SQL] CreateViewCommand should not t...

Posted by liancheng <gi...@git.apache.org>.
Github user liancheng commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14297#discussion_r72029905
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala ---
    @@ -44,7 +50,11 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
      *                 unless they are specified with full qualified table name with database prefix.
      */
     case class CreateViewCommand(
    -    tableDesc: CatalogTable,
    +    name: TableIdentifier,
    +    userSpecifiedColumns: Seq[(String, Option[String])],
    +    comment: Option[String],
    +    properties: Map[String, String],
    +    originalText: Option[String],
         child: LogicalPlan,
         allowExisting: Boolean,
         replace: Boolean,
    --- End diff --
    
    Shall we replace `allowExisting` and `replace` with `SaveMode`? We have the following mapping here:
    
    | `allowExisting` | `replace` |    `SaveMode`   |
    |:---------------:|:---------:|:---------------:|
    |      `true`     |  `false`  |     `Ignore`    |
    |     `false`     |  `false`  | `ErrorIfExists` |
    |     `false`     |   `true`  |   `Overwrite`   |
    |      `true`     |   `true`  |   `Overwrite`   |
    
    `Append` can't be used here though.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14297: [SPARK-16660][SQL] CreateViewCommand should not t...

Posted by liancheng <gi...@git.apache.org>.
Github user liancheng commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14297#discussion_r72014267
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala ---
    @@ -44,7 +50,11 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
      *                 unless they are specified with full qualified table name with database prefix.
      */
     case class CreateViewCommand(
    -    tableDesc: CatalogTable,
    +    name: TableIdentifier,
    +    userSpecifiedColumns: Seq[(String, Option[String])],
    +    comment: Option[String],
    +    properties: Map[String, String],
    +    originalText: Option[String],
         child: LogicalPlan,
    --- End diff --
    
    I think it's OK that `CreateViewCommand` shows the original parsed logical plan in its physical plan tree since this logical plan is just an argument of the command, it's not executed when creating the view anyway.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14297: [SPARK-16660][SQL] CreateViewCommand should not t...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14297#discussion_r71819083
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala ---
    @@ -88,23 +96,23 @@ case class CreateViewCommand(
         qe.assertAnalyzed()
         val analyzedPlan = qe.analyzed
     
    -    if (tableDesc.schema != Nil && tableDesc.schema.length != analyzedPlan.output.length) {
    +    if (userSpecifiedColumns.nonEmpty &&
    --- End diff --
    
    We allow users to create a table containing zero column, right? See my previous PR: https://github.com/apache/spark/pull/14007


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    **[Test build #62775 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62775/consoleFull)** for PR 14297 at commit [`5072959`](https://github.com/apache/spark/commit/50729594c1956d185bde4c2b41891ce1567cfd5a).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/62661/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14297: [SPARK-16660][SQL] CreateViewCommand should not t...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14297#discussion_r72031776
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala ---
    @@ -44,7 +50,11 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
      *                 unless they are specified with full qualified table name with database prefix.
      */
     case class CreateViewCommand(
    -    tableDesc: CatalogTable,
    +    name: TableIdentifier,
    +    userSpecifiedColumns: Seq[(String, Option[String])],
    +    comment: Option[String],
    +    properties: Map[String, String],
    +    originalText: Option[String],
         child: LogicalPlan,
         allowExisting: Boolean,
         replace: Boolean,
    --- End diff --
    
    I'd like to do it in another PR, to move `SaveMode` into catalyst module and use it for CREATE TABLE/VIEW


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14297: [SPARK-16660][SQL] CreateViewCommand should not t...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14297#discussion_r71816585
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala ---
    @@ -44,7 +50,11 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
      *                 unless they are specified with full qualified table name with database prefix.
      */
     case class CreateViewCommand(
    -    tableDesc: CatalogTable,
    +    name: TableIdentifier,
    +    userSpecifiedColumns: Seq[(String, Option[String])],
    +    comment: Option[String],
    +    properties: Map[String, String],
    +    originalText: Option[String],
         child: LogicalPlan,
    --- End diff --
    
    is it an new issue after this PR?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by liancheng <gi...@git.apache.org>.
Github user liancheng commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    Merging to master, thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14297: [SPARK-16660][SQL] CreateViewCommand should not t...

Posted by liancheng <gi...@git.apache.org>.
Github user liancheng commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14297#discussion_r72032392
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala ---
    @@ -44,7 +50,11 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
      *                 unless they are specified with full qualified table name with database prefix.
      */
     case class CreateViewCommand(
    -    tableDesc: CatalogTable,
    +    name: TableIdentifier,
    +    userSpecifiedColumns: Seq[(String, Option[String])],
    +    comment: Option[String],
    +    properties: Map[String, String],
    +    originalText: Option[String],
         child: LogicalPlan,
         allowExisting: Boolean,
         replace: Boolean,
    --- End diff --
    
    Yea, sure.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    **[Test build #62671 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62671/consoleFull)** for PR 14297 at commit [`3e22096`](https://github.com/apache/spark/commit/3e220961449a906518567863215d214c451697a0).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14297: [SPARK-16660][SQL] CreateViewCommand should not take Cat...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14297
  
    **[Test build #62775 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62775/consoleFull)** for PR 14297 at commit [`5072959`](https://github.com/apache/spark/commit/50729594c1956d185bde4c2b41891ce1567cfd5a).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14297: [SPARK-16660][SQL] CreateViewCommand should not t...

Posted by liancheng <gi...@git.apache.org>.
Github user liancheng commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14297#discussion_r72032682
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala ---
    @@ -58,10 +68,8 @@ case class CreateViewCommand(
     
       override def output: Seq[Attribute] = Seq.empty[Attribute]
     
    -  require(tableDesc.tableType == CatalogTableType.VIEW,
    -    "The type of the table to created with CREATE VIEW must be 'CatalogTableType.VIEW'.")
       if (!isTemporary) {
    -    require(tableDesc.viewText.isDefined,
    +    require(originalText.isDefined,
           "The table to created with CREATE VIEW must have 'viewText'.")
    --- End diff --
    
    Nit: "viewText" to "originalText".


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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