You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by cloud-fan <gi...@git.apache.org> on 2017/11/02 13:31:20 UTC

[GitHub] spark pull request #19644: [SPARK-22306][SQL] alter table schema should not ...

GitHub user cloud-fan opened a pull request:

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

    [SPARK-22306][SQL] alter table schema should not erase the bucketing metadata at hive side

    forward-port https://github.com/apache/spark/pull/19622 to master branch.
    
    This bug doesn't exist in master because we've added hive bucketing support and the hive bucketing metadata can be recognized by Spark, but we should still port it to master: 1) there may be other unsupported hive metadata removed by Spark. 2) reduce code difference between master and 2.2 to ease the backport in the feature.
    
    When we alter table schema, we set the new schema to spark `CatalogTable`, convert it to hive table, and finally call `hive.alterTable`. This causes a problem in Spark 2.2, because hive bucketing metedata is not recognized by Spark, which means a Spark `CatalogTable` representing a hive table is always non-bucketed, and when we convert it to hive table and call `hive.alterTable`, the original hive bucketing metadata will be removed.
    
    To fix this bug, we should read out the raw hive table metadata, update its schema, and call `hive.alterTable`. By doing this we can guarantee only the schema is changed, and nothing else.

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

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

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

    https://github.com/apache/spark/pull/19644.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 #19644
    
----
commit d73d7771b5b29d16a930069473ffee28548044c2
Author: Wenchen Fan <we...@databricks.com>
Date:   2017-11-02T11:37:52Z

    [SPARK-22306][SQL] alter table schema should not erase the bucketing metadata at hive side
    
    When we alter table schema, we set the new schema to spark `CatalogTable`, convert it to hive table, and finally call `hive.alterTable`. This causes a problem in Spark 2.2, because hive bucketing metedata is not recognized by Spark, which means a Spark `CatalogTable` representing a hive table is always non-bucketed, and when we convert it to hive table and call `hive.alterTable`, the original hive bucketing metadata will be removed.
    
    To fix this bug, we should read out the raw hive table metadata, update its schema, and call `hive.alterTable`. By doing this we can guarantee only the schema is changed, and nothing else.
    
    Note that this bug doesn't exist in the master branch, because we've added hive bucketing support and the hive bucketing metadata can be recognized by Spark. I think we should merge this PR to master too, for code cleanup and reduce the difference between master and 2.2 branch for backporting.
    
    new regression test
    
    Author: Wenchen Fan <we...@databricks.com>
    
    Closes #19622 from cloud-fan/infer.

----


---

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


[GitHub] spark issue #19644: [SPARK-22306][SQL] alter table schema should not erase t...

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

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


---

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


[GitHub] spark pull request #19644: [SPARK-22306][SQL] alter table schema should not ...

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

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


---

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


[GitHub] spark issue #19644: [SPARK-22306][SQL] alter table schema should not erase t...

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

    https://github.com/apache/spark/pull/19644
  
    cc @gatorsmile 


---

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


[GitHub] spark pull request #19644: [SPARK-22306][SQL] alter table schema should not ...

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/19644#discussion_r148678873
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala ---
    @@ -306,10 +305,10 @@ class ParquetFileFormat
         hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName)
         hadoopConf.set(
           ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA,
    -      ParquetSchemaConverter.checkFieldNames(requiredSchema).json)
    +      requiredSchema.json)
         hadoopConf.set(
           ParquetWriteSupport.SPARK_ROW_SCHEMA,
    -      ParquetSchemaConverter.checkFieldNames(requiredSchema).json)
    --- End diff --
    
    at read path, the parquet files are already there, so the field names must be valid :)


---

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


[GitHub] spark issue #19644: [SPARK-22306][SQL] alter table schema should not erase t...

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

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


---

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


[GitHub] spark issue #19644: [SPARK-22306][SQL] alter table schema should not erase t...

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

    https://github.com/apache/spark/pull/19644
  
    +1, LGTM. Thank you, @cloud-fan .
    
    BTW, although this is a forward-port, `checkFieldNames` part is new. I'm wondering if you are going to backport this into branch-2.2 again.


---

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


[GitHub] spark pull request #19644: [SPARK-22306][SQL] alter table schema should not ...

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

    https://github.com/apache/spark/pull/19644#discussion_r148613092
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala ---
    @@ -306,10 +305,10 @@ class ParquetFileFormat
         hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName)
         hadoopConf.set(
           ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA,
    -      ParquetSchemaConverter.checkFieldNames(requiredSchema).json)
    +      requiredSchema.json)
         hadoopConf.set(
           ParquetWriteSupport.SPARK_ROW_SCHEMA,
    -      ParquetSchemaConverter.checkFieldNames(requiredSchema).json)
    --- End diff --
    
    This is for the read path. Not sure whether this also matters. 
    
    The other checks are for DDL.


---

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


[GitHub] spark issue #19644: [SPARK-22306][SQL] alter table schema should not erase t...

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

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


---

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


[GitHub] spark issue #19644: [SPARK-22306][SQL] alter table schema should not erase t...

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

    https://github.com/apache/spark/pull/19644
  
    **[Test build #83345 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83345/testReport)** for PR 19644 at commit [`16714bc`](https://github.com/apache/spark/commit/16714bc43d51b5248b67f44e3ed46374ed36c1ea).


---

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


[GitHub] spark issue #19644: [SPARK-22306][SQL] alter table schema should not erase t...

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

    https://github.com/apache/spark/pull/19644
  
    `checkFieldNames` is just a small cleanup, I think we can keep it in master.


---

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


[GitHub] spark issue #19644: [SPARK-22306][SQL] alter table schema should not erase t...

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

    https://github.com/apache/spark/pull/19644
  
    thanks, merging to master!


---

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


[GitHub] spark pull request #19644: [SPARK-22306][SQL] alter table schema should not ...

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/19644#discussion_r148550770
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala ---
    @@ -306,10 +305,10 @@ class ParquetFileFormat
         hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName)
         hadoopConf.set(
           ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA,
    -      ParquetSchemaConverter.checkFieldNames(requiredSchema).json)
    +      requiredSchema.json)
         hadoopConf.set(
           ParquetWriteSupport.SPARK_ROW_SCHEMA,
    -      ParquetSchemaConverter.checkFieldNames(requiredSchema).json)
    --- End diff --
    
    I don't think we need to check field names at this last step, cc @dongjoon-hyun 


---

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


[GitHub] spark pull request #19644: [SPARK-22306][SQL] alter table schema should not ...

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

    https://github.com/apache/spark/pull/19644#discussion_r148575508
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
    @@ -857,19 +857,23 @@ object DDLUtils {
         }
       }
     
    -  private[sql] def checkDataSchemaFieldNames(table: CatalogTable): Unit = {
    +  private[sql] def checkDataColNames(table: CatalogTable): Unit = {
    +    checkDataColNames(table, table.dataSchema.fieldNames)
    +  }
    +
    +  private[sql] def checkDataColNames(table: CatalogTable, colNames: Seq[String]): Unit = {
    --- End diff --
    
    Do you mean `checkFieldNames` in [line 870](https://github.com/apache/spark/pull/19644/files#diff-54979ed5797b4a6193cf663dc23baca5R870) instead of `checkDataColNames` or `checkDataSchemaFieldNames`?


---

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


[GitHub] spark pull request #19644: [SPARK-22306][SQL] alter table schema should not ...

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/19644#discussion_r148550602
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
    @@ -857,19 +857,23 @@ object DDLUtils {
         }
       }
     
    -  private[sql] def checkDataSchemaFieldNames(table: CatalogTable): Unit = {
    +  private[sql] def checkDataColNames(table: CatalogTable): Unit = {
    +    checkDataColNames(table, table.dataSchema.fieldNames)
    +  }
    +
    +  private[sql] def checkDataColNames(table: CatalogTable, colNames: Seq[String]): Unit = {
    --- End diff --
    
    I think it's better to let a checking method return void. cc @dongjoon-hyun 


---

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


[GitHub] spark issue #19644: [SPARK-22306][SQL] alter table schema should not erase t...

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

    https://github.com/apache/spark/pull/19644
  
    **[Test build #83345 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83345/testReport)** for PR 19644 at commit [`16714bc`](https://github.com/apache/spark/commit/16714bc43d51b5248b67f44e3ed46374ed36c1ea).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #19644: [SPARK-22306][SQL] alter table schema should not ...

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

    https://github.com/apache/spark/pull/19644#discussion_r148575793
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
    @@ -857,19 +857,23 @@ object DDLUtils {
         }
       }
     
    -  private[sql] def checkDataSchemaFieldNames(table: CatalogTable): Unit = {
    +  private[sql] def checkDataColNames(table: CatalogTable): Unit = {
    +    checkDataColNames(table, table.dataSchema.fieldNames)
    +  }
    +
    +  private[sql] def checkDataColNames(table: CatalogTable, colNames: Seq[String]): Unit = {
    --- End diff --
    
    +1!


---

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


[GitHub] spark pull request #19644: [SPARK-22306][SQL] alter table schema should not ...

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

    https://github.com/apache/spark/pull/19644#discussion_r148578288
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala ---
    @@ -306,10 +305,10 @@ class ParquetFileFormat
         hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName)
         hadoopConf.set(
           ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA,
    -      ParquetSchemaConverter.checkFieldNames(requiredSchema).json)
    +      requiredSchema.json)
         hadoopConf.set(
           ParquetWriteSupport.SPARK_ROW_SCHEMA,
    -      ParquetSchemaConverter.checkFieldNames(requiredSchema).json)
    --- End diff --
    
    Thank you for pinging me. I see. That logic is introduced at [SPARK-13883](https://github.com/cloud-fan/spark/blame/da54abfd8730ef752eca921089bcf568773bd24a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala#L302) about 2 years ago by @marmbrus .


---

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


[GitHub] spark issue #19644: [SPARK-22306][SQL] alter table schema should not erase t...

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

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


---

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