-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-19107][SQL] support creating hive table with DataFrameWriter and Catalog #16487
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -205,6 +205,11 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { | |
| * @since 1.4.0 | ||
| */ | ||
| def save(): Unit = { | ||
| if (source.toLowerCase == DDLUtils.HIVE_PROVIDER) { | ||
| throw new AnalysisException("Hive data source can only be used with tables, you can not " + | ||
| "write files of Hive data source directly.") | ||
| } | ||
|
|
||
| assertNotBucketed("save") | ||
| val dataSource = DataSource( | ||
| df.sparkSession, | ||
|
|
@@ -361,10 +366,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { | |
| } | ||
|
|
||
| private def saveAsTable(tableIdent: TableIdentifier): Unit = { | ||
| if (source.toLowerCase == DDLUtils.HIVE_PROVIDER) { | ||
| throw new AnalysisException("Cannot create hive serde table with saveAsTable API") | ||
| } | ||
|
|
||
| val catalog = df.sparkSession.sessionState.catalog | ||
| val tableExists = catalog.tableExists(tableIdent) | ||
| val db = tableIdent.database.getOrElse(catalog.getCurrentDatabase) | ||
|
|
@@ -385,6 +386,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { | |
| } | ||
| EliminateSubqueryAliases(catalog.lookupRelation(tableIdentWithDB)) match { | ||
| // Only do the check if the table is a data source table (the relation is a BaseRelation). | ||
| // TODO(cloud-fan): also check hive table relation here when we support overwrite mode | ||
| // for creating hive tables. | ||
|
||
| case LogicalRelation(dest: BaseRelation, _, _) if srcRelations.contains(dest) => | ||
| throw new AnalysisException( | ||
| s"Cannot overwrite table $tableName that is also being read from") | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -1169,26 +1169,6 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv | |
| } | ||
| } | ||
|
|
||
| test("save API - format hive") { | ||
| withTempDir { dir => | ||
| val path = dir.getCanonicalPath | ||
| val e = intercept[ClassNotFoundException] { | ||
| spark.range(10).write.format("hive").mode(SaveMode.Ignore).save(path) | ||
| }.getMessage | ||
| assert(e.contains("Failed to find data source: hive")) | ||
| } | ||
| } | ||
|
|
||
| test("saveAsTable API - format hive") { | ||
| val tableName = "tab1" | ||
| withTable(tableName) { | ||
| val e = intercept[AnalysisException] { | ||
| spark.range(10).write.format("hive").mode(SaveMode.Overwrite).saveAsTable(tableName) | ||
| }.getMessage | ||
| assert(e.contains("Cannot create hive serde table with saveAsTable API")) | ||
| } | ||
| } | ||
|
|
||
| test("create a temp view using hive") { | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. |
||
| val tableName = "tab1" | ||
| withTable (tableName) { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -33,6 +33,7 @@ import org.apache.spark.sql.hive.test.TestHiveSingleton | |
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION | ||
| import org.apache.spark.sql.test.SQLTestUtils | ||
| import org.apache.spark.sql.types.StructType | ||
|
|
||
| class HiveDDLSuite | ||
| extends QueryTest with SQLTestUtils with TestHiveSingleton with BeforeAndAfterEach { | ||
|
|
@@ -1289,4 +1290,80 @@ class HiveDDLSuite | |
| } | ||
| } | ||
| } | ||
|
|
||
| test("create hive serde table with Catalog") { | ||
| withTable("t") { | ||
| withTempDir { dir => | ||
| val df = spark.catalog.createExternalTable( | ||
| "t", | ||
| "hive", | ||
| new StructType().add("i", "int"), | ||
| Map("path" -> dir.getCanonicalPath, "fileFormat" -> "parquet")) | ||
|
||
| assert(df.collect().isEmpty) | ||
|
|
||
| val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) | ||
| assert(DDLUtils.isHiveTable(table)) | ||
| assert(table.storage.inputFormat == | ||
| Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat")) | ||
| assert(table.storage.outputFormat == | ||
| Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat")) | ||
| assert(table.storage.serde == | ||
| Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe")) | ||
|
|
||
| sql("INSERT INTO t SELECT 1") | ||
| checkAnswer(spark.table("t"), Row(1)) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| test("create hive serde table with DataFrameWriter.saveAsTable") { | ||
| withTable("t", "t2") { | ||
| Seq(1 -> "a").toDF("i", "j") | ||
| .write.format("hive").option("fileFormat", "avro").saveAsTable("t") | ||
| checkAnswer(spark.table("t"), Row(1, "a")) | ||
|
|
||
| val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) | ||
| assert(DDLUtils.isHiveTable(table)) | ||
| assert(table.storage.inputFormat == | ||
| Some("org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat")) | ||
| assert(table.storage.outputFormat == | ||
| Some("org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat")) | ||
| assert(table.storage.serde == | ||
| Some("org.apache.hadoop.hive.serde2.avro.AvroSerDe")) | ||
|
|
||
| sql("INSERT INTO t SELECT 2, 'b'") | ||
| checkAnswer(spark.table("t"), Row(1, "a") :: Row(2, "b") :: Nil) | ||
|
|
||
| val e = intercept[AnalysisException] { | ||
| Seq(1 -> "a").toDF("i", "j").write.format("hive").partitionBy("i").saveAsTable("t2") | ||
| } | ||
| assert(e.message.contains("A Create Table As Select (CTAS) statement is not allowed " + | ||
| "to create a partitioned table using Hive")) | ||
|
|
||
| val e2 = intercept[AnalysisException] { | ||
| Seq(1 -> "a").toDF("i", "j").write.format("hive").bucketBy(4, "i").saveAsTable("t2") | ||
| } | ||
| assert(e2.message.contains("Creating bucketed Hive serde table is not supported yet")) | ||
|
|
||
| val e3 = intercept[AnalysisException] { | ||
| spark.table("t").write.format("hive").mode("overwrite").saveAsTable("t") | ||
| } | ||
| assert(e3.message.contains( | ||
| "CTAS for hive serde tables does not support append or overwrite semantics")) | ||
| } | ||
| } | ||
|
|
||
| test("read/write files with hive data source is not allowed") { | ||
| withTempDir { dir => | ||
| val e = intercept[AnalysisException] { | ||
| spark.read.format("hive").load(dir.getAbsolutePath) | ||
| } | ||
| assert(e.message.contains("Hive data source can only be used with tables")) | ||
|
|
||
| val e2 = intercept[AnalysisException] { | ||
| Seq(1 -> "a").toDF("i", "j").write.format("hive").save(dir.getAbsolutePath) | ||
| } | ||
| assert(e2.message.contains("Hive data source can only be used with tables")) | ||
| } | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1