Skip to content

Commit

Permalink
Refactor code and update test cases.
Browse files Browse the repository at this point in the history
  • Loading branch information
yhuai committed Jan 8, 2015
1 parent f1cffe4 commit f5c22b0
Show file tree
Hide file tree
Showing 6 changed files with 201 additions and 184 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ private[sql] class DefaultSource extends SchemaRelationProvider {
override def createRelation(
sqlContext: SQLContext,
parameters: Map[String, String],
schema: Option[StructType] = None): BaseRelation = {
schema: Option[StructType]): BaseRelation = {
val fileName = parameters.getOrElse("path", sys.error("Option 'path' not specified"))
val samplingRatio = parameters.get("samplingRatio").map(_.toDouble).getOrElse(1.0)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ class DefaultSource extends SchemaRelationProvider {
override def createRelation(
sqlContext: SQLContext,
parameters: Map[String, String],
schema: Option[StructType] = None): BaseRelation = {
schema: Option[StructType]): BaseRelation = {
val path =
parameters.getOrElse("path", sys.error("'path' must be specified for parquet tables."))

Expand Down
29 changes: 11 additions & 18 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 @@ -64,14 +64,14 @@ private[sql] class DDLParser extends StandardTokenParsers with PackratParsers wi

// Data types.
protected val STRING = Keyword("STRING")
protected val FLOAT = Keyword("FLOAT")
protected val INT = Keyword("INT")
protected val BINARY = Keyword("BINARY")
protected val BOOLEAN = Keyword("BOOLEAN")
protected val TINYINT = Keyword("TINYINT")
protected val SMALLINT = Keyword("SMALLINT")
protected val DOUBLE = Keyword("DOUBLE")
protected val INT = Keyword("INT")
protected val BIGINT = Keyword("BIGINT")
protected val BINARY = Keyword("BINARY")
protected val BOOLEAN = Keyword("BOOLEAN")
protected val FLOAT = Keyword("FLOAT")
protected val DOUBLE = Keyword("DOUBLE")
protected val DECIMAL = Keyword("DECIMAL")
protected val DATE = Keyword("DATE")
protected val TIMESTAMP = Keyword("TIMESTAMP")
Expand Down Expand Up @@ -105,8 +105,8 @@ private[sql] class DDLParser extends StandardTokenParsers with PackratParsers wi
CREATE ~ TEMPORARY ~ TABLE ~> ident
~ (tableCols).? ~ (USING ~> className) ~ (OPTIONS ~> options) ^^ {
case tableName ~ columns ~ provider ~ opts =>
val tblColumns = if(columns.isEmpty) Seq.empty else columns.get
CreateTableUsing(tableName, tblColumns, provider, opts)
val userSpecifiedSchema = columns.flatMap(fields => Some(StructType(fields)))
CreateTableUsing(tableName, userSpecifiedSchema, provider, opts)
}
)

Expand Down Expand Up @@ -184,7 +184,7 @@ private[sql] class DDLParser extends StandardTokenParsers with PackratParsers wi

private[sql] case class CreateTableUsing(
tableName: String,
tableCols: Seq[StructField],
userSpecifiedSchema: Option[StructType],
provider: String,
options: Map[String, String]) extends RunnableCommand {

Expand All @@ -203,16 +203,9 @@ private[sql] case class CreateTableUsing(
.asInstanceOf[org.apache.spark.sql.sources.RelationProvider]
.createRelation(sqlContext, new CaseInsensitiveMap(options))
case dataSource: org.apache.spark.sql.sources.SchemaRelationProvider =>
if(tableCols.isEmpty) {
dataSource
.asInstanceOf[org.apache.spark.sql.sources.SchemaRelationProvider]
.createRelation(sqlContext, new CaseInsensitiveMap(options))
} else {
dataSource
.asInstanceOf[org.apache.spark.sql.sources.SchemaRelationProvider]
.createRelation(
sqlContext, new CaseInsensitiveMap(options), Some(StructType(tableCols)))
}
dataSource
.asInstanceOf[org.apache.spark.sql.sources.SchemaRelationProvider]
.createRelation(sqlContext, new CaseInsensitiveMap(options), userSpecifiedSchema)
}

sqlContext.baseRelationToSchemaRDD(relation).registerTempTable(tableName)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,7 @@ trait SchemaRelationProvider {
def createRelation(
sqlContext: SQLContext,
parameters: Map[String, String],
schema: Option[StructType] = None): BaseRelation
schema: Option[StructType]): BaseRelation
}

/**
Expand Down

This file was deleted.

Loading

0 comments on commit f5c22b0

Please sign in to comment.