Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-5135][SQL] Add support for describe [extended] table to DDL in SQLContext #4127

Closed
wants to merge 5 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -227,8 +227,9 @@ abstract class DataType {
def json: String = compact(render(jsonValue))

def prettyJson: String = pretty(render(jsonValue))
}

def toSimpleString: String = typeName
}

/**
* :: DeveloperApi ::
Expand All @@ -240,9 +241,9 @@ abstract class DataType {
@DeveloperApi
case object NullType extends DataType {
override def defaultSize: Int = 1
override def toSimpleString = "null"
}


protected[sql] object NativeType {
val all = Seq(
IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType)
Expand Down Expand Up @@ -300,6 +301,8 @@ case object StringType extends NativeType with PrimitiveType {
* The default size of a value of the StringType is 4096 bytes.
*/
override def defaultSize: Int = 4096

override def toSimpleString = "string"
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Isn't this what typeName is?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Here to replace string with TypeName is ok.
But not all typeName equals simple string.
i.e
LongType, the typeName of it is long but not bigint... etc...

def typeName: String = this.getClass.getSimpleName.stripSuffix("$").dropRight(4).toLowerCase

}


Expand Down Expand Up @@ -329,6 +332,8 @@ case object BinaryType extends NativeType with PrimitiveType {
* The default size of a value of the BinaryType is 4096 bytes.
*/
override def defaultSize: Int = 4096

override def toSimpleString = "binary"
}


Expand All @@ -349,6 +354,8 @@ case object BooleanType extends NativeType with PrimitiveType {
* The default size of a value of the BooleanType is 1 byte.
*/
override def defaultSize: Int = 1

override def toSimpleString = "boolean"
}


Expand All @@ -374,6 +381,8 @@ case object TimestampType extends NativeType {
* The default size of a value of the TimestampType is 8 bytes.
*/
override def defaultSize: Int = 8

override def toSimpleString = "timestamp"
}


Expand All @@ -399,6 +408,8 @@ case object DateType extends NativeType {
* The default size of a value of the DateType is 8 bytes.
*/
override def defaultSize: Int = 8

override def toSimpleString = "date"
}


Expand Down Expand Up @@ -450,6 +461,8 @@ case object LongType extends IntegralType {
* The default size of a value of the LongType is 8 bytes.
*/
override def defaultSize: Int = 8

override def toSimpleString = "bigint"
}


Expand All @@ -472,6 +485,8 @@ case object IntegerType extends IntegralType {
* The default size of a value of the IntegerType is 4 bytes.
*/
override def defaultSize: Int = 4

override def toSimpleString = "int"
}


Expand All @@ -494,6 +509,8 @@ case object ShortType extends IntegralType {
* The default size of a value of the ShortType is 2 bytes.
*/
override def defaultSize: Int = 2

override def toSimpleString = "smallint"
}


Expand All @@ -516,6 +533,8 @@ case object ByteType extends IntegralType {
* The default size of a value of the ByteType is 1 byte.
*/
override def defaultSize: Int = 1

override def toSimpleString = "tinyint"
}


Expand Down Expand Up @@ -575,6 +594,11 @@ case class DecimalType(precisionInfo: Option[PrecisionInfo]) extends FractionalT
* The default size of a value of the DecimalType is 4096 bytes.
*/
override def defaultSize: Int = 4096

override def toSimpleString = precisionInfo match {
case Some(PrecisionInfo(precision, scale)) => s"decimal($precision,$scale)"
case None => "decimal(10,0)"
}
}


Expand Down Expand Up @@ -630,6 +654,8 @@ case object DoubleType extends FractionalType {
* The default size of a value of the DoubleType is 8 bytes.
*/
override def defaultSize: Int = 8

override def toSimpleString = "double"
}


Expand All @@ -653,6 +679,8 @@ case object FloatType extends FractionalType {
* The default size of a value of the FloatType is 4 bytes.
*/
override def defaultSize: Int = 4

override def toSimpleString = "float"
}


Expand Down Expand Up @@ -697,6 +725,8 @@ case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataT
* (We assume that there are 100 elements).
*/
override def defaultSize: Int = 100 * elementType.defaultSize

override def toSimpleString = s"array<${elementType.toSimpleString}>"
}


Expand Down Expand Up @@ -871,6 +901,11 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru
* The default size of a value of the StructType is the total default sizes of all field types.
*/
override def defaultSize: Int = fields.map(_.dataType.defaultSize).sum

override def toSimpleString = {
val fieldTypes = fields.map(field => s"${field.name}:${field.dataType.toSimpleString}")
s"struct<${fieldTypes.mkString(",")}>"
}
}


Expand Down Expand Up @@ -921,6 +956,8 @@ case class MapType(
* (We assume that there are 100 elements).
*/
override def defaultSize: Int = 100 * (keyType.defaultSize + valueType.defaultSize)

override def toSimpleString = s"map<${keyType.toSimpleString},${valueType.toSimpleString}>"
}


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,12 @@ import org.apache.spark.Logging
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{SchemaRDD, SQLConf, SQLContext}
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.expressions.{Row, Attribute}
import org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import scala.collection.mutable.ArrayBuffer

/**
* A logical command that is executed for its side-effects. `RunnableCommand`s are
Expand Down Expand Up @@ -178,3 +180,34 @@ case class DescribeCommand(
child.output.map(field => Row(field.name, field.dataType.toString, null))
}
}

/**
* :: DeveloperApi ::
*/
@DeveloperApi
case class DDLDescribeCommand(
dbName: Option[String],
tableName: String, isExtended: Boolean) extends RunnableCommand {

override def run(sqlContext: SQLContext) = {
val tblRelation = dbName match {
case Some(db) => UnresolvedRelation(Seq(db, tableName))
case None => UnresolvedRelation(Seq(tableName))
}
val logicalRelation = sqlContext.executePlan(tblRelation).analyzed
val rows = new ArrayBuffer[Row]()
rows ++= logicalRelation.schema.fields.map{field =>
Row(field.name, field.dataType.toSimpleString, null)}

/*
* TODO if future support partition table, add header below:
* # Partition Information
* # col_name data_type comment
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this way of returning the info is pretty weird. If we are going to return partition information that should probably be its own column or something.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Here what I mean is to display the normal columns information first and then append the partitioned columns at bottom of the normal columns description. Like below:

CREATE TABLE temp_shengli (
  viewTime int,
  userid bigint,
  page_url string,
  referrer_url string,
  ip string comment 'IP Address of the User'
)
comment 'This is the page view table'
PARTITIONED BY(date string, pos string)

To describe it:

viewtime                int                     None                
userid                  bigint                  None                
page_url                string                  None                
referrer_url            string                  None                
ip                      string                  IP Address of the User
date                    string                  None                
pos                     string                  None                

# Partition Information          
# col_name              data_type               comment             

date                    string                  None                
pos                     string                  None        

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I realize that was your intention and that is how hive did it, but I
disagree with that implementation. It means that in order to know which
columns are partition columns you have to look a other rows of the
results. Any given row should be self contained.

I should be able to query the catalog just like I can query normal tables.
On Jan 20, 2015 7:40 PM, "Sheng, Li" notifications@github.com wrote:

In sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
#4127 (comment):

  • tableName: String, isExtended: Boolean) extends RunnableCommand {
  • override def run(sqlContext: SQLContext) = {
  • val tblRelation = dbName match {
  •  case Some(db) => UnresolvedRelation(Seq(db, tableName))
    
  •  case None => UnresolvedRelation(Seq(tableName))
    
  • }
  • val logicalRelation = sqlContext.executePlan(tblRelation).analyzed
  • val rows = new ArrayBufferRow
  • rows ++= logicalRelation.schema.fields.map{field =>
  •  Row(field.name, field.dataType.toSimpleString, null)}
    
  • /*
  • \* TODO if future support partition table, add header below:
    
  • \* # Partition Information
    
  • \* # col_name data_type comment
    

Here what I mean is to display the normal columns information first and
then append the partitioned columns at bottom of the normal columns
description. Like below:

CREATE TABLE temp_shengli (
viewTime int,
userid bigint,
page_url string,
referrer_url string,
ip string comment 'IP Address of the User'
)
comment 'This is the page view table'
PARTITIONED BY(date string, pos string)

To describe it:

viewtime int None
userid bigint None
page_url string None
referrer_url string None
ip string IP Address of the User
date string None
pos string None

Partition Information

col_name data_type comment

date string None
pos string None


Reply to this email directly or view it on GitHub
https://github.com/apache/spark/pull/4127/files#r23276556.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@marmbrus yeah, I understand your concern.
I refereed hive's describe implementation.
We should provide a getPartitionedColumns method in partition table. Maybe the partition info description can be done after SPARK-5182.
What about just make the PR only do describe table. After SPARK-5182 , we can do extended operation and show partition columns info ? /cc @rxin

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

after finish SPARK-5182. we can do it like this:

    val logicalRelation = sqlContext.executePlan(tblRelation).analyzed
    val rows = new ArrayBuffer[Row]()
    rows ++= logicalRelation.schema.fields.map{field =>
      Row(field.name, field.dataType.toSimpleString, null)}
    val partitionFields = logicalRelation.schema.getPartitionedCols()
    if (partitionFields.nonEmpty) {
      val partColumnRows =
        partitionFields.map(field => Row(field.getName, field.getType.toSimpleString, null))
      rows ++=
          Seq(("# Partition Information", "", "")) ++
              ("col_name","data_type","comment") ++ 
          partColumnRows
    }

For extended we can do it later and discuss to show detail information.

    if (isExtended) {
      rows ++= Seq(("Detailed Table Information", //get some detail info from table, ""))
    }

*/
if (isExtended) { // TODO describe extended table
// rows += Row("# extended", null, null)
}
rows
}
}

17 changes: 15 additions & 2 deletions sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala
Original file line number Diff line number Diff line change
Expand Up @@ -25,9 +25,10 @@ import org.apache.spark.Logging
import org.apache.spark.sql.{SchemaRDD, SQLContext}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.SqlLexical
import org.apache.spark.sql.execution.RunnableCommand
import org.apache.spark.sql.execution.{DDLDescribeCommand, RunnableCommand}
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation

/**
* A parser for foreign DDL commands.
Expand Down Expand Up @@ -61,6 +62,8 @@ private[sql] class DDLParser extends StandardTokenParsers with PackratParsers wi
protected val TABLE = Keyword("TABLE")
protected val USING = Keyword("USING")
protected val OPTIONS = Keyword("OPTIONS")
protected val DESCRIBE = Keyword("DESCRIBE")
protected val EXTENDED = Keyword("EXTENDED")

// Data types.
protected val STRING = Keyword("STRING")
Expand Down Expand Up @@ -89,7 +92,7 @@ private[sql] class DDLParser extends StandardTokenParsers with PackratParsers wi

override val lexical = new SqlLexical(reservedWords)

protected lazy val ddl: Parser[LogicalPlan] = createTable
protected lazy val ddl: Parser[LogicalPlan] = createTable | describeTable

/**
* `CREATE [TEMPORARY] TABLE avroTable
Expand All @@ -112,6 +115,16 @@ private[sql] class DDLParser extends StandardTokenParsers with PackratParsers wi

protected lazy val tableCols: Parser[Seq[StructField]] = "(" ~> repsep(column, ",") <~ ")"

/*
* describe [extended] table avroTable
* This will display all columns of table `avroTable` includes column_name,column_type,nullable
*/
protected lazy val describeTable: Parser[LogicalPlan] =
(DESCRIBE ~> opt(EXTENDED)) ~ (ident <~ ".").? ~ ident ^^ {
case e ~ db ~ tbl =>
DDLDescribeCommand(db, tbl, e.nonEmpty)
}

protected lazy val options: Parser[Map[String, String]] =
"(" ~> repsep(pair, ",") <~ ")" ^^ { case s: Seq[(String, String)] => s.toMap }

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,121 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.sources

import org.apache.spark.sql._
import org.apache.spark.sql.types._

class DDLScanSource extends RelationProvider {
override def createRelation(
sqlContext: SQLContext,
parameters: Map[String, String]): BaseRelation = {
SimpleDDLScan(parameters("from").toInt, parameters("TO").toInt)(sqlContext)
}
}

case class SimpleDDLScan(from: Int, to: Int)(@transient val sqlContext: SQLContext)
extends TableScan {

override def schema =
StructType(Seq(
StructField("intType", IntegerType, nullable = false),
StructField("stringType", StringType, nullable = false),
StructField("dateType", DateType, nullable = false),
StructField("timestampType", TimestampType, nullable = false),
StructField("doubleType", DoubleType, nullable = false),
StructField("bigintType", LongType, nullable = false),
StructField("tinyintType", ByteType, nullable = false),
StructField("decimalType", DecimalType.Unlimited, nullable = false),
StructField("fixedDecimalType", DecimalType(5,1), nullable = false),
StructField("binaryType", BinaryType, nullable = false),
StructField("booleanType", BooleanType, nullable = false),
StructField("smallIntType", ShortType, nullable = false),
StructField("floatType", FloatType, nullable = false),
StructField("mapType", MapType(StringType, StringType)),
StructField("arrayType", ArrayType(StringType)),
StructField("structType",
StructType(StructField("f1",StringType) ::
StructField("f2",IntegerType) :: Nil
)
)
))


override def buildScan() = sqlContext.sparkContext.parallelize(from to to).map { e =>
Row(s"people$e", e * 2)
}
}

class DDLTestSuite extends DataSourceTest {
import caseInsensisitiveContext._

before {
sql(
"""
|CREATE TEMPORARY TABLE ddlPeople
|USING org.apache.spark.sql.sources.DDLScanSource
|OPTIONS (
| From '1',
| To '10'
|)
""".stripMargin)
}

sqlTest(
"describe ddlPeople",
Seq(
Row("intType", "int", null),
Row("stringType", "string", null),
Row("dateType", "date", null),
Row("timestampType", "timestamp", null),
Row("doubleType", "double", null),
Row("bigintType", "bigint", null),
Row("tinyintType", "tinyint", null),
Row("decimalType", "decimal(10,0)", null),
Row("fixedDecimalType", "decimal(5,1)", null),
Row("binaryType", "binary", null),
Row("booleanType", "boolean", null),
Row("smallIntType", "smallint", null),
Row("floatType", "float", null),
Row("mapType", "map<string,string>", null),
Row("arrayType", "array<string>", null),
Row("structType", "struct<f1:string,f2:int>", null)
))

sqlTest(
"describe extended ddlPeople",
Seq(
Row("intType", "int", null),
Row("stringType", "string", null),
Row("dateType", "date", null),
Row("timestampType", "timestamp", null),
Row("doubleType", "double", null),
Row("bigintType", "bigint", null),
Row("tinyintType", "tinyint", null),
Row("decimalType", "decimal(10,0)", null),
Row("fixedDecimalType", "decimal(5,1)", null),
Row("binaryType", "binary", null),
Row("booleanType", "boolean", null),
Row("smallIntType", "smallint", null),
Row("floatType", "float", null),
Row("mapType", "map<string,string>", null),
Row("arrayType", "array<string>", null),
Row("structType", "struct<f1:string,f2:int>", null)
// Row("# extended", null, null)
))
}
Loading