Skip to content

Commit 08c781c

Browse files
sameeragarwalyhuai
authored andcommitted
[SPARK-12682][SQL] Add support for (optionally) not storing tables in hive metadata format
This PR adds a new table option (`skip_hive_metadata`) that'd allow the user to skip storing the table metadata in hive metadata format. While this could be useful in general, the specific use-case for this change is that Hive doesn't handle wide schemas well (see https://issues.apache.org/jira/browse/SPARK-12682 and https://issues.apache.org/jira/browse/SPARK-6024) which in turn prevents such tables from being queried in SparkSQL. Author: Sameer Agarwal <[email protected]> Closes #10826 from sameeragarwal/skip-hive-metadata.
1 parent ae0309a commit 08c781c

File tree

2 files changed

+39
-0
lines changed

2 files changed

+39
-0
lines changed

sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -327,7 +327,14 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive
327327

328328
// TODO: Support persisting partitioned data source relations in Hive compatible format
329329
val qualifiedTableName = tableIdent.quotedString
330+
val skipHiveMetadata = options.getOrElse("skipHiveMetadata", "false").toBoolean
330331
val (hiveCompatibleTable, logMessage) = (maybeSerDe, dataSource.relation) match {
332+
case _ if skipHiveMetadata =>
333+
val message =
334+
s"Persisting partitioned data source relation $qualifiedTableName into " +
335+
"Hive metastore in Spark SQL specific format, which is NOT compatible with Hive."
336+
(None, message)
337+
331338
case (Some(serde), relation: HadoopFsRelation)
332339
if relation.paths.length == 1 && relation.partitionColumns.isEmpty =>
333340
val hiveTable = newHiveCompatibleMetastoreTable(relation, serde)

sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala

Lines changed: 32 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -900,4 +900,36 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv
900900
sqlContext.sql("""use default""")
901901
sqlContext.sql("""drop database if exists testdb8156 CASCADE""")
902902
}
903+
904+
test("skip hive metadata on table creation") {
905+
val schema = StructType((1 to 5).map(i => StructField(s"c_$i", StringType)))
906+
907+
catalog.createDataSourceTable(
908+
tableIdent = TableIdentifier("not_skip_hive_metadata"),
909+
userSpecifiedSchema = Some(schema),
910+
partitionColumns = Array.empty[String],
911+
bucketSpec = None,
912+
provider = "parquet",
913+
options = Map("path" -> "just a dummy path", "skipHiveMetadata" -> "false"),
914+
isExternal = false)
915+
916+
// As a proxy for verifying that the table was stored in Hive compatible format, we verify that
917+
// each column of the table is of native type StringType.
918+
assert(catalog.client.getTable("default", "not_skip_hive_metadata").schema
919+
.forall(column => HiveMetastoreTypes.toDataType(column.hiveType) == StringType))
920+
921+
catalog.createDataSourceTable(
922+
tableIdent = TableIdentifier("skip_hive_metadata"),
923+
userSpecifiedSchema = Some(schema),
924+
partitionColumns = Array.empty[String],
925+
bucketSpec = None,
926+
provider = "parquet",
927+
options = Map("path" -> "just a dummy path", "skipHiveMetadata" -> "true"),
928+
isExternal = false)
929+
930+
// As a proxy for verifying that the table was stored in SparkSQL format, we verify that
931+
// the table has a column type as array of StringType.
932+
assert(catalog.client.getTable("default", "skip_hive_metadata").schema
933+
.forall(column => HiveMetastoreTypes.toDataType(column.hiveType) == ArrayType(StringType)))
934+
}
903935
}

0 commit comments

Comments
 (0)