diff options
author | Reynold Xin <rxin@databricks.com> | 2016-03-14 23:09:10 -0700 |
---|---|---|
committer | Reynold Xin <rxin@databricks.com> | 2016-03-14 23:09:10 -0700 |
commit | e64958001cb95d53c441131f8c7a92556f49fd7d (patch) | |
tree | b2b1471353c076d8f9c0cdc50700072910867030 /sql/core | |
parent | f72743d971a38d3d08984ef4b66e0955945d2f58 (diff) | |
download | spark-e64958001cb95d53c441131f8c7a92556f49fd7d.tar.gz spark-e64958001cb95d53c441131f8c7a92556f49fd7d.tar.bz2 spark-e64958001cb95d53c441131f8c7a92556f49fd7d.zip |
[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 <rxin@databricks.com>
Closes #11710 from rxin/SPARK-13884.
Diffstat (limited to 'sql/core')
4 files changed, 12 insertions, 12 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala index 8dde308f96..11391bd12a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala @@ -271,15 +271,14 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly // issue. val tableIdent = TableIdentifier( cleanIdentifier(tableName), Some(cleanIdentifier(dbName))) - datasources.DescribeCommand( - UnresolvedRelation(tableIdent, None), isExtended = extended.isDefined) + datasources.DescribeCommand(tableIdent, isExtended = extended.isDefined) case Token(dbName, Nil) :: Token(tableName, Nil) :: Token(colName, Nil) :: Nil => // It is describing a column with the format like "describe db.table column". nodeToDescribeFallback(node) case tableName :: Nil => // It is describing a table with the format like "describe table". datasources.DescribeCommand( - UnresolvedRelation(TableIdentifier(cleanIdentifier(tableName.text)), None), + TableIdentifier(cleanIdentifier(tableName.text)), isExtended = extended.isDefined) case _ => nodeToDescribeFallback(node) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index bae0750788..6352c48c76 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -398,11 +398,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { sys.error("Tables created with SQLContext must be TEMPORARY. Use a HiveContext instead.") case describe @ LogicalDescribeCommand(table, isExtended) => - val resultPlan = self.sqlContext.executePlan(table).executedPlan - ExecutedCommand( - RunnableDescribeCommand(resultPlan, describe.output, isExtended)) :: Nil + ExecutedCommand(RunnableDescribeCommand(table, describe.output, isExtended)) :: Nil - case logical.ShowFunctions(db, pattern) => ExecutedCommand(ShowFunctions(db, pattern)) :: Nil + case logical.ShowFunctions(db, pattern) => + ExecutedCommand(ShowFunctions(db, pattern)) :: Nil case logical.DescribeFunction(function, extended) => ExecutedCommand(DescribeFunction(function, extended)) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 54cdcb10ac..6e36a15a6d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -22,7 +22,7 @@ import java.util.NoSuchElementException import org.apache.spark.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Dataset, Row, SQLContext} -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical @@ -293,13 +293,14 @@ case object ClearCacheCommand extends RunnableCommand { case class DescribeCommand( - child: SparkPlan, + table: TableIdentifier, override val output: Seq[Attribute], isExtended: Boolean) extends RunnableCommand { override def run(sqlContext: SQLContext): Seq[Row] = { - child.schema.fields.map { field => + val relation = sqlContext.sessionState.catalog.lookupRelation(table) + relation.schema.fields.map { field => val cmtKey = "comment" val comment = if (field.metadata.contains(cmtKey)) field.metadata.getString(cmtKey) else "" Row(field.name, field.dataType.simpleString, comment) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala index 903c9913ac..04e51735c4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala @@ -33,8 +33,9 @@ import org.apache.spark.sql.types._ * It is effective only when the table is a Hive table. */ case class DescribeCommand( - table: LogicalPlan, - isExtended: Boolean) extends LogicalPlan with logical.Command { + table: TableIdentifier, + isExtended: Boolean) + extends LogicalPlan with logical.Command { override def children: Seq[LogicalPlan] = Seq.empty |