You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by xwu0226 <gi...@git.apache.org> on 2017/01/17 22:46:31 UTC

[GitHub] spark pull request #16626: [SPARK-19261][SQL] Alter add columns for Hive tab...

GitHub user xwu0226 opened a pull request:

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

    [SPARK-19261][SQL] Alter add columns for Hive tables

    ## What changes were proposed in this pull request?
    Support ALTER TABLE ADD COLUMNS (...) syntax for Hive tables. 
    In this PR, we consider a few aspects:
    1. View is not supported for alter add
    2. Since tables created in SparkSQL with Hive DDL syntax will populate table properties with schema information, we need make sure the consistency of the schema before and after ALTER operation in order for future use. Because of this issue, there will be a new PR need to support ALTER TABLE ADD COLUMNS for datasource tables, carefully.
    3. For embedded-schema type of format, such as `parquet`, we need to make sure that the predicate on the newly-added columns can be evaluated properly, or pushed down properly. In case of the data file does not have the columns for the newly-added columns, such predicates should return as if the column values are NULLs. 
    
    ## How was this patch tested?
    Add new test cases 
    Please review http://spark.apache.org/contributing.html before opening a pull request.

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

    $ git pull https://github.com/xwu0226/spark alter_add_columns

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

    https://github.com/apache/spark/pull/16626.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 #16626
    
----
commit dba7dcdda36568b910b57de9bea368cec6d793dd
Author: Xin Wu <xi...@us.ibm.com>
Date:   2016-11-21T18:41:15Z

    alter_add_col: initial changes

commit 3750541f7ccd73c7f92fb916e16c1ab4575ec140
Author: Xin Wu <xi...@us.ibm.com>
Date:   2016-12-01T22:03:28Z

    add testcases

commit f7d9f031303d273daa8973517c6a09684e3ba52f
Author: Xin Wu <xi...@us.ibm.com>
Date:   2016-12-01T23:00:38Z

    negative testcases

commit 4cd7848c291ff019e17a3841c755d5befc58178a
Author: Xin Wu <xi...@us.ibm.com>
Date:   2016-12-05T18:35:14Z

    remove non-support testcase

commit 29347f6f52409e8c330101af8dba9bc335786c34
Author: Xin Wu <xi...@us.ibm.com>
Date:   2016-12-05T20:31:55Z

    fix testcase

commit 7cc03483f3546a1d8d38bf67a347bea261e0a1b6
Author: Xin Wu <xi...@us.ibm.com>
Date:   2016-12-07T21:56:40Z

    update testcases

commit 0272fbad90815c3d92f5d06ff3a11299305630e3
Author: Xin Wu <xi...@us.ibm.com>
Date:   2016-12-07T23:10:54Z

    update testcases

commit 96fb6777e74435bc96baac5fa2d0a17183e3f363
Author: Xin Wu <xi...@us.ibm.com>
Date:   2017-01-13T20:53:44Z

    update testcases

----


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r107063558
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -455,8 +455,10 @@ abstract class SessionCatalogSuite extends PlanTest {
         withBasicCatalog { sessionCatalog =>
           sessionCatalog.createTable(newTable("t1", "default"), ignoreIfExists = false)
           val oldTab = sessionCatalog.externalCatalog.getTable("default", "t1")
    +      val partitionSchema = oldTab.partitionSchema
           sessionCatalog.alterTableSchema(
    -        TableIdentifier("t1", Some("default")), oldTab.schema.add("c3", IntegerType))
    +        TableIdentifier("t1", Some("default")),
    +        StructType(oldTab.dataSchema.add("c3", IntegerType) ++ partitionSchema))
    --- End diff --
    
    `partitionSchema ` -> `oldTab.partitionSchema`


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r107049953
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -296,6 +311,51 @@ class SessionCatalog(
       }
     
       /**
    +   * Alter the schema of a table identified by the provided table identifier. The new schema
    +   * should still contain the existing bucket columns and partition columns used by the table. This
    +   * method will also update any Spark SQL-related parameters stored as Hive table properties (such
    +   * as the schema itself).
    +   *
    +   * @param identifier TableIdentifier
    +   * @param newSchema Updated schema to be used for the table (must contain existing partition and
    +   *                  bucket columns)
    +   */
    +  def alterTableSchema(
    +      identifier: TableIdentifier,
    +      newSchema: StructType): Unit = {
    +    val db = formatDatabaseName(identifier.database.getOrElse(getCurrentDatabase))
    +    val table = formatTableName(identifier.table)
    +    val tableIdentifier = TableIdentifier(table, Some(db))
    +    requireDbExists(db)
    +    requireTableExists(tableIdentifier)
    +    checkDuplication(newSchema)
    +
    +    val catalogTable = externalCatalog.getTable(db, table)
    +    val oldSchema = catalogTable.schema
    +
    +    // not supporting dropping columns yet
    +    val nonExistentColumnNames = oldSchema.map(_.name).filterNot(columnNameResolved(newSchema, _))
    +    if (nonExistentColumnNames.nonEmpty) {
    +      throw new AnalysisException(
    +        s"""
    +           |Some existing schema fields (${nonExistentColumnNames.mkString("[", ",", "]")}) are
    +           |not present in the new schema. We don't support dropping columns yet.
    +         """.stripMargin)
    +    }
    +
    +    // make sure partition columns are at the end
    +    val partitionSchema = catalogTable.partitionSchema
    --- End diff --
    
    @cloud-fan Thanks! My understanding is that the caller may pass in a new schema that may not follow the order in that partition column is added to the end.  So i want to ensure that before passing to `exernalCatalog.alterTableSchema`.
    
    How about I change the definition of asking caller to ensure the column ordering in the `newSchema` before calling this function?


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74583 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74583/testReport)** for PR 16626 at commit [`f5e93bc`](https://github.com/apache/spark/commit/f5e93bc8f158a1a0cba501cf5874bbbe9ab78784).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive tables

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

    https://github.com/apache/spark/pull/16626
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71749/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r107049979
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala ---
    @@ -2178,4 +2178,138 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
           }
         }
       }
    +
    +  val supportedNativeFileFormatsForAlterTableAddColumns = Seq("parquet", "json", "csv")
    +
    +  supportedNativeFileFormatsForAlterTableAddColumns.foreach { provider =>
    +    test(s"alter datasource table add columns - $provider") {
    +      withTable("t1") {
    +        sql(s"CREATE TABLE t1 (c1 int) USING $provider")
    +        sql("INSERT INTO t1 VALUES (1)")
    +        sql("ALTER TABLE t1 ADD COLUMNS (c2 int)")
    +        checkAnswer(
    +          spark.table("t1"),
    +          Seq(Row(1, null))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c2 is null"),
    +          Seq(Row(1, null))
    +        )
    +
    +        sql("INSERT INTO t1 VALUES (3, 2)")
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c2 = 2"),
    +          Seq(Row(3, 2))
    +        )
    +      }
    +    }
    +  }
    +
    +  supportedNativeFileFormatsForAlterTableAddColumns.foreach { provider =>
    +    test(s"alter datasource table add columns - partitioned - $provider") {
    +      withTable("t1") {
    +        sql(s"CREATE TABLE t1 (c1 int, c2 int) USING $provider PARTITIONED BY (c2)")
    +        sql("INSERT INTO t1 PARTITION(c2 = 2) VALUES (1)")
    +        sql("ALTER TABLE t1 ADD COLUMNS (c3 int)")
    +        checkAnswer(
    +          spark.table("t1"),
    +          Seq(Row(1, null, 2))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c3 is null"),
    +          Seq(Row(1, null, 2))
    +        )
    +        sql("INSERT INTO t1 PARTITION(c2 =1) VALUES (2, 3)")
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c3 = 3"),
    +          Seq(Row(2, 3, 1))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c2 = 1"),
    +          Seq(Row(2, 3, 1))
    +        )
    +      }
    +    }
    +  }
    +
    +  test("alter datasource table add columns - text format not supported") {
    +    withTable("t1") {
    +      sql("CREATE TABLE t1 (c1 int) USING text")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE t1 ADD COLUMNS (c2 int)")
    +      }.getMessage
    +      assert(e.contains("ALTER ADD COLUMNS does not support datasource table with type"))
    +    }
    +  }
    +
    +  test("alter table add columns -- not support temp view") {
    +    withTempView("tmp_v") {
    +      sql("CREATE TEMPORARY VIEW tmp_v AS SELECT 1 AS c1, 2 AS c2")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE tmp_v ADD COLUMNS (c3 INT)")
    +      }
    +      assert(e.message.contains("ALTER ADD COLUMNS does not support views"))
    +    }
    +  }
    +
    +  test("alter table add columns -- not support view") {
    +    withView("v1") {
    +      sql("CREATE VIEW v1 AS SELECT 1 AS c1, 2 AS c2")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE v1 ADD COLUMNS (c3 INT)")
    +      }
    +      assert(e.message.contains("ALTER ADD COLUMNS does not support views"))
    +    }
    +  }
    +
    +  test("alter table add columns with existing column name") {
    +    withTable("t1") {
    +      sql("CREATE TABLE t1 (c1 int) USING PARQUET")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE t1 ADD COLUMNS (c1 string)")
    +      }.getMessage
    +      assert(e.contains("Found duplicate column(s)"))
    +    }
    +  }
    +
    +  test("alter table add columns to table referenced by a view") {
    --- End diff --
    
    ok. I will remove this test. 


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74929/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74634 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74634/testReport)** for PR 16626 at commit [`7fbfc71`](https://github.com/apache/spark/commit/7fbfc7165e3bce388d4dc6e2c58487d4abf8d098).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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/16626#discussion_r107050210
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala ---
    @@ -1860,4 +1861,115 @@ class HiveDDLSuite
           }
         }
       }
    +
    +  hiveFormats.foreach { tableType =>
    +    test(s"alter hive serde table add columns -- partitioned - $tableType") {
    +      withTable("tab") {
    +        sql(
    +          s"""
    +             |CREATE TABLE tab (c1 int, c2 int)
    +             |PARTITIONED BY (c3 int) STORED AS $tableType
    +          """.stripMargin)
    +
    +        sql("INSERT INTO tab PARTITION (c3=1) VALUES (1, 2)")
    +        sql("ALTER TABLE tab ADD COLUMNS (c4 int)")
    +
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c3 = 1"),
    +          Seq(Row(1, 2, null, 1))
    +        )
    +        assert(spark.table("tab").schema
    +          .contains(StructField("c4", IntegerType)))
    +        sql("INSERT INTO tab PARTITION (c3=2) VALUES (2, 3, 4)")
    +        checkAnswer(
    +          spark.table("tab"),
    +          Seq(Row(1, 2, null, 1), Row(2, 3, 4, 2))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c3 = 2 AND c4 IS NOT NULL"),
    +          Seq(Row(2, 3, 4, 2))
    +        )
    +
    +        sql("ALTER TABLE tab ADD COLUMNS (c5 char(10))")
    +        assert(spark.table("tab").schema.find(_.name == "c5")
    +          .get.metadata.getString("HIVE_TYPE_STRING") == "char(10)")
    +      }
    +    }
    +  }
    +
    +  hiveFormats.foreach { tableType =>
    +    test(s"alter hive serde table add columns -- with predicate - $tableType ") {
    +      withTable("tab") {
    +        sql(s"CREATE TABLE tab (c1 int, c2 int) STORED AS $tableType")
    +        sql("INSERT INTO tab VALUES (1, 2)")
    +        sql("ALTER TABLE tab ADD COLUMNS (c4 int)")
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c4 IS NULL"),
    +          Seq(Row(1, 2, null))
    +        )
    +        assert(spark.table("tab").schema
    +          .contains(StructField("c4", IntegerType)))
    +        sql("INSERT INTO tab VALUES (2, 3, 4)")
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c4 = 4 "),
    +          Seq(Row(2, 3, 4))
    +        )
    +        checkAnswer(
    +          spark.table("tab"),
    +          Seq(Row(1, 2, null), Row(2, 3, 4))
    +        )
    +      }
    +    }
    +  }
    +
    +  Seq("orc", "ORC", "org.apache.spark.sql.hive.orc",
    +    "org.apache.spark.sql.hive.orc.DefaultSource").foreach { source =>
    +    test(s"alter datasource table add columns - $source format not supported") {
    +      withTable("tab") {
    +        sql(s"CREATE TABLE tab (c1 int) USING $source")
    +        val e = intercept[AnalysisException] {
    +          sql("ALTER TABLE tab ADD COLUMNS (c2 int)")
    +        }.getMessage
    +        assert(
    +          e.contains(s"ALTER ADD COLUMNS does not support datasource table with type"))
    +      }
    +    }
    +  }
    +
    +  Seq("true", "false").foreach { caseSensitive =>
    +    test(s"alter add columns with existing column name - caseSensitive $caseSensitive") {
    +      withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) {
    +        withTable("tab") {
    +          sql("CREATE TABLE tab (c1 int) PARTITIONED BY (c2 int) STORED AS PARQUET")
    +          if (caseSensitive == "false") {
    --- End diff --
    
    if (!caseSensitive)


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive tables

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

    https://github.com/apache/spark/pull/16626
  
    @gatorsmile @cloud-fan Thanks for reviewing!


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106786022
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -296,6 +297,64 @@ class SessionCatalog(
       }
     
       /**
    +   * Alter the schema of a table identified by the provided table identifier. The new schema
    +   * should still contain the existing bucket columns and partition columns used by the table. This
    +   * method will also update any Spark SQL-related parameters stored as Hive table properties (such
    +   * as the schema itself).
    +   *
    +   * @param identifier TableIdentifier
    +   * @param newSchema Updated schema to be used for the table (must contain existing partition and
    +   *                  bucket columns)
    +   */
    +  def alterTableSchema(
    +      identifier: TableIdentifier,
    +      newSchema: StructType): Unit = {
    +    val db = formatDatabaseName(identifier.database.getOrElse(getCurrentDatabase))
    +    val table = formatTableName(identifier.table)
    +    val tableIdentifier = TableIdentifier(table, Some(db))
    +    requireDbExists(db)
    +    requireTableExists(tableIdentifier)
    +    val catalogTable = externalCatalog.getTable(db, table)
    +    val oldSchema = catalogTable.schema
    +
    +    // no supporting dropping columns yet
    +    if (!oldSchema.forall(f => columnNameResolved(newSchema, f.name ))) {
    +      throw new AnalysisException(
    +        s"""
    +          |Some existing schema fields are not present in the new schema.
    +          |We don't support dropping columns yet.
    +         """.stripMargin)
    +    }
    +
    +    checkDuplication(newSchema)
    +    // make sure partition columns are at the end
    +    val partitionSchema = catalogTable.partitionSchema
    +    val reorderedSchema = newSchema
    +      .filterNot(f => columnNameResolved(partitionSchema, f.name)) ++ partitionSchema
    +
    +    externalCatalog.alterTableSchema(
    +      db, table, oldSchema.copy(fields = reorderedSchema.toArray))
    +  }
    +
    +  private def columnNameResolved(schema: StructType, colName: String): Boolean = {
    +    schema.fields.map(_.name).find(conf.resolver(_, colName)).isDefined
    +  }
    +
    +  private def checkDuplication(fields: Seq[StructField]): Unit = {
    +    val columnNames = if (conf.caseSensitiveAnalysis) {
    +      fields.map(_.name)
    +    } else {
    +      fields.map(_.name.toLowerCase)
    +    }
    +    if (columnNames.distinct.length != columnNames.length) {
    +      val duplicateColumns = columnNames.groupBy(identity).collect {
    +        case (x, ys) if ys.length > 1 => x
    +      }
    +      throw new AnalysisException(s"Found duplicate column(s): ${duplicateColumns.mkString(", ")}")
    +    }
    +  }
    --- End diff --
    
    Move these functions to[ Line 164 ](https://github.com/xwu0226/spark/blob/ae324d3103358a045cab81677ab1f28c13303b7c/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala#L164)


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r107051308
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala ---
    @@ -2178,4 +2178,138 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
           }
         }
       }
    +
    +  val supportedNativeFileFormatsForAlterTableAddColumns = Seq("parquet", "json", "csv")
    +
    +  supportedNativeFileFormatsForAlterTableAddColumns.foreach { provider =>
    +    test(s"alter datasource table add columns - $provider") {
    +      withTable("t1") {
    +        sql(s"CREATE TABLE t1 (c1 int) USING $provider")
    +        sql("INSERT INTO t1 VALUES (1)")
    +        sql("ALTER TABLE t1 ADD COLUMNS (c2 int)")
    +        checkAnswer(
    +          spark.table("t1"),
    +          Seq(Row(1, null))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c2 is null"),
    +          Seq(Row(1, null))
    +        )
    +
    +        sql("INSERT INTO t1 VALUES (3, 2)")
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c2 = 2"),
    +          Seq(Row(3, 2))
    +        )
    +      }
    +    }
    +  }
    +
    +  supportedNativeFileFormatsForAlterTableAddColumns.foreach { provider =>
    +    test(s"alter datasource table add columns - partitioned - $provider") {
    +      withTable("t1") {
    +        sql(s"CREATE TABLE t1 (c1 int, c2 int) USING $provider PARTITIONED BY (c2)")
    +        sql("INSERT INTO t1 PARTITION(c2 = 2) VALUES (1)")
    +        sql("ALTER TABLE t1 ADD COLUMNS (c3 int)")
    +        checkAnswer(
    +          spark.table("t1"),
    +          Seq(Row(1, null, 2))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c3 is null"),
    +          Seq(Row(1, null, 2))
    +        )
    +        sql("INSERT INTO t1 PARTITION(c2 =1) VALUES (2, 3)")
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c3 = 3"),
    +          Seq(Row(2, 3, 1))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c2 = 1"),
    +          Seq(Row(2, 3, 1))
    +        )
    +      }
    +    }
    +  }
    +
    +  test("alter datasource table add columns - text format not supported") {
    +    withTable("t1") {
    +      sql("CREATE TABLE t1 (c1 int) USING text")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE t1 ADD COLUMNS (c2 int)")
    +      }.getMessage
    +      assert(e.contains("ALTER ADD COLUMNS does not support datasource table with type"))
    +    }
    +  }
    +
    +  test("alter table add columns -- not support temp view") {
    +    withTempView("tmp_v") {
    +      sql("CREATE TEMPORARY VIEW tmp_v AS SELECT 1 AS c1, 2 AS c2")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE tmp_v ADD COLUMNS (c3 INT)")
    +      }
    +      assert(e.message.contains("ALTER ADD COLUMNS does not support views"))
    +    }
    +  }
    +
    +  test("alter table add columns -- not support view") {
    +    withView("v1") {
    +      sql("CREATE VIEW v1 AS SELECT 1 AS c1, 2 AS c2")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE v1 ADD COLUMNS (c3 INT)")
    +      }
    +      assert(e.message.contains("ALTER ADD COLUMNS does not support views"))
    +    }
    +  }
    +
    +  test("alter table add columns with existing column name") {
    +    withTable("t1") {
    +      sql("CREATE TABLE t1 (c1 int) USING PARQUET")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE t1 ADD COLUMNS (c1 string)")
    +      }.getMessage
    +      assert(e.contains("Found duplicate column(s)"))
    +    }
    +  }
    +
    +  test("alter table add columns to table referenced by a view") {
    +    withTable("t1") {
    +      withView("v1") {
    +        sql("CREATE TABLE t1 (c1 int, c2 int) USING PARQUET")
    +        sql("CREATE VIEW v1 AS SELECT * FROM t1")
    +        val originViewSchema = sql("SELECT * FROM v1").schema
    +        sql("ALTER TABLE t1 ADD COLUMNS (c3 int)")
    +        assert(sql("SELECT * FROM v1").schema == originViewSchema)
    +      }
    +    }
    +  }
    +
    +  Seq("true", "false").foreach { caseSensitive =>
    +    test(s"alter table add columns with existing column name - caseSensitive $caseSensitive") {
    +      withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) {
    +        withTable("t1") {
    +          sql("CREATE TABLE t1 (c1 int) USING PARQUET")
    +          if (caseSensitive == "false") {
    +            val e = intercept[AnalysisException] {
    +              sql("ALTER TABLE t1 ADD COLUMNS (C1 string)")
    +            }.getMessage
    +            assert(e.contains("Found duplicate column(s)"))
    +          } else {
    +            if (isUsingHiveMetastore) {
    +              // hive catalog will still complains that c1 is duplicate column name because hive
    +              // identifiers are case insensitive.
    --- End diff --
    
    It was the` hive.ql` error when I pass a column name that is different case from an existing column name. The error was thrown from hive client complaining that it is duplicate name, even though SparkSQL thinks it is a different name. Can the schema stored in table properties influence how hive sees the sensitivity of the column name? 


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72530/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r99677955
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
    @@ -763,7 +763,9 @@ object DDLUtils {
       val HIVE_PROVIDER = "hive"
     
       def isHiveTable(table: CatalogTable): Boolean = {
    -    table.provider.isDefined && table.provider.get.toLowerCase == HIVE_PROVIDER
    +    // When `CatalogTable` is directly fetched from the catalog,
    +    // CatalogTable.provider = None means the table is a Hive serde table.
    +    !table.provider.isDefined || table.provider.get.toLowerCase == HIVE_PROVIDER
    --- End diff --
    
    The provider could be empty if the table is a VIEW. Thus, please do not modify the utility function 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106792988
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala ---
    @@ -2178,4 +2177,136 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
           }
         }
       }
    +
    +  Seq("parquet", "json", "csv").foreach { provider =>
    --- End diff --
    
    Let us introduce another variable.
    
    ```Scala
    val supportedNativeFileFormatsForAlterTableAddColumns = Seq("parquet", "json", "csv")
    ```


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74733/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106792941
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---
    @@ -175,6 +178,74 @@ case class AlterTableRenameCommand(
     }
     
     /**
    + * A command that add columns to a table
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   ALTER TABLE table_identifier
    + *   ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
    + * }}}
    +*/
    +case class AlterTableAddColumnsCommand(
    +    table: TableIdentifier,
    +    columns: Seq[StructField]) extends RunnableCommand {
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    val catalog = sparkSession.sessionState.catalog
    +    val catalogTable = verifyAlterTableAddColumn(catalog, table)
    +
    +    try {
    +      sparkSession.catalog.uncacheTable(table.quotedString)
    +    } catch {
    +      case NonFatal(e) =>
    +        log.warn(s"Exception when attempting to uncache table ${table.quotedString}", e)
    +    }
    +    catalog.refreshTable(table)
    +    catalog.alterTableSchema(
    +      table, catalogTable.schema.copy(fields = catalogTable.schema.fields ++ columns))
    +
    +    Seq.empty[Row]
    +  }
    +
    +  /**
    +   * ALTER TABLE ADD COLUMNS command does not support temporary view/table,
    +   * view, or datasource table with text, orc formats or external provider.
    +   * For datasource table, it currently only supports parquet, json, csv.
    +   */
    +  private def verifyAlterTableAddColumn(
    +      catalog: SessionCatalog,
    +      table: TableIdentifier): CatalogTable = {
    +    val catalogTable = catalog.getTempViewOrPermanentTableMetadata(table)
    +
    +    if (catalogTable.tableType == CatalogTableType.VIEW) {
    +      throw new AnalysisException(
    +        s"""
    +          |ALTER ADD COLUMNS does not support views.
    +          |You must drop and re-create the views for adding the new columns. Views: $table
    +         """.stripMargin)
    +    }
    +
    +    if (DDLUtils.isDatasourceTable(catalogTable)) {
    +      DataSource.lookupDataSource(catalogTable.provider.get).newInstance() match {
    +        // For datasource table, this command can only support the following File format.
    +        // TextFileFormat only default to one column "value"
    +        // OrcFileFormat can not handle difference between user-specified schema and
    +        // inferred schema yet. TODO, once this issue is resolved , we can add Orc back.
    +        // Hive type is already considered as hive serde table, so the logic will not
    +        // come in here.
    +        case _: JsonFileFormat | _: CSVFileFormat | _: ParquetFileFormat =>
    +        case s =>
    +          throw new AnalysisException(
    +            s"""
    +               |ALTER ADD COLUMNS does not support datasource table with type $s.
    +               |You must drop and re-create the table for adding the new columns. Tables: $table
    +         """.stripMargin)
    --- End diff --
    
    Nit: fix the format


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106785796
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -296,6 +297,64 @@ class SessionCatalog(
       }
     
       /**
    +   * Alter the schema of a table identified by the provided table identifier. The new schema
    +   * should still contain the existing bucket columns and partition columns used by the table. This
    +   * method will also update any Spark SQL-related parameters stored as Hive table properties (such
    +   * as the schema itself).
    +   *
    +   * @param identifier TableIdentifier
    +   * @param newSchema Updated schema to be used for the table (must contain existing partition and
    +   *                  bucket columns)
    +   */
    +  def alterTableSchema(
    +      identifier: TableIdentifier,
    +      newSchema: StructType): Unit = {
    +    val db = formatDatabaseName(identifier.database.getOrElse(getCurrentDatabase))
    +    val table = formatTableName(identifier.table)
    +    val tableIdentifier = TableIdentifier(table, Some(db))
    +    requireDbExists(db)
    +    requireTableExists(tableIdentifier)
    +    val catalogTable = externalCatalog.getTable(db, table)
    +    val oldSchema = catalogTable.schema
    +
    +    // no supporting dropping columns yet
    +    if (!oldSchema.forall(f => columnNameResolved(newSchema, f.name ))) {
    +      throw new AnalysisException(
    +        s"""
    +          |Some existing schema fields are not present in the new schema.
    +          |We don't support dropping columns yet.
    +         """.stripMargin)
    +    }
    --- End diff --
    
    Let us output the column names.
    ```
        // not supporting dropping columns yet
        val nonExistentColumnNames = oldSchema.map(_.name).map(isColumnNameResolved(newSchema, _))
        if (nonExistentColumnNames.nonEmpty) {
          throw new AnalysisException(
            s"""
              |Some existing schema fields (${nonExistentColumnNames.mkString("[", ",", "]")}) are
              |not present in the new schema. We don't support dropping columns yet.
             """.stripMargin)
        }
    ```



---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106765877
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -296,6 +297,45 @@ class SessionCatalog(
       }
     
       /**
    +   * Alter the schema of a table identified by the provided table identifier to add new columns
    +   * @param identifier TableIdentifier
    +   * @param columns new columns
    +   * @param caseSensitive enforce case sensitivity for column names
    +   */
    +  def alterTableAddColumns(
    --- End diff --
    
    I will change back to use `alterTableSchema` to make it more generic for other schema evolution features. 


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r99681098
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
    @@ -814,4 +816,50 @@ object DDLUtils {
           }
         }
       }
    +
    +  /**
    +   * ALTER TABLE ADD COLUMNS command does not support temporary view/table,
    +   * view, or datasource table with text, orc formats or external provider.
    +   */
    +  def verifyAlterTableAddColumn(
    +      catalog: SessionCatalog,
    +      table: TableIdentifier): CatalogTable = {
    +    if (catalog.isTemporaryTable(table)) {
    +      throw new AnalysisException(
    +        s"${table.toString} is a temporary VIEW, which does not support ALTER ADD COLUMNS.")
    +    }
    +
    +    val catalogTable = catalog.getTableMetadata(table)
    --- End diff --
    
    I see. Will do. 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r103024279
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
    @@ -563,35 +574,47 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
             //       want to alter the table location to a file path, we will fail. This should be fixed
             //       in the future.
     
    -        val newLocation = tableDefinition.storage.locationUri
    -        val storageWithPathOption = tableDefinition.storage.copy(
    -          properties = tableDefinition.storage.properties ++ newLocation.map("path" -> _))
    +        val newLocation = newTableDefinition.storage.locationUri
    +        val storageWithPathOption = newTableDefinition.storage.copy(
    +          properties = newTableDefinition.storage.properties ++ newLocation.map("path" -> _))
     
    -        val oldLocation = getLocationFromStorageProps(oldTableDef)
    +        val oldLocation = getLocationFromStorageProps(oldRawTableDef)
             if (oldLocation == newLocation) {
    -          storageWithPathOption.copy(locationUri = oldTableDef.storage.locationUri)
    +          storageWithPathOption.copy(locationUri = oldRawTableDef.storage.locationUri)
             } else {
               storageWithPathOption
             }
           }
     
    -      val partitionProviderProp = if (tableDefinition.tracksPartitionsInCatalog) {
    +      val partitionProviderProp = if (newTableDefinition.tracksPartitionsInCatalog) {
             TABLE_PARTITION_PROVIDER -> TABLE_PARTITION_PROVIDER_CATALOG
           } else {
             TABLE_PARTITION_PROVIDER -> TABLE_PARTITION_PROVIDER_FILESYSTEM
           }
     
    -      // Sets the `schema`, `partitionColumnNames` and `bucketSpec` from the old table definition,
    +      // Sets the `partitionColumnNames` and `bucketSpec` from the old table definition,
           // to retain the spark specific format if it is. Also add old data source properties to table
           // properties, to retain the data source table format.
    -      val oldDataSourceProps = oldTableDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX))
    -      val newTableProps = oldDataSourceProps ++ withStatsProps.properties + partitionProviderProp
    -      val newDef = withStatsProps.copy(
    +      val dataSourceProps = if (schemaChange) {
    +        val props =
    +          tableMetaToTableProps(newTableDefinition).filter(_._1.startsWith(DATASOURCE_PREFIX))
    +        if (newTableDefinition.provider.isDefined
    +          && newTableDefinition.provider.get.toLowerCase != DDLUtils.HIVE_PROVIDER) {
    +          // we only need to populate non-hive provider to the tableprops
    +          props.put(DATASOURCE_PROVIDER, newTableDefinition.provider.get)
    +        }
    +        props
    +      } else {
    +        oldRawTableDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX))
    +      }
    +      val newTableProps =
    +        dataSourceProps ++ maybeWithStatsPropsTable.properties + partitionProviderProp
    --- End diff --
    
    The code for constructing the new table properties is a bit of not easy to make it stand alone, it is relatively easy to refactor out the `withStatsProps` and `newStorage` part to 2 private helper functions. I am trying this and testing now. 


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106472274
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---
    @@ -175,6 +178,87 @@ case class AlterTableRenameCommand(
     }
     
     /**
    + * A command that add columns to a table
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   ALTER TABLE table_identifier
    + *   ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
    + * }}}
    +*/
    +case class AlterTableAddColumnsCommand(
    +    table: TableIdentifier,
    +    columns: Seq[StructField]) extends RunnableCommand {
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    val catalog = sparkSession.sessionState.catalog
    +    val catalogTable = verifyAlterTableAddColumn(catalog, table)
    +
    +    try {
    +      sparkSession.catalog.uncacheTable(table.unquotedString)
    +    } catch {
    +      case NonFatal(e) =>
    +        log.warn(s"Exception when attempting to uncache table ${table.unquotedString}", e)
    +    }
    +
    +    // Invalidate the table last, otherwise uncaching the table would load the logical plan
    +    // back into the hive metastore cache
    +    catalog.refreshTable(table)
    +    val partitionFields = catalogTable.schema.takeRight(catalogTable.partitionColumnNames.length)
    +    val newSchemaFields = catalogTable.schema
    +      .take(catalogTable.schema.length - catalogTable.partitionColumnNames.length) ++
    +      columns ++ partitionFields
    +    checkDuplication(newSchemaFields.map(_.name))
    +    catalog.alterTableSchema(table, newSchema =
    +      catalogTable.schema.copy(fields = newSchemaFields.toArray))
    +
    +    Seq.empty[Row]
    +  }
    +
    +  private def checkDuplication(colNames: Seq[String]): Unit = {
    +    if (colNames.distinct.length != colNames.length) {
    +      val duplicateColumns = colNames.groupBy(identity).collect {
    +        case (x, ys) if ys.length > 1 => x
    +      }
    +      throw new AnalysisException(
    +        s"Found duplicate column(s): ${duplicateColumns.mkString(", ")}")
    +    }
    +  }
    +
    +  /**
    +   * ALTER TABLE ADD COLUMNS command does not support temporary view/table,
    +   * view, or datasource table with text, orc formats or external provider.
    +   * For datasource table, it currently only supports parquet, json, csv.
    +   */
    +  private def verifyAlterTableAddColumn(
    +      catalog: SessionCatalog,
    +      table: TableIdentifier): CatalogTable = {
    +    val catalogTable = catalog.getTempViewOrPermanentTableMetadata(table)
    +
    +    if (catalogTable.tableType == CatalogTableType.VIEW) {
    +      throw new AnalysisException(
    +        s"${table.toString} is a VIEW, which does not support ALTER ADD COLUMNS.")
    --- End diff --
    
    How about?
    > ALTER ADD COLUMNS does not support views. You must drop and re-create the views for adding the new columns. Views: $table.


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r99680331
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
    @@ -814,4 +816,50 @@ object DDLUtils {
           }
         }
       }
    +
    +  /**
    +   * ALTER TABLE ADD COLUMNS command does not support temporary view/table,
    +   * view, or datasource table with text, orc formats or external provider.
    +   */
    +  def verifyAlterTableAddColumn(
    +      catalog: SessionCatalog,
    +      table: TableIdentifier): CatalogTable = {
    +    if (catalog.isTemporaryTable(table)) {
    +      throw new AnalysisException(
    +        s"${table.toString} is a temporary VIEW, which does not support ALTER ADD COLUMNS.")
    +    }
    +
    +    val catalogTable = catalog.getTableMetadata(table)
    +    if (catalogTable.tableType == CatalogTableType.VIEW) {
    +      throw new AnalysisException(
    +        s"${table.toString} is a VIEW, which does not support ALTER ADD COLUMNS.")
    +    }
    +
    +    if (isDatasourceTable(catalogTable)) {
    +      catalogTable.provider.get match {
    +        case provider if provider.toLowerCase == "text" =>
    +          // TextFileFormat can not support adding column either because text datasource table
    +          // is resolved as a single-column table only.
    +          throw new AnalysisException(
    +            s"""${table.toString} is a text format datasource table,
    +               |which does not support ALTER ADD COLUMNS.""".stripMargin)
    +        case provider if provider.toLowerCase == "orc"
    +          || provider.startsWith("org.apache.spark.sql.hive.orc") =>
    +          // TODO Current native orc reader can not handle the difference between
    +          // user-specified schema and inferred schema from ORC data file yet.
    +          throw new AnalysisException(
    +            s"""${table.toString} is an ORC datasource table,
    +               |which does not support ALTER ADD COLUMNS.""".stripMargin)
    +        case provider
    +          if (!DataSource.lookupDataSource(provider).newInstance().isInstanceOf[FileFormat]) =>
    --- End diff --
    
    `FileFormat` only covers a few cases. It does not cover the other external data sources. How about using a white list here in this function?


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive tables

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #71728 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71728/testReport)** for PR 16626 at commit [`ffb8b55`](https://github.com/apache/spark/commit/ffb8b5544822851085fc6ca101852b6d28ac413b).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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/16626#discussion_r107050001
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala ---
    @@ -965,9 +965,27 @@ class JDBCSuite extends SparkFunSuite
               |CREATE TEMPORARY VIEW people_view
               |USING org.apache.spark.sql.jdbc
               |OPTIONS (uRl '$url', DbTaBlE 'TEST.PEOPLE', User 'testUser', PassWord 'testPass')
    -        """.stripMargin.replaceAll("\n", " "))
    +        """.stripMargin)
     
           assert(sql("select * from people_view").count() == 3)
         }
       }
    +
    +  test("ALTER TABLE ADD COLUMNS") {
    --- End diff --
    
    are we going to test all the unsupported data sources? that's a lot, and unnecessary. I think the text format test is enough, let's remove others.


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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/16626#discussion_r106621508
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -450,6 +451,26 @@ abstract class SessionCatalogSuite extends PlanTest {
         }
       }
     
    +  test("alter table add columns") {
    +    withBasicCatalog { sessionCatalog =>
    +      sessionCatalog.createTable(newTable("t1", "default"), ignoreIfExists = false)
    +      val oldTab = sessionCatalog.externalCatalog.getTable("default", "t1")
    +      sessionCatalog.alterTableSchema(TableIdentifier("t1", Some("default")),
    +        oldTab.schema.add("c3", IntegerType))
    +      val newTab = sessionCatalog.externalCatalog.getTable("default", "t1")
    +      if (sessionCatalog.externalCatalog.isInstanceOf[InMemoryCatalog]) {
    +        assert(newTab.schema.toString == oldTab.schema.add("c3", IntegerType).toString)
    +      } else {
    +        // HiveExternalCatalog will always arrange the partition columns to the end
    --- End diff --
    
    We need to do this(arrange the partition columns to the end) in `SessionCatalog.alterTableSchema`.


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106786355
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala ---
    @@ -2178,4 +2197,107 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
           }
         }
       }
    +
    +  Seq("parquet", "json", "csv").foreach { provider =>
    +    test(s"alter datasource table add columns - $provider") {
    +      withTable("t1") {
    +        sql(s"CREATE TABLE t1 (c1 int) USING $provider")
    +        sql("INSERT INTO t1 VALUES (1)")
    +        sql("ALTER TABLE t1 ADD COLUMNS (c2 int)")
    +        checkAnswer(
    +          sql("SELECT * FROM t1"),
    --- End diff --
    
    spark.table("t1")


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72406/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72447/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive tables

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #71535 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71535/testReport)** for PR 16626 at commit [`96fb677`](https://github.com/apache/spark/commit/96fb6777e74435bc96baac5fa2d0a17183e3f363).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r107050116
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala ---
    @@ -965,9 +965,27 @@ class JDBCSuite extends SparkFunSuite
               |CREATE TEMPORARY VIEW people_view
               |USING org.apache.spark.sql.jdbc
               |OPTIONS (uRl '$url', DbTaBlE 'TEST.PEOPLE', User 'testUser', PassWord 'testPass')
    -        """.stripMargin.replaceAll("\n", " "))
    +        """.stripMargin)
     
           assert(sql("select * from people_view").count() == 3)
         }
       }
    +
    +  test("ALTER TABLE ADD COLUMNS") {
    --- End diff --
    
    oh. You mean remove the tests from `JDBCSuite` and `TableScanSuite`? 


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive tab...

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

    https://github.com/apache/spark/pull/16626#discussion_r97191587
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---
    @@ -168,6 +168,43 @@ case class AlterTableRenameCommand(
     }
     
     /**
    + * A command that add columns to a table
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   ALTER TABLE table_identifier
    + *   ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
    + * }}}
    +*/
    +case class AlterTableAddColumnsCommand(
    +    table: TableIdentifier,
    +    columns: Seq[StructField]) extends RunnableCommand {
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    val catalog = sparkSession.sessionState.catalog
    +    val catalogTable = DDLUtils.verifyAlterTableAddColumn(catalog, table)
    +
    +    // If an exception is thrown here we can just assume the table is uncached;
    +    // this can happen with Hive tables when the underlying catalog is in-memory.
    +    val wasCached = Try(sparkSession.catalog.isCached(table.unquotedString)).getOrElse(false)
    +    if (wasCached) {
    +      try {
    +        sparkSession.catalog.uncacheTable(table.unquotedString)
    +      } catch {
    +        case NonFatal(e) => log.warn(e.toString, e)
    +      }
    +    }
    +    // Invalidate the table last, otherwise uncaching the table would load the logical plan
    +    // back into the hive metastore cache
    +    catalog.refreshTable(table)
    +
    +    val newSchema = catalogTable.schema.copy(fields = catalogTable.schema.fields ++ columns)
    --- End diff --
    
    We support partitioned tables. The test cases add include this case. 
    However, we don't support ALTER ADD COLUMNS to a particular partition, as what Hive can do today. EX: `ALTER TABLE T1 PARTITION(c3=1) ADD COLUMNS .... `. This is another potential feature to add if we maintain schema for a partition. 


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #73094 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73094/testReport)** for PR 16626 at commit [`193c0c3`](https://github.com/apache/spark/commit/193c0c34a7ec55007fe93e397dace43223b32f58).
     * 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    We also need to test the support of `InMemoryCatalog`. Please do not add a test case yet. I think I really need to finish https://github.com/apache/spark/pull/16592 ASAP. It will make everyone simple to test both Catalogs.


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74937 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74937/testReport)** for PR 16626 at commit [`7d8437d`](https://github.com/apache/spark/commit/7d8437dbba76e1e88c9f230dc7ea895d343dceb8).
     * 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74952/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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/16626#discussion_r107050048
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala ---
    @@ -416,4 +416,22 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext {
         val comments = planned.schema.fields.map(_.getComment().getOrElse("NO_COMMENT")).mkString(",")
         assert(comments === "SN,SA,NO_COMMENT")
       }
    +
    +  test("ALTER TABLE ADD COLUMNS does not support RelationProvider") {
    --- End diff --
    
    let's remove 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74733 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74733/testReport)** for PR 16626 at commit [`61de6b7`](https://github.com/apache/spark/commit/61de6b705d1006ddbde68713a86284634f160166).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106682311
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---
    @@ -175,6 +178,98 @@ case class AlterTableRenameCommand(
     }
     
     /**
    + * A command that add columns to a table
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   ALTER TABLE table_identifier
    + *   ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
    + * }}}
    +*/
    +case class AlterTableAddColumnsCommand(
    +    table: TableIdentifier,
    +    columns: Seq[StructField]) extends RunnableCommand {
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    val catalog = sparkSession.sessionState.catalog
    +    val catalogTable = verifyAlterTableAddColumn(catalog, table)
    +
    +    try {
    +      sparkSession.catalog.uncacheTable(table.quotedString)
    +    } catch {
    +      case NonFatal(e) =>
    +        log.warn(s"Exception when attempting to uncache table ${table.quotedString}", e)
    +    }
    +
    +    // Invalidate the table last, otherwise uncaching the table would load the logical plan
    +    // back into the hive metastore cache
    +    catalog.refreshTable(table)
    +    val partitionFields = catalogTable.schema.takeRight(catalogTable.partitionColumnNames.length)
    +    val newSchemaFields = catalogTable.schema
    +      .take(catalogTable.schema.length - catalogTable.partitionColumnNames.length) ++
    +      columns ++ partitionFields
    +    checkDuplication(sparkSession, newSchemaFields)
    --- End diff --
    
    Will do. 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106097304
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala ---
    @@ -166,6 +166,42 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSQLContext with Befo
         }
       }
     
    +  Seq("parquet", "json", "csv").foreach { provider =>
    +    test(s"Alter table add columns -- ${provider} format") {
    +      assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory")
    +      withTable("t") {
    +        sql(s"create table t (c1 int) using ${provider}")
    +        sql("insert into table t values (1)")
    +        sql("alter table t add columns (c2 int)")
    +        checkAnswer(sql("select * from t"), Seq(Row(1, null)))
    +        sql("insert into table t values (2, 2)")
    +        checkAnswer(sql("select * from t where c2 is not null"), Seq(Row(2, 2)))
    +        checkAnswer(
    +          sql("select * from t"),
    +          Seq(Row(1, null), Row(2, 2))
    +        )
    +      }
    +    }
    +  }
    +
    +  Seq("parquet", "json", "csv").foreach { provider =>
    +    test(s"Alter table add columns with partitions -- ${provider} format") {
    +      assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory")
    +      withTable("t") {
    +        sql(s"create table t (c1 int, c2 int) using ${provider} partitioned by (c2)")
    +        sql("insert into table t values (1, 1)")
    +        sql("alter table t add columns (c3 int)")
    +        checkAnswer(sql("select * from t"), Seq(Row(1, null, 1)))
    +        sql("insert into table t values (2, 2, 3)")
    +        checkAnswer(sql("select * from t where c3 is not null"), Seq(Row(2, 2, 3)))
    +        checkAnswer(sql("select * from t where c2 = 3"), Seq(Row(2, 2, 3)))
    +        checkAnswer(
    +          sql("select * from t"),
    +          Seq(Row(1, null, 1), Row(2, 2, 3))
    +        )
    +      }
    +    }
    +  }
    --- End diff --
    
    Why we need these two test cases? We already have them in `DDLSuite`, which is extended by `InMemoryCatalogedDDLSuite`.


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74680 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74680/testReport)** for PR 16626 at commit [`95a471b`](https://github.com/apache/spark/commit/95a471b566392a1fd32e051e17fd439d5c0b671d).
     * 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74943 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74943/testReport)** for PR 16626 at commit [`7d8437d`](https://github.com/apache/spark/commit/7d8437dbba76e1e88c9f230dc7ea895d343dceb8).
     * 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 pull request #16626: [SPARK-19261][SQL] Alter add columns for Hive tab...

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

    https://github.com/apache/spark/pull/16626#discussion_r97189688
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
    @@ -814,4 +814,28 @@ object DDLUtils {
           }
         }
       }
    +
    +  /**
    +   * ALTER TABLE ADD COLUMNS command does not support temporary view/table,
    +   * view, or datasource table yet.
    +   */
    +  def verifyAlterTableAddColumn(
    +      catalog: SessionCatalog,
    +      table: TableIdentifier): CatalogTable = {
    +    if (catalog.isTemporaryTable(table)) {
    +      throw new AnalysisException(
    +        s"${table.toString} is a temporary VIEW, which does not support ALTER ADD COLUMNS.")
    +    }
    +
    +    val catalogTable = catalog.getTableMetadata(table)
    +    if (catalogTable.tableType == CatalogTableType.VIEW) {
    +      throw new AnalysisException(
    +        s"${table.toString} is a VIEW, which does not support ALTER ADD COLUMNS.")
    +    }
    +    if (isDatasourceTable(catalogTable)) {
    --- End diff --
    
    What is the reason why data source tables are not supported?


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r99679239
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
    @@ -814,4 +816,50 @@ object DDLUtils {
           }
         }
       }
    +
    +  /**
    +   * ALTER TABLE ADD COLUMNS command does not support temporary view/table,
    +   * view, or datasource table with text, orc formats or external provider.
    +   */
    +  def verifyAlterTableAddColumn(
    --- End diff --
    
    Ok. I will move to AlterTableAddColumnsCommand class


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive tab...

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

    https://github.com/apache/spark/pull/16626#discussion_r97195490
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
    @@ -584,14 +593,18 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
           // Sets the `schema`, `partitionColumnNames` and `bucketSpec` from the old table definition,
           // to retain the spark specific format if it is. Also add old data source properties to table
           // properties, to retain the data source table format.
    -      val oldDataSourceProps = oldTableDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX))
    --- End diff --
    
    Hmm, I think this branch is for datasource table, but looks like you don't support datasource table yet in this change?


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73094/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106792952
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala ---
    @@ -165,7 +165,6 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSQLContext with Befo
           assert(e.contains("Hive support is required to CREATE Hive TABLE (AS SELECT)"))
         }
       }
    -
    --- End diff --
    
    revert it back


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106342123
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala ---
    @@ -1860,4 +1860,72 @@ class HiveDDLSuite
           }
         }
       }
    +
    +  Seq("PARQUET", "ORC", "TEXTFILE", "SEQUENCEFILE", "RCFILE", "AVRO").foreach { tableType =>
    +    test(s"alter hive serde table add columns -- partitioned - $tableType") {
    +      withTable("alter_add_partitioned") {
    --- End diff --
    
    The name is confusing. Let us just simplify it to `tab`. We already can know the scenario by the test case name.


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #72530 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72530/testReport)** for PR 16626 at commit [`0b7c0b1`](https://github.com/apache/spark/commit/0b7c0b146f69f7deba02b032c14bb1e8a1ed56d5).
     * 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106786366
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala ---
    @@ -2178,4 +2197,107 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
           }
         }
       }
    +
    +  Seq("parquet", "json", "csv").foreach { provider =>
    +    test(s"alter datasource table add columns - $provider") {
    +      withTable("t1") {
    +        sql(s"CREATE TABLE t1 (c1 int) USING $provider")
    +        sql("INSERT INTO t1 VALUES (1)")
    +        sql("ALTER TABLE t1 ADD COLUMNS (c2 int)")
    +        checkAnswer(
    +          sql("SELECT * FROM t1"),
    +          Seq(Row(1, null))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c2 is null"),
    +          Seq(Row(1, null))
    +        )
    +
    +        sql("INSERT INTO t1 VALUES (3, 2)")
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c2 = 2"),
    +          Seq(Row(3, 2))
    +        )
    +      }
    +    }
    +  }
    +
    +  Seq("parquet", "json", "csv").foreach { provider =>
    +    test(s"alter datasource table add columns - partitioned - $provider") {
    +      withTable("t1") {
    +        sql(s"CREATE TABLE t1 (c1 int, c2 int) USING $provider PARTITIONED BY (c2)")
    +        sql("INSERT INTO t1 PARTITION(c2 = 2) VALUES (1)")
    +        sql("ALTER TABLE t1 ADD COLUMNS (c3 int)")
    +        checkAnswer(
    +          sql("SELECT * FROM t1"),
    +          Seq(Row(1, null, 2))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c3 is null"),
    +          Seq(Row(1, null, 2))
    +        )
    +        sql("INSERT INTO t1 PARTITION(c2 =1) VALUES (2, 3)")
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c3 = 3"),
    +          Seq(Row(2, 3, 1))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c2 = 1"),
    +          Seq(Row(2, 3, 1))
    +        )
    +      }
    +    }
    +  }
    +
    +  test("alter datasource table add columns - text format not supported") {
    +    withTable("t1") {
    +      sql(s"CREATE TABLE t1 (c1 int) USING text")
    --- End diff --
    
    `sql("CREATE TABLE t1 (c1 int) USING text")`


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r99660704
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
    @@ -814,4 +816,50 @@ object DDLUtils {
           }
         }
       }
    +
    +  /**
    +   * ALTER TABLE ADD COLUMNS command does not support temporary view/table,
    +   * view, or datasource table with text, orc formats or external provider.
    +   */
    +  def verifyAlterTableAddColumn(
    --- End diff --
    
    oh. this is ddl util function. 


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r99679303
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
    @@ -814,4 +816,50 @@ object DDLUtils {
           }
         }
       }
    +
    +  /**
    +   * ALTER TABLE ADD COLUMNS command does not support temporary view/table,
    +   * view, or datasource table with text, orc formats or external provider.
    +   */
    +  def verifyAlterTableAddColumn(
    +      catalog: SessionCatalog,
    +      table: TableIdentifier): CatalogTable = {
    +    if (catalog.isTemporaryTable(table)) {
    +      throw new AnalysisException(
    +        s"${table.toString} is a temporary VIEW, which does not support ALTER ADD COLUMNS.")
    +    }
    +
    +    val catalogTable = catalog.getTableMetadata(table)
    --- End diff --
    
    Call `getTempViewOrPermanentTableMetadata` instead of `getTableMetadata`. Then, you do not need the above check for temporary views. In addition, it also covers the cases for global views.


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74791 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74791/testReport)** for PR 16626 at commit [`a28fc42`](https://github.com/apache/spark/commit/a28fc42cee6dc52516e074ced7c4351ee6baa45d).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    @cloud-fan @gatorsmile Thanks again! I updated the code based @cloud-fan 's review. 


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106472336
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---
    @@ -175,6 +178,87 @@ case class AlterTableRenameCommand(
     }
     
     /**
    + * A command that add columns to a table
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   ALTER TABLE table_identifier
    + *   ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
    + * }}}
    +*/
    +case class AlterTableAddColumnsCommand(
    +    table: TableIdentifier,
    +    columns: Seq[StructField]) extends RunnableCommand {
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    val catalog = sparkSession.sessionState.catalog
    +    val catalogTable = verifyAlterTableAddColumn(catalog, table)
    +
    +    try {
    +      sparkSession.catalog.uncacheTable(table.unquotedString)
    +    } catch {
    +      case NonFatal(e) =>
    +        log.warn(s"Exception when attempting to uncache table ${table.unquotedString}", e)
    +    }
    +
    +    // Invalidate the table last, otherwise uncaching the table would load the logical plan
    +    // back into the hive metastore cache
    +    catalog.refreshTable(table)
    +    val partitionFields = catalogTable.schema.takeRight(catalogTable.partitionColumnNames.length)
    +    val newSchemaFields = catalogTable.schema
    +      .take(catalogTable.schema.length - catalogTable.partitionColumnNames.length) ++
    +      columns ++ partitionFields
    +    checkDuplication(newSchemaFields.map(_.name))
    +    catalog.alterTableSchema(table, newSchema =
    +      catalogTable.schema.copy(fields = newSchemaFields.toArray))
    +
    +    Seq.empty[Row]
    +  }
    +
    +  private def checkDuplication(colNames: Seq[String]): Unit = {
    +    if (colNames.distinct.length != colNames.length) {
    +      val duplicateColumns = colNames.groupBy(identity).collect {
    +        case (x, ys) if ys.length > 1 => x
    +      }
    +      throw new AnalysisException(
    +        s"Found duplicate column(s): ${duplicateColumns.mkString(", ")}")
    +    }
    +  }
    +
    +  /**
    +   * ALTER TABLE ADD COLUMNS command does not support temporary view/table,
    +   * view, or datasource table with text, orc formats or external provider.
    +   * For datasource table, it currently only supports parquet, json, csv.
    +   */
    +  private def verifyAlterTableAddColumn(
    +      catalog: SessionCatalog,
    +      table: TableIdentifier): CatalogTable = {
    +    val catalogTable = catalog.getTempViewOrPermanentTableMetadata(table)
    +
    +    if (catalogTable.tableType == CatalogTableType.VIEW) {
    +      throw new AnalysisException(
    +        s"${table.toString} is a VIEW, which does not support ALTER ADD COLUMNS.")
    +    }
    +
    +    if (DDLUtils.isDatasourceTable(catalogTable)) {
    +      DataSource.lookupDataSource(catalogTable.provider.get).newInstance() match {
    +        // For datasource table, this command can only support the following File format.
    +        // TextFileFormat only default to one column "value"
    +        // OrcFileFormat can not handle difference between user-specified schema and
    +        // inferred schema yet. TODO, once this issue is resolved , we can add Orc back.
    +        // Hive type is already considered as hive serde table, so the logic will not
    +        // come in here.
    +        case _: JsonFileFormat | _: CSVFileFormat | _: ParquetFileFormat =>
    +        case s =>
    +          throw new AnalysisException(
    +            s"Datasource table $table with type $s, which does not support ALTER ADD COLUMNS.")
    --- End diff --
    
    The same 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 issue #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Maybe you can resolve the conflict now?


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106786279
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala ---
    @@ -1860,4 +1861,119 @@ class HiveDDLSuite
           }
         }
       }
    +
    +  hiveFormats.foreach { tableType =>
    +    test(s"alter hive serde table add columns -- partitioned - $tableType") {
    +      withTable("tab") {
    +        sql(
    +          s"""
    +             |CREATE TABLE tab (c1 int, c2 int)
    +             |PARTITIONED BY (c3 int) STORED AS $tableType
    +          """.stripMargin)
    +
    +        sql("INSERT INTO tab PARTITION (c3=1) VALUES (1, 2)")
    +        sql("ALTER TABLE tab ADD COLUMNS (c4 int)")
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c3 = 1"),
    +          Seq(Row(1, 2, null, 1))
    +        )
    +        assert(sql("SELECT * FROM tab").schema
    +          .contains(StructField("c4", IntegerType)))
    +        sql("INSERT INTO tab PARTITION (c3=2) VALUES (2, 3, 4)")
    +        checkAnswer(
    +          sql("SELECT * FROM tab"),
    +          Seq(Row(1, 2, null, 1), Row(2, 3, 4, 2))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c3 = 2 AND c4 IS NOT NULL"),
    +          Seq(Row(2, 3, 4, 2))
    +        )
    +      }
    +    }
    +  }
    +
    +  hiveFormats.foreach { tableType =>
    +    test(s"alter hive serde table add columns -- with predicate - $tableType ") {
    +      withTable("tab") {
    +        sql(s"CREATE TABLE tab (c1 int, c2 int) STORED AS $tableType")
    +        sql("INSERT INTO tab VALUES (1, 2)")
    +        sql("ALTER TABLE tab ADD COLUMNS (c4 int)")
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c4 IS NULL"),
    +          Seq(Row(1, 2, null))
    +        )
    +        assert(sql("SELECT * FROM tab").schema
    +          .contains(StructField("c4", IntegerType)))
    +        sql("INSERT INTO tab VALUES (2, 3, 4)")
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c4 = 4 "),
    +          Seq(Row(2, 3, 4))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM tab"),
    --- End diff --
    
    `spark.table("tab")`


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #73450 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73450/testReport)** for PR 16626 at commit [`4bd04fc`](https://github.com/apache/spark/commit/4bd04fc556fde3805f58e39a0e6ad50c2a1e8aec).
     * 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74952 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74952/testReport)** for PR 16626 at commit [`7d8437d`](https://github.com/apache/spark/commit/7d8437dbba76e1e88c9f230dc7ea895d343dceb8).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106467818
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---
    @@ -175,6 +178,87 @@ case class AlterTableRenameCommand(
     }
     
     /**
    + * A command that add columns to a table
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   ALTER TABLE table_identifier
    + *   ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
    + * }}}
    +*/
    +case class AlterTableAddColumnsCommand(
    +    table: TableIdentifier,
    +    columns: Seq[StructField]) extends RunnableCommand {
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    val catalog = sparkSession.sessionState.catalog
    +    val catalogTable = verifyAlterTableAddColumn(catalog, table)
    +
    +    try {
    +      sparkSession.catalog.uncacheTable(table.unquotedString)
    +    } catch {
    +      case NonFatal(e) =>
    +        log.warn(s"Exception when attempting to uncache table ${table.unquotedString}", e)
    +    }
    +
    +    // Invalidate the table last, otherwise uncaching the table would load the logical plan
    +    // back into the hive metastore cache
    +    catalog.refreshTable(table)
    +    val partitionFields = catalogTable.schema.takeRight(catalogTable.partitionColumnNames.length)
    +    val newSchemaFields = catalogTable.schema
    +      .take(catalogTable.schema.length - catalogTable.partitionColumnNames.length) ++
    +      columns ++ partitionFields
    +    checkDuplication(newSchemaFields.map(_.name))
    +    catalog.alterTableSchema(table, newSchema =
    +      catalogTable.schema.copy(fields = newSchemaFields.toArray))
    +
    +    Seq.empty[Row]
    +  }
    +
    +  private def checkDuplication(colNames: Seq[String]): Unit = {
    +    if (colNames.distinct.length != colNames.length) {
    +      val duplicateColumns = colNames.groupBy(identity).collect {
    --- End diff --
    
    This does not consider the case sensitivity 


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74761/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106785833
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -296,6 +297,64 @@ class SessionCatalog(
       }
     
       /**
    +   * Alter the schema of a table identified by the provided table identifier. The new schema
    +   * should still contain the existing bucket columns and partition columns used by the table. This
    +   * method will also update any Spark SQL-related parameters stored as Hive table properties (such
    +   * as the schema itself).
    +   *
    +   * @param identifier TableIdentifier
    +   * @param newSchema Updated schema to be used for the table (must contain existing partition and
    +   *                  bucket columns)
    +   */
    +  def alterTableSchema(
    +      identifier: TableIdentifier,
    +      newSchema: StructType): Unit = {
    +    val db = formatDatabaseName(identifier.database.getOrElse(getCurrentDatabase))
    +    val table = formatTableName(identifier.table)
    +    val tableIdentifier = TableIdentifier(table, Some(db))
    +    requireDbExists(db)
    +    requireTableExists(tableIdentifier)
    +    val catalogTable = externalCatalog.getTable(db, table)
    +    val oldSchema = catalogTable.schema
    +
    +    // no supporting dropping columns yet
    +    if (!oldSchema.forall(f => columnNameResolved(newSchema, f.name ))) {
    +      throw new AnalysisException(
    +        s"""
    +          |Some existing schema fields are not present in the new schema.
    +          |We don't support dropping columns yet.
    +         """.stripMargin)
    +    }
    +
    +    checkDuplication(newSchema)
    --- End diff --
    
    Move this just below `requireTableExists(tableIdentifier)`


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive tables

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #71749 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71749/testReport)** for PR 16626 at commit [`73b0243`](https://github.com/apache/spark/commit/73b024309674dc6d76e853547ef2a64da4836ce8).
     * 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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/16626#discussion_r107049525
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala ---
    @@ -2178,4 +2178,138 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
           }
         }
       }
    +
    +  val supportedNativeFileFormatsForAlterTableAddColumns = Seq("parquet", "json", "csv")
    +
    +  supportedNativeFileFormatsForAlterTableAddColumns.foreach { provider =>
    +    test(s"alter datasource table add columns - $provider") {
    +      withTable("t1") {
    +        sql(s"CREATE TABLE t1 (c1 int) USING $provider")
    +        sql("INSERT INTO t1 VALUES (1)")
    +        sql("ALTER TABLE t1 ADD COLUMNS (c2 int)")
    +        checkAnswer(
    +          spark.table("t1"),
    +          Seq(Row(1, null))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c2 is null"),
    +          Seq(Row(1, null))
    +        )
    +
    +        sql("INSERT INTO t1 VALUES (3, 2)")
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c2 = 2"),
    +          Seq(Row(3, 2))
    +        )
    +      }
    +    }
    +  }
    +
    +  supportedNativeFileFormatsForAlterTableAddColumns.foreach { provider =>
    +    test(s"alter datasource table add columns - partitioned - $provider") {
    +      withTable("t1") {
    +        sql(s"CREATE TABLE t1 (c1 int, c2 int) USING $provider PARTITIONED BY (c2)")
    +        sql("INSERT INTO t1 PARTITION(c2 = 2) VALUES (1)")
    +        sql("ALTER TABLE t1 ADD COLUMNS (c3 int)")
    +        checkAnswer(
    +          spark.table("t1"),
    +          Seq(Row(1, null, 2))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c3 is null"),
    +          Seq(Row(1, null, 2))
    +        )
    +        sql("INSERT INTO t1 PARTITION(c2 =1) VALUES (2, 3)")
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c3 = 3"),
    +          Seq(Row(2, 3, 1))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c2 = 1"),
    +          Seq(Row(2, 3, 1))
    +        )
    +      }
    +    }
    +  }
    +
    +  test("alter datasource table add columns - text format not supported") {
    +    withTable("t1") {
    +      sql("CREATE TABLE t1 (c1 int) USING text")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE t1 ADD COLUMNS (c2 int)")
    +      }.getMessage
    +      assert(e.contains("ALTER ADD COLUMNS does not support datasource table with type"))
    +    }
    +  }
    +
    +  test("alter table add columns -- not support temp view") {
    +    withTempView("tmp_v") {
    +      sql("CREATE TEMPORARY VIEW tmp_v AS SELECT 1 AS c1, 2 AS c2")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE tmp_v ADD COLUMNS (c3 INT)")
    +      }
    +      assert(e.message.contains("ALTER ADD COLUMNS does not support views"))
    +    }
    +  }
    +
    +  test("alter table add columns -- not support view") {
    +    withView("v1") {
    +      sql("CREATE VIEW v1 AS SELECT 1 AS c1, 2 AS c2")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE v1 ADD COLUMNS (c3 INT)")
    +      }
    +      assert(e.message.contains("ALTER ADD COLUMNS does not support views"))
    +    }
    +  }
    +
    +  test("alter table add columns with existing column name") {
    +    withTable("t1") {
    +      sql("CREATE TABLE t1 (c1 int) USING PARQUET")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE t1 ADD COLUMNS (c1 string)")
    +      }.getMessage
    +      assert(e.contains("Found duplicate column(s)"))
    +    }
    +  }
    +
    +  test("alter table add columns to table referenced by a view") {
    --- End diff --
    
    this is not needed, the view test already covered the case when the referenced table change its schema


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive tab...

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

    https://github.com/apache/spark/pull/16626#discussion_r97191445
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
    @@ -814,4 +814,28 @@ object DDLUtils {
           }
         }
       }
    +
    +  /**
    +   * ALTER TABLE ADD COLUMNS command does not support temporary view/table,
    +   * view, or datasource table yet.
    +   */
    +  def verifyAlterTableAddColumn(
    +      catalog: SessionCatalog,
    +      table: TableIdentifier): CatalogTable = {
    +    if (catalog.isTemporaryTable(table)) {
    +      throw new AnalysisException(
    +        s"${table.toString} is a temporary VIEW, which does not support ALTER ADD COLUMNS.")
    +    }
    +
    +    val catalogTable = catalog.getTableMetadata(table)
    +    if (catalogTable.tableType == CatalogTableType.VIEW) {
    +      throw new AnalysisException(
    +        s"${table.toString} is a VIEW, which does not support ALTER ADD COLUMNS.")
    +    }
    +    if (isDatasourceTable(catalogTable)) {
    --- End diff --
    
    Currently, their code paths for managing hive serde tables and data source tables have been combined. Thus, it can be easily handled together. 


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r107063564
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -455,8 +455,10 @@ abstract class SessionCatalogSuite extends PlanTest {
         withBasicCatalog { sessionCatalog =>
           sessionCatalog.createTable(newTable("t1", "default"), ignoreIfExists = false)
           val oldTab = sessionCatalog.externalCatalog.getTable("default", "t1")
    +      val partitionSchema = oldTab.partitionSchema
    --- End diff --
    
    Remove this line


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74281 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74281/testReport)** for PR 16626 at commit [`50564f3`](https://github.com/apache/spark/commit/50564f333de95e7c2e2232f6940eef144fb51e3e).
     * This patch passes all tests.
     * This patch **does not merge 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106786328
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala ---
    @@ -1860,4 +1861,119 @@ class HiveDDLSuite
           }
         }
       }
    +
    +  hiveFormats.foreach { tableType =>
    +    test(s"alter hive serde table add columns -- partitioned - $tableType") {
    +      withTable("tab") {
    +        sql(
    +          s"""
    +             |CREATE TABLE tab (c1 int, c2 int)
    +             |PARTITIONED BY (c3 int) STORED AS $tableType
    +          """.stripMargin)
    +
    +        sql("INSERT INTO tab PARTITION (c3=1) VALUES (1, 2)")
    +        sql("ALTER TABLE tab ADD COLUMNS (c4 int)")
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c3 = 1"),
    +          Seq(Row(1, 2, null, 1))
    +        )
    +        assert(sql("SELECT * FROM tab").schema
    --- End diff --
    
    Use `catalog.getTableMetadata` to get the table metadata and then verify the schema


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74652 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74652/testReport)** for PR 16626 at commit [`211287e`](https://github.com/apache/spark/commit/211287edc3437a951b2c6579d31e71f38f875306).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74943/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106344896
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala ---
    @@ -71,7 +71,6 @@ class JDBCSuite extends SparkFunSuite
         conn.prepareStatement("insert into test.people values ('mary', 2)").executeUpdate()
         conn.prepareStatement(
           "insert into test.people values ('joe ''foo'' \"bar\"', 3)").executeUpdate()
    -    conn.commit()
    --- End diff --
    
    This is a mistake


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74680 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74680/testReport)** for PR 16626 at commit [`95a471b`](https://github.com/apache/spark/commit/95a471b566392a1fd32e051e17fd439d5c0b671d).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #72447 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72447/testReport)** for PR 16626 at commit [`88c2f48`](https://github.com/apache/spark/commit/88c2f48f730460c38aef50d02af08dd1df5c2097).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106097921
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -296,6 +297,25 @@ class SessionCatalog(
       }
     
       /**
    +   * Alter the schema of a table identified by the provided table identifier. The new schema
    +   * should still contain the existing bucket columns and partition columns used by the table. This
    +   * method will also update any Spark SQL-related parameters stored as Hive table properties (such
    +   * as the schema itself).
    +   *
    +   * @param identifier  TableIdentifier
    +   * @param newSchema Updated schema to be used for the table (must contain existing partition and
    +   *               bucket columns)
    --- End diff --
    
    Nit: please fix the styles of the above two @ parm 


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 pull request #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r101960271
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
    @@ -523,18 +523,29 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
               statsProperties += (columnStatKeyPropName(colName, k) -> v)
             }
           }
    -      tableDefinition.copy(properties = tableDefinition.properties ++ statsProperties)
    +      newTableDefinition.copy(properties = newTableDefinition.properties ++ statsProperties)
         } else {
    -      tableDefinition
    +      newTableDefinition
         }
     
    -    if (tableDefinition.tableType == VIEW) {
    -      client.alterTable(withStatsProps)
    +    if (newTableDefinition.tableType == VIEW) {
    +      client.alterTable(maybeWithStatsPropsTable)
         } else {
    -      val oldTableDef = getRawTable(db, withStatsProps.identifier.table)
    --- End diff --
    
    To the other reviewers: `oldTableDef ` actually is storing the raw table metadata. In the new changes, it is renamed to `oldRawTableDef `


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106786292
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala ---
    @@ -1860,4 +1861,119 @@ class HiveDDLSuite
           }
         }
       }
    +
    +  hiveFormats.foreach { tableType =>
    +    test(s"alter hive serde table add columns -- partitioned - $tableType") {
    +      withTable("tab") {
    +        sql(
    +          s"""
    +             |CREATE TABLE tab (c1 int, c2 int)
    +             |PARTITIONED BY (c3 int) STORED AS $tableType
    +          """.stripMargin)
    +
    +        sql("INSERT INTO tab PARTITION (c3=1) VALUES (1, 2)")
    +        sql("ALTER TABLE tab ADD COLUMNS (c4 int)")
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c3 = 1"),
    +          Seq(Row(1, 2, null, 1))
    +        )
    +        assert(sql("SELECT * FROM tab").schema
    +          .contains(StructField("c4", IntegerType)))
    +        sql("INSERT INTO tab PARTITION (c3=2) VALUES (2, 3, 4)")
    +        checkAnswer(
    +          sql("SELECT * FROM tab"),
    --- End diff --
    
    `spark.table("tab")`


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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/16626#discussion_r107048698
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -296,6 +311,51 @@ class SessionCatalog(
       }
     
       /**
    +   * Alter the schema of a table identified by the provided table identifier. The new schema
    +   * should still contain the existing bucket columns and partition columns used by the table. This
    +   * method will also update any Spark SQL-related parameters stored as Hive table properties (such
    +   * as the schema itself).
    +   *
    +   * @param identifier TableIdentifier
    +   * @param newSchema Updated schema to be used for the table (must contain existing partition and
    +   *                  bucket columns)
    +   */
    +  def alterTableSchema(
    +      identifier: TableIdentifier,
    +      newSchema: StructType): Unit = {
    +    val db = formatDatabaseName(identifier.database.getOrElse(getCurrentDatabase))
    +    val table = formatTableName(identifier.table)
    +    val tableIdentifier = TableIdentifier(table, Some(db))
    +    requireDbExists(db)
    +    requireTableExists(tableIdentifier)
    +    checkDuplication(newSchema)
    +
    +    val catalogTable = externalCatalog.getTable(db, table)
    +    val oldSchema = catalogTable.schema
    +
    +    // not supporting dropping columns yet
    +    val nonExistentColumnNames = oldSchema.map(_.name).filterNot(columnNameResolved(newSchema, _))
    +    if (nonExistentColumnNames.nonEmpty) {
    +      throw new AnalysisException(
    +        s"""
    +           |Some existing schema fields (${nonExistentColumnNames.mkString("[", ",", "]")}) are
    +           |not present in the new schema. We don't support dropping columns yet.
    +         """.stripMargin)
    +    }
    +
    +    // make sure partition columns are at the end
    +    val partitionSchema = catalogTable.partitionSchema
    --- End diff --
    
    `CatalogTable.partitionSchema` will throw exception if partition columns are not at the end, so we can just call `partitionSchema`, no need to do the reordering.


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive tables

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106341499
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---
    @@ -175,6 +178,78 @@ case class AlterTableRenameCommand(
     }
     
     /**
    + * A command that add columns to a table
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   ALTER TABLE table_identifier
    + *   ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
    + * }}}
    +*/
    +case class AlterTableAddColumnsCommand(
    +    table: TableIdentifier,
    +    columns: Seq[StructField]) extends RunnableCommand {
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    val catalog = sparkSession.sessionState.catalog
    +    val catalogTable = verifyAlterTableAddColumn(catalog, table)
    +
    +    // If an exception is thrown here we can just assume the table is uncached;
    +    // this can happen with Hive tables when the underlying catalog is in-memory.
    +    val wasCached = Try(sparkSession.catalog.isCached(table.unquotedString)).getOrElse(false)
    +    if (wasCached) {
    +      try {
    +        sparkSession.catalog.uncacheTable(table.unquotedString)
    +      } catch {
    +        case NonFatal(e) => log.warn(e.toString, e)
    +      }
    +    }
    --- End diff --
    
    No need to check if it is cached or not. Just uncache 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r99680029
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
    @@ -814,4 +816,50 @@ object DDLUtils {
           }
         }
       }
    +
    +  /**
    +   * ALTER TABLE ADD COLUMNS command does not support temporary view/table,
    +   * view, or datasource table with text, orc formats or external provider.
    +   */
    +  def verifyAlterTableAddColumn(
    +      catalog: SessionCatalog,
    +      table: TableIdentifier): CatalogTable = {
    +    if (catalog.isTemporaryTable(table)) {
    +      throw new AnalysisException(
    +        s"${table.toString} is a temporary VIEW, which does not support ALTER ADD COLUMNS.")
    +    }
    +
    +    val catalogTable = catalog.getTableMetadata(table)
    +    if (catalogTable.tableType == CatalogTableType.VIEW) {
    +      throw new AnalysisException(
    +        s"${table.toString} is a VIEW, which does not support ALTER ADD COLUMNS.")
    +    }
    +
    +    if (isDatasourceTable(catalogTable)) {
    +      catalogTable.provider.get match {
    +        case provider if provider.toLowerCase == "text" =>
    +          // TextFileFormat can not support adding column either because text datasource table
    +          // is resolved as a single-column table only.
    +          throw new AnalysisException(
    +            s"""${table.toString} is a text format datasource table,
    +               |which does not support ALTER ADD COLUMNS.""".stripMargin)
    +        case provider if provider.toLowerCase == "orc"
    +          || provider.startsWith("org.apache.spark.sql.hive.orc") =>
    --- End diff --
    
    When we store the metadata in the catalog, we unify different representations to `orc`, right? Can you find any case to break 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74035/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r101961067
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
    @@ -563,35 +574,47 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
             //       want to alter the table location to a file path, we will fail. This should be fixed
             //       in the future.
     
    -        val newLocation = tableDefinition.storage.locationUri
    -        val storageWithPathOption = tableDefinition.storage.copy(
    -          properties = tableDefinition.storage.properties ++ newLocation.map("path" -> _))
    +        val newLocation = newTableDefinition.storage.locationUri
    +        val storageWithPathOption = newTableDefinition.storage.copy(
    +          properties = newTableDefinition.storage.properties ++ newLocation.map("path" -> _))
     
    -        val oldLocation = getLocationFromStorageProps(oldTableDef)
    +        val oldLocation = getLocationFromStorageProps(oldRawTableDef)
             if (oldLocation == newLocation) {
    -          storageWithPathOption.copy(locationUri = oldTableDef.storage.locationUri)
    +          storageWithPathOption.copy(locationUri = oldRawTableDef.storage.locationUri)
             } else {
               storageWithPathOption
             }
           }
     
    -      val partitionProviderProp = if (tableDefinition.tracksPartitionsInCatalog) {
    +      val partitionProviderProp = if (newTableDefinition.tracksPartitionsInCatalog) {
             TABLE_PARTITION_PROVIDER -> TABLE_PARTITION_PROVIDER_CATALOG
           } else {
             TABLE_PARTITION_PROVIDER -> TABLE_PARTITION_PROVIDER_FILESYSTEM
           }
     
    -      // Sets the `schema`, `partitionColumnNames` and `bucketSpec` from the old table definition,
    +      // Sets the `partitionColumnNames` and `bucketSpec` from the old table definition,
           // to retain the spark specific format if it is. Also add old data source properties to table
           // properties, to retain the data source table format.
    -      val oldDataSourceProps = oldTableDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX))
    -      val newTableProps = oldDataSourceProps ++ withStatsProps.properties + partitionProviderProp
    -      val newDef = withStatsProps.copy(
    +      val dataSourceProps = if (schemaChange) {
    +        val props =
    +          tableMetaToTableProps(newTableDefinition).filter(_._1.startsWith(DATASOURCE_PREFIX))
    +        if (newTableDefinition.provider.isDefined
    +          && newTableDefinition.provider.get.toLowerCase != DDLUtils.HIVE_PROVIDER) {
    +          // we only need to populate non-hive provider to the tableprops
    +          props.put(DATASOURCE_PROVIDER, newTableDefinition.provider.get)
    +        }
    +        props
    +      } else {
    +        oldRawTableDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX))
    +      }
    +      val newTableProps =
    +        dataSourceProps ++ maybeWithStatsPropsTable.properties + partitionProviderProp
    --- End diff --
    
    Let's create a new helper function for generating the table properties. Now, `alterTable` has 100+ lines


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r99679185
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
    @@ -763,7 +763,9 @@ object DDLUtils {
       val HIVE_PROVIDER = "hive"
     
       def isHiveTable(table: CatalogTable): Boolean = {
    -    table.provider.isDefined && table.provider.get.toLowerCase == HIVE_PROVIDER
    +    // When `CatalogTable` is directly fetched from the catalog,
    +    // CatalogTable.provider = None means the table is a Hive serde table.
    +    !table.provider.isDefined || table.provider.get.toLowerCase == HIVE_PROVIDER
    --- End diff --
    
    I see. I will find another way. 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106786118
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala ---
    @@ -166,6 +166,25 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSQLContext with Befo
         }
       }
     
    +  Seq("true", "false").foreach { caseSensitive =>
    +    test(s"alter table add columns with existing column name - caseSensitive $caseSensitive") {
    +      withSQLConf(("spark.sql.caseSensitive", caseSensitive)) {
    --- End diff --
    
    `withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive)`


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive tab...

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

    https://github.com/apache/spark/pull/16626#discussion_r97189715
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---
    @@ -168,6 +168,43 @@ case class AlterTableRenameCommand(
     }
     
     /**
    + * A command that add columns to a table
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   ALTER TABLE table_identifier
    + *   ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
    + * }}}
    +*/
    +case class AlterTableAddColumnsCommand(
    +    table: TableIdentifier,
    +    columns: Seq[StructField]) extends RunnableCommand {
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    val catalog = sparkSession.sessionState.catalog
    +    val catalogTable = DDLUtils.verifyAlterTableAddColumn(catalog, table)
    +
    +    // If an exception is thrown here we can just assume the table is uncached;
    +    // this can happen with Hive tables when the underlying catalog is in-memory.
    +    val wasCached = Try(sparkSession.catalog.isCached(table.unquotedString)).getOrElse(false)
    +    if (wasCached) {
    +      try {
    +        sparkSession.catalog.uncacheTable(table.unquotedString)
    +      } catch {
    +        case NonFatal(e) => log.warn(e.toString, e)
    +      }
    +    }
    +    // Invalidate the table last, otherwise uncaching the table would load the logical plan
    +    // back into the hive metastore cache
    +    catalog.refreshTable(table)
    +
    +    val newSchema = catalogTable.schema.copy(fields = catalogTable.schema.fields ++ columns)
    --- End diff --
    
    We are not supporting partitioned tables, 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 issue #16626: [SPARK-19261][SQL] Alter add columns for Hive tables

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #71728 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71728/testReport)** for PR 16626 at commit [`ffb8b55`](https://github.com/apache/spark/commit/ffb8b5544822851085fc6ca101852b6d28ac413b).
     * 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #72447 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72447/testReport)** for PR 16626 at commit [`88c2f48`](https://github.com/apache/spark/commit/88c2f48f730460c38aef50d02af08dd1df5c2097).
     * 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74583/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r107063379
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -343,15 +343,11 @@ class SessionCatalog(
              """.stripMargin)
         }
     
    -    // make sure partition columns are at the end
    -    val partitionSchema = catalogTable.partitionSchema
    -    val reorderedSchema = newSchema
    -      .filterNot(f => columnNameResolved(partitionSchema, f.name)) ++ partitionSchema
    -
    -    externalCatalog.alterTableSchema(db, table, StructType(reorderedSchema))
    +    // assuming the newSchema has all partition columns at the end as required
    +    externalCatalog.alterTableSchema(db, table, StructType(newSchema))
    --- End diff --
    
    `StructType(newSchema)` -> `newSchema`


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r101959389
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
    @@ -504,15 +504,15 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
        * Note: As of now, this doesn't support altering table schema, partition column names and bucket
        * specification. We will ignore them even if users do specify different values for these fields.
        */
    -  override def alterTable(tableDefinition: CatalogTable): Unit = withClient {
    -    assert(tableDefinition.identifier.database.isDefined)
    -    val db = tableDefinition.identifier.database.get
    -    requireTableExists(db, tableDefinition.identifier.table)
    -    verifyTableProperties(tableDefinition)
    +  override def alterTable(newTableDefinition: CatalogTable): Unit = withClient {
    +    assert(newTableDefinition.identifier.database.isDefined)
    +    val db = newTableDefinition.identifier.database.get
    +    requireTableExists(db, newTableDefinition.identifier.table)
    +    verifyTableProperties(newTableDefinition)
     
         // convert table statistics to properties so that we can persist them through hive api
    -    val withStatsProps = if (tableDefinition.stats.isDefined) {
    -      val stats = tableDefinition.stats.get
    +    val maybeWithStatsPropsTable: CatalogTable = if (newTableDefinition.stats.isDefined) {
    --- End diff --
    
    Keep the original name ` withStatsProps`


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Please also add a test case: ALTER TABLE does not affect any view that references the table being altered. Also includes the views that have an "*" in their SELECT list. 


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Adding a check in the existing test case to see if `HIVE_TYPE_STRING` is correctly populated in the metadata. 
    
    LGTM except a few minor comments
    
    cc @cloud-fan 


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72320/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74652/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74761 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74761/testReport)** for PR 16626 at commit [`b219178`](https://github.com/apache/spark/commit/b2191788f5d60261946da63e8c2634d5c6dfe6f5).
     * 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106097855
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -296,6 +297,25 @@ class SessionCatalog(
       }
     
       /**
    +   * Alter the schema of a table identified by the provided table identifier. The new schema
    +   * should still contain the existing bucket columns and partition columns used by the table. This
    +   * method will also update any Spark SQL-related parameters stored as Hive table properties (such
    +   * as the schema itself).
    +   *
    +   * @param identifier  TableIdentifier
    +   * @param newSchema Updated schema to be used for the table (must contain existing partition and
    +   *               bucket columns)
    +   */
    +  def alterTableSchema(identifier: TableIdentifier, newSchema: StructType): Unit = {
    +    val db = formatDatabaseName(identifier.database.getOrElse(getCurrentDatabase))
    +    val table = formatTableName(identifier.table)
    +    val tableIdentifier = TableIdentifier(table, Some(db))
    +    requireDbExists(db)
    +    requireTableExists(tableIdentifier)
    +    externalCatalog.alterTableSchema(db, table, newSchema)
    +  }
    --- End diff --
    
    Please add a test cases to SessionCatalogSuite.scala


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74736 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74736/testReport)** for PR 16626 at commit [`8bf7a02`](https://github.com/apache/spark/commit/8bf7a0252e47abd336d831e8c7a771bebad3b6ee).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106098075
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala ---
    @@ -2182,4 +2218,85 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
           }
         }
       }
    +
    +  Seq("parquet", "json", "csv").foreach { provider =>
    +    test(s"alter datasource table add columns - $provider") {
    +      withTable("alter_add_ds") {
    +        sql(s"CREATE TABLE alter_add_ds (c1 int) USING $provider")
    +        sql("INSERT INTO alter_add_ds VALUES (1)")
    +        sql("ALTER TABLE alter_add_ds ADD COLUMNS (c2 int)")
    +        checkAnswer(
    +          sql("SELECT * FROM alter_add_ds"),
    +          Seq(Row(1, null))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM alter_add_ds where c2 is null"),
    +          Seq(Row(1, null))
    +        )
    +
    +        sql("INSERT INTO alter_add_ds VALUES (3, 2)")
    +        checkAnswer(
    +          sql("SELECT * FROM alter_add_ds where c2 = 2"),
    +          Seq(Row(3, 2))
    +        )
    +      }
    +    }
    +  }
    +
    +  Seq("parquet", "json", "csv").foreach { provider =>
    +    test(s"alter datasource table add columns - partitioned - $provider") {
    +      withTable("alter_add_ds") {
    +        sql(s"CREATE TABLE alter_add_ds (c1 int, c2 int) USING $provider partitioned by (c2)")
    +        sql("INSERT INTO alter_add_ds partition(c2 = 2) VALUES (1)")
    +        sql("ALTER TABLE alter_add_ds ADD COLUMNS (c3 int)")
    +        checkAnswer(
    +          sql("SELECT * FROM alter_add_ds"),
    +          Seq(Row(1, null, 2))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM alter_add_ds where c3 is null"),
    +          Seq(Row(1, null, 2))
    +        )
    +        sql("INSERT INTO alter_add_ds partition(c2 =1) VALUES (2, 3)")
    +        checkAnswer(
    +          sql("SELECT * FROM alter_add_ds where c3 = 3"),
    +          Seq(Row(2, 3, 1))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM alter_add_ds where c2 = 1"),
    +          Seq(Row(2, 3, 1))
    +        )
    +      }
    +    }
    +  }
    +
    +  test("alter datasource table add columns - text format not supported") {
    +    withTable("alter_add_ds_text") {
    +      sql(s"CREATE TABLE alter_add_ds_text (c1 int) USING text")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE alter_add_ds_text ADD COLUMNS (c2 int)")
    +      }.getMessage
    +      assert(e.contains("does not support ALTER ADD COLUMNS"))
    +    }
    +  }
    +
    +  test("alter table add columns -- not support temp view") {
    +    withTempView("tmp_v") {
    +      sql("create temporary view tmp_v as select 1 as c1, 2 as c2")
    --- End diff --
    
    Nit: make all the SQL keywords upper cases.


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

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


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74816/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74816 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74816/testReport)** for PR 16626 at commit [`1eb7cd3`](https://github.com/apache/spark/commit/1eb7cd3ccc6e8ba02510ef5d5cce0beb31ffd8f6).
     * 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r107063198
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -296,6 +311,47 @@ class SessionCatalog(
       }
     
       /**
    +   * Alter the schema of a table identified by the provided table identifier. The new schema
    +   * should still contain the existing bucket columns and partition columns used by the table. This
    +   * method will also update any Spark SQL-related parameters stored as Hive table properties (such
    +   * as the schema itself).
    +   *
    +   * @param identifier TableIdentifier
    +   * @param newSchema Updated schema to be used for the table (must contain existing partition and
    +   *                  bucket columns, and partition columns need to be at the end)
    +   */
    +  def alterTableSchema(
    +      identifier: TableIdentifier,
    +      newSchema: StructType): Unit = {
    +    val db = formatDatabaseName(identifier.database.getOrElse(getCurrentDatabase))
    +    val table = formatTableName(identifier.table)
    +    val tableIdentifier = TableIdentifier(table, Some(db))
    +    requireDbExists(db)
    +    requireTableExists(tableIdentifier)
    +    checkDuplication(newSchema)
    +
    +    val catalogTable = externalCatalog.getTable(db, table)
    +    val oldSchema = catalogTable.schema
    +
    +    // not supporting dropping columns yet
    +    val nonExistentColumnNames = oldSchema.map(_.name).filterNot(columnNameResolved(newSchema, _))
    +    if (nonExistentColumnNames.nonEmpty) {
    +      throw new AnalysisException(
    +        s"""
    +           |Some existing schema fields (${nonExistentColumnNames.mkString("[", ",", "]")}) are
    +           |not present in the new schema. We don't support dropping columns yet.
    +         """.stripMargin)
    +    }
    +
    +    // assuming the newSchema has all partition columns at the end as required
    +    externalCatalog.alterTableSchema(db, table, StructType(newSchema))
    +  }
    +
    +  def columnNameResolved(schema: StructType, colName: String): Boolean = {
    --- End diff --
    
    private?


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106465419
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---
    @@ -175,6 +178,87 @@ case class AlterTableRenameCommand(
     }
     
     /**
    + * A command that add columns to a table
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   ALTER TABLE table_identifier
    + *   ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
    + * }}}
    +*/
    +case class AlterTableAddColumnsCommand(
    +    table: TableIdentifier,
    +    columns: Seq[StructField]) extends RunnableCommand {
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    val catalog = sparkSession.sessionState.catalog
    +    val catalogTable = verifyAlterTableAddColumn(catalog, table)
    +
    +    try {
    +      sparkSession.catalog.uncacheTable(table.unquotedString)
    --- End diff --
    
    `table.quotedString`


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106097394
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
    @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, Resolver}
     import org.apache.spark.sql.catalyst.catalog._
     import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
     import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
    -import org.apache.spark.sql.execution.datasources.PartitioningUtils
    +import org.apache.spark.sql.execution.datasources.{DataSource, FileFormat, PartitioningUtils}
    --- End diff --
    
    Please revert them back.


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72341/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106097589
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---
    @@ -175,6 +178,80 @@ case class AlterTableRenameCommand(
     }
     
     /**
    + * A command that add columns to a table
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   ALTER TABLE table_identifier
    + *   ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
    + * }}}
    +*/
    +case class AlterTableAddColumnsCommand(
    +    table: TableIdentifier,
    +    columns: Seq[StructField]) extends RunnableCommand {
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    val catalog = sparkSession.sessionState.catalog
    +    val catalogTable = verifyAlterTableAddColumn(catalog, table)
    +
    +    // If an exception is thrown here we can just assume the table is uncached;
    +    // this can happen with Hive tables when the underlying catalog is in-memory.
    +    val wasCached = Try(sparkSession.catalog.isCached(table.unquotedString)).getOrElse(false)
    +    if (wasCached) {
    +      try {
    +        sparkSession.catalog.uncacheTable(table.unquotedString)
    +      } catch {
    +        case NonFatal(e) => log.warn(e.toString, e)
    +      }
    +    }
    +    // Invalidate the table last, otherwise uncaching the table would load the logical plan
    +    // back into the hive metastore cache
    +    catalog.refreshTable(table)
    +    val partitionFields = catalogTable.schema.takeRight(catalogTable.partitionColumnNames.length)
    +    val dataSchema = catalogTable.schema
    +      .take(catalogTable.schema.length - catalogTable.partitionColumnNames.length)
    +    catalog.alterTableSchema(table, newSchema =
    +      catalogTable.schema.copy(fields = (dataSchema ++ columns ++ partitionFields).toArray))
    +
    +    Seq.empty[Row]
    +  }
    +
    +  /**
    +   * ALTER TABLE ADD COLUMNS command does not support temporary view/table,
    +   * view, or datasource table with text, orc formats or external provider.
    +   * For datasource table, it currently only supports parquet, json, csv.
    +   */
    +  private def verifyAlterTableAddColumn(
    +      catalog: SessionCatalog,
    +      table: TableIdentifier): CatalogTable = {
    +    val catalogTable = catalog.getTempViewOrPermanentTableMetadata(table)
    +
    +    if (catalogTable.tableType == CatalogTableType.VIEW) {
    +      throw new AnalysisException(
    +        s"${table.toString} is a VIEW, which does not support ALTER ADD COLUMNS.")
    +    }
    +
    +    if (DDLUtils.isDatasourceTable(catalogTable)) {
    +      DataSource.lookupDataSource(catalogTable.provider.get).newInstance() match {
    +        // For datasource table, this command can only support the following File format.
    +        // TextFileFormat only default to one column "value"
    +        // OrcFileFormat can not handle difference between user-specified schema and
    +        // inferred schema yet. TODO, once this issue is resolved , we can add Orc back.
    +        // Hive type is already considered as hive serde table, so the logic will not
    +        // come in here.
    +        case _: JsonFileFormat | _: CSVFileFormat | _: ParquetFileFormat =>
    +        case s =>
    +          throw new AnalysisException(
    +            s"""${table} is a datasource table with type $s,
    +               |which does not support ALTER ADD COLUMNS.
    +            """.stripMargin)
    --- End diff --
    
    Nit:
    ```Scala
              throw new AnalysisException(
                s"$table is a datasource table with type $s, which does not support ALTER ADD 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 issue #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #72320 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72320/testReport)** for PR 16626 at commit [`26e0940`](https://github.com/apache/spark/commit/26e0940b7f5e619fee2117a0a04b31897eaa90d7).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r101958020
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---
    @@ -174,6 +177,79 @@ case class AlterTableRenameCommand(
     }
     
     /**
    + * A command that add columns to a table
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   ALTER TABLE table_identifier
    + *   ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
    + * }}}
    +*/
    +case class AlterTableAddColumnsCommand(
    +    table: TableIdentifier,
    +    columns: Seq[StructField]) extends RunnableCommand {
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    val catalog = sparkSession.sessionState.catalog
    +    val catalogTable = verifyAlterTableAddColumn(catalog, table)
    +
    +    // If an exception is thrown here we can just assume the table is uncached;
    +    // this can happen with Hive tables when the underlying catalog is in-memory.
    +    val wasCached = Try(sparkSession.catalog.isCached(table.unquotedString)).getOrElse(false)
    +    if (wasCached) {
    +      try {
    +        sparkSession.catalog.uncacheTable(table.unquotedString)
    +      } catch {
    +        case NonFatal(e) => log.warn(e.toString, e)
    +      }
    +    }
    +    // Invalidate the table last, otherwise uncaching the table would load the logical plan
    +    // back into the hive metastore cache
    +    catalog.refreshTable(table)
    +    val partitionFields = catalogTable.schema.takeRight(catalogTable.partitionColumnNames.length)
    +    val dataSchema = catalogTable.schema
    +      .take(catalogTable.schema.length - catalogTable.partitionColumnNames.length)
    +    catalog.alterTable(catalogTable.copy(schema =
    +      catalogTable.schema.copy(fields = (dataSchema ++ columns ++ partitionFields).toArray)))
    +
    +    Seq.empty[Row]
    +  }
    +
    +  /**
    +   * ALTER TABLE ADD COLUMNS command does not support temporary view/table,
    +   * view, or datasource table with text, orc formats or external provider.
    +   */
    +  private def verifyAlterTableAddColumn(
    +    catalog: SessionCatalog,
    +    table: TableIdentifier): CatalogTable = {
    +    val catalogTable = catalog.getTempViewOrPermanentTableMetadata(table)
    +
    +    if (catalogTable.tableType == CatalogTableType.VIEW) {
    +      throw new AnalysisException(
    +        s"${table.toString} is a VIEW, which does not support ALTER ADD COLUMNS.")
    +    }
    +
    +    if (DDLUtils.isDatasourceTable(catalogTable)) {
    +      DataSource.lookupDataSource(catalogTable.provider.get).newInstance() match {
    +        // For datasource table, this command can only support the following File format.
    +        // TextFileFormat only default to one column "value"
    +        // OrcFileFormat can not handle difference between user-specified schema and
    +        // inferred schema yet. TODO, once this issue is resolved , we can add Orc back.
    +        // Hive type is already considered as hive serde table, so the logic will not
    +        // come in here.
    +        case _: JsonFileFormat | _: CSVFileFormat | _: ParquetFileFormat =>
    +        case s =>
    +          throw new AnalysisException(
    +            s"""${table.toString} is a datasource table with type $s,
    --- End diff --
    
    `toString` is not needed?


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r101958217
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala ---
    @@ -416,4 +416,21 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext {
         val comments = planned.schema.fields.map(_.getComment().getOrElse("NO_COMMENT")).mkString(",")
         assert(comments === "SN,SA,NO_COMMENT")
       }
    +
    +  test("ALTER TABLE ADD COLUMNS does not support RelationProvider") {
    +    withTable("ds_relationProvider") {
    +      sql(
    +        """
    +          |CREATE TABLE ds_relationProvider
    +          |USING org.apache.spark.sql.sources.SimpleScanSource
    +          |OPTIONS (
    +          |  From '1',
    +          |  To '10'
    +          |)""".stripMargin)
    --- End diff --
    
    Syntax 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 pull request #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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/16626#discussion_r107049695
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala ---
    @@ -2178,4 +2178,138 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
           }
         }
       }
    +
    +  val supportedNativeFileFormatsForAlterTableAddColumns = Seq("parquet", "json", "csv")
    +
    +  supportedNativeFileFormatsForAlterTableAddColumns.foreach { provider =>
    +    test(s"alter datasource table add columns - $provider") {
    +      withTable("t1") {
    +        sql(s"CREATE TABLE t1 (c1 int) USING $provider")
    +        sql("INSERT INTO t1 VALUES (1)")
    +        sql("ALTER TABLE t1 ADD COLUMNS (c2 int)")
    +        checkAnswer(
    +          spark.table("t1"),
    +          Seq(Row(1, null))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c2 is null"),
    +          Seq(Row(1, null))
    +        )
    +
    +        sql("INSERT INTO t1 VALUES (3, 2)")
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c2 = 2"),
    +          Seq(Row(3, 2))
    +        )
    +      }
    +    }
    +  }
    +
    +  supportedNativeFileFormatsForAlterTableAddColumns.foreach { provider =>
    +    test(s"alter datasource table add columns - partitioned - $provider") {
    +      withTable("t1") {
    +        sql(s"CREATE TABLE t1 (c1 int, c2 int) USING $provider PARTITIONED BY (c2)")
    +        sql("INSERT INTO t1 PARTITION(c2 = 2) VALUES (1)")
    +        sql("ALTER TABLE t1 ADD COLUMNS (c3 int)")
    +        checkAnswer(
    +          spark.table("t1"),
    +          Seq(Row(1, null, 2))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c3 is null"),
    +          Seq(Row(1, null, 2))
    +        )
    +        sql("INSERT INTO t1 PARTITION(c2 =1) VALUES (2, 3)")
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c3 = 3"),
    +          Seq(Row(2, 3, 1))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c2 = 1"),
    +          Seq(Row(2, 3, 1))
    +        )
    +      }
    +    }
    +  }
    +
    +  test("alter datasource table add columns - text format not supported") {
    +    withTable("t1") {
    +      sql("CREATE TABLE t1 (c1 int) USING text")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE t1 ADD COLUMNS (c2 int)")
    +      }.getMessage
    +      assert(e.contains("ALTER ADD COLUMNS does not support datasource table with type"))
    +    }
    +  }
    +
    +  test("alter table add columns -- not support temp view") {
    +    withTempView("tmp_v") {
    +      sql("CREATE TEMPORARY VIEW tmp_v AS SELECT 1 AS c1, 2 AS c2")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE tmp_v ADD COLUMNS (c3 INT)")
    +      }
    +      assert(e.message.contains("ALTER ADD COLUMNS does not support views"))
    +    }
    +  }
    +
    +  test("alter table add columns -- not support view") {
    +    withView("v1") {
    +      sql("CREATE VIEW v1 AS SELECT 1 AS c1, 2 AS c2")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE v1 ADD COLUMNS (c3 INT)")
    +      }
    +      assert(e.message.contains("ALTER ADD COLUMNS does not support views"))
    +    }
    +  }
    +
    +  test("alter table add columns with existing column name") {
    +    withTable("t1") {
    +      sql("CREATE TABLE t1 (c1 int) USING PARQUET")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE t1 ADD COLUMNS (c1 string)")
    +      }.getMessage
    +      assert(e.contains("Found duplicate column(s)"))
    +    }
    +  }
    +
    +  test("alter table add columns to table referenced by a view") {
    +    withTable("t1") {
    +      withView("v1") {
    +        sql("CREATE TABLE t1 (c1 int, c2 int) USING PARQUET")
    +        sql("CREATE VIEW v1 AS SELECT * FROM t1")
    +        val originViewSchema = sql("SELECT * FROM v1").schema
    +        sql("ALTER TABLE t1 ADD COLUMNS (c3 int)")
    +        assert(sql("SELECT * FROM v1").schema == originViewSchema)
    +      }
    +    }
    +  }
    +
    +  Seq("true", "false").foreach { caseSensitive =>
    +    test(s"alter table add columns with existing column name - caseSensitive $caseSensitive") {
    +      withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) {
    +        withTable("t1") {
    +          sql("CREATE TABLE t1 (c1 int) USING PARQUET")
    +          if (caseSensitive == "false") {
    +            val e = intercept[AnalysisException] {
    +              sql("ALTER TABLE t1 ADD COLUMNS (C1 string)")
    +            }.getMessage
    +            assert(e.contains("Found duplicate column(s)"))
    +          } else {
    +            if (isUsingHiveMetastore) {
    +              // hive catalog will still complains that c1 is duplicate column name because hive
    +              // identifiers are case insensitive.
    --- End diff --
    
    actually we can fix this, as we store the schema in table properties.


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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/16626#discussion_r107057320
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -296,6 +311,51 @@ class SessionCatalog(
       }
     
       /**
    +   * Alter the schema of a table identified by the provided table identifier. The new schema
    +   * should still contain the existing bucket columns and partition columns used by the table. This
    +   * method will also update any Spark SQL-related parameters stored as Hive table properties (such
    +   * as the schema itself).
    +   *
    +   * @param identifier TableIdentifier
    +   * @param newSchema Updated schema to be used for the table (must contain existing partition and
    +   *                  bucket columns)
    +   */
    +  def alterTableSchema(
    +      identifier: TableIdentifier,
    +      newSchema: StructType): Unit = {
    +    val db = formatDatabaseName(identifier.database.getOrElse(getCurrentDatabase))
    +    val table = formatTableName(identifier.table)
    +    val tableIdentifier = TableIdentifier(table, Some(db))
    +    requireDbExists(db)
    +    requireTableExists(tableIdentifier)
    +    checkDuplication(newSchema)
    +
    +    val catalogTable = externalCatalog.getTable(db, table)
    +    val oldSchema = catalogTable.schema
    +
    +    // not supporting dropping columns yet
    +    val nonExistentColumnNames = oldSchema.map(_.name).filterNot(columnNameResolved(newSchema, _))
    +    if (nonExistentColumnNames.nonEmpty) {
    +      throw new AnalysisException(
    +        s"""
    +           |Some existing schema fields (${nonExistentColumnNames.mkString("[", ",", "]")}) are
    +           |not present in the new schema. We don't support dropping columns yet.
    +         """.stripMargin)
    +    }
    +
    +    // make sure partition columns are at the end
    +    val partitionSchema = catalogTable.partitionSchema
    --- End diff --
    
    yea


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r99678116
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
    @@ -814,4 +816,50 @@ object DDLUtils {
           }
         }
       }
    +
    +  /**
    +   * ALTER TABLE ADD COLUMNS command does not support temporary view/table,
    +   * view, or datasource table with text, orc formats or external provider.
    +   */
    +  def verifyAlterTableAddColumn(
    --- End diff --
    
    Since this checking is only used in `AlterTableAddColumnsCommand `, we do not need to move it 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 issue #16626: [SPARK-19261][SQL] Alter add columns for Hive tables

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

    https://github.com/apache/spark/pull/16626
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71728/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #73450 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73450/testReport)** for PR 16626 at commit [`4bd04fc`](https://github.com/apache/spark/commit/4bd04fc556fde3805f58e39a0e6ad50c2a1e8aec).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74943 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74943/testReport)** for PR 16626 at commit [`7d8437d`](https://github.com/apache/spark/commit/7d8437dbba76e1e88c9f230dc7ea895d343dceb8).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r107053921
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala ---
    @@ -2178,4 +2178,138 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
           }
         }
       }
    +
    +  val supportedNativeFileFormatsForAlterTableAddColumns = Seq("parquet", "json", "csv")
    +
    +  supportedNativeFileFormatsForAlterTableAddColumns.foreach { provider =>
    +    test(s"alter datasource table add columns - $provider") {
    +      withTable("t1") {
    +        sql(s"CREATE TABLE t1 (c1 int) USING $provider")
    +        sql("INSERT INTO t1 VALUES (1)")
    +        sql("ALTER TABLE t1 ADD COLUMNS (c2 int)")
    +        checkAnswer(
    +          spark.table("t1"),
    +          Seq(Row(1, null))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c2 is null"),
    +          Seq(Row(1, null))
    +        )
    +
    +        sql("INSERT INTO t1 VALUES (3, 2)")
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c2 = 2"),
    +          Seq(Row(3, 2))
    +        )
    +      }
    +    }
    +  }
    +
    +  supportedNativeFileFormatsForAlterTableAddColumns.foreach { provider =>
    +    test(s"alter datasource table add columns - partitioned - $provider") {
    +      withTable("t1") {
    +        sql(s"CREATE TABLE t1 (c1 int, c2 int) USING $provider PARTITIONED BY (c2)")
    +        sql("INSERT INTO t1 PARTITION(c2 = 2) VALUES (1)")
    +        sql("ALTER TABLE t1 ADD COLUMNS (c3 int)")
    +        checkAnswer(
    +          spark.table("t1"),
    +          Seq(Row(1, null, 2))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c3 is null"),
    +          Seq(Row(1, null, 2))
    +        )
    +        sql("INSERT INTO t1 PARTITION(c2 =1) VALUES (2, 3)")
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c3 = 3"),
    +          Seq(Row(2, 3, 1))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c2 = 1"),
    +          Seq(Row(2, 3, 1))
    +        )
    +      }
    +    }
    +  }
    +
    +  test("alter datasource table add columns - text format not supported") {
    +    withTable("t1") {
    +      sql("CREATE TABLE t1 (c1 int) USING text")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE t1 ADD COLUMNS (c2 int)")
    +      }.getMessage
    +      assert(e.contains("ALTER ADD COLUMNS does not support datasource table with type"))
    +    }
    +  }
    +
    +  test("alter table add columns -- not support temp view") {
    +    withTempView("tmp_v") {
    +      sql("CREATE TEMPORARY VIEW tmp_v AS SELECT 1 AS c1, 2 AS c2")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE tmp_v ADD COLUMNS (c3 INT)")
    +      }
    +      assert(e.message.contains("ALTER ADD COLUMNS does not support views"))
    +    }
    +  }
    +
    +  test("alter table add columns -- not support view") {
    +    withView("v1") {
    +      sql("CREATE VIEW v1 AS SELECT 1 AS c1, 2 AS c2")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE v1 ADD COLUMNS (c3 INT)")
    +      }
    +      assert(e.message.contains("ALTER ADD COLUMNS does not support views"))
    +    }
    +  }
    +
    +  test("alter table add columns with existing column name") {
    +    withTable("t1") {
    +      sql("CREATE TABLE t1 (c1 int) USING PARQUET")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE t1 ADD COLUMNS (c1 string)")
    +      }.getMessage
    +      assert(e.contains("Found duplicate column(s)"))
    +    }
    +  }
    +
    +  test("alter table add columns to table referenced by a view") {
    +    withTable("t1") {
    +      withView("v1") {
    +        sql("CREATE TABLE t1 (c1 int, c2 int) USING PARQUET")
    +        sql("CREATE VIEW v1 AS SELECT * FROM t1")
    +        val originViewSchema = sql("SELECT * FROM v1").schema
    +        sql("ALTER TABLE t1 ADD COLUMNS (c3 int)")
    +        assert(sql("SELECT * FROM v1").schema == originViewSchema)
    +      }
    +    }
    +  }
    +
    +  Seq("true", "false").foreach { caseSensitive =>
    +    test(s"alter table add columns with existing column name - caseSensitive $caseSensitive") {
    +      withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) {
    +        withTable("t1") {
    +          sql("CREATE TABLE t1 (c1 int) USING PARQUET")
    +          if (caseSensitive == "false") {
    +            val e = intercept[AnalysisException] {
    +              sql("ALTER TABLE t1 ADD COLUMNS (C1 string)")
    +            }.getMessage
    +            assert(e.contains("Found duplicate column(s)"))
    +          } else {
    +            if (isUsingHiveMetastore) {
    +              // hive catalog will still complains that c1 is duplicate column name because hive
    +              // identifiers are case insensitive.
    --- End diff --
    
    @cloud-fan I just tested the data source table, like `create table t1 (c1 int, C1 int) using parquet` with `spark.sql.caseSensitive = true`, spark sql does not complain.. it just bounce back the exception from hive, but logged as WARN message. And the table was created  successfully and I am able to insert and select. But if i create a hive serde table with `create table t2 (c1 int, C1 int) stored as parquet`, hive will complain and fail to create the table.  So for the data source case, should we fix anything regarding the WARN message? 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 issue #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #72320 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72320/testReport)** for PR 16626 at commit [`26e0940`](https://github.com/apache/spark/commit/26e0940b7f5e619fee2117a0a04b31897eaa90d7).
     * This patch **fails SparkR 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 pull request #16626: [SPARK-19261][SQL] Alter add columns for Hive tab...

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

    https://github.com/apache/spark/pull/16626#discussion_r97212553
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -736,6 +736,22 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder {
       }
     
       /**
    +   * Create a [[AlterTableAddColumnsCommand]] command.
    +   *
    +   * For example:
    +   * {{{
    +   *   ALTER TABLE table1
    +   *   ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
    +   * }}}
    +   */
    +  override def visitAddTableColumns(ctx: AddTableColumnsContext): LogicalPlan = withOrigin(ctx) {
    +    AlterTableAddColumnsCommand(
    +      visitTableIdentifier(ctx.tableIdentifier),
    +      Option(ctx.columns).map(visitColTypeList).getOrElse(Nil)
    --- End diff --
    
    columns are not optimal for this case.


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Thanks! Merging to master.


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106786357
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala ---
    @@ -2178,4 +2197,107 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
           }
         }
       }
    +
    +  Seq("parquet", "json", "csv").foreach { provider =>
    +    test(s"alter datasource table add columns - $provider") {
    +      withTable("t1") {
    +        sql(s"CREATE TABLE t1 (c1 int) USING $provider")
    +        sql("INSERT INTO t1 VALUES (1)")
    +        sql("ALTER TABLE t1 ADD COLUMNS (c2 int)")
    +        checkAnswer(
    +          sql("SELECT * FROM t1"),
    +          Seq(Row(1, null))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c2 is null"),
    +          Seq(Row(1, null))
    +        )
    +
    +        sql("INSERT INTO t1 VALUES (3, 2)")
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c2 = 2"),
    +          Seq(Row(3, 2))
    +        )
    +      }
    +    }
    +  }
    +
    +  Seq("parquet", "json", "csv").foreach { provider =>
    +    test(s"alter datasource table add columns - partitioned - $provider") {
    +      withTable("t1") {
    +        sql(s"CREATE TABLE t1 (c1 int, c2 int) USING $provider PARTITIONED BY (c2)")
    +        sql("INSERT INTO t1 PARTITION(c2 = 2) VALUES (1)")
    +        sql("ALTER TABLE t1 ADD COLUMNS (c3 int)")
    +        checkAnswer(
    +          sql("SELECT * FROM t1"),
    --- End diff --
    
    spark.table("t1")


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73450/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74774 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74774/testReport)** for PR 16626 at commit [`ae324d3`](https://github.com/apache/spark/commit/ae324d3103358a045cab81677ab1f28c13303b7c).
     * 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74952 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74952/testReport)** for PR 16626 at commit [`7d8437d`](https://github.com/apache/spark/commit/7d8437dbba76e1e88c9f230dc7ea895d343dceb8).
     * 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r101960398
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
    @@ -504,15 +504,15 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
        * Note: As of now, this doesn't support altering table schema, partition column names and bucket
        * specification. We will ignore them even if users do specify different values for these fields.
        */
    -  override def alterTable(tableDefinition: CatalogTable): Unit = withClient {
    -    assert(tableDefinition.identifier.database.isDefined)
    -    val db = tableDefinition.identifier.database.get
    -    requireTableExists(db, tableDefinition.identifier.table)
    -    verifyTableProperties(tableDefinition)
    +  override def alterTable(newTableDefinition: CatalogTable): Unit = withClient {
    +    assert(newTableDefinition.identifier.database.isDefined)
    +    val db = newTableDefinition.identifier.database.get
    +    requireTableExists(db, newTableDefinition.identifier.table)
    +    verifyTableProperties(newTableDefinition)
     
         // convert table statistics to properties so that we can persist them through hive api
    -    val withStatsProps = if (tableDefinition.stats.isDefined) {
    -      val stats = tableDefinition.stats.get
    +    val maybeWithStatsPropsTable: CatalogTable = if (newTableDefinition.stats.isDefined) {
    --- End diff --
    
    `: CatalogTable ` is not needed.


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r101958919
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---
    @@ -174,6 +177,79 @@ case class AlterTableRenameCommand(
     }
     
     /**
    + * A command that add columns to a table
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   ALTER TABLE table_identifier
    + *   ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
    + * }}}
    +*/
    +case class AlterTableAddColumnsCommand(
    +    table: TableIdentifier,
    +    columns: Seq[StructField]) extends RunnableCommand {
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    val catalog = sparkSession.sessionState.catalog
    +    val catalogTable = verifyAlterTableAddColumn(catalog, table)
    +
    +    // If an exception is thrown here we can just assume the table is uncached;
    +    // this can happen with Hive tables when the underlying catalog is in-memory.
    +    val wasCached = Try(sparkSession.catalog.isCached(table.unquotedString)).getOrElse(false)
    +    if (wasCached) {
    +      try {
    +        sparkSession.catalog.uncacheTable(table.unquotedString)
    +      } catch {
    +        case NonFatal(e) => log.warn(e.toString, e)
    +      }
    +    }
    +    // Invalidate the table last, otherwise uncaching the table would load the logical plan
    +    // back into the hive metastore cache
    +    catalog.refreshTable(table)
    +    val partitionFields = catalogTable.schema.takeRight(catalogTable.partitionColumnNames.length)
    +    val dataSchema = catalogTable.schema
    +      .take(catalogTable.schema.length - catalogTable.partitionColumnNames.length)
    +    catalog.alterTable(catalogTable.copy(schema =
    +      catalogTable.schema.copy(fields = (dataSchema ++ columns ++ partitionFields).toArray)))
    +
    +    Seq.empty[Row]
    +  }
    +
    +  /**
    +   * ALTER TABLE ADD COLUMNS command does not support temporary view/table,
    +   * view, or datasource table with text, orc formats or external provider.
    --- End diff --
    
    Also need to explain what are supported too.


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74281 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74281/testReport)** for PR 16626 at commit [`50564f3`](https://github.com/apache/spark/commit/50564f333de95e7c2e2232f6940eef144fb51e3e).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74774/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74810/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r99465918
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java ---
    @@ -107,7 +107,13 @@ public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptCont
           footer = readFooter(configuration, file, range(split.getStart(), split.getEnd()));
           MessageType fileSchema = footer.getFileMetaData().getSchema();
           FilterCompat.Filter filter = getFilter(configuration);
    -      blocks = filterRowGroups(filter, footer.getBlocks(), fileSchema);
    +      try {
    +        blocks = filterRowGroups(filter, footer.getBlocks(), fileSchema);
    +      } catch (IllegalArgumentException e) {
    +        // In the case where a particular parquet files does not contain
    --- End diff --
    
    We already upgrade Parquet, so we don't need this now.


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r107051682
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala ---
    @@ -1860,4 +1861,115 @@ class HiveDDLSuite
           }
         }
       }
    +
    +  hiveFormats.foreach { tableType =>
    +    test(s"alter hive serde table add columns -- partitioned - $tableType") {
    +      withTable("tab") {
    +        sql(
    +          s"""
    +             |CREATE TABLE tab (c1 int, c2 int)
    +             |PARTITIONED BY (c3 int) STORED AS $tableType
    +          """.stripMargin)
    +
    +        sql("INSERT INTO tab PARTITION (c3=1) VALUES (1, 2)")
    +        sql("ALTER TABLE tab ADD COLUMNS (c4 int)")
    +
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c3 = 1"),
    +          Seq(Row(1, 2, null, 1))
    +        )
    +        assert(spark.table("tab").schema
    +          .contains(StructField("c4", IntegerType)))
    +        sql("INSERT INTO tab PARTITION (c3=2) VALUES (2, 3, 4)")
    +        checkAnswer(
    +          spark.table("tab"),
    +          Seq(Row(1, 2, null, 1), Row(2, 3, 4, 2))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c3 = 2 AND c4 IS NOT NULL"),
    +          Seq(Row(2, 3, 4, 2))
    +        )
    +
    +        sql("ALTER TABLE tab ADD COLUMNS (c5 char(10))")
    +        assert(spark.table("tab").schema.find(_.name == "c5")
    +          .get.metadata.getString("HIVE_TYPE_STRING") == "char(10)")
    +      }
    +    }
    +  }
    +
    +  hiveFormats.foreach { tableType =>
    +    test(s"alter hive serde table add columns -- with predicate - $tableType ") {
    +      withTable("tab") {
    +        sql(s"CREATE TABLE tab (c1 int, c2 int) STORED AS $tableType")
    +        sql("INSERT INTO tab VALUES (1, 2)")
    +        sql("ALTER TABLE tab ADD COLUMNS (c4 int)")
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c4 IS NULL"),
    +          Seq(Row(1, 2, null))
    +        )
    +        assert(spark.table("tab").schema
    +          .contains(StructField("c4", IntegerType)))
    +        sql("INSERT INTO tab VALUES (2, 3, 4)")
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c4 = 4 "),
    +          Seq(Row(2, 3, 4))
    +        )
    +        checkAnswer(
    +          spark.table("tab"),
    +          Seq(Row(1, 2, null), Row(2, 3, 4))
    +        )
    +      }
    +    }
    +  }
    +
    +  Seq("orc", "ORC", "org.apache.spark.sql.hive.orc",
    --- End diff --
    
    OK. will do. 


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r99680917
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---
    @@ -168,6 +168,43 @@ case class AlterTableRenameCommand(
     }
     
     /**
    + * A command that add columns to a table
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   ALTER TABLE table_identifier
    + *   ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
    + * }}}
    +*/
    +case class AlterTableAddColumnsCommand(
    +    table: TableIdentifier,
    +    columns: Seq[StructField]) extends RunnableCommand {
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    val catalog = sparkSession.sessionState.catalog
    +    val catalogTable = DDLUtils.verifyAlterTableAddColumn(catalog, table)
    +
    +    // If an exception is thrown here we can just assume the table is uncached;
    +    // this can happen with Hive tables when the underlying catalog is in-memory.
    +    val wasCached = Try(sparkSession.catalog.isCached(table.unquotedString)).getOrElse(false)
    --- End diff --
    
    The current way is right. The implementation should not rely on the internal behavior of another function.


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74680/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive tab...

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

    https://github.com/apache/spark/pull/16626#discussion_r97213702
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
    @@ -584,14 +593,18 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
           // Sets the `schema`, `partitionColumnNames` and `bucketSpec` from the old table definition,
           // to retain the spark specific format if it is. Also add old data source properties to table
           // properties, to retain the data source table format.
    -      val oldDataSourceProps = oldTableDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX))
    --- End diff --
    
    I think the valuable name needs to change since now the hive table and datasource table both populate the table properties with the schema. Both cases will go through this path. I temporarily block the datasource table ALTER ADD columns because I am not confident yet if I have holes. But according to @gatorsmile , it may be safe to support datasource table too. So I am actually adding more test cases to confirm. I may remove the condition in 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 pull request #16626: [SPARK-19261][SQL] Alter add columns for Hive tab...

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

    https://github.com/apache/spark/pull/16626#discussion_r97195247
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---
    @@ -168,6 +168,43 @@ case class AlterTableRenameCommand(
     }
     
     /**
    + * A command that add columns to a table
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   ALTER TABLE table_identifier
    + *   ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
    + * }}}
    +*/
    +case class AlterTableAddColumnsCommand(
    +    table: TableIdentifier,
    +    columns: Seq[StructField]) extends RunnableCommand {
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    val catalog = sparkSession.sessionState.catalog
    +    val catalogTable = DDLUtils.verifyAlterTableAddColumn(catalog, table)
    +
    +    // If an exception is thrown here we can just assume the table is uncached;
    +    // this can happen with Hive tables when the underlying catalog is in-memory.
    +    val wasCached = Try(sparkSession.catalog.isCached(table.unquotedString)).getOrElse(false)
    --- End diff --
    
    Do we need the following to check cache status? I think `uncacheTable` is no-op if the table is not cached.


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r99660453
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
    @@ -814,4 +816,50 @@ object DDLUtils {
           }
         }
       }
    +
    +  /**
    +   * ALTER TABLE ADD COLUMNS command does not support temporary view/table,
    +   * view, or datasource table with text, orc formats or external provider.
    +   */
    +  def verifyAlterTableAddColumn(
    --- End diff --
    
    yes. you are right.  I will change it to private.


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    @gatorsmile Thanks for your comments! I will update the code. 


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74791/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #72341 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72341/testReport)** for PR 16626 at commit [`e2c53a2`](https://github.com/apache/spark/commit/e2c53a2ac61ec8fc65f1f26053c190d72d2dad31).
     * This patch **fails SparkR 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 pull request #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r99681895
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
    @@ -814,4 +816,50 @@ object DDLUtils {
           }
         }
       }
    +
    +  /**
    +   * ALTER TABLE ADD COLUMNS command does not support temporary view/table,
    +   * view, or datasource table with text, orc formats or external provider.
    +   */
    +  def verifyAlterTableAddColumn(
    +      catalog: SessionCatalog,
    +      table: TableIdentifier): CatalogTable = {
    +    if (catalog.isTemporaryTable(table)) {
    +      throw new AnalysisException(
    +        s"${table.toString} is a temporary VIEW, which does not support ALTER ADD COLUMNS.")
    +    }
    +
    +    val catalogTable = catalog.getTableMetadata(table)
    +    if (catalogTable.tableType == CatalogTableType.VIEW) {
    +      throw new AnalysisException(
    +        s"${table.toString} is a VIEW, which does not support ALTER ADD COLUMNS.")
    +    }
    +
    +    if (isDatasourceTable(catalogTable)) {
    +      catalogTable.provider.get match {
    +        case provider if provider.toLowerCase == "text" =>
    +          // TextFileFormat can not support adding column either because text datasource table
    +          // is resolved as a single-column table only.
    +          throw new AnalysisException(
    +            s"""${table.toString} is a text format datasource table,
    +               |which does not support ALTER ADD COLUMNS.""".stripMargin)
    +        case provider if provider.toLowerCase == "orc"
    +          || provider.startsWith("org.apache.spark.sql.hive.orc") =>
    +          // TODO Current native orc reader can not handle the difference between
    +          // user-specified schema and inferred schema from ORC data file yet.
    +          throw new AnalysisException(
    +            s"""${table.toString} is an ORC datasource table,
    +               |which does not support ALTER ADD COLUMNS.""".stripMargin)
    +        case provider
    +          if (!DataSource.lookupDataSource(provider).newInstance().isInstanceOf[FileFormat]) =>
    --- End diff --
    
    OK. I will use the white list of allowed FileFormat implementations. 


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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/16626#discussion_r106621151
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -450,6 +451,26 @@ abstract class SessionCatalogSuite extends PlanTest {
         }
       }
     
    +  test("alter table add columns") {
    +    withBasicCatalog { sessionCatalog =>
    +      sessionCatalog.createTable(newTable("t1", "default"), ignoreIfExists = false)
    +      val oldTab = sessionCatalog.externalCatalog.getTable("default", "t1")
    +      sessionCatalog.alterTableSchema(TableIdentifier("t1", Some("default")),
    +        oldTab.schema.add("c3", IntegerType))
    +      val newTab = sessionCatalog.externalCatalog.getTable("default", "t1")
    +      if (sessionCatalog.externalCatalog.isInstanceOf[InMemoryCatalog]) {
    +        assert(newTab.schema.toString == oldTab.schema.add("c3", IntegerType).toString)
    --- End diff --
    
    why compare the schema with `toString`?


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74634/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #72341 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72341/testReport)** for PR 16626 at commit [`e2c53a2`](https://github.com/apache/spark/commit/e2c53a2ac61ec8fc65f1f26053c190d72d2dad31).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106341966
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala ---
    @@ -1860,4 +1860,72 @@ class HiveDDLSuite
           }
         }
       }
    +
    +  Seq("PARQUET", "ORC", "TEXTFILE", "SEQUENCEFILE", "RCFILE", "AVRO").foreach { tableType =>
    --- End diff --
    
    If the list is complete, we can create a variable and reuse it in the future test cases in `HiveCatalogedDDLSuite `. Let us create it now?


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106786173
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala ---
    @@ -1860,4 +1861,119 @@ class HiveDDLSuite
           }
         }
       }
    +
    +  hiveFormats.foreach { tableType =>
    +    test(s"alter hive serde table add columns -- partitioned - $tableType") {
    +      withTable("tab") {
    +        sql(
    +          s"""
    +             |CREATE TABLE tab (c1 int, c2 int)
    +             |PARTITIONED BY (c3 int) STORED AS $tableType
    +          """.stripMargin)
    +
    +        sql("INSERT INTO tab PARTITION (c3=1) VALUES (1, 2)")
    +        sql("ALTER TABLE tab ADD COLUMNS (c4 int)")
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c3 = 1"),
    +          Seq(Row(1, 2, null, 1))
    +        )
    +        assert(sql("SELECT * FROM tab").schema
    +          .contains(StructField("c4", IntegerType)))
    +        sql("INSERT INTO tab PARTITION (c3=2) VALUES (2, 3, 4)")
    +        checkAnswer(
    +          sql("SELECT * FROM tab"),
    +          Seq(Row(1, 2, null, 1), Row(2, 3, 4, 2))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c3 = 2 AND c4 IS NOT NULL"),
    +          Seq(Row(2, 3, 4, 2))
    +        )
    +      }
    +    }
    +  }
    +
    +  hiveFormats.foreach { tableType =>
    +    test(s"alter hive serde table add columns -- with predicate - $tableType ") {
    +      withTable("tab") {
    +        sql(s"CREATE TABLE tab (c1 int, c2 int) STORED AS $tableType")
    +        sql("INSERT INTO tab VALUES (1, 2)")
    +        sql("ALTER TABLE tab ADD COLUMNS (c4 int)")
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c4 IS NULL"),
    +          Seq(Row(1, 2, null))
    +        )
    +        assert(sql("SELECT * FROM tab").schema
    +          .contains(StructField("c4", IntegerType)))
    +        sql("INSERT INTO tab VALUES (2, 3, 4)")
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c4 = 4 "),
    +          Seq(Row(2, 3, 4))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM tab"),
    +          Seq(Row(1, 2, null), Row(2, 3, 4))
    +        )
    +      }
    +    }
    +  }
    +
    +  Seq("orc", "ORC", "org.apache.spark.sql.hive.orc",
    +    "org.apache.spark.sql.hive.orc.DefaultSource").foreach { source =>
    +    test(s"alter datasource table add columns - $source format not supported") {
    +      withTable("tab") {
    +        sql(s"CREATE TABLE tab (c1 int) USING $source")
    +        val e = intercept[AnalysisException] {
    +          sql("ALTER TABLE tab ADD COLUMNS (c2 int)")
    +        }.getMessage
    +        assert(
    +          e.contains(s"ALTER ADD COLUMNS does not support datasource table with type"))
    +      }
    +    }
    +  }
    +
    +  Seq("true", "false").foreach { caseSensitive =>
    +    test(s"alter add columns with existing partition column name - caseSensitive $caseSensitive") {
    +      withSQLConf(("spark.sql.caseSensitive", caseSensitive)) {
    +        withTable("tab") {
    +          sql("CREATE TABLE tab (c1 int) PARTITIONED BY (c2 int) STORED AS PARQUET")
    +          if (caseSensitive == "false") {
    +            val e = intercept[AnalysisException] {
    +              sql("ALTER TABLE tab ADD COLUMNS (C2 string)")
    +            }.getMessage
    +            assert(e.contains("Found duplicate column(s)"))
    +          } else {
    +            // hive catalog will still complains that c1 is duplicate column name because hive
    +            // identifiers are case insensitive.
    +            val e = intercept[AnalysisException] {
    +              sql("ALTER TABLE tab ADD COLUMNS (C2 string)")
    +            }.getMessage
    +            assert(e.contains("HiveException"))
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  Seq("true", "false").foreach { caseSensitive =>
    +    test(s"alter add columns with existing column name - caseSensitive $caseSensitive") {
    +      withSQLConf(("spark.sql.caseSensitive", caseSensitive)) {
    +        withTable("t1") {
    +          sql("CREATE TABLE t1 (c1 int) USING PARQUET")
    +          if (caseSensitive == "false") {
    +            val e = intercept[AnalysisException] {
    +              sql("ALTER TABLE t1 ADD COLUMNS (C1 string)")
    +            }.getMessage
    +            assert(e.contains("Found duplicate column(s)"))
    +          } else {
    +            // hive catalog will still complains that c1 is duplicate column name because hive
    +            // identifiers are case insensitive.
    +            val e = intercept[AnalysisException] {
    +              sql("ALTER TABLE t1 ADD COLUMNS (C1 string)")
    +            }.getMessage
    +            assert(e.contains("HiveException"))
    +          }
    +        }
    +      }
    +    }
    +  }
    --- End diff --
    
    You still can combine it with the one in `InMemoryCatalogedDDLSuite` by using `isUsingHiveMetastore`


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106098143
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala ---
    @@ -970,4 +969,22 @@ class JDBCSuite extends SparkFunSuite
           assert(sql("select * from people_view").count() == 3)
         }
       }
    +
    +  test("ALTER TABLE ADD COLUMNS") {
    +    withTable("ds_jdbc") {
    +      sql(
    +        s"""
    +          |CREATE TABLE IF NOT EXISTS ds_jdbc
    +          |USING org.apache.spark.sql.jdbc
    +          |OPTIONS (url '$url', dbtable 'TEST.PEOPLE', user 'testUser', password 'testPass')
    +         """.stripMargin.replaceAll("\n", " "))
    --- End diff --
    
    Nit: remove `.replaceAll("\n", " ")`


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r105289122
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
    @@ -504,94 +504,57 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
        * Note: As of now, this doesn't support altering table schema, partition column names and bucket
        * specification. We will ignore them even if users do specify different values for these fields.
        */
    -  override def alterTable(tableDefinition: CatalogTable): Unit = withClient {
    -    assert(tableDefinition.identifier.database.isDefined)
    -    val db = tableDefinition.identifier.database.get
    -    requireTableExists(db, tableDefinition.identifier.table)
    -    verifyTableProperties(tableDefinition)
    +  override def alterTable(newTableDefinition: CatalogTable): Unit = withClient {
    --- End diff --
    
    @cloud-fan I see this new API is added today.. I will modify my code. 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106792918
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -450,6 +451,21 @@ abstract class SessionCatalogSuite extends PlanTest {
         }
       }
     
    +  test("alter table add columns") {
    --- End diff --
    
    Also add a negative test case for dropping columns, although we do not support it now.


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive tab...

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

    https://github.com/apache/spark/pull/16626#discussion_r97192776
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java ---
    @@ -107,7 +107,13 @@ public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptCont
           footer = readFooter(configuration, file, range(split.getStart(), split.getEnd()));
           MessageType fileSchema = footer.getFileMetaData().getSchema();
           FilterCompat.Filter filter = getFilter(configuration);
    -      blocks = filterRowGroups(filter, footer.getBlocks(), fileSchema);
    +      try {
    +        blocks = filterRowGroups(filter, footer.getBlocks(), fileSchema);
    +      } catch (IllegalArgumentException e) {
    +        // In the case where a particular parquet files does not contain
    --- End diff --
    
    Can we add a TODO? I think the newer Parquet can handle this issue. Once we upgrade Parquet version, we don't need this.


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r99681470
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
    @@ -814,4 +816,50 @@ object DDLUtils {
           }
         }
       }
    +
    +  /**
    +   * ALTER TABLE ADD COLUMNS command does not support temporary view/table,
    +   * view, or datasource table with text, orc formats or external provider.
    +   */
    +  def verifyAlterTableAddColumn(
    +      catalog: SessionCatalog,
    +      table: TableIdentifier): CatalogTable = {
    +    if (catalog.isTemporaryTable(table)) {
    +      throw new AnalysisException(
    +        s"${table.toString} is a temporary VIEW, which does not support ALTER ADD COLUMNS.")
    +    }
    +
    +    val catalogTable = catalog.getTableMetadata(table)
    +    if (catalogTable.tableType == CatalogTableType.VIEW) {
    +      throw new AnalysisException(
    +        s"${table.toString} is a VIEW, which does not support ALTER ADD COLUMNS.")
    +    }
    +
    +    if (isDatasourceTable(catalogTable)) {
    +      catalogTable.provider.get match {
    +        case provider if provider.toLowerCase == "text" =>
    +          // TextFileFormat can not support adding column either because text datasource table
    +          // is resolved as a single-column table only.
    +          throw new AnalysisException(
    +            s"""${table.toString} is a text format datasource table,
    +               |which does not support ALTER ADD COLUMNS.""".stripMargin)
    +        case provider if provider.toLowerCase == "orc"
    +          || provider.startsWith("org.apache.spark.sql.hive.orc") =>
    --- End diff --
    
    I will double check with this case.. If `orc` is the only representation in CatalogTable.provider, I will reduce the logic 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive tab...

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

    https://github.com/apache/spark/pull/16626#discussion_r97213829
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---
    @@ -168,6 +168,43 @@ case class AlterTableRenameCommand(
     }
     
     /**
    + * A command that add columns to a table
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   ALTER TABLE table_identifier
    + *   ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
    + * }}}
    +*/
    +case class AlterTableAddColumnsCommand(
    +    table: TableIdentifier,
    +    columns: Seq[StructField]) extends RunnableCommand {
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    val catalog = sparkSession.sessionState.catalog
    +    val catalogTable = DDLUtils.verifyAlterTableAddColumn(catalog, table)
    +
    +    // If an exception is thrown here we can just assume the table is uncached;
    +    // this can happen with Hive tables when the underlying catalog is in-memory.
    +    val wasCached = Try(sparkSession.catalog.isCached(table.unquotedString)).getOrElse(false)
    --- End diff --
    
    `AlterTableRenameCommand` has similar way to do the uncaching. I thought there might be a reason it exists there. So I did the same. But looking  at the code, it seems you are right. 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106790141
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala ---
    @@ -1860,4 +1861,119 @@ class HiveDDLSuite
           }
         }
       }
    +
    +  hiveFormats.foreach { tableType =>
    +    test(s"alter hive serde table add columns -- partitioned - $tableType") {
    +      withTable("tab") {
    +        sql(
    +          s"""
    +             |CREATE TABLE tab (c1 int, c2 int)
    +             |PARTITIONED BY (c3 int) STORED AS $tableType
    +          """.stripMargin)
    +
    +        sql("INSERT INTO tab PARTITION (c3=1) VALUES (1, 2)")
    +        sql("ALTER TABLE tab ADD COLUMNS (c4 int)")
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c3 = 1"),
    +          Seq(Row(1, 2, null, 1))
    +        )
    +        assert(sql("SELECT * FROM tab").schema
    --- End diff --
    
    maybe `spark.table("tab")` is shorter? because to use getTableMetaData, i need to use `spark.sessionState.catalog.getTableMetadata`


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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/16626#discussion_r107050121
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala ---
    @@ -1860,4 +1861,115 @@ class HiveDDLSuite
           }
         }
       }
    +
    +  hiveFormats.foreach { tableType =>
    +    test(s"alter hive serde table add columns -- partitioned - $tableType") {
    +      withTable("tab") {
    +        sql(
    +          s"""
    +             |CREATE TABLE tab (c1 int, c2 int)
    +             |PARTITIONED BY (c3 int) STORED AS $tableType
    +          """.stripMargin)
    +
    +        sql("INSERT INTO tab PARTITION (c3=1) VALUES (1, 2)")
    +        sql("ALTER TABLE tab ADD COLUMNS (c4 int)")
    +
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c3 = 1"),
    +          Seq(Row(1, 2, null, 1))
    +        )
    +        assert(spark.table("tab").schema
    +          .contains(StructField("c4", IntegerType)))
    +        sql("INSERT INTO tab PARTITION (c3=2) VALUES (2, 3, 4)")
    +        checkAnswer(
    +          spark.table("tab"),
    +          Seq(Row(1, 2, null, 1), Row(2, 3, 4, 2))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c3 = 2 AND c4 IS NOT NULL"),
    +          Seq(Row(2, 3, 4, 2))
    +        )
    +
    +        sql("ALTER TABLE tab ADD COLUMNS (c5 char(10))")
    +        assert(spark.table("tab").schema.find(_.name == "c5")
    +          .get.metadata.getString("HIVE_TYPE_STRING") == "char(10)")
    +      }
    +    }
    +  }
    +
    +  hiveFormats.foreach { tableType =>
    +    test(s"alter hive serde table add columns -- with predicate - $tableType ") {
    +      withTable("tab") {
    +        sql(s"CREATE TABLE tab (c1 int, c2 int) STORED AS $tableType")
    +        sql("INSERT INTO tab VALUES (1, 2)")
    +        sql("ALTER TABLE tab ADD COLUMNS (c4 int)")
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c4 IS NULL"),
    +          Seq(Row(1, 2, null))
    +        )
    +        assert(spark.table("tab").schema
    +          .contains(StructField("c4", IntegerType)))
    +        sql("INSERT INTO tab VALUES (2, 3, 4)")
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c4 = 4 "),
    +          Seq(Row(2, 3, 4))
    +        )
    +        checkAnswer(
    +          spark.table("tab"),
    +          Seq(Row(1, 2, null), Row(2, 3, 4))
    +        )
    +      }
    +    }
    +  }
    +
    +  Seq("orc", "ORC", "org.apache.spark.sql.hive.orc",
    --- End diff --
    
    let's remove 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Could you add a scenario when users add a column name that already exists in the table schema?


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74283 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74283/testReport)** for PR 16626 at commit [`0bb55a7`](https://github.com/apache/spark/commit/0bb55a7c632c1156ac1a5b88c442c6e8f3c97192).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74035 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74035/testReport)** for PR 16626 at commit [`ec4a044`](https://github.com/apache/spark/commit/ec4a044ba7b009483b7f728da49a24b6aa942284).
     * 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r101960044
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
    @@ -563,35 +574,47 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
             //       want to alter the table location to a file path, we will fail. This should be fixed
             //       in the future.
     
    -        val newLocation = tableDefinition.storage.locationUri
    -        val storageWithPathOption = tableDefinition.storage.copy(
    -          properties = tableDefinition.storage.properties ++ newLocation.map("path" -> _))
    +        val newLocation = newTableDefinition.storage.locationUri
    +        val storageWithPathOption = newTableDefinition.storage.copy(
    +          properties = newTableDefinition.storage.properties ++ newLocation.map("path" -> _))
     
    -        val oldLocation = getLocationFromStorageProps(oldTableDef)
    +        val oldLocation = getLocationFromStorageProps(oldRawTableDef)
             if (oldLocation == newLocation) {
    -          storageWithPathOption.copy(locationUri = oldTableDef.storage.locationUri)
    +          storageWithPathOption.copy(locationUri = oldRawTableDef.storage.locationUri)
             } else {
               storageWithPathOption
             }
           }
     
    -      val partitionProviderProp = if (tableDefinition.tracksPartitionsInCatalog) {
    +      val partitionProviderProp = if (newTableDefinition.tracksPartitionsInCatalog) {
             TABLE_PARTITION_PROVIDER -> TABLE_PARTITION_PROVIDER_CATALOG
           } else {
             TABLE_PARTITION_PROVIDER -> TABLE_PARTITION_PROVIDER_FILESYSTEM
           }
     
    -      // Sets the `schema`, `partitionColumnNames` and `bucketSpec` from the old table definition,
    +      // Sets the `partitionColumnNames` and `bucketSpec` from the old table definition,
           // to retain the spark specific format if it is. Also add old data source properties to table
           // properties, to retain the data source table format.
    -      val oldDataSourceProps = oldTableDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX))
    -      val newTableProps = oldDataSourceProps ++ withStatsProps.properties + partitionProviderProp
    -      val newDef = withStatsProps.copy(
    +      val dataSourceProps = if (schemaChange) {
    +        val props =
    +          tableMetaToTableProps(newTableDefinition).filter(_._1.startsWith(DATASOURCE_PREFIX))
    +        if (newTableDefinition.provider.isDefined
    +          && newTableDefinition.provider.get.toLowerCase != DDLUtils.HIVE_PROVIDER) {
    --- End diff --
    
    `&&` should be moved up to the line # 601.


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74582 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74582/testReport)** for PR 16626 at commit [`c83e852`](https://github.com/apache/spark/commit/c83e8526c3fc864ed80a15846737a912a7744434).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive tab...

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

    https://github.com/apache/spark/pull/16626#discussion_r97174479
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -736,6 +736,21 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder {
       }
     
       /**
    +   * Create a [[AlterTableAddColumnsCommand]] command.
    +   *
    +   * For example:
    +   * {{{
    +   *   ALTER TABLE table1 ADD COLUMNS (c1 int);
    --- End diff --
    
    Please check the Hive syntax. At least, we can support the column comment. 


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72540/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive tab...

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

    https://github.com/apache/spark/pull/16626#discussion_r97191366
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
    @@ -814,4 +814,28 @@ object DDLUtils {
           }
         }
       }
    +
    +  /**
    +   * ALTER TABLE ADD COLUMNS command does not support temporary view/table,
    +   * view, or datasource table yet.
    +   */
    +  def verifyAlterTableAddColumn(
    +      catalog: SessionCatalog,
    +      table: TableIdentifier): CatalogTable = {
    +    if (catalog.isTemporaryTable(table)) {
    +      throw new AnalysisException(
    +        s"${table.toString} is a temporary VIEW, which does not support ALTER ADD COLUMNS.")
    +    }
    +
    +    val catalogTable = catalog.getTableMetadata(table)
    +    if (catalogTable.tableType == CatalogTableType.VIEW) {
    +      throw new AnalysisException(
    +        s"${table.toString} is a VIEW, which does not support ALTER ADD COLUMNS.")
    +    }
    +    if (isDatasourceTable(catalogTable)) {
    --- End diff --
    
    I am thinking that there are different ways to create a datasource table, such as df.write.saveAsTable, or create with "create table " DDL with/without schema. Plus JDBC datasource table maybe not be supported.. I just want to spend more time on trying different scenarios to see if there is any hole before claiming supporting it. I will submit another PR once I am sure it is handled correctly. 


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r101961354
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
    @@ -563,35 +574,47 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
             //       want to alter the table location to a file path, we will fail. This should be fixed
             //       in the future.
     
    -        val newLocation = tableDefinition.storage.locationUri
    -        val storageWithPathOption = tableDefinition.storage.copy(
    -          properties = tableDefinition.storage.properties ++ newLocation.map("path" -> _))
    +        val newLocation = newTableDefinition.storage.locationUri
    +        val storageWithPathOption = newTableDefinition.storage.copy(
    +          properties = newTableDefinition.storage.properties ++ newLocation.map("path" -> _))
     
    -        val oldLocation = getLocationFromStorageProps(oldTableDef)
    +        val oldLocation = getLocationFromStorageProps(oldRawTableDef)
             if (oldLocation == newLocation) {
    -          storageWithPathOption.copy(locationUri = oldTableDef.storage.locationUri)
    +          storageWithPathOption.copy(locationUri = oldRawTableDef.storage.locationUri)
             } else {
               storageWithPathOption
             }
           }
     
    -      val partitionProviderProp = if (tableDefinition.tracksPartitionsInCatalog) {
    +      val partitionProviderProp = if (newTableDefinition.tracksPartitionsInCatalog) {
             TABLE_PARTITION_PROVIDER -> TABLE_PARTITION_PROVIDER_CATALOG
           } else {
             TABLE_PARTITION_PROVIDER -> TABLE_PARTITION_PROVIDER_FILESYSTEM
           }
     
    -      // Sets the `schema`, `partitionColumnNames` and `bucketSpec` from the old table definition,
    +      // Sets the `partitionColumnNames` and `bucketSpec` from the old table definition,
           // to retain the spark specific format if it is. Also add old data source properties to table
           // properties, to retain the data source table format.
    -      val oldDataSourceProps = oldTableDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX))
    -      val newTableProps = oldDataSourceProps ++ withStatsProps.properties + partitionProviderProp
    -      val newDef = withStatsProps.copy(
    +      val dataSourceProps = if (schemaChange) {
    --- End diff --
    
    Could we move the whole logics when we find the table has a schema change?


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74283 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74283/testReport)** for PR 16626 at commit [`0bb55a7`](https://github.com/apache/spark/commit/0bb55a7c632c1156ac1a5b88c442c6e8f3c97192).
     * 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106785853
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -296,6 +297,64 @@ class SessionCatalog(
       }
     
       /**
    +   * Alter the schema of a table identified by the provided table identifier. The new schema
    +   * should still contain the existing bucket columns and partition columns used by the table. This
    +   * method will also update any Spark SQL-related parameters stored as Hive table properties (such
    +   * as the schema itself).
    +   *
    +   * @param identifier TableIdentifier
    +   * @param newSchema Updated schema to be used for the table (must contain existing partition and
    +   *                  bucket columns)
    +   */
    +  def alterTableSchema(
    +      identifier: TableIdentifier,
    +      newSchema: StructType): Unit = {
    +    val db = formatDatabaseName(identifier.database.getOrElse(getCurrentDatabase))
    +    val table = formatTableName(identifier.table)
    +    val tableIdentifier = TableIdentifier(table, Some(db))
    +    requireDbExists(db)
    +    requireTableExists(tableIdentifier)
    +    val catalogTable = externalCatalog.getTable(db, table)
    +    val oldSchema = catalogTable.schema
    +
    +    // no supporting dropping columns yet
    +    if (!oldSchema.forall(f => columnNameResolved(newSchema, f.name ))) {
    +      throw new AnalysisException(
    +        s"""
    +          |Some existing schema fields are not present in the new schema.
    +          |We don't support dropping columns yet.
    +         """.stripMargin)
    +    }
    +
    +    checkDuplication(newSchema)
    +    // make sure partition columns are at the end
    +    val partitionSchema = catalogTable.partitionSchema
    +    val reorderedSchema = newSchema
    +      .filterNot(f => columnNameResolved(partitionSchema, f.name)) ++ partitionSchema
    +
    +    externalCatalog.alterTableSchema(
    +      db, table, oldSchema.copy(fields = reorderedSchema.toArray))
    +  }
    +
    +  private def columnNameResolved(schema: StructType, colName: String): Boolean = {
    +    schema.fields.map(_.name).find(conf.resolver(_, colName)).isDefined
    --- End diff --
    
    ```Scala
    schema.fields.map(_.name).exists(conf.resolver(_, colName))
    ```



---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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/16626#discussion_r105286751
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
    @@ -504,94 +504,57 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
        * Note: As of now, this doesn't support altering table schema, partition column names and bucket
        * specification. We will ignore them even if users do specify different values for these fields.
        */
    -  override def alterTable(tableDefinition: CatalogTable): Unit = withClient {
    -    assert(tableDefinition.identifier.database.isDefined)
    -    val db = tableDefinition.identifier.database.get
    -    requireTableExists(db, tableDefinition.identifier.table)
    -    verifyTableProperties(tableDefinition)
    +  override def alterTable(newTableDefinition: CatalogTable): Unit = withClient {
    --- End diff --
    
    now we have a `alterSchema` method in `ExternalCatalog`, we can use that instead.


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74736 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74736/testReport)** for PR 16626 at commit [`8bf7a02`](https://github.com/apache/spark/commit/8bf7a0252e47abd336d831e8c7a771bebad3b6ee).
     * 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive tables

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #71749 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71749/testReport)** for PR 16626 at commit [`73b0243`](https://github.com/apache/spark/commit/73b024309674dc6d76e853547ef2a64da4836ce8).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74761 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74761/testReport)** for PR 16626 at commit [`b219178`](https://github.com/apache/spark/commit/b2191788f5d60261946da63e8c2634d5c6dfe6f5).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74281/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive tables

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #71535 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71535/testReport)** for PR 16626 at commit [`96fb677`](https://github.com/apache/spark/commit/96fb6777e74435bc96baac5fa2d0a17183e3f363).
     * 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74937 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74937/testReport)** for PR 16626 at commit [`7d8437d`](https://github.com/apache/spark/commit/7d8437dbba76e1e88c9f230dc7ea895d343dceb8).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106682149
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -450,6 +451,26 @@ abstract class SessionCatalogSuite extends PlanTest {
         }
       }
     
    +  test("alter table add columns") {
    +    withBasicCatalog { sessionCatalog =>
    +      sessionCatalog.createTable(newTable("t1", "default"), ignoreIfExists = false)
    +      val oldTab = sessionCatalog.externalCatalog.getTable("default", "t1")
    +      sessionCatalog.alterTableSchema(TableIdentifier("t1", Some("default")),
    +        oldTab.schema.add("c3", IntegerType))
    +      val newTab = sessionCatalog.externalCatalog.getTable("default", "t1")
    +      if (sessionCatalog.externalCatalog.isInstanceOf[InMemoryCatalog]) {
    +        assert(newTab.schema.toString == oldTab.schema.add("c3", IntegerType).toString)
    --- End diff --
    
    `StructField` returned from `CatalogTable` has metadata field that contains some hive related information, which the manually added schema field does not have. So directly comparing 2 `StructType` returns false.  `toString` method of `StructField` does not include metadata field..  But I think this may not be a good way to test anyway.. I will modify this test case.. 


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    LGTM pending Jenkins. 


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74810 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74810/testReport)** for PR 16626 at commit [`1eb7cd3`](https://github.com/apache/spark/commit/1eb7cd3ccc6e8ba02510ef5d5cce0beb31ffd8f6).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106097351
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -742,6 +742,22 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder {
       }
     
       /**
    +   * Create a [[AlterTableAddColumnsCommand]] command.
    +   *
    +   * For example:
    +   * {{{
    +   *   ALTER TABLE table1
    +   *   ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
    +   * }}}
    +   */
    +  override def visitAddTableColumns(ctx: AddTableColumnsContext): LogicalPlan = withOrigin(ctx) {
    +    AlterTableAddColumnsCommand(
    +      visitTableIdentifier(ctx.tableIdentifier),
    +      Option(ctx.columns).map(visitColTypeList).getOrElse(Nil)
    --- End diff --
    
    -> `visitColTypeList(ctx.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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r101958137
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala ---
    @@ -71,8 +71,20 @@ class JDBCSuite extends SparkFunSuite
         conn.prepareStatement("insert into test.people values ('mary', 2)").executeUpdate()
         conn.prepareStatement(
           "insert into test.people values ('joe ''foo'' \"bar\"', 3)").executeUpdate()
    +
    +    conn.prepareStatement("create table test.t_alter_add(c1 int, c2 int)").executeUpdate()
    +    conn.prepareStatement("insert into test.t_alter_add values (1, 2)").executeUpdate()
    +    conn.prepareStatement("insert into test.t_alter_add values (2, 4)").executeUpdate()
    --- End diff --
    
    We do not need to add the extra table for the invalid case.


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r107050148
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala ---
    @@ -416,4 +416,22 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext {
         val comments = planned.schema.fields.map(_.getComment().getOrElse("NO_COMMENT")).mkString(",")
         assert(comments === "SN,SA,NO_COMMENT")
       }
    +
    +  test("ALTER TABLE ADD COLUMNS does not support RelationProvider") {
    --- End diff --
    
    ok. Will do. 


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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/16626#discussion_r106621736
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---
    @@ -175,6 +178,98 @@ case class AlterTableRenameCommand(
     }
     
     /**
    + * A command that add columns to a table
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   ALTER TABLE table_identifier
    + *   ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
    + * }}}
    +*/
    +case class AlterTableAddColumnsCommand(
    +    table: TableIdentifier,
    +    columns: Seq[StructField]) extends RunnableCommand {
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    val catalog = sparkSession.sessionState.catalog
    +    val catalogTable = verifyAlterTableAddColumn(catalog, table)
    +
    +    try {
    +      sparkSession.catalog.uncacheTable(table.quotedString)
    +    } catch {
    +      case NonFatal(e) =>
    +        log.warn(s"Exception when attempting to uncache table ${table.quotedString}", e)
    +    }
    +
    +    // Invalidate the table last, otherwise uncaching the table would load the logical plan
    +    // back into the hive metastore cache
    +    catalog.refreshTable(table)
    +    val partitionFields = catalogTable.schema.takeRight(catalogTable.partitionColumnNames.length)
    --- End diff --
    
    `catalog.partitionSchema`


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106786264
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala ---
    @@ -1860,4 +1861,119 @@ class HiveDDLSuite
           }
         }
       }
    +
    +  hiveFormats.foreach { tableType =>
    +    test(s"alter hive serde table add columns -- partitioned - $tableType") {
    +      withTable("tab") {
    +        sql(
    +          s"""
    +             |CREATE TABLE tab (c1 int, c2 int)
    +             |PARTITIONED BY (c3 int) STORED AS $tableType
    +          """.stripMargin)
    +
    +        sql("INSERT INTO tab PARTITION (c3=1) VALUES (1, 2)")
    +        sql("ALTER TABLE tab ADD COLUMNS (c4 int)")
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c3 = 1"),
    +          Seq(Row(1, 2, null, 1))
    +        )
    +        assert(sql("SELECT * FROM tab").schema
    +          .contains(StructField("c4", IntegerType)))
    +        sql("INSERT INTO tab PARTITION (c3=2) VALUES (2, 3, 4)")
    +        checkAnswer(
    +          sql("SELECT * FROM tab"),
    +          Seq(Row(1, 2, null, 1), Row(2, 3, 4, 2))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c3 = 2 AND c4 IS NOT NULL"),
    +          Seq(Row(2, 3, 4, 2))
    +        )
    +      }
    +    }
    +  }
    +
    +  hiveFormats.foreach { tableType =>
    +    test(s"alter hive serde table add columns -- with predicate - $tableType ") {
    +      withTable("tab") {
    +        sql(s"CREATE TABLE tab (c1 int, c2 int) STORED AS $tableType")
    +        sql("INSERT INTO tab VALUES (1, 2)")
    +        sql("ALTER TABLE tab ADD COLUMNS (c4 int)")
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c4 IS NULL"),
    +          Seq(Row(1, 2, null))
    +        )
    +        assert(sql("SELECT * FROM tab").schema
    +          .contains(StructField("c4", IntegerType)))
    +        sql("INSERT INTO tab VALUES (2, 3, 4)")
    +        checkAnswer(
    +          sql("SELECT * FROM tab WHERE c4 = 4 "),
    +          Seq(Row(2, 3, 4))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM tab"),
    +          Seq(Row(1, 2, null), Row(2, 3, 4))
    +        )
    +      }
    +    }
    +  }
    +
    +  Seq("orc", "ORC", "org.apache.spark.sql.hive.orc",
    +    "org.apache.spark.sql.hive.orc.DefaultSource").foreach { source =>
    +    test(s"alter datasource table add columns - $source format not supported") {
    +      withTable("tab") {
    +        sql(s"CREATE TABLE tab (c1 int) USING $source")
    +        val e = intercept[AnalysisException] {
    +          sql("ALTER TABLE tab ADD COLUMNS (c2 int)")
    +        }.getMessage
    +        assert(
    +          e.contains(s"ALTER ADD COLUMNS does not support datasource table with type"))
    +      }
    +    }
    +  }
    +
    +  Seq("true", "false").foreach { caseSensitive =>
    +    test(s"alter add columns with existing partition column name - caseSensitive $caseSensitive") {
    --- End diff --
    
    combine this with the next one. just create a partitioned table and add two scenarios: one is for partition columns; another is for data 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive tab...

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

    https://github.com/apache/spark/pull/16626#discussion_r97174715
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---
    @@ -168,6 +168,43 @@ case class AlterTableRenameCommand(
     }
     
     /**
    + * A command that add columns to a table
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   ALTER TABLE table_identifier
    + *   ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
    --- End diff --
    
    This is better. Please copy this to SparkSqlParser.scala


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive tables

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

    https://github.com/apache/spark/pull/16626
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71535/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74652 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74652/testReport)** for PR 16626 at commit [`211287e`](https://github.com/apache/spark/commit/211287edc3437a951b2c6579d31e71f38f875306).
     * 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74733 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74733/testReport)** for PR 16626 at commit [`61de6b7`](https://github.com/apache/spark/commit/61de6b705d1006ddbde68713a86284634f160166).
     * 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74634 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74634/testReport)** for PR 16626 at commit [`7fbfc71`](https://github.com/apache/spark/commit/7fbfc7165e3bce388d4dc6e2c58487d4abf8d098).
     * 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106785992
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -296,6 +297,64 @@ class SessionCatalog(
       }
     
       /**
    +   * Alter the schema of a table identified by the provided table identifier. The new schema
    +   * should still contain the existing bucket columns and partition columns used by the table. This
    +   * method will also update any Spark SQL-related parameters stored as Hive table properties (such
    +   * as the schema itself).
    +   *
    +   * @param identifier TableIdentifier
    +   * @param newSchema Updated schema to be used for the table (must contain existing partition and
    +   *                  bucket columns)
    +   */
    +  def alterTableSchema(
    +      identifier: TableIdentifier,
    +      newSchema: StructType): Unit = {
    +    val db = formatDatabaseName(identifier.database.getOrElse(getCurrentDatabase))
    +    val table = formatTableName(identifier.table)
    +    val tableIdentifier = TableIdentifier(table, Some(db))
    +    requireDbExists(db)
    +    requireTableExists(tableIdentifier)
    +    val catalogTable = externalCatalog.getTable(db, table)
    +    val oldSchema = catalogTable.schema
    +
    +    // no supporting dropping columns yet
    +    if (!oldSchema.forall(f => columnNameResolved(newSchema, f.name ))) {
    +      throw new AnalysisException(
    +        s"""
    +          |Some existing schema fields are not present in the new schema.
    +          |We don't support dropping columns yet.
    +         """.stripMargin)
    +    }
    +
    +    checkDuplication(newSchema)
    +    // make sure partition columns are at the end
    +    val partitionSchema = catalogTable.partitionSchema
    +    val reorderedSchema = newSchema
    +      .filterNot(f => columnNameResolved(partitionSchema, f.name)) ++ partitionSchema
    +
    +    externalCatalog.alterTableSchema(
    +      db, table, oldSchema.copy(fields = reorderedSchema.toArray))
    --- End diff --
    
    `oldSchema.copy(fields = reorderedSchema.toArray)` 
    -> `StructType(reorderedSchema)`


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #72540 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72540/testReport)** for PR 16626 at commit [`d67042f`](https://github.com/apache/spark/commit/d67042fe043b092d7ca57af55429883a9208bf46).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106682210
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -450,6 +451,26 @@ abstract class SessionCatalogSuite extends PlanTest {
         }
       }
     
    +  test("alter table add columns") {
    +    withBasicCatalog { sessionCatalog =>
    +      sessionCatalog.createTable(newTable("t1", "default"), ignoreIfExists = false)
    +      val oldTab = sessionCatalog.externalCatalog.getTable("default", "t1")
    +      sessionCatalog.alterTableSchema(TableIdentifier("t1", Some("default")),
    +        oldTab.schema.add("c3", IntegerType))
    +      val newTab = sessionCatalog.externalCatalog.getTable("default", "t1")
    +      if (sessionCatalog.externalCatalog.isInstanceOf[InMemoryCatalog]) {
    +        assert(newTab.schema.toString == oldTab.schema.add("c3", IntegerType).toString)
    +      } else {
    +        // HiveExternalCatalog will always arrange the partition columns to the end
    --- End diff --
    
    Will do. 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106792896
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -450,6 +451,21 @@ abstract class SessionCatalogSuite extends PlanTest {
         }
       }
     
    +  test("alter table add columns") {
    +    withBasicCatalog { sessionCatalog =>
    +      sessionCatalog.createTable(newTable("t1", "default"), ignoreIfExists = false)
    +      val oldTab = sessionCatalog.externalCatalog.getTable("default", "t1")
    +      sessionCatalog.alterTableSchema(
    +        TableIdentifier("t1", Some("default")), oldTab.schema.add("c3", IntegerType))
    +
    +      val newTab = sessionCatalog.externalCatalog.getTable("default", "t1")
    +      // construct the expected table schema
    +      val oldTabSchema = StructType(oldTab.dataSchema.fields ++
    --- End diff --
    
    `oldTabSchema ` -> `expectedTabSchema `


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106786038
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---
    @@ -175,6 +178,75 @@ case class AlterTableRenameCommand(
     }
     
     /**
    + * A command that add columns to a table
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   ALTER TABLE table_identifier
    + *   ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
    + * }}}
    +*/
    +case class AlterTableAddColumnsCommand(
    +    table: TableIdentifier,
    +    columns: Seq[StructField]) extends RunnableCommand {
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    val catalog = sparkSession.sessionState.catalog
    +    val catalogTable = verifyAlterTableAddColumn(catalog, table)
    +
    +    try {
    +      sparkSession.catalog.uncacheTable(table.quotedString)
    +    } catch {
    +      case NonFatal(e) =>
    +        log.warn(s"Exception when attempting to uncache table ${table.quotedString}", e)
    +    }
    +
    --- End diff --
    
    Remove this empty line


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive tab...

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

    https://github.com/apache/spark/pull/16626#discussion_r97213578
  
    --- Diff: sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java ---
    @@ -107,7 +107,13 @@ public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptCont
           footer = readFooter(configuration, file, range(split.getStart(), split.getEnd()));
           MessageType fileSchema = footer.getFileMetaData().getSchema();
           FilterCompat.Filter filter = getFilter(configuration);
    -      blocks = filterRowGroups(filter, footer.getBlocks(), fileSchema);
    +      try {
    +        blocks = filterRowGroups(filter, footer.getBlocks(), fileSchema);
    +      } catch (IllegalArgumentException e) {
    +        // In the case where a particular parquet files does not contain
    --- End diff --
    
    Yes. we can. 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r101958045
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---
    @@ -174,6 +177,79 @@ case class AlterTableRenameCommand(
     }
     
     /**
    + * A command that add columns to a table
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   ALTER TABLE table_identifier
    + *   ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
    + * }}}
    +*/
    +case class AlterTableAddColumnsCommand(
    +    table: TableIdentifier,
    +    columns: Seq[StructField]) extends RunnableCommand {
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    val catalog = sparkSession.sessionState.catalog
    +    val catalogTable = verifyAlterTableAddColumn(catalog, table)
    +
    +    // If an exception is thrown here we can just assume the table is uncached;
    +    // this can happen with Hive tables when the underlying catalog is in-memory.
    +    val wasCached = Try(sparkSession.catalog.isCached(table.unquotedString)).getOrElse(false)
    +    if (wasCached) {
    +      try {
    +        sparkSession.catalog.uncacheTable(table.unquotedString)
    +      } catch {
    +        case NonFatal(e) => log.warn(e.toString, e)
    +      }
    +    }
    +    // Invalidate the table last, otherwise uncaching the table would load the logical plan
    +    // back into the hive metastore cache
    +    catalog.refreshTable(table)
    +    val partitionFields = catalogTable.schema.takeRight(catalogTable.partitionColumnNames.length)
    +    val dataSchema = catalogTable.schema
    +      .take(catalogTable.schema.length - catalogTable.partitionColumnNames.length)
    +    catalog.alterTable(catalogTable.copy(schema =
    +      catalogTable.schema.copy(fields = (dataSchema ++ columns ++ partitionFields).toArray)))
    +
    +    Seq.empty[Row]
    +  }
    +
    +  /**
    +   * ALTER TABLE ADD COLUMNS command does not support temporary view/table,
    +   * view, or datasource table with text, orc formats or external provider.
    +   */
    +  private def verifyAlterTableAddColumn(
    +    catalog: SessionCatalog,
    +    table: TableIdentifier): CatalogTable = {
    --- End diff --
    
    indent


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74035 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74035/testReport)** for PR 16626 at commit [`ec4a044`](https://github.com/apache/spark/commit/ec4a044ba7b009483b7f728da49a24b6aa942284).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74774 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74774/testReport)** for PR 16626 at commit [`ae324d3`](https://github.com/apache/spark/commit/ae324d3103358a045cab81677ab1f28c13303b7c).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #73094 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73094/testReport)** for PR 16626 at commit [`193c0c3`](https://github.com/apache/spark/commit/193c0c34a7ec55007fe93e397dace43223b32f58).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74937/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74929 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74929/testReport)** for PR 16626 at commit [`04ce8f4`](https://github.com/apache/spark/commit/04ce8f4853d7ffb8d3b33853c88add6d122e9aed).
     * 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r99659988
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
    @@ -814,4 +816,50 @@ object DDLUtils {
           }
         }
       }
    +
    +  /**
    +   * ALTER TABLE ADD COLUMNS command does not support temporary view/table,
    +   * view, or datasource table with text, orc formats or external provider.
    +   */
    +  def verifyAlterTableAddColumn(
    --- End diff --
    
    This function should be a private function of `AlterTableAddColumnsCommand `, 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 issue #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    @gatorsmile yes. I will do so. 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 issue #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74736/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive tables

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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/16626#discussion_r106622058
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---
    @@ -175,6 +178,98 @@ case class AlterTableRenameCommand(
     }
     
     /**
    + * A command that add columns to a table
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   ALTER TABLE table_identifier
    + *   ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
    + * }}}
    +*/
    +case class AlterTableAddColumnsCommand(
    +    table: TableIdentifier,
    +    columns: Seq[StructField]) extends RunnableCommand {
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    val catalog = sparkSession.sessionState.catalog
    +    val catalogTable = verifyAlterTableAddColumn(catalog, table)
    +
    +    try {
    +      sparkSession.catalog.uncacheTable(table.quotedString)
    +    } catch {
    +      case NonFatal(e) =>
    +        log.warn(s"Exception when attempting to uncache table ${table.quotedString}", e)
    +    }
    +
    +    // Invalidate the table last, otherwise uncaching the table would load the logical plan
    +    // back into the hive metastore cache
    +    catalog.refreshTable(table)
    +    val partitionFields = catalogTable.schema.takeRight(catalogTable.partitionColumnNames.length)
    +    val newSchemaFields = catalogTable.schema
    +      .take(catalogTable.schema.length - catalogTable.partitionColumnNames.length) ++
    +      columns ++ partitionFields
    +    checkDuplication(sparkSession, newSchemaFields)
    --- End diff --
    
    let's move the partition column arrangement and the duplication check into `SessionCatalog.alterTableSchema`


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    @gatorsmile @cloud-fan In order to use `ExternalCatalog.alterTableSchema` API, I need to add `SessionCatalog.alterTableSchema` that calls `ExternalCatalog.alterTableSchema`. Please check if this makes sense. 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 issue #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #72406 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72406/testReport)** for PR 16626 at commit [`88c2f48`](https://github.com/apache/spark/commit/88c2f48f730460c38aef50d02af08dd1df5c2097).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74791 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74791/testReport)** for PR 16626 at commit [`a28fc42`](https://github.com/apache/spark/commit/a28fc42cee6dc52516e074ced7c4351ee6baa45d).
     * 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive tables

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74816 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74816/testReport)** for PR 16626 at commit [`1eb7cd3`](https://github.com/apache/spark/commit/1eb7cd3ccc6e8ba02510ef5d5cce0beb31ffd8f6).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74283/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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

    https://github.com/apache/spark/pull/16626#discussion_r106342233
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala ---
    @@ -71,7 +71,6 @@ class JDBCSuite extends SparkFunSuite
         conn.prepareStatement("insert into test.people values ('mary', 2)").executeUpdate()
         conn.prepareStatement(
           "insert into test.people values ('joe ''foo'' \"bar\"', 3)").executeUpdate()
    -    conn.commit()
    --- End diff --
    
    Why?


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #74929 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74929/testReport)** for PR 16626 at commit [`04ce8f4`](https://github.com/apache/spark/commit/04ce8f4853d7ffb8d3b33853c88add6d122e9aed).


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive ser...

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/16626#discussion_r107057509
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala ---
    @@ -2178,4 +2178,138 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
           }
         }
       }
    +
    +  val supportedNativeFileFormatsForAlterTableAddColumns = Seq("parquet", "json", "csv")
    +
    +  supportedNativeFileFormatsForAlterTableAddColumns.foreach { provider =>
    +    test(s"alter datasource table add columns - $provider") {
    +      withTable("t1") {
    +        sql(s"CREATE TABLE t1 (c1 int) USING $provider")
    +        sql("INSERT INTO t1 VALUES (1)")
    +        sql("ALTER TABLE t1 ADD COLUMNS (c2 int)")
    +        checkAnswer(
    +          spark.table("t1"),
    +          Seq(Row(1, null))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c2 is null"),
    +          Seq(Row(1, null))
    +        )
    +
    +        sql("INSERT INTO t1 VALUES (3, 2)")
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c2 = 2"),
    +          Seq(Row(3, 2))
    +        )
    +      }
    +    }
    +  }
    +
    +  supportedNativeFileFormatsForAlterTableAddColumns.foreach { provider =>
    +    test(s"alter datasource table add columns - partitioned - $provider") {
    +      withTable("t1") {
    +        sql(s"CREATE TABLE t1 (c1 int, c2 int) USING $provider PARTITIONED BY (c2)")
    +        sql("INSERT INTO t1 PARTITION(c2 = 2) VALUES (1)")
    +        sql("ALTER TABLE t1 ADD COLUMNS (c3 int)")
    +        checkAnswer(
    +          spark.table("t1"),
    +          Seq(Row(1, null, 2))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c3 is null"),
    +          Seq(Row(1, null, 2))
    +        )
    +        sql("INSERT INTO t1 PARTITION(c2 =1) VALUES (2, 3)")
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c3 = 3"),
    +          Seq(Row(2, 3, 1))
    +        )
    +        checkAnswer(
    +          sql("SELECT * FROM t1 WHERE c2 = 1"),
    +          Seq(Row(2, 3, 1))
    +        )
    +      }
    +    }
    +  }
    +
    +  test("alter datasource table add columns - text format not supported") {
    +    withTable("t1") {
    +      sql("CREATE TABLE t1 (c1 int) USING text")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE t1 ADD COLUMNS (c2 int)")
    +      }.getMessage
    +      assert(e.contains("ALTER ADD COLUMNS does not support datasource table with type"))
    +    }
    +  }
    +
    +  test("alter table add columns -- not support temp view") {
    +    withTempView("tmp_v") {
    +      sql("CREATE TEMPORARY VIEW tmp_v AS SELECT 1 AS c1, 2 AS c2")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE tmp_v ADD COLUMNS (c3 INT)")
    +      }
    +      assert(e.message.contains("ALTER ADD COLUMNS does not support views"))
    +    }
    +  }
    +
    +  test("alter table add columns -- not support view") {
    +    withView("v1") {
    +      sql("CREATE VIEW v1 AS SELECT 1 AS c1, 2 AS c2")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE v1 ADD COLUMNS (c3 INT)")
    +      }
    +      assert(e.message.contains("ALTER ADD COLUMNS does not support views"))
    +    }
    +  }
    +
    +  test("alter table add columns with existing column name") {
    +    withTable("t1") {
    +      sql("CREATE TABLE t1 (c1 int) USING PARQUET")
    +      val e = intercept[AnalysisException] {
    +        sql("ALTER TABLE t1 ADD COLUMNS (c1 string)")
    +      }.getMessage
    +      assert(e.contains("Found duplicate column(s)"))
    +    }
    +  }
    +
    +  test("alter table add columns to table referenced by a view") {
    +    withTable("t1") {
    +      withView("v1") {
    +        sql("CREATE TABLE t1 (c1 int, c2 int) USING PARQUET")
    +        sql("CREATE VIEW v1 AS SELECT * FROM t1")
    +        val originViewSchema = sql("SELECT * FROM v1").schema
    +        sql("ALTER TABLE t1 ADD COLUMNS (c3 int)")
    +        assert(sql("SELECT * FROM v1").schema == originViewSchema)
    +      }
    +    }
    +  }
    +
    +  Seq("true", "false").foreach { caseSensitive =>
    +    test(s"alter table add columns with existing column name - caseSensitive $caseSensitive") {
    +      withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) {
    +        withTable("t1") {
    +          sql("CREATE TABLE t1 (c1 int) USING PARQUET")
    +          if (caseSensitive == "false") {
    +            val e = intercept[AnalysisException] {
    +              sql("ALTER TABLE t1 ADD COLUMNS (C1 string)")
    +            }.getMessage
    +            assert(e.contains("Found duplicate column(s)"))
    +          } else {
    +            if (isUsingHiveMetastore) {
    +              // hive catalog will still complains that c1 is duplicate column name because hive
    +              // identifiers are case insensitive.
    --- End diff --
    
    ah right, for hive, we can only make it case-preserving, not case-sensitive, I was wrong


---
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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74582/
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #72540 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72540/testReport)** for PR 16626 at commit [`d67042f`](https://github.com/apache/spark/commit/d67042fe043b092d7ca57af55429883a9208bf46).
     * 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 #16626: [SPARK-19261][SQL] Alter add columns for Hive serde and ...

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

    https://github.com/apache/spark/pull/16626
  
    **[Test build #72530 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72530/testReport)** for PR 16626 at commit [`0b7c0b1`](https://github.com/apache/spark/commit/0b7c0b146f69f7deba02b032c14bb1e8a1ed56d5).


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