Skip to content

Commit 04e81e5

Browse files
rxinroygao94
authored andcommitted
[SPARK-13884][SQL] Remove DescribeCommand's dependency on LogicalPlan
## What changes were proposed in this pull request? This patch removes DescribeCommand's dependency on LogicalPlan. After this patch, DescribeCommand simply accepts a TableIdentifier. It minimizes the dependency, and blocks my next patch (removes SQLContext dependency from SparkPlanner). ## How was this patch tested? Should be covered by existing unit tests and Hive compatibility tests that run describe table. Author: Reynold Xin <[email protected]> Closes apache#11710 from rxin/SPARK-13884.
1 parent 6c2495a commit 04e81e5

File tree

6 files changed

+49
-46
lines changed

6 files changed

+49
-46
lines changed

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

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -271,15 +271,14 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly
271271
// issue.
272272
val tableIdent = TableIdentifier(
273273
cleanIdentifier(tableName), Some(cleanIdentifier(dbName)))
274-
datasources.DescribeCommand(
275-
UnresolvedRelation(tableIdent, None), isExtended = extended.isDefined)
274+
datasources.DescribeCommand(tableIdent, isExtended = extended.isDefined)
276275
case Token(dbName, Nil) :: Token(tableName, Nil) :: Token(colName, Nil) :: Nil =>
277276
// It is describing a column with the format like "describe db.table column".
278277
nodeToDescribeFallback(node)
279278
case tableName :: Nil =>
280279
// It is describing a table with the format like "describe table".
281280
datasources.DescribeCommand(
282-
UnresolvedRelation(TableIdentifier(cleanIdentifier(tableName.text)), None),
281+
TableIdentifier(cleanIdentifier(tableName.text)),
283282
isExtended = extended.isDefined)
284283
case _ =>
285284
nodeToDescribeFallback(node)

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

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -398,11 +398,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
398398
sys.error("Tables created with SQLContext must be TEMPORARY. Use a HiveContext instead.")
399399

400400
case describe @ LogicalDescribeCommand(table, isExtended) =>
401-
val resultPlan = self.sqlContext.executePlan(table).executedPlan
402-
ExecutedCommand(
403-
RunnableDescribeCommand(resultPlan, describe.output, isExtended)) :: Nil
401+
ExecutedCommand(RunnableDescribeCommand(table, describe.output, isExtended)) :: Nil
404402

405-
case logical.ShowFunctions(db, pattern) => ExecutedCommand(ShowFunctions(db, pattern)) :: Nil
403+
case logical.ShowFunctions(db, pattern) =>
404+
ExecutedCommand(ShowFunctions(db, pattern)) :: Nil
406405

407406
case logical.DescribeFunction(function, extended) =>
408407
ExecutedCommand(DescribeFunction(function, extended)) :: Nil

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

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@ import java.util.NoSuchElementException
2222
import org.apache.spark.Logging
2323
import org.apache.spark.rdd.RDD
2424
import org.apache.spark.sql.{Dataset, Row, SQLContext}
25-
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
25+
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, TableIdentifier}
2626
import org.apache.spark.sql.catalyst.errors.TreeNodeException
2727
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
2828
import org.apache.spark.sql.catalyst.plans.logical
@@ -293,13 +293,14 @@ case object ClearCacheCommand extends RunnableCommand {
293293

294294

295295
case class DescribeCommand(
296-
child: SparkPlan,
296+
table: TableIdentifier,
297297
override val output: Seq[Attribute],
298298
isExtended: Boolean)
299299
extends RunnableCommand {
300300

301301
override def run(sqlContext: SQLContext): Seq[Row] = {
302-
child.schema.fields.map { field =>
302+
val relation = sqlContext.sessionState.catalog.lookupRelation(table)
303+
relation.schema.fields.map { field =>
303304
val cmtKey = "comment"
304305
val comment = if (field.metadata.contains(cmtKey)) field.metadata.getString(cmtKey) else ""
305306
Row(field.name, field.dataType.simpleString, comment)

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -33,8 +33,9 @@ import org.apache.spark.sql.types._
3333
* It is effective only when the table is a Hive table.
3434
*/
3535
case class DescribeCommand(
36-
table: LogicalPlan,
37-
isExtended: Boolean) extends LogicalPlan with logical.Command {
36+
table: TableIdentifier,
37+
isExtended: Boolean)
38+
extends LogicalPlan with logical.Command {
3839

3940
override def children: Seq[LogicalPlan] = Seq.empty
4041

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

Lines changed: 2 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -102,18 +102,8 @@ private[hive] trait HiveStrategies {
102102
case class HiveCommandStrategy(context: HiveContext) extends Strategy {
103103
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
104104
case describe: DescribeCommand =>
105-
val resolvedTable = context.executePlan(describe.table).analyzed
106-
resolvedTable match {
107-
case t: MetastoreRelation =>
108-
ExecutedCommand(
109-
DescribeHiveTableCommand(t, describe.output, describe.isExtended)) :: Nil
110-
111-
case o: LogicalPlan =>
112-
val resultPlan = context.executePlan(o).executedPlan
113-
ExecutedCommand(RunnableDescribeCommand(
114-
resultPlan, describe.output, describe.isExtended)) :: Nil
115-
}
116-
105+
ExecutedCommand(
106+
DescribeHiveTableCommand(describe.table, describe.output, describe.isExtended)) :: Nil
117107
case _ => Nil
118108
}
119109
}

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

Lines changed: 35 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -22,42 +22,55 @@ import scala.collection.JavaConverters._
2222
import org.apache.hadoop.hive.metastore.api.FieldSchema
2323

2424
import org.apache.spark.sql.{Row, SQLContext}
25+
import org.apache.spark.sql.catalyst.TableIdentifier
2526
import org.apache.spark.sql.catalyst.expressions.Attribute
26-
import org.apache.spark.sql.execution.command.RunnableCommand
27+
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
28+
import org.apache.spark.sql.execution.command.{DescribeCommand, RunnableCommand}
2729
import org.apache.spark.sql.hive.MetastoreRelation
2830

2931
/**
3032
* Implementation for "describe [extended] table".
3133
*/
3234
private[hive]
3335
case class DescribeHiveTableCommand(
34-
table: MetastoreRelation,
36+
tableId: TableIdentifier,
3537
override val output: Seq[Attribute],
3638
isExtended: Boolean) extends RunnableCommand {
3739

3840
override def run(sqlContext: SQLContext): Seq[Row] = {
39-
// Trying to mimic the format of Hive's output. But not exactly the same.
40-
var results: Seq[(String, String, String)] = Nil
41-
42-
val columns: Seq[FieldSchema] = table.hiveQlTable.getCols.asScala
43-
val partitionColumns: Seq[FieldSchema] = table.hiveQlTable.getPartCols.asScala
44-
results ++= columns.map(field => (field.getName, field.getType, field.getComment))
45-
if (partitionColumns.nonEmpty) {
46-
val partColumnInfo =
47-
partitionColumns.map(field => (field.getName, field.getType, field.getComment))
48-
results ++=
49-
partColumnInfo ++
50-
Seq(("# Partition Information", "", "")) ++
51-
Seq((s"# ${output(0).name}", output(1).name, output(2).name)) ++
52-
partColumnInfo
53-
}
41+
// There are two modes here:
42+
// For metastore tables, create an output similar to Hive's.
43+
// For other tables, delegate to DescribeCommand.
5444

55-
if (isExtended) {
56-
results ++= Seq(("Detailed Table Information", table.hiveQlTable.getTTable.toString, ""))
57-
}
45+
// In the future, we will consolidate the two and simply report what the catalog reports.
46+
sqlContext.sessionState.catalog.lookupRelation(tableId) match {
47+
case table: MetastoreRelation =>
48+
// Trying to mimic the format of Hive's output. But not exactly the same.
49+
var results: Seq[(String, String, String)] = Nil
50+
51+
val columns: Seq[FieldSchema] = table.hiveQlTable.getCols.asScala
52+
val partitionColumns: Seq[FieldSchema] = table.hiveQlTable.getPartCols.asScala
53+
results ++= columns.map(field => (field.getName, field.getType, field.getComment))
54+
if (partitionColumns.nonEmpty) {
55+
val partColumnInfo =
56+
partitionColumns.map(field => (field.getName, field.getType, field.getComment))
57+
results ++=
58+
partColumnInfo ++
59+
Seq(("# Partition Information", "", "")) ++
60+
Seq((s"# ${output(0).name}", output(1).name, output(2).name)) ++
61+
partColumnInfo
62+
}
63+
64+
if (isExtended) {
65+
results ++= Seq(("Detailed Table Information", table.hiveQlTable.getTTable.toString, ""))
66+
}
67+
68+
results.map { case (name, dataType, comment) =>
69+
Row(name, dataType, comment)
70+
}
5871

59-
results.map { case (name, dataType, comment) =>
60-
Row(name, dataType, comment)
72+
case o: LogicalPlan =>
73+
DescribeCommand(tableId, output, isExtended).run(sqlContext)
6174
}
6275
}
6376
}

0 commit comments

Comments
 (0)