[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-11-03 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r86470657
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -417,11 +429,12 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
   }
 
   override def renameTable(db: String, oldName: String, newName: String): 
Unit = withClient {
-val rawTable = client.getTable(db, oldName)
+val rawTable = getRawTable(db, oldName)
 
-val storageWithNewPath = if (rawTable.tableType == MANAGED) {
-  // If it's a managed table and we are renaming it, then the path 
option becomes inaccurate
-  // and we need to update it according to the new table name.
+val hasPathOption = new 
CaseInsensitiveMap(rawTable.storage.properties).contains("path")
+val storageWithNewPath = if (rawTable.tableType == MANAGED && 
hasPathOption) {
--- End diff --

Let's also explain why we are not changing locationUri at here (need to 
mention that Hive will change it inside alterTable, line 
https://github.com/apache/spark/pull/14750/files#diff-159191585e10542f013cb3a714f26075R448).


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

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-11-03 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r86469470
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -95,8 +95,11 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 }
   }
 
-  private def requireTableExists(db: String, table: String): Unit = {
-withClient { getTable(db, table) }
+  /**
+   * Get the raw table metadata from hive metastore directly.
--- End diff --

Let's also define `raw` at 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-11-03 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r86471858
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -537,22 +559,11 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
   table
 } else {
   getProviderFromTableProperties(table).map { provider =>
--- End diff --

Seems we need to make this part more reader friendly. Maybe ?
```
getProviderFromTableProperties(table) match {
  case None => // regular hive tables
  case Some("hive") => // hive serde table created by spark 2.1 or higher 
version
  case Some(other) => // data source 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-11-03 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r86469682
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -255,6 +267,12 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 .getOrElse("skipHiveMetadata", "false").toBoolean
 
   val (hiveCompatibleTable, logMessage) = maybeSerde match {
+case _ if tableDefinition.provider.get == "hive" =>
+  val message = s"Persisting Hive serde table $qualifiedTableName 
into Hive metastore."
+  val tableWithDataSourceProps = tableDefinition.copy(
+properties = tableDefinition.properties ++ tableProperties)
+  (Some(tableWithDataSourceProps), message)
--- End diff --

Add some comments at 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-11-03 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r86471149
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -475,18 +490,27 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
   tableDefinition
 }
 
-if (DDLUtils.isDatasourceTable(withStatsProps)) {
-  val oldTableDef = client.getTable(db, 
withStatsProps.identifier.table)
+if (tableDefinition.tableType == VIEW) {
+  client.alterTable(withStatsProps)
+} else {
+  val oldTableDef = getRawTable(db, withStatsProps.identifier.table)
 
-  val oldLocation = getLocationFromStorageProps(oldTableDef)
-  val newLocation = tableDefinition.storage.locationUri
-  // Only update the `locationUri` field if the location is really 
changed, because this table
-  // may be not Hive-compatible and can not set the `locationUri` 
field. We should respect the
-  // old `locationUri` even it's None.
-  val storageWithNewLocation = if (oldLocation == newLocation) {
-oldTableDef.storage
+  val newStorage = if (tableDefinition.provider.get == "hive") {
+tableDefinition.storage
   } else {
-updateLocationInStorageProps(oldTableDef, 
newLocation).copy(locationUri = newLocation)
+val newLocation = tableDefinition.storage.locationUri
+val storageWithPathOption = tableDefinition.storage.copy(
+  properties = tableDefinition.storage.properties ++ 
newLocation.map("path" -> _))
+
+// For data source tables, only update the `locationUri` field if 
the location is really
+// changed, because this table may be not Hive-compatible and can 
not set the `locationUri`
+// field. We should respect the old `locationUri` even it's None.
+val oldLocation = getLocationFromStorageProps(oldTableDef)
+if (oldLocation == newLocation) {
+  storageWithPathOption.copy(locationUri = 
oldTableDef.storage.locationUri)
--- End diff --

Let's use an example to explain what we are doing at 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-11-03 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r86472353
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -620,7 +667,9 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 
 val orderedPartitionSpec = new util.LinkedHashMap[String, String]()
 getTable(db, table).partitionColumnNames.foreach { colName =>
-  orderedPartitionSpec.put(colName, partition(colName))
+  // Lowercase the partition column names before passing the partition 
spec to Hive client, as
+  // Hive metastore is not case preserving.
--- End diff --

Let's mention that Hive will lower case the part column names. So, if we do 
not convert the name to its lower case, hive will complain.


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

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-11-03 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r86470287
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -372,10 +386,8 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
   }
 
   private def saveTableIntoHive(tableDefinition: CatalogTable, 
ignoreIfExists: Boolean): Unit = {
-assert(DDLUtils.isDatasourceTable(tableDefinition),
-  "saveTableIntoHive only takes data source table.")
 // If this is an external data source table...
-if (tableDefinition.tableType == EXTERNAL &&
+if (tableDefinition.provider.get != "hive" && 
tableDefinition.tableType == EXTERNAL &&
--- End diff --

we should avoid calling this method for hive table in the try-catch


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

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-10-24 Thread ericl
Github user ericl commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r84817845
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -446,29 +452,46 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
   table
 } else {
   getProviderFromTableProperties(table).map { provider =>
-assert(provider != "hive", "Hive serde table should not save 
provider in table properties.")
-// SPARK-15269: Persisted data source tables always store the 
location URI as a storage
-// property named "path" instead of standard Hive `dataLocation`, 
because Hive only
-// allows directory paths as location URIs while Spark SQL data 
source tables also
-// allows file paths. So the standard Hive `dataLocation` is 
meaningless for Spark SQL
-// data source tables.
-// Spark SQL may also save external data source in Hive compatible 
format when
-// possible, so that these tables can be directly accessed by 
Hive. For these tables,
-// `dataLocation` is still necessary. Here we also check for input 
format because only
-// these Hive compatible tables set this field.
-val storage = if (table.tableType == EXTERNAL && 
table.storage.inputFormat.isEmpty) {
-  table.storage.copy(locationUri = None)
+if (provider == "hive") {
+  val schemaFromTableProps = getSchemaFromTableProperties(table)
+  if 
(DataType.equalsIgnoreCaseAndNullability(schemaFromTableProps, table.schema)) {
+table.copy(
+  schema = schemaFromTableProps,
+  provider = Some(provider),
+  partitionColumnNames = 
getPartitionColumnsFromTableProperties(table),
+  bucketSpec = getBucketSpecFromTableProperties(table),
+  properties = getOriginalTableProperties(table))
+  } else {
+// Hive metastore may change the table schema, e.g. schema 
inference. If the table
+// schema we read back is different from the one in table 
properties which was written
+// when creating table, we should respect the table schema 
from hive.
+table.copy(properties = getOriginalTableProperties(table))
--- End diff --

Shouldn't we still restore the original column names when possible?


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

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-10-24 Thread ericl
Github user ericl commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r84817656
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
 ---
@@ -373,7 +373,7 @@ case class InsertIntoTable(
   // Note: The parser (visitPartitionSpec in AstBuilder) already turns
   // keys in partition to their lowercase forms.
   val staticPartCols = partition.filter(_._2.isDefined).keySet
-  Some(table.output.filterNot(a => staticPartCols.contains(a.name)))
+  Some(table.output.filterNot(a => 
staticPartCols.contains(a.name.toLowerCase)))
--- End diff --

Seems like this will be addressed by 
https://github.com/apache/spark/pull/15566


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

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-09-13 Thread clockfly
Github user clockfly commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r78666218
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -93,8 +92,11 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 }
   }
 
-  private def requireTableExists(db: String, table: String): Unit = {
-withClient { getTable(db, table) }
+  /**
+   * Get the raw table metadata from hive metastore directly.
+   */
+  private def getRawTable(db: String, table: String): CatalogTable = 
withClient {
--- End diff --

I see.


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

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-09-13 Thread clockfly
Github user clockfly commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r78659438
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
 ---
@@ -373,7 +373,7 @@ case class InsertIntoTable(
   // Note: The parser (visitPartitionSpec in AstBuilder) already turns
   // keys in partition to their lowercase forms.
   val staticPartCols = partition.filter(_._2.isDefined).keySet
-  Some(table.output.filterNot(a => staticPartCols.contains(a.name)))
+  Some(table.output.filterNot(a => 
staticPartCols.contains(a.name.toLowerCase)))
--- End diff --

visitPartitionSpec is used by 
```
ShowPartitionsCommand
LoadDataCommand
TruncateTableCommand
AlterTableAddPartitionCommand
AlterTableRenamePartitionCommand
AlterTableDropPartitionCommand
AlterTableSetLocationCommand
```

I peeked several class above, seems the case sensitivity is not handled 
well. 


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

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-09-13 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r78641530
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -93,8 +92,11 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 }
   }
 
-  private def requireTableExists(db: String, table: String): Unit = {
-withClient { getTable(db, table) }
+  /**
+   * Get the raw table metadata from hive metastore directly.
+   */
+  private def getRawTable(db: String, table: String): CatalogTable = 
withClient {
--- End diff --

This is from my original comment: 
https://github.com/apache/spark/pull/14750#issuecomment-245788615 

Personally, I think it is easier for the other coders to understand the 
differences between `getTable` and `client.getTable`


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

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-09-12 Thread clockfly
Github user clockfly commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r78494405
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -93,8 +92,11 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 }
   }
 
-  private def requireTableExists(db: String, table: String): Unit = {
-withClient { getTable(db, table) }
+  /**
+   * Get the raw table metadata from hive metastore directly.
+   */
+  private def getRawTable(db: String, table: String): CatalogTable = 
withClient {
--- End diff --

Is it worthy to create a new method instead of using `client.getTable(db, 
table)` directly?


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

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-09-12 Thread clockfly
Github user clockfly commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r78474805
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala ---
@@ -245,4 +245,28 @@ object DataType {
   case (fromDataType, toDataType) => fromDataType == toDataType
 }
   }
+
+  /**
+   * Compares two types, ignoring nullability of ArrayType, MapType, 
StructType, and ignoring case
+   * sensitivity of field names in StructType.
+   */
+  private[sql] def equalsIgnoreCaseAndNullability(from: DataType, to: 
DataType): Boolean = {
--- End diff --

It is really tricky to add this method in DataType. 


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

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-09-08 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r78129508
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
 ---
@@ -169,7 +169,10 @@ case class InsertIntoHiveTable(
 
 // All partition column names in the format of "//..."
 val partitionColumns = 
fileSinkConf.getTableInfo.getProperties.getProperty("partition_columns")
-val partitionColumnNames = 
Option(partitionColumns).map(_.split("/")).getOrElse(Array.empty)
+// As the keys of partition spec `partition` is always lowercase, we 
should also lowercase the
+// partition column names of the table here.
+val partitionColumnNames =
+  
Option(partitionColumns).map(_.split("/").map(_.toLowerCase)).getOrElse(Array.empty)
--- End diff --

We can combine the above two lines into a single line?
```Scala
val partitionColumnNames = 
table.catalogTable.partitionColumnNames.map(_.toLowerCase)
```


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

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-09-08 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r78124662
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
 ---
@@ -169,7 +169,10 @@ case class InsertIntoHiveTable(
 
 // All partition column names in the format of "//..."
 val partitionColumns = 
fileSinkConf.getTableInfo.getProperties.getProperty("partition_columns")
-val partitionColumnNames = 
Option(partitionColumns).map(_.split("/")).getOrElse(Array.empty)
+// As the keys of partition spec `partition` is always lowercase, we 
should also lowercase the
+// partition column names of the table here.
+val partitionColumnNames =
+  
Option(partitionColumns).map(_.split("/").map(_.toLowerCase)).getOrElse(Array.empty)
--- End diff --

After more investigation, 
[`fileSinkConf.getTableInfo`](https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala#L171)
 is 
[`table.tableDesc`](https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala#L145).
 That means, we do not need to convert it to lower 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-09-08 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r78120149
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
 ---
@@ -169,7 +169,10 @@ case class InsertIntoHiveTable(
 
 // All partition column names in the format of "//..."
 val partitionColumns = 
fileSinkConf.getTableInfo.getProperties.getProperty("partition_columns")
-val partitionColumnNames = 
Option(partitionColumns).map(_.split("/")).getOrElse(Array.empty)
+// As the keys of partition spec `partition` is always lowercase, we 
should also lowercase the
+// partition column names of the table here.
+val partitionColumnNames =
+  
Option(partitionColumns).map(_.split("/").map(_.toLowerCase)).getOrElse(Array.empty)
--- End diff --

It sounds like this is from Hive side. Thus, it should be always Lower 
Case? Without these changes, I ran the build. It sounds like it works well.

Can we use ExternalCatalog to get the partition info? I can make a try to 
see if it still works.


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

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-09-08 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r78114385
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -446,29 +449,46 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
   table
 } else {
   getProviderFromTableProperties(table).map { provider =>
-assert(provider != "hive", "Hive serde table should not save 
provider in table properties.")
-// SPARK-15269: Persisted data source tables always store the 
location URI as a storage
-// property named "path" instead of standard Hive `dataLocation`, 
because Hive only
-// allows directory paths as location URIs while Spark SQL data 
source tables also
-// allows file paths. So the standard Hive `dataLocation` is 
meaningless for Spark SQL
-// data source tables.
-// Spark SQL may also save external data source in Hive compatible 
format when
-// possible, so that these tables can be directly accessed by 
Hive. For these tables,
-// `dataLocation` is still necessary. Here we also check for input 
format because only
-// these Hive compatible tables set this field.
-val storage = if (table.tableType == EXTERNAL && 
table.storage.inputFormat.isEmpty) {
-  table.storage.copy(locationUri = None)
+if (provider == "hive") {
+  val schemaFromTableProps = getSchemaFromTableProperties(table)
+  if 
(DataType.equalsIgnoreCaseAndNullability(schemaFromTableProps, table.schema)) {
--- End diff --

Yeah, but Hive allows users to change physical storage by issuing `ALTER 
TABLE` DDL. 
```
ALTER TABLE table_name CLUSTERED BY (col_name, col_name, ...) [SORTED BY 
(col_name, ...)]
  INTO num_buckets BUCKETS;
```

That means, even if they are part of schema, but the `bucketSpect` could be 
different.


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

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-09-08 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r78113922
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -446,29 +449,46 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
   table
 } else {
   getProviderFromTableProperties(table).map { provider =>
-assert(provider != "hive", "Hive serde table should not save 
provider in table properties.")
-// SPARK-15269: Persisted data source tables always store the 
location URI as a storage
-// property named "path" instead of standard Hive `dataLocation`, 
because Hive only
-// allows directory paths as location URIs while Spark SQL data 
source tables also
-// allows file paths. So the standard Hive `dataLocation` is 
meaningless for Spark SQL
-// data source tables.
-// Spark SQL may also save external data source in Hive compatible 
format when
-// possible, so that these tables can be directly accessed by 
Hive. For these tables,
-// `dataLocation` is still necessary. Here we also check for input 
format because only
-// these Hive compatible tables set this field.
-val storage = if (table.tableType == EXTERNAL && 
table.storage.inputFormat.isEmpty) {
-  table.storage.copy(locationUri = None)
+if (provider == "hive") {
+  val schemaFromTableProps = getSchemaFromTableProperties(table)
+  if 
(DataType.equalsIgnoreCaseAndNullability(schemaFromTableProps, table.schema)) {
--- End diff --

> Schema includes partitioning columns, but it does not include the info of 
bucketSpec

hmmm? don't bucket columns and sort columns must be part of 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-09-08 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r78106864
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -446,29 +449,46 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
   table
 } else {
   getProviderFromTableProperties(table).map { provider =>
-assert(provider != "hive", "Hive serde table should not save 
provider in table properties.")
-// SPARK-15269: Persisted data source tables always store the 
location URI as a storage
-// property named "path" instead of standard Hive `dataLocation`, 
because Hive only
-// allows directory paths as location URIs while Spark SQL data 
source tables also
-// allows file paths. So the standard Hive `dataLocation` is 
meaningless for Spark SQL
-// data source tables.
-// Spark SQL may also save external data source in Hive compatible 
format when
-// possible, so that these tables can be directly accessed by 
Hive. For these tables,
-// `dataLocation` is still necessary. Here we also check for input 
format because only
-// these Hive compatible tables set this field.
-val storage = if (table.tableType == EXTERNAL && 
table.storage.inputFormat.isEmpty) {
-  table.storage.copy(locationUri = None)
+if (provider == "hive") {
+  val schemaFromTableProps = getSchemaFromTableProperties(table)
+  if 
(DataType.equalsIgnoreCaseAndNullability(schemaFromTableProps, table.schema)) {
--- End diff --

Schema includes partitioning columns, but it does not include the info of 
`bucketSpec`. Based on the previous answer about `bucketSpec`, I am not sure 
whether we also need to check whether `bucketSpec` is the same? If not, should 
we leave a TODO here; otherwise, we might forget it when we support 
`bucketSpec` for Hive serde tables. 


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

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-09-08 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r78104317
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -408,7 +411,9 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
   tableDefinition
 }
 
-if (DDLUtils.isDatasourceTable(withStatsProps)) {
+if (tableDefinition.tableType == VIEW) {
+  client.alterTable(withStatsProps)
+} else {
   val oldDef = client.getTable(db, withStatsProps.identifier.table)
   // 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
--- End diff --

We need to update the comment and variable name `oldDataSourceProps` below. 
It also affects Hive serde tables after this PR.


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

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-09-06 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r77636703
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -413,29 +415,48 @@ private[spark] class HiveExternalCatalog(client: 
HiveClient, hadoopConf: Configu
   table
 } else {
   getProviderFromTableProperties(table).map { provider =>
-assert(provider != "hive", "Hive serde table should not save 
provider in table properties.")
-// SPARK-15269: Persisted data source tables always store the 
location URI as a storage
-// property named "path" instead of standard Hive `dataLocation`, 
because Hive only
-// allows directory paths as location URIs while Spark SQL data 
source tables also
-// allows file paths. So the standard Hive `dataLocation` is 
meaningless for Spark SQL
-// data source tables.
-// Spark SQL may also save external data source in Hive compatible 
format when
-// possible, so that these tables can be directly accessed by 
Hive. For these tables,
-// `dataLocation` is still necessary. Here we also check for input 
format because only
-// these Hive compatible tables set this field.
-val storage = if (table.tableType == EXTERNAL && 
table.storage.inputFormat.isEmpty) {
-  table.storage.copy(locationUri = None)
+if (provider == "hive") {
+  val schemaFromTableProps = getSchemaFromTableProperties(table)
+  if 
(DataType.equalsIgnoreCaseAndNullability(schemaFromTableProps, table.schema)) {
+table.copy(
+  schema = schemaFromTableProps,
+  provider = Some(provider),
+  partitionColumnNames = 
getPartitionColumnsFromTableProperties(table),
+  bucketSpec = getBucketSpecFromTableProperties(table),
--- End diff --

yea, it should always be `None`, as we disallow bucketed hive tables in 
public API. However, we do write bucket spec into table properties, so 
logically we can support it. I also write a test for it: 
https://github.com/apache/spark/pull/14750/files#diff-8c4108666a6639034f0ddbfa075bcb37R290


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

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-08-29 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r76709732
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -413,29 +415,48 @@ private[spark] class HiveExternalCatalog(client: 
HiveClient, hadoopConf: Configu
   table
 } else {
   getProviderFromTableProperties(table).map { provider =>
-assert(provider != "hive", "Hive serde table should not save 
provider in table properties.")
-// SPARK-15269: Persisted data source tables always store the 
location URI as a storage
-// property named "path" instead of standard Hive `dataLocation`, 
because Hive only
-// allows directory paths as location URIs while Spark SQL data 
source tables also
-// allows file paths. So the standard Hive `dataLocation` is 
meaningless for Spark SQL
-// data source tables.
-// Spark SQL may also save external data source in Hive compatible 
format when
-// possible, so that these tables can be directly accessed by 
Hive. For these tables,
-// `dataLocation` is still necessary. Here we also check for input 
format because only
-// these Hive compatible tables set this field.
-val storage = if (table.tableType == EXTERNAL && 
table.storage.inputFormat.isEmpty) {
-  table.storage.copy(locationUri = None)
+if (provider == "hive") {
+  val schemaFromTableProps = getSchemaFromTableProperties(table)
+  if 
(DataType.equalsIgnoreCaseAndNullability(schemaFromTableProps, table.schema)) {
+table.copy(
+  schema = schemaFromTableProps,
+  provider = Some(provider),
+  partitionColumnNames = 
getPartitionColumnsFromTableProperties(table),
+  bucketSpec = getBucketSpecFromTableProperties(table),
--- End diff --

This should be always `None`, right? 

In the future, if we can support Hive bucketing tables, we also need to 
compare the bucket spec besides 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-08-29 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r76703496
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -413,29 +415,48 @@ private[spark] class HiveExternalCatalog(client: 
HiveClient, hadoopConf: Configu
   table
 } else {
   getProviderFromTableProperties(table).map { provider =>
-assert(provider != "hive", "Hive serde table should not save 
provider in table properties.")
-// SPARK-15269: Persisted data source tables always store the 
location URI as a storage
-// property named "path" instead of standard Hive `dataLocation`, 
because Hive only
-// allows directory paths as location URIs while Spark SQL data 
source tables also
-// allows file paths. So the standard Hive `dataLocation` is 
meaningless for Spark SQL
-// data source tables.
-// Spark SQL may also save external data source in Hive compatible 
format when
-// possible, so that these tables can be directly accessed by 
Hive. For these tables,
-// `dataLocation` is still necessary. Here we also check for input 
format because only
-// these Hive compatible tables set this field.
-val storage = if (table.tableType == EXTERNAL && 
table.storage.inputFormat.isEmpty) {
-  table.storage.copy(locationUri = None)
+if (provider == "hive") {
+  val schemaFromTableProps = getSchemaFromTableProperties(table)
+  if 
(DataType.equalsIgnoreCaseAndNullability(schemaFromTableProps, table.schema)) {
+table.copy(
+  schema = schemaFromTableProps,
+  provider = Some(provider),
+  partitionColumnNames = 
getPartitionColumnsFromTableProperties(table),
+  bucketSpec = getBucketSpecFromTableProperties(table),
+  properties = getOriginalTableProperties(table))
+  } else {
+// Hive metastore may change the table schema, e.g. schema 
inference. If the table
+// schema we read back is different from the one in table 
properties which was written
+// when creating table, we should respect the table schema 
from hive.
+table.copy(
+  properties = getOriginalTableProperties(table))
+  }
 } else {
-  table.storage
+  // SPARK-15269: Persisted data source tables always store the 
location URI as a storage
+  // property named "path" instead of standard Hive 
`dataLocation`, because Hive only
+  // allows directory paths as location URIs while Spark SQL data 
source tables also
+  // allows file paths. So the standard Hive `dataLocation` is 
meaningless for Spark SQL
+  // data source tables.
+  // Spark SQL may also save external data source in Hive 
compatible format when
+  // possible, so that these tables can be directly accessed by 
Hive. For these tables,
+  // `dataLocation` is still necessary. Here we also check for 
input format because only
+  // these Hive compatible tables set this field.
+  val storage = if (provider != "hive" && table.tableType == 
EXTERNAL &&
+table.storage.inputFormat.isEmpty) {
+table.storage.copy(locationUri = None)
+  } else {
+table.storage
+  }
+  table.copy(
+storage = storage,
+schema = getSchemaFromTableProperties(table),
+provider = Some(provider),
+partitionColumnNames = 
getPartitionColumnsFromTableProperties(table),
+bucketSpec = getBucketSpecFromTableProperties(table),
+properties = getOriginalTableProperties(table))
 }
-table.copy(
-  storage = storage,
-  schema = getSchemaFromTableProperties(table),
-  provider = Some(provider),
-  partitionColumnNames = 
getPartitionColumnsFromTableProperties(table),
-  bucketSpec = getBucketSpecFromTableProperties(table),
-  properties = getOriginalTableProperties(table))
   } getOrElse {
+// Hive serde tables that are created at Hive side will hit this 
branch.
--- End diff --

It sounds like we do not have any test case to cover this scenario. Do you 
want me to add 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-

[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-08-29 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r76702974
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -413,29 +415,48 @@ private[spark] class HiveExternalCatalog(client: 
HiveClient, hadoopConf: Configu
   table
 } else {
   getProviderFromTableProperties(table).map { provider =>
-assert(provider != "hive", "Hive serde table should not save 
provider in table properties.")
-// SPARK-15269: Persisted data source tables always store the 
location URI as a storage
-// property named "path" instead of standard Hive `dataLocation`, 
because Hive only
-// allows directory paths as location URIs while Spark SQL data 
source tables also
-// allows file paths. So the standard Hive `dataLocation` is 
meaningless for Spark SQL
-// data source tables.
-// Spark SQL may also save external data source in Hive compatible 
format when
-// possible, so that these tables can be directly accessed by 
Hive. For these tables,
-// `dataLocation` is still necessary. Here we also check for input 
format because only
-// these Hive compatible tables set this field.
-val storage = if (table.tableType == EXTERNAL && 
table.storage.inputFormat.isEmpty) {
-  table.storage.copy(locationUri = None)
+if (provider == "hive") {
+  val schemaFromTableProps = getSchemaFromTableProperties(table)
+  if 
(DataType.equalsIgnoreCaseAndNullability(schemaFromTableProps, table.schema)) {
+table.copy(
+  schema = schemaFromTableProps,
+  provider = Some(provider),
+  partitionColumnNames = 
getPartitionColumnsFromTableProperties(table),
+  bucketSpec = getBucketSpecFromTableProperties(table),
+  properties = getOriginalTableProperties(table))
+  } else {
+// Hive metastore may change the table schema, e.g. schema 
inference. If the table
+// schema we read back is different from the one in table 
properties which was written
+// when creating table, we should respect the table schema 
from hive.
+table.copy(
+  properties = getOriginalTableProperties(table))
+  }
 } else {
-  table.storage
+  // SPARK-15269: Persisted data source tables always store the 
location URI as a storage
+  // property named "path" instead of standard Hive 
`dataLocation`, because Hive only
+  // allows directory paths as location URIs while Spark SQL data 
source tables also
+  // allows file paths. So the standard Hive `dataLocation` is 
meaningless for Spark SQL
+  // data source tables.
+  // Spark SQL may also save external data source in Hive 
compatible format when
+  // possible, so that these tables can be directly accessed by 
Hive. For these tables,
+  // `dataLocation` is still necessary. Here we also check for 
input format because only
+  // these Hive compatible tables set this field.
+  val storage = if (provider != "hive" && table.tableType == 
EXTERNAL &&
--- End diff --

`provider != "hive"` is always true, 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-08-29 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r76699427
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
 ---
@@ -373,7 +373,7 @@ case class InsertIntoTable(
   // Note: The parser (visitPartitionSpec in AstBuilder) already turns
   // keys in partition to their lowercase forms.
   val staticPartCols = partition.filter(_._2.isDefined).keySet
-  Some(table.output.filterNot(a => staticPartCols.contains(a.name)))
+  Some(table.output.filterNot(a => 
staticPartCols.contains(a.name.toLowerCase)))
--- End diff --

If `visitPartitionSpec` change the keys in partition spec to lower case, it 
sounds we are having multiple bugs in name comparison for partition 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-08-29 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14750#discussion_r76697635
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -166,15 +165,19 @@ private[spark] class HiveExternalCatalog(client: 
HiveClient, hadoopConf: Configu
 requireDbExists(db)
 verifyTableProperties(tableDefinition)
 
-// Before saving data source table metadata into Hive metastore, we 
should:
-//  1. Put table schema, partition column names and bucket 
specification in table properties.
-//  2. Check if this table is hive compatible
-//2.1  If it's not hive compatible, set schema, partition columns 
and bucket spec to empty
-// and save table metadata to Hive.
-//2.1  If it's hive compatible, set serde information in table 
metadata and try to save
-// it to Hive. If it fails, treat it as not hive compatible 
and go back to 2.1
-if (DDLUtils.isDatasourceTable(tableDefinition)) {
-  // data source table always have a provider, it's guaranteed by 
`DDLUtils.isDatasourceTable`.
+if (tableDefinition.tableType == VIEW) {
+  client.createTable(tableDefinition, ignoreIfExists)
+} else {
+  // To work around some hive metastore issues, e.g. not 
case-preserving, bad decimal type
+  // support, no column nullability, etc., we have some pretreatments 
before saving
+  // data source/Hive serde table metadata into Hive metastore:
+  //  1. Put provider, table schema, partition column names and bucket 
specification in table
+  // properties.
+  //  2. Check if this table is hive compatible
+  //2.1  If it's not hive compatible, set schema, partition 
columns and bucket spec to empty
+  // before save table metadata to Hive.
+  //2.1  If it's hive compatible, set serde information in table 
metadata and try to save
--- End diff --

`2.1` -> `2.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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14750: [SPARK-17183][SQL] put hive serde table schema to...

2016-08-22 Thread cloud-fan
GitHub user cloud-fan opened a pull request:

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

[SPARK-17183][SQL] put hive serde table schema to table properties like 
data source table

## What changes were proposed in this pull request?

For data source tables, we will put its table schema, partition columns, 
etc. to table properties, to work around some hive metastore issues, e.g. not 
case-preserving, bad decimal type support, etc.

We should also do this for hive serde tables, to reduce the difference 
between hive serde tables and data source tables, e.g. column names should be 
case preserving.

## How was this patch tested?

existing tests, and a new test in `HiveExternalCatalog`

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

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

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

https://github.com/apache/spark/pull/14750.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 #14750


commit 167fd433c9f64f179eaf5d506d6e1f5283320b13
Author: Wenchen Fan 
Date:   2016-08-22T12:51:08Z

put hive serde table schema to table properties like data source 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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