-
Notifications
You must be signed in to change notification settings - Fork 28.5k
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -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 | ||
|
@@ -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 | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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. There was a problem hiding this comment. Choose a reason for hiding this commentThe 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
To describe it:
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 I should be able to query the catalog just like I can query normal tables.
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @marmbrus yeah, I understand your concern. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. after finish SPARK-5182. we can do it like this:
For
|
||
*/ | ||
if (isExtended) { // TODO describe extended table | ||
// rows += Row("# extended", null, null) | ||
} | ||
rows | ||
} | ||
} | ||
|
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) | ||
)) | ||
} |
There was a problem hiding this comment.
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?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Here to replace
string
withTypeName
is ok.But not all
typeName
equalssimple string
.i.e
LongType, the
typeName
of it islong
but notbigint
... etc...