aboutsummaryrefslogtreecommitdiff
path: root/sql/hive/src/test/scala/org
diff options
context:
space:
mode:
authorwindpiger <songjun@outlook.com>2017-01-24 20:40:27 +0800
committerWenchen Fan <wenchen@databricks.com>2017-01-24 20:40:27 +0800
commit3c86fdddf4bb1eac985654f80c3c716b7ae7540b (patch)
tree9be99b014de4f96cfbd7bd674e936492f8fbc456 /sql/hive/src/test/scala/org
parentcca8680047bb2ec312ffc296a561abd5cbc8323c (diff)
downloadspark-3c86fdddf4bb1eac985654f80c3c716b7ae7540b.tar.gz
spark-3c86fdddf4bb1eac985654f80c3c716b7ae7540b.tar.bz2
spark-3c86fdddf4bb1eac985654f80c3c716b7ae7540b.zip
[SPARK-19152][SQL] DataFrameWriter.saveAsTable support hive append
## What changes were proposed in this pull request? After [SPARK-19107](https://issues.apache.org/jira/browse/SPARK-19107), we now can treat hive as a data source and create hive tables with DataFrameWriter and Catalog. However, the support is not completed, there are still some cases we do not support. This PR implement: DataFrameWriter.saveAsTable work with hive format with append mode ## How was this patch tested? unit test added Author: windpiger <songjun@outlook.com> Closes #16552 from windpiger/saveAsTableWithHiveAppend.
Diffstat (limited to 'sql/hive/src/test/scala/org')
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala13
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala36
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala17
3 files changed, 52 insertions, 14 deletions
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
index f0e2c9369b..c262095df6 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
@@ -419,12 +419,6 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv
sql(s"CREATE TABLE $tableName STORED AS SEQUENCEFILE AS SELECT 1 AS key, 'abc' AS value")
val df = sql(s"SELECT key, value FROM $tableName")
- val e = intercept[AnalysisException] {
- df.write.mode(SaveMode.Append).saveAsTable(tableName)
- }.getMessage
- assert(e.contains("Saving data in the Hive serde table default.tab1 is not supported " +
- "yet. Please use the insertInto() API as an alternative."))
-
df.write.insertInto(tableName)
checkAnswer(
sql(s"SELECT * FROM $tableName"),
@@ -1167,8 +1161,8 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv
test("create a temp view using hive") {
val tableName = "tab1"
- withTable (tableName) {
- val e = intercept[ClassNotFoundException] {
+ withTable(tableName) {
+ val e = intercept[AnalysisException] {
sql(
s"""
|CREATE TEMPORARY VIEW $tableName
@@ -1176,7 +1170,8 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv
|USING hive
""".stripMargin)
}.getMessage
- assert(e.contains("Failed to find data source: hive"))
+ assert(e.contains("Hive data source can only be used with tables, you can't use it with " +
+ "CREATE TEMP VIEW USING"))
}
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
index a77c68339d..2827183456 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
@@ -1326,7 +1326,7 @@ class HiveDDLSuite
}
test("create hive serde table with DataFrameWriter.saveAsTable") {
- withTable("t", "t2") {
+ withTable("t", "t1") {
Seq(1 -> "a").toDF("i", "j")
.write.format("hive").option("fileFormat", "avro").saveAsTable("t")
checkAnswer(spark.table("t"), Row(1, "a"))
@@ -1357,11 +1357,8 @@ class HiveDDLSuite
assert(table.storage.serde ==
Some("org.apache.hadoop.hive.serde2.avro.AvroSerDe"))
- sql("INSERT INTO t SELECT 2, 'b'")
- checkAnswer(spark.table("t"), Row(9, "x") :: Row(2, "b") :: Nil)
-
val e2 = intercept[AnalysisException] {
- Seq(1 -> "a").toDF("i", "j").write.format("hive").bucketBy(4, "i").saveAsTable("t2")
+ Seq(1 -> "a").toDF("i", "j").write.format("hive").bucketBy(4, "i").saveAsTable("t1")
}
assert(e2.message.contains("Creating bucketed Hive serde table is not supported yet"))
@@ -1372,6 +1369,35 @@ class HiveDDLSuite
}
}
+ test("append data to hive serde table") {
+ withTable("t", "t1") {
+ Seq(1 -> "a").toDF("i", "j")
+ .write.format("hive").option("fileFormat", "avro").saveAsTable("t")
+ checkAnswer(spark.table("t"), Row(1, "a"))
+
+ sql("INSERT INTO t SELECT 2, 'b'")
+ checkAnswer(spark.table("t"), Row(1, "a") :: Row(2, "b") :: Nil)
+
+ Seq(3 -> "c").toDF("i", "j")
+ .write.format("hive").mode("append").saveAsTable("t")
+ checkAnswer(spark.table("t"), Row(1, "a") :: Row(2, "b") :: Row(3, "c") :: Nil)
+
+ Seq("c" -> 3).toDF("i", "j")
+ .write.format("hive").mode("append").saveAsTable("t")
+ checkAnswer(spark.table("t"), Row(1, "a") :: Row(2, "b") :: Row(3, "c")
+ :: Row(null, "3") :: Nil)
+
+ Seq(4 -> "d").toDF("i", "j").write.saveAsTable("t1")
+
+ val e = intercept[AnalysisException] {
+ Seq(5 -> "e").toDF("i", "j")
+ .write.format("hive").mode("append").saveAsTable("t1")
+ }
+ assert(e.message.contains("The format of the existing table default.t1 is " +
+ "`ParquetFileFormat`. It doesn't match the specified format `HiveFileFormat`."))
+ }
+ }
+
test("create partitioned hive serde table as select") {
withTable("t", "t1") {
withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") {
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
index 1a28c4c84a..20f30e48ab 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
@@ -1461,6 +1461,23 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
})
}
+ test("run sql directly on files - hive") {
+ withTempPath(f => {
+ spark.range(100).toDF.write.parquet(f.getCanonicalPath)
+
+ var e = intercept[AnalysisException] {
+ sql(s"select id from hive.`${f.getCanonicalPath}`")
+ }
+ assert(e.message.contains("Unsupported data source type for direct query on files: hive"))
+
+ // data source type is case insensitive
+ e = intercept[AnalysisException] {
+ sql(s"select id from HIVE.`${f.getCanonicalPath}`")
+ }
+ assert(e.message.contains("Unsupported data source type for direct query on files: HIVE"))
+ })
+ }
+
test("SPARK-8976 Wrong Result for Rollup #1") {
checkAnswer(sql(
"SELECT count(*) AS cnt, key % 5, grouping_id() FROM src GROUP BY key%5 WITH ROLLUP"),