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 2016/04/15 03:46:46 UTC

[GitHub] spark pull request: [SPARK-14346] [SQL][WIP] Show create table

GitHub user xwu0226 opened a pull request:

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

    [SPARK-14346] [SQL][WIP] Show create table 

    ## What changes were proposed in this pull request?
    Allow users to issue "`SHOW CREATE TABLE`" command natively in SparkSQL. 
    1. For tables that are created by Hive, this command will display the DDL in hive syntax. If the syntax includes `CLUSTERED BY, SKEWED BY or STORED BY` clause, there will be a warning message saying that this DDL is not supported in SparkSQL native DDL yet. 
    
    2. For tables that are created by datasource DDL, such as "`CREATE TABLE... USING ... OPTIONS (...)`", it will show the DDL in this syntax. 
    
    3. For tables that are created by dataframe API, such as "`df.write.partitionBy(...).saveAsTable(...)`", currently the command will display DDL with the syntax "CREATE TABLE.. USING...OPTIONS(...)". However, this syntax lose the partitioning information. It is proposed to display create table in the dataframe API format (in progress)
    
    ## How was this patch tested?
    Unit tests are created. 
    
    


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

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

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

    https://github.com/apache/spark/pull/12406.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 #12406
    
----
commit 0ebb0142e13db3ce8fb474ee5682528b0f87d2d2
Author: xin Wu <xi...@us.ibm.com>
Date:   2016-04-02T01:46:16Z

    show create table DDL -- hive metastore table

commit 6d060be797d4127f0b86fa59c1bc848d75215533
Author: xin Wu <xi...@us.ibm.com>
Date:   2016-04-02T06:01:46Z

    update upon review

commit 2799672162d715b209cad9a5c103d6f09692d8dc
Author: xin Wu <xi...@us.ibm.com>
Date:   2016-04-02T18:19:26Z

    ignoring sqlContext temp table and considering datasource table ddl

commit 98c020aa9a5374861d1470fa0c305148e8314ada
Author: xin Wu <xi...@us.ibm.com>
Date:   2016-04-04T21:54:32Z

    fix scala style issue

commit efd889821bf84e328ef6dd8d0b6a645729248251
Author: xin Wu <xi...@us.ibm.com>
Date:   2016-04-04T22:40:26Z

    fix scala style issue in testcase

commit b370630f5827071bc5076e9b3fa9c92720b27eb2
Author: xin Wu <xi...@us.ibm.com>
Date:   2016-04-05T01:31:46Z

    fix testcase for test failure

commit 8cb7a7299df84f2608b91b092a7df6795b85d41e
Author: xin Wu <xi...@us.ibm.com>
Date:   2016-04-06T18:12:07Z

    continue the database ddl generation

commit 8b67d22c5ed8fd6b309df772e4a372e741acf630
Author: xin Wu <xi...@us.ibm.com>
Date:   2016-04-08T20:57:12Z

    support datasource ddl

commit 9ab863fb7f8127d1acd083b1ba857f5c1fd2769c
Author: xin Wu <xi...@us.ibm.com>
Date:   2016-04-08T22:04:05Z

    scala style fix

commit a40273c7989bebdf62b93ce6e604bb14cacce100
Author: xin Wu <xi...@us.ibm.com>
Date:   2016-04-13T22:54:16Z

    merge the code committed by CREATE TABLE native support

commit d214a3b0c54641a6234ba39eef82b2b8ac4c87dd
Author: xin Wu <xi...@us.ibm.com>
Date:   2016-04-14T23:49:03Z

    rework show create ddl based on new native supported create table DDL work

commit 1680ea0403f0d29185d9a3f8f81d15599be81aac
Author: xin Wu <xi...@us.ibm.com>
Date:   2016-04-14T23:51:03Z

    Merge branch 'show_create_table_1' into show_create_table_2

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-14346] [SQL] Show create table

Posted by wangmiao1981 <gi...@git.apache.org>.
Github user wangmiao1981 commented on the pull request:

    https://github.com/apache/spark/pull/12406#issuecomment-212752658
  
    @xwu0226 use git rebase upstream/master. Do not use git merge upstream/master. I have the same issue before. git merge will add others' commits to your PR. git rebase will discard others' commits.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-14346] [SQL] Show create table

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

    https://github.com/apache/spark/pull/12406#issuecomment-211491518
  
    CC @yhuai @andrewor14 The PR is ready for review. 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: [SPARK-14346] [SQL][WIP] Show create table

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

    https://github.com/apache/spark/pull/12406#issuecomment-210244038
  
    Can one of the admins verify this patch?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-14346] [SQL][WIP] Show create table

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

    https://github.com/apache/spark/pull/12406#discussion_r59891913
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -109,6 +109,15 @@ class SparkSqlAstBuilder extends AstBuilder {
       }
     
       /**
    +   * Create a [[ShowCreateTableCommand]] logical plan
    +   */
    +  override def visitShowCreateTable(ctx: ShowCreateTableContext): LogicalPlan = withOrigin(ctx) {
    +    ShowCreateTableCommand(
    --- End diff --
    
    @hvanhovell Agree. Will change. 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: [SPARK-14346] [SQL][WIP] Show create table

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

    https://github.com/apache/spark/pull/12406#discussion_r59891868
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -423,6 +423,39 @@ case class ShowTablePropertiesCommand(
     }
     
     /**
    + * A command for users to get the DDL of an existing table
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   SHOW CREATE TABLE tableIdentifier
    + * }}}
    + */
    +case class ShowCreateTableCommand(
    +    tableName: String,
    --- End diff --
    
    @hvanhovell Thanks!. Yes, I should have. Will change 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: [SPARK-14346] [SQL] Show create table

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

    https://github.com/apache/spark/pull/12406#issuecomment-212752759
  
    Thanks @wangmiao1981 .
    
    This happens again. all the commits after my last commits got pulled into this PR. I need to close it and open a new PR. Will submit a new 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: [SPARK-14346] [SQL][WIP] Show create table

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

    https://github.com/apache/spark/pull/12406#discussion_r59857255
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -423,6 +423,39 @@ case class ShowTablePropertiesCommand(
     }
     
     /**
    + * A command for users to get the DDL of an existing table
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   SHOW CREATE TABLE tableIdentifier
    + * }}}
    + */
    +case class ShowCreateTableCommand(
    +    tableName: String,
    --- End diff --
    
    Shouldn't we just pass in a `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 pull request: [SPARK-14346] [SQL][WIP] Show create table

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

    https://github.com/apache/spark/pull/12406#issuecomment-210244218
  
    @yhuai @andrewor14 I need to resubmit this PR and closed the earlier one because somehow my merge brought in a lot of other PR's stuff, which make the other PR un-reviewable. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-14346] [SQL][WIP] Show create table

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

    https://github.com/apache/spark/pull/12406#discussion_r59857287
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -109,6 +109,15 @@ class SparkSqlAstBuilder extends AstBuilder {
       }
     
       /**
    +   * Create a [[ShowCreateTableCommand]] logical plan
    +   */
    +  override def visitShowCreateTable(ctx: ShowCreateTableContext): LogicalPlan = withOrigin(ctx) {
    +    ShowCreateTableCommand(
    --- End diff --
    
    Use `visitTableIdentifier`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-14346] [SQL] Show create table

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-14346] [SQL][WIP] Show create table

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

    https://github.com/apache/spark/pull/12406#discussion_r59893181
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala ---
    @@ -625,11 +629,215 @@ private[hive] class HiveClientImpl(
           }
       }
     
    +  override def showCreateTable(db: String, tableName: String): String = withHiveState {
    +    Option(client.getTable(db, tableName, false)).map { hiveTable =>
    +      val tblProperties = hiveTable.getParameters.asScala.toMap
    +      if (tblProperties.get("spark.sql.sources.provider").isDefined) {
    +        generateDataSourceDDL(hiveTable)
    +      } else {
    +        generateHiveDDL(hiveTable)
    +      }
    +    }.get
    +  }
    +
     
       /* -------------------------------------------------------- *
        |  Helper methods for converting to and from Hive classes  |
        * -------------------------------------------------------- */
     
    +  private def generateCreateTableHeader(
    +              hiveTable: HiveTable,
    +              processedProps: scala.collection.mutable.ArrayBuffer[String]): String = {
    +    val sb = new StringBuilder("CREATE ")
    +    if(hiveTable.isTemporary) {
    +      sb.append("TEMPORARY ")
    +    }
    +    if (hiveTable.getTableType == HiveTableType.EXTERNAL_TABLE) {
    +      processedProps += "EXTERNAL"
    +      sb.append("EXTERNAL TABLE " +
    +        quoteIdentifier(hiveTable.getDbName) + "." + quoteIdentifier(hiveTable.getTableName))
    +    } else {
    +      sb.append("TABLE " +
    +        quoteIdentifier(hiveTable.getDbName) + "." + quoteIdentifier(hiveTable.getTableName))
    +    }
    +    sb.toString()
    +  }
    +
    +  private def generateColsDataSource(
    +              hiveTable: HiveTable,
    +              processedProps: scala.collection.mutable.ArrayBuffer[String]): String = {
    +    val schemaStringFromParts: Option[String] = {
    +      val props = hiveTable.getParameters.asScala
    +      props.get("spark.sql.sources.schema.numParts").map { numParts =>
    +        val parts = (0 until numParts.toInt).map { index =>
    +          val part = props.get(s"spark.sql.sources.schema.part.$index").orNull
    +          if (part == null) {
    +            throw new AnalysisException(
    +              "Could not read schema from the metastore because it is corrupted " +
    +                s"(missing part $index of the schema, $numParts parts are expected).")
    +          }
    +          part
    +        }
    +        // Stick all parts back to a single schema string.
    +        parts.mkString
    +      }
    +    }
    +
    +    if (schemaStringFromParts.isDefined) {
    +      (schemaStringFromParts.map(s => DataType.fromJson(s).asInstanceOf[StructType]).
    +        get map { f => s"${quoteIdentifier(f.name)} ${f.dataType.sql}" })
    +        .mkString("( ", ", ", " )")
    +    } else {
    +      ""
    +    }
    +  }
    +
    +  private def generateDataSourceDDL(hiveTable: HiveTable): String = {
    --- End diff --
    
    @hvanhovell Are you referring to generateDataSourceDDL or generateHiveDDL? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-14346] [SQL][WIP] Show create table

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

    https://github.com/apache/spark/pull/12406#discussion_r59909616
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala ---
    @@ -625,11 +629,215 @@ private[hive] class HiveClientImpl(
           }
       }
     
    +  override def showCreateTable(db: String, tableName: String): String = withHiveState {
    +    Option(client.getTable(db, tableName, false)).map { hiveTable =>
    +      val tblProperties = hiveTable.getParameters.asScala.toMap
    +      if (tblProperties.get("spark.sql.sources.provider").isDefined) {
    +        generateDataSourceDDL(hiveTable)
    +      } else {
    +        generateHiveDDL(hiveTable)
    +      }
    +    }.get
    +  }
    +
     
       /* -------------------------------------------------------- *
        |  Helper methods for converting to and from Hive classes  |
        * -------------------------------------------------------- */
     
    +  private def generateCreateTableHeader(
    +              hiveTable: HiveTable,
    +              processedProps: scala.collection.mutable.ArrayBuffer[String]): String = {
    +    val sb = new StringBuilder("CREATE ")
    +    if(hiveTable.isTemporary) {
    +      sb.append("TEMPORARY ")
    +    }
    +    if (hiveTable.getTableType == HiveTableType.EXTERNAL_TABLE) {
    +      processedProps += "EXTERNAL"
    +      sb.append("EXTERNAL TABLE " +
    +        quoteIdentifier(hiveTable.getDbName) + "." + quoteIdentifier(hiveTable.getTableName))
    +    } else {
    +      sb.append("TABLE " +
    +        quoteIdentifier(hiveTable.getDbName) + "." + quoteIdentifier(hiveTable.getTableName))
    +    }
    +    sb.toString()
    +  }
    +
    +  private def generateColsDataSource(
    +              hiveTable: HiveTable,
    +              processedProps: scala.collection.mutable.ArrayBuffer[String]): String = {
    +    val schemaStringFromParts: Option[String] = {
    +      val props = hiveTable.getParameters.asScala
    +      props.get("spark.sql.sources.schema.numParts").map { numParts =>
    +        val parts = (0 until numParts.toInt).map { index =>
    +          val part = props.get(s"spark.sql.sources.schema.part.$index").orNull
    +          if (part == null) {
    +            throw new AnalysisException(
    +              "Could not read schema from the metastore because it is corrupted " +
    +                s"(missing part $index of the schema, $numParts parts are expected).")
    +          }
    +          part
    +        }
    +        // Stick all parts back to a single schema string.
    +        parts.mkString
    +      }
    +    }
    +
    +    if (schemaStringFromParts.isDefined) {
    +      (schemaStringFromParts.map(s => DataType.fromJson(s).asInstanceOf[StructType]).
    +        get map { f => s"${quoteIdentifier(f.name)} ${f.dataType.sql}" })
    +        .mkString("( ", ", ", " )")
    +    } else {
    +      ""
    +    }
    +  }
    +
    +  private def generateDataSourceDDL(hiveTable: HiveTable): String = {
    --- End diff --
    
    @hvanhovell  I looked through the Hive's Hive, Table and MetaStoreClient APIs and have not found an API function that can do this for us. The way I see Hive implements this is through DDLTask.showCreateTable, which is executed through normal hive command parsing and executing route.. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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: [SPARK-14346] [SQL][WIP] Show create table

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

    https://github.com/apache/spark/pull/12406#discussion_r59857950
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala ---
    @@ -625,11 +629,215 @@ private[hive] class HiveClientImpl(
           }
       }
     
    +  override def showCreateTable(db: String, tableName: String): String = withHiveState {
    +    Option(client.getTable(db, tableName, false)).map { hiveTable =>
    +      val tblProperties = hiveTable.getParameters.asScala.toMap
    +      if (tblProperties.get("spark.sql.sources.provider").isDefined) {
    +        generateDataSourceDDL(hiveTable)
    +      } else {
    +        generateHiveDDL(hiveTable)
    +      }
    +    }.get
    +  }
    +
     
       /* -------------------------------------------------------- *
        |  Helper methods for converting to and from Hive classes  |
        * -------------------------------------------------------- */
     
    +  private def generateCreateTableHeader(
    +              hiveTable: HiveTable,
    +              processedProps: scala.collection.mutable.ArrayBuffer[String]): String = {
    +    val sb = new StringBuilder("CREATE ")
    +    if(hiveTable.isTemporary) {
    +      sb.append("TEMPORARY ")
    +    }
    +    if (hiveTable.getTableType == HiveTableType.EXTERNAL_TABLE) {
    +      processedProps += "EXTERNAL"
    +      sb.append("EXTERNAL TABLE " +
    +        quoteIdentifier(hiveTable.getDbName) + "." + quoteIdentifier(hiveTable.getTableName))
    +    } else {
    +      sb.append("TABLE " +
    +        quoteIdentifier(hiveTable.getDbName) + "." + quoteIdentifier(hiveTable.getTableName))
    +    }
    +    sb.toString()
    +  }
    +
    +  private def generateColsDataSource(
    +              hiveTable: HiveTable,
    +              processedProps: scala.collection.mutable.ArrayBuffer[String]): String = {
    +    val schemaStringFromParts: Option[String] = {
    +      val props = hiveTable.getParameters.asScala
    +      props.get("spark.sql.sources.schema.numParts").map { numParts =>
    +        val parts = (0 until numParts.toInt).map { index =>
    +          val part = props.get(s"spark.sql.sources.schema.part.$index").orNull
    +          if (part == null) {
    +            throw new AnalysisException(
    +              "Could not read schema from the metastore because it is corrupted " +
    +                s"(missing part $index of the schema, $numParts parts are expected).")
    +          }
    +          part
    +        }
    +        // Stick all parts back to a single schema string.
    +        parts.mkString
    +      }
    +    }
    +
    +    if (schemaStringFromParts.isDefined) {
    +      (schemaStringFromParts.map(s => DataType.fromJson(s).asInstanceOf[StructType]).
    +        get map { f => s"${quoteIdentifier(f.name)} ${f.dataType.sql}" })
    +        .mkString("( ", ", ", " )")
    +    } else {
    +      ""
    +    }
    +  }
    +
    +  private def generateDataSourceDDL(hiveTable: HiveTable): String = {
    --- End diff --
    
    Can't we ask Hive to do this for us?


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

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