aboutsummaryrefslogtreecommitdiff
path: root/sql/hive
diff options
context:
space:
mode:
authorDongjoon Hyun <dongjoon@apache.org>2016-09-29 15:30:18 -0700
committerHerman van Hovell <hvanhovell@databricks.com>2016-09-29 15:30:18 -0700
commit4ecc648ad713f9d618adf0406b5d39981779059d (patch)
treed9dbfa340c81ebce37ae2d593aa2ce4302367bf0 /sql/hive
parent566d7f28275f90f7b9bed6a75e90989ad0c59931 (diff)
downloadspark-4ecc648ad713f9d618adf0406b5d39981779059d.tar.gz
spark-4ecc648ad713f9d618adf0406b5d39981779059d.tar.bz2
spark-4ecc648ad713f9d618adf0406b5d39981779059d.zip
[SPARK-17612][SQL] Support `DESCRIBE table PARTITION` SQL syntax
## What changes were proposed in this pull request? This PR implements `DESCRIBE table PARTITION` SQL Syntax again. It was supported until Spark 1.6.2, but was dropped since 2.0.0. **Spark 1.6.2** ```scala scala> sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)") res1: org.apache.spark.sql.DataFrame = [result: string] scala> sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)") res2: org.apache.spark.sql.DataFrame = [result: string] scala> sql("DESC partitioned_table PARTITION (c='Us', d=1)").show(false) +----------------------------------------------------------------+ |result | +----------------------------------------------------------------+ |a string | |b int | |c string | |d string | | | |# Partition Information | |# col_name data_type comment | | | |c string | |d string | +----------------------------------------------------------------+ ``` **Spark 2.0** - **Before** ```scala scala> sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)") res0: org.apache.spark.sql.DataFrame = [] scala> sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)") res1: org.apache.spark.sql.DataFrame = [] scala> sql("DESC partitioned_table PARTITION (c='Us', d=1)").show(false) org.apache.spark.sql.catalyst.parser.ParseException: Unsupported SQL statement ``` - **After** ```scala scala> sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)") res0: org.apache.spark.sql.DataFrame = [] scala> sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)") res1: org.apache.spark.sql.DataFrame = [] scala> sql("DESC partitioned_table PARTITION (c='Us', d=1)").show(false) +-----------------------+---------+-------+ |col_name |data_type|comment| +-----------------------+---------+-------+ |a |string |null | |b |int |null | |c |string |null | |d |string |null | |# Partition Information| | | |# col_name |data_type|comment| |c |string |null | |d |string |null | +-----------------------+---------+-------+ scala> sql("DESC EXTENDED partitioned_table PARTITION (c='Us', d=1)").show(100,false) +-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+---------+-------+ |col_name |data_type|comment| +-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+---------+-------+ |a |string |null | |b |int |null | |c |string |null | |d |string |null | |# Partition Information | | | |# col_name |data_type|comment| |c |string |null | |d |string |null | | | | | |Detailed Partition Information CatalogPartition( Partition Values: [Us, 1] Storage(Location: file:/Users/dhyun/SPARK-17612-DESC-PARTITION/spark-warehouse/partitioned_table/c=Us/d=1, InputFormat: org.apache.hadoop.mapred.TextInputFormat, OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, Serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, Properties: [serialization.format=1]) Partition Parameters:{transient_lastDdlTime=1475001066})| | | +-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+---------+-------+ scala> sql("DESC FORMATTED partitioned_table PARTITION (c='Us', d=1)").show(100,false) +--------------------------------+---------------------------------------------------------------------------------------+-------+ |col_name |data_type |comment| +--------------------------------+---------------------------------------------------------------------------------------+-------+ |a |string |null | |b |int |null | |c |string |null | |d |string |null | |# Partition Information | | | |# col_name |data_type |comment| |c |string |null | |d |string |null | | | | | |# Detailed Partition Information| | | |Partition Value: |[Us, 1] | | |Database: |default | | |Table: |partitioned_table | | |Location: |file:/Users/dhyun/SPARK-17612-DESC-PARTITION/spark-warehouse/partitioned_table/c=Us/d=1| | |Partition Parameters: | | | | transient_lastDdlTime |1475001066 | | | | | | |# Storage Information | | | |SerDe Library: |org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe | | |InputFormat: |org.apache.hadoop.mapred.TextInputFormat | | |OutputFormat: |org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat | | |Compressed: |No | | |Storage Desc Parameters: | | | | serialization.format |1 | | +--------------------------------+---------------------------------------------------------------------------------------+-------+ ``` ## How was this patch tested? Pass the Jenkins tests with a new testcase. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #15168 from dongjoon-hyun/SPARK-17612.
Diffstat (limited to 'sql/hive')
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala77
1 files changed, 76 insertions, 1 deletions
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 dc4d099f0f..6c77a0deb5 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
@@ -26,7 +26,7 @@ import org.apache.hadoop.fs.Path
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, FunctionRegistry}
+import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, FunctionRegistry, NoSuchPartitionException}
import org.apache.spark.sql.catalyst.catalog.CatalogTableType
import org.apache.spark.sql.catalyst.parser.ParseException
import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation}
@@ -341,6 +341,81 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
}
}
+ test("describe partition") {
+ withTable("partitioned_table") {
+ sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)")
+ sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)")
+
+ checkKeywordsExist(sql("DESC partitioned_table PARTITION (c='Us', d=1)"),
+ "# Partition Information",
+ "# col_name")
+
+ checkKeywordsExist(sql("DESC EXTENDED partitioned_table PARTITION (c='Us', d=1)"),
+ "# Partition Information",
+ "# col_name",
+ "Detailed Partition Information CatalogPartition(",
+ "Partition Values: [Us, 1]",
+ "Storage(Location:",
+ "Partition Parameters")
+
+ checkKeywordsExist(sql("DESC FORMATTED partitioned_table PARTITION (c='Us', d=1)"),
+ "# Partition Information",
+ "# col_name",
+ "# Detailed Partition Information",
+ "Partition Value:",
+ "Database:",
+ "Table:",
+ "Location:",
+ "Partition Parameters:",
+ "# Storage Information")
+ }
+ }
+
+ test("describe partition - error handling") {
+ withTable("partitioned_table", "datasource_table") {
+ sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)")
+ sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)")
+
+ val m = intercept[NoSuchPartitionException] {
+ sql("DESC partitioned_table PARTITION (c='Us', d=2)")
+ }.getMessage()
+ assert(m.contains("Partition not found in table"))
+
+ val m2 = intercept[AnalysisException] {
+ sql("DESC partitioned_table PARTITION (c='Us')")
+ }.getMessage()
+ assert(m2.contains("Partition spec is invalid"))
+
+ val m3 = intercept[ParseException] {
+ sql("DESC partitioned_table PARTITION (c='Us', d)")
+ }.getMessage()
+ assert(m3.contains("PARTITION specification is incomplete: `d`"))
+
+ spark
+ .range(1).select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd).write
+ .partitionBy("d")
+ .saveAsTable("datasource_table")
+ val m4 = intercept[AnalysisException] {
+ sql("DESC datasource_table PARTITION (d=2)")
+ }.getMessage()
+ assert(m4.contains("DESC PARTITION is not allowed on a datasource table"))
+
+ val m5 = intercept[AnalysisException] {
+ spark.range(10).select('id as 'a, 'id as 'b).createTempView("view1")
+ sql("DESC view1 PARTITION (c='Us', d=1)")
+ }.getMessage()
+ assert(m5.contains("DESC PARTITION is not allowed on a temporary view"))
+
+ withView("permanent_view") {
+ val m = intercept[AnalysisException] {
+ sql("CREATE VIEW permanent_view AS SELECT * FROM partitioned_table")
+ sql("DESC permanent_view PARTITION (c='Us', d=1)")
+ }.getMessage()
+ assert(m.contains("DESC PARTITION is not allowed on a view"))
+ }
+ }
+ }
+
test("SPARK-5371: union with null and sum") {
val df = Seq((1, 1)).toDF("c1", "c2")
df.createOrReplaceTempView("table1")