Skip to content

Commit ff4d5a2

Browse files
yhuairxin
authored andcommitted
[SPARK-2177][SQL] describe table result contains only one column
``` scala> hql("describe src").collect().foreach(println) [key string None ] [value string None ] ``` The result should contain 3 columns instead of one. This screws up JDBC or even the downstream consumer of the Scala/Java/Python APIs. I am providing a workaround. We handle a subset of describe commands in Spark SQL, which are defined by ... ``` DESCRIBE [EXTENDED] [db_name.]table_name ``` All other cases are treated as Hive native commands. Also, if we upgrade Hive to 0.13, we need to check the results of context.sessionState.isHiveServerQuery() to determine how to split the result. This method is introduced by https://issues.apache.org/jira/browse/HIVE-4545. We may want to set Hive to use JsonMetaDataFormatter for the output of a DDL statement (`set hive.ddl.output.format=json` introduced by https://issues.apache.org/jira/browse/HIVE-2822). The link to JIRA: https://issues.apache.org/jira/browse/SPARK-2177 Author: Yin Huai <[email protected]> Closes #1118 from yhuai/SPARK-2177 and squashes the following commits: fd2534c [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-2177 b9b9aa5 [Yin Huai] rxin's comments. e7c4e72 [Yin Huai] Fix unit test. 656b068 [Yin Huai] 100 characters. 6387217 [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-2177 8003cf3 [Yin Huai] Generate strings with the format like Hive for unit tests. 9787fff [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-2177 440c5af [Yin Huai] rxin's comments. f1a417e [Yin Huai] Update doc. 83adb2f [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-2177 366f891 [Yin Huai] Add describe command. 74bd1d4 [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-2177 342fdf7 [Yin Huai] Split to up to 3 parts. 725e88c [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-2177 bb8bbef [Yin Huai] Split every string in the result of a describe command. (cherry picked from commit f397e92) Signed-off-by: Reynold Xin <[email protected]>
1 parent cc2e4ca commit ff4d5a2

File tree

9 files changed

+294
-31
lines changed

9 files changed

+294
-31
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -60,3 +60,19 @@ case class ExplainCommand(plan: LogicalPlan) extends Command {
6060
* Returned for the "CACHE TABLE tableName" and "UNCACHE TABLE tableName" command.
6161
*/
6262
case class CacheCommand(tableName: String, doCache: Boolean) extends Command
63+
64+
/**
65+
* Returned for the "DESCRIBE [EXTENDED] [dbName.]tableName" command.
66+
* @param table The table to be described.
67+
* @param isExtended True if "DESCRIBE EXTENDED" is used. Otherwise, false.
68+
* It is effective only when the table is a Hive table.
69+
*/
70+
case class DescribeCommand(
71+
table: LogicalPlan,
72+
isExtended: Boolean) extends Command {
73+
override def output = Seq(
74+
// Column names are based on Hive.
75+
BoundReference(0, AttributeReference("col_name", StringType, nullable = false)()),
76+
BoundReference(1, AttributeReference("data_type", StringType, nullable = false)()),
77+
BoundReference(2, AttributeReference("comment", StringType, nullable = false)()))
78+
}

sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala

Lines changed: 21 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -121,3 +121,24 @@ case class CacheCommand(tableName: String, doCache: Boolean)(@transient context:
121121

122122
override def output: Seq[Attribute] = Seq.empty
123123
}
124+
125+
/**
126+
* :: DeveloperApi ::
127+
*/
128+
@DeveloperApi
129+
case class DescribeCommand(child: SparkPlan, output: Seq[Attribute])(
130+
@transient context: SQLContext)
131+
extends LeafNode with Command {
132+
133+
override protected[sql] lazy val sideEffectResult: Seq[(String, String, String)] = {
134+
Seq(("# Registered as a temporary table", null, null)) ++
135+
child.output.map(field => (field.name, field.dataType.toString, null))
136+
}
137+
138+
override def execute(): RDD[Row] = {
139+
val rows = sideEffectResult.map {
140+
case (name, dataType, comment) => new GenericRow(Array[Any](name, dataType, comment))
141+
}
142+
context.sparkContext.parallelize(rows, 1)
143+
}
144+
}

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

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -38,6 +38,7 @@ import org.apache.spark.sql.catalyst.plans.logical._
3838
import org.apache.spark.sql.catalyst.types._
3939
import org.apache.spark.sql.execution.QueryExecutionException
4040
import org.apache.spark.sql.execution.{Command => PhysicalCommand}
41+
import org.apache.spark.sql.hive.execution.DescribeHiveTableCommand
4142

4243
/**
4344
* Starts up an instance of hive where metadata is stored locally. An in-process metadata data is
@@ -291,6 +292,10 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
291292
* execution is simply passed back to Hive.
292293
*/
293294
def stringResult(): Seq[String] = executedPlan match {
295+
case describeHiveTableCommand: DescribeHiveTableCommand =>
296+
// If it is a describe command for a Hive table, we want to have the output format
297+
// be similar with Hive.
298+
describeHiveTableCommand.hiveString
294299
case command: PhysicalCommand =>
295300
command.sideEffectResult.map(_.toString)
296301

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

Lines changed: 54 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -52,7 +52,6 @@ private[hive] case class AddFile(filePath: String) extends Command
5252
private[hive] object HiveQl {
5353
protected val nativeCommands = Seq(
5454
"TOK_DESCFUNCTION",
55-
"TOK_DESCTABLE",
5655
"TOK_DESCDATABASE",
5756
"TOK_SHOW_TABLESTATUS",
5857
"TOK_SHOWDATABASES",
@@ -120,6 +119,12 @@ private[hive] object HiveQl {
120119
"TOK_SWITCHDATABASE"
121120
)
122121

122+
// Commands that we do not need to explain.
123+
protected val noExplainCommands = Seq(
124+
"TOK_CREATETABLE",
125+
"TOK_DESCTABLE"
126+
) ++ nativeCommands
127+
123128
/**
124129
* A set of implicit transformations that allow Hive ASTNodes to be rewritten by transformations
125130
* similar to [[catalyst.trees.TreeNode]].
@@ -362,13 +367,20 @@ private[hive] object HiveQl {
362367
}
363368
}
364369

370+
protected def extractDbNameTableName(tableNameParts: Node): (Option[String], String) = {
371+
val (db, tableName) =
372+
tableNameParts.getChildren.map { case Token(part, Nil) => cleanIdentifier(part) } match {
373+
case Seq(tableOnly) => (None, tableOnly)
374+
case Seq(databaseName, table) => (Some(databaseName), table)
375+
}
376+
377+
(db, tableName)
378+
}
379+
365380
protected def nodeToPlan(node: Node): LogicalPlan = node match {
366381
// Just fake explain for any of the native commands.
367-
case Token("TOK_EXPLAIN", explainArgs) if nativeCommands contains explainArgs.head.getText =>
368-
ExplainCommand(NoRelation)
369-
// Create tables aren't native commands due to CTAS queries, but we still don't need to
370-
// explain them.
371-
case Token("TOK_EXPLAIN", explainArgs) if explainArgs.head.getText == "TOK_CREATETABLE" =>
382+
case Token("TOK_EXPLAIN", explainArgs)
383+
if noExplainCommands.contains(explainArgs.head.getText) =>
372384
ExplainCommand(NoRelation)
373385
case Token("TOK_EXPLAIN", explainArgs) =>
374386
// Ignore FORMATTED if present.
@@ -377,6 +389,39 @@ private[hive] object HiveQl {
377389
// TODO: support EXTENDED?
378390
ExplainCommand(nodeToPlan(query))
379391

392+
case Token("TOK_DESCTABLE", describeArgs) =>
393+
// Reference: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL
394+
val Some(tableType) :: formatted :: extended :: pretty :: Nil =
395+
getClauses(Seq("TOK_TABTYPE", "FORMATTED", "EXTENDED", "PRETTY"), describeArgs)
396+
if (formatted.isDefined || pretty.isDefined) {
397+
// FORMATTED and PRETTY are not supported and this statement will be treated as
398+
// a Hive native command.
399+
NativePlaceholder
400+
} else {
401+
tableType match {
402+
case Token("TOK_TABTYPE", nameParts) if nameParts.size == 1 => {
403+
nameParts.head match {
404+
case Token(".", dbName :: tableName :: Nil) =>
405+
// It is describing a table with the format like "describe db.table".
406+
// TODO: Actually, a user may mean tableName.columnName. Need to resolve this issue.
407+
val (db, tableName) = extractDbNameTableName(nameParts.head)
408+
DescribeCommand(
409+
UnresolvedRelation(db, tableName, None), extended.isDefined)
410+
case Token(".", dbName :: tableName :: colName :: Nil) =>
411+
// It is describing a column with the format like "describe db.table column".
412+
NativePlaceholder
413+
case tableName =>
414+
// It is describing a table with the format like "describe table".
415+
DescribeCommand(
416+
UnresolvedRelation(None, tableName.getText, None),
417+
extended.isDefined)
418+
}
419+
}
420+
// All other cases.
421+
case _ => NativePlaceholder
422+
}
423+
}
424+
380425
case Token("TOK_CREATETABLE", children)
381426
if children.collect { case t@Token("TOK_QUERY", _) => t }.nonEmpty =>
382427
// TODO: Parse other clauses.
@@ -414,11 +459,8 @@ private[hive] object HiveQl {
414459
s"Unhandled clauses: ${notImplemented.flatten.map(dumpTree(_)).mkString("\n")}")
415460
}
416461

417-
val (db, tableName) =
418-
tableNameParts.getChildren.map{ case Token(part, Nil) => cleanIdentifier(part)} match {
419-
case Seq(tableOnly) => (None, tableOnly)
420-
case Seq(databaseName, table) => (Some(databaseName), table)
421-
}
462+
val (db, tableName) = extractDbNameTableName(tableNameParts)
463+
422464
InsertIntoCreatedTable(db, tableName, nodeToPlan(query))
423465

424466
// If its not a "CREATE TABLE AS" like above then just pass it back to hive as a native command.
@@ -736,11 +778,7 @@ private[hive] object HiveQl {
736778
val Some(tableNameParts) :: partitionClause :: Nil =
737779
getClauses(Seq("TOK_TABNAME", "TOK_PARTSPEC"), tableArgs)
738780

739-
val (db, tableName) =
740-
tableNameParts.getChildren.map{ case Token(part, Nil) => cleanIdentifier(part)} match {
741-
case Seq(tableOnly) => (None, tableOnly)
742-
case Seq(databaseName, table) => (Some(databaseName), table)
743-
}
781+
val (db, tableName) = extractDbNameTableName(tableNameParts)
744782

745783
val partitionKeys = partitionClause.map(_.getChildren.map {
746784
// Parse partitions. We also make keys case insensitive.

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

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -81,6 +81,16 @@ private[hive] trait HiveStrategies {
8181
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
8282
case logical.NativeCommand(sql) =>
8383
NativeCommand(sql, plan.output)(context) :: Nil
84+
case describe: logical.DescribeCommand => {
85+
val resolvedTable = context.executePlan(describe.table).analyzed
86+
resolvedTable match {
87+
case t: MetastoreRelation =>
88+
Seq(DescribeHiveTableCommand(
89+
t, describe.output, describe.isExtended)(context))
90+
case o: LogicalPlan =>
91+
Seq(DescribeCommand(planLater(o), describe.output)(context))
92+
}
93+
}
8494
case _ => Nil
8595
}
8696
}

sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/hiveOperators.scala

Lines changed: 60 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -20,8 +20,10 @@ package org.apache.spark.sql.hive.execution
2020
import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar}
2121
import org.apache.hadoop.hive.conf.HiveConf
2222
import org.apache.hadoop.hive.metastore.MetaStoreUtils
23+
import org.apache.hadoop.hive.metastore.api.FieldSchema
2324
import org.apache.hadoop.hive.ql.Context
2425
import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Hive}
26+
import org.apache.hadoop.hive.ql.metadata.formatting.MetaDataFormatUtils
2527
import org.apache.hadoop.hive.ql.plan.{TableDesc, FileSinkDesc}
2628
import org.apache.hadoop.hive.serde.serdeConstants
2729
import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption
@@ -462,3 +464,61 @@ case class NativeCommand(
462464

463465
override def otherCopyArgs = context :: Nil
464466
}
467+
468+
/**
469+
* :: DeveloperApi ::
470+
*/
471+
@DeveloperApi
472+
case class DescribeHiveTableCommand(
473+
table: MetastoreRelation,
474+
output: Seq[Attribute],
475+
isExtended: Boolean)(
476+
@transient context: HiveContext)
477+
extends LeafNode with Command {
478+
479+
// Strings with the format like Hive. It is used for result comparison in our unit tests.
480+
lazy val hiveString: Seq[String] = {
481+
val alignment = 20
482+
val delim = "\t"
483+
484+
sideEffectResult.map {
485+
case (name, dataType, comment) =>
486+
String.format("%-" + alignment + "s", name) + delim +
487+
String.format("%-" + alignment + "s", dataType) + delim +
488+
String.format("%-" + alignment + "s", Option(comment).getOrElse("None"))
489+
}
490+
}
491+
492+
override protected[sql] lazy val sideEffectResult: Seq[(String, String, String)] = {
493+
// Trying to mimic the format of Hive's output. But not exactly the same.
494+
var results: Seq[(String, String, String)] = Nil
495+
496+
val columns: Seq[FieldSchema] = table.hiveQlTable.getCols
497+
val partitionColumns: Seq[FieldSchema] = table.hiveQlTable.getPartCols
498+
results ++= columns.map(field => (field.getName, field.getType, field.getComment))
499+
if (!partitionColumns.isEmpty) {
500+
val partColumnInfo =
501+
partitionColumns.map(field => (field.getName, field.getType, field.getComment))
502+
results ++=
503+
partColumnInfo ++
504+
Seq(("# Partition Information", "", "")) ++
505+
Seq((s"# ${output.get(0).name}", output.get(1).name, output.get(2).name)) ++
506+
partColumnInfo
507+
}
508+
509+
if (isExtended) {
510+
results ++= Seq(("Detailed Table Information", table.hiveQlTable.getTTable.toString, ""))
511+
}
512+
513+
results
514+
}
515+
516+
override def execute(): RDD[Row] = {
517+
val rows = sideEffectResult.map {
518+
case (name, dataType, comment) => new GenericRow(Array[Any](name, dataType, comment))
519+
}
520+
context.sparkContext.parallelize(rows, 1)
521+
}
522+
523+
override def otherCopyArgs = context :: Nil
524+
}

sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala

Lines changed: 23 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -144,6 +144,12 @@ abstract class HiveComparisonTest
144144
case _: SetCommand => Seq("0")
145145
case _: LogicalNativeCommand => answer.filterNot(nonDeterministicLine).filterNot(_ == "")
146146
case _: ExplainCommand => answer
147+
case _: DescribeCommand =>
148+
// Filter out non-deterministic lines and lines which do not have actual results but
149+
// can introduce problems because of the way Hive formats these lines.
150+
// Then, remove empty lines. Do not sort the results.
151+
answer.filterNot(
152+
r => nonDeterministicLine(r) || ignoredLine(r)).map(_.trim).filterNot(_ == "")
147153
case plan => if (isSorted(plan)) answer else answer.sorted
148154
}
149155
orderedAnswer.map(cleanPaths)
@@ -169,6 +175,16 @@ abstract class HiveComparisonTest
169175
protected def nonDeterministicLine(line: String) =
170176
nonDeterministicLineIndicators.exists(line contains _)
171177

178+
// This list contains indicators for those lines which do not have actual results and we
179+
// want to ignore.
180+
lazy val ignoredLineIndicators = Seq(
181+
"# Partition Information",
182+
"# col_name"
183+
)
184+
185+
protected def ignoredLine(line: String) =
186+
ignoredLineIndicators.exists(line contains _)
187+
172188
/**
173189
* Removes non-deterministic paths from `str` so cached answers will compare correctly.
174190
*/
@@ -329,11 +345,17 @@ abstract class HiveComparisonTest
329345

330346
if ((!hiveQuery.logical.isInstanceOf[ExplainCommand]) && preparedHive != catalyst) {
331347

332-
val hivePrintOut = s"== HIVE - ${hive.size} row(s) ==" +: preparedHive
348+
val hivePrintOut = s"== HIVE - ${preparedHive.size} row(s) ==" +: preparedHive
333349
val catalystPrintOut = s"== CATALYST - ${catalyst.size} row(s) ==" +: catalyst
334350

335351
val resultComparison = sideBySide(hivePrintOut, catalystPrintOut).mkString("\n")
336352

353+
println("hive output")
354+
hive.foreach(println)
355+
356+
println("catalyst printout")
357+
catalyst.foreach(println)
358+
337359
if (recomputeCache) {
338360
logger.warn(s"Clearing cache files for failed test $testCaseName")
339361
hiveCacheFiles.foreach(_.delete())

sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala

Lines changed: 11 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -78,7 +78,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
7878
"alter_merge",
7979
"alter_concatenate_indexed_table",
8080
"protectmode2",
81-
"describe_table",
81+
//"describe_table",
8282
"describe_comment_nonascii",
8383
"udf5",
8484
"udf_java_method",
@@ -177,7 +177,16 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
177177
// After stop taking the `stringOrError` route, exceptions are thrown from these cases.
178178
// See SPARK-2129 for details.
179179
"join_view",
180-
"mergejoins_mixed"
180+
"mergejoins_mixed",
181+
182+
// Returning the result of a describe state as a JSON object is not supported.
183+
"describe_table_json",
184+
"describe_database_json",
185+
"describe_formatted_view_partitioned_json",
186+
187+
// Hive returns the results of describe as plain text. Comments with multiple lines
188+
// introduce extra lines in the Hive results, which make the result comparison fail.
189+
"describe_comment_indent"
181190
)
182191

183192
/**
@@ -292,11 +301,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
292301
"default_partition_name",
293302
"delimiter",
294303
"desc_non_existent_tbl",
295-
"describe_comment_indent",
296-
"describe_database_json",
297304
"describe_formatted_view_partitioned",
298-
"describe_formatted_view_partitioned_json",
299-
"describe_table_json",
300305
"diff_part_input_formats",
301306
"disable_file_format_check",
302307
"drop_function",

0 commit comments

Comments
 (0)