Skip to content

Commit 0ba70df

Browse files
committed
draft version
1 parent 040d6f2 commit 0ba70df

File tree

5 files changed

+187
-117
lines changed

5 files changed

+187
-117
lines changed

sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala

Lines changed: 10 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -18,31 +18,38 @@
1818
package org.apache.spark.sql.json
1919

2020
import org.apache.spark.sql.SQLContext
21+
import org.apache.spark.sql.catalyst.types.StructType
2122
import org.apache.spark.sql.sources._
2223

2324
private[sql] class DefaultSource extends RelationProvider {
2425
/** Returns a new base relation with the given parameters. */
2526
override def createRelation(
2627
sqlContext: SQLContext,
27-
parameters: Map[String, String]): BaseRelation = {
28+
parameters: Map[String, String],
29+
schema: Option[StructType]): BaseRelation = {
2830
val fileName = parameters.getOrElse("path", sys.error("Option 'path' not specified"))
2931
val samplingRatio = parameters.get("samplingRatio").map(_.toDouble).getOrElse(1.0)
3032

31-
JSONRelation(fileName, samplingRatio)(sqlContext)
33+
JSONRelation(fileName, samplingRatio, schema)(sqlContext)
3234
}
3335
}
3436

35-
private[sql] case class JSONRelation(fileName: String, samplingRatio: Double)(
37+
private[sql] case class JSONRelation(
38+
fileName: String,
39+
samplingRatio: Double,
40+
userSpecifiedSchema: Option[StructType])(
3641
@transient val sqlContext: SQLContext)
3742
extends TableScan {
3843

3944
private def baseRDD = sqlContext.sparkContext.textFile(fileName)
4045

4146
override val schema =
47+
userSpecifiedSchema.getOrElse(
4248
JsonRDD.inferSchema(
4349
baseRDD,
4450
samplingRatio,
4551
sqlContext.columnNameOfCorruptRecord)
52+
)
4653

4754
override def buildScan() =
4855
JsonRDD.jsonStringToRow(baseRDD, schema, sqlContext.columnNameOfCorruptRecord)

sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala

Lines changed: 17 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -22,22 +22,21 @@ import org.apache.hadoop.fs.{FileStatus, FileSystem, Path}
2222
import org.apache.hadoop.conf.{Configurable, Configuration}
2323
import org.apache.hadoop.io.Writable
2424
import org.apache.hadoop.mapreduce.{JobContext, InputSplit, Job}
25-
import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
26-
2725
import parquet.hadoop.ParquetInputFormat
2826
import parquet.hadoop.util.ContextUtil
2927

3028
import org.apache.spark.annotation.DeveloperApi
3129
import org.apache.spark.{Partition => SparkPartition, Logging}
3230
import org.apache.spark.rdd.{NewHadoopPartition, RDD}
33-
34-
import org.apache.spark.sql.{SQLConf, Row, SQLContext}
31+
import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
3532
import org.apache.spark.sql.catalyst.expressions._
36-
import org.apache.spark.sql.catalyst.types.{StringType, IntegerType, StructField, StructType}
33+
import org.apache.spark.sql.catalyst.types.{IntegerType, StructField, StructType}
3734
import org.apache.spark.sql.sources._
35+
import org.apache.spark.sql.{SQLConf, SQLContext}
3836

3937
import scala.collection.JavaConversions._
4038

39+
4140
/**
4241
* Allows creation of parquet based tables using the syntax
4342
* `CREATE TEMPORARY TABLE ... USING org.apache.spark.sql.parquet`. Currently the only option
@@ -48,11 +47,12 @@ class DefaultSource extends RelationProvider {
4847
/** Returns a new base relation with the given parameters. */
4948
override def createRelation(
5049
sqlContext: SQLContext,
51-
parameters: Map[String, String]): BaseRelation = {
50+
parameters: Map[String, String],
51+
schema: Option[StructType]): BaseRelation = {
5252
val path =
5353
parameters.getOrElse("path", sys.error("'path' must be specified for parquet tables."))
5454

55-
ParquetRelation2(path)(sqlContext)
55+
ParquetRelation2(path, schema)(sqlContext)
5656
}
5757
}
5858

@@ -82,7 +82,9 @@ private[parquet] case class Partition(partitionValues: Map[String, Any], files:
8282
* discovery.
8383
*/
8484
@DeveloperApi
85-
case class ParquetRelation2(path: String)(@transient val sqlContext: SQLContext)
85+
case class ParquetRelation2(
86+
path: String,
87+
userSpecifiedSchema: Option[StructType])(@transient val sqlContext: SQLContext)
8688
extends CatalystScan with Logging {
8789

8890
def sparkContext = sqlContext.sparkContext
@@ -133,12 +135,13 @@ case class ParquetRelation2(path: String)(@transient val sqlContext: SQLContext)
133135

134136
override val sizeInBytes = partitions.flatMap(_.files).map(_.getLen).sum
135137

136-
val dataSchema = StructType.fromAttributes( // TODO: Parquet code should not deal with attributes.
137-
ParquetTypesConverter.readSchemaFromFile(
138-
partitions.head.files.head.getPath,
139-
Some(sparkContext.hadoopConfiguration),
140-
sqlContext.isParquetBinaryAsString))
141-
138+
val dataSchema = userSpecifiedSchema.getOrElse(
139+
StructType.fromAttributes( // TODO: Parquet code should not deal with attributes.
140+
ParquetTypesConverter.readSchemaFromFile(
141+
partitions.head.files.head.getPath,
142+
Some(sparkContext.hadoopConfiguration),
143+
sqlContext.isParquetBinaryAsString))
144+
)
142145
val dataIncludesKey =
143146
partitionKeys.headOption.map(dataSchema.fieldNames.contains(_)).getOrElse(true)
144147

sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala

Lines changed: 130 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -17,16 +17,16 @@
1717

1818
package org.apache.spark.sql.sources
1919

20+
import scala.language.implicitConversions
21+
import scala.util.parsing.combinator.syntactical.StandardTokenParsers
22+
import scala.util.parsing.combinator.{RegexParsers, PackratParsers}
23+
2024
import org.apache.spark.Logging
25+
import org.apache.spark.annotation.DeveloperApi
2126
import org.apache.spark.sql.SQLContext
27+
import org.apache.spark.sql.catalyst.types._
2228
import org.apache.spark.sql.execution.RunnableCommand
2329
import org.apache.spark.util.Utils
24-
25-
import scala.language.implicitConversions
26-
import scala.util.parsing.combinator.lexical.StdLexical
27-
import scala.util.parsing.combinator.syntactical.StandardTokenParsers
28-
import scala.util.parsing.combinator.PackratParsers
29-
3030
import org.apache.spark.sql.catalyst.plans.logical._
3131
import org.apache.spark.sql.catalyst.SqlLexical
3232

@@ -49,6 +49,21 @@ private[sql] class DDLParser extends StandardTokenParsers with PackratParsers wi
4949
protected implicit def asParser(k: Keyword): Parser[String] =
5050
lexical.allCaseVersions(k.str).map(x => x : Parser[String]).reduce(_ | _)
5151

52+
// data types
53+
protected val STRING = Keyword("STRING")
54+
protected val DOUBLE = Keyword("DOUBLE")
55+
protected val BOOLEAN = Keyword("BOOLEAN")
56+
protected val FLOAT = Keyword("FLOAT")
57+
protected val INT = Keyword("INT")
58+
protected val TINYINT = Keyword("TINYINT")
59+
protected val SMALLINT = Keyword("SMALLINT")
60+
protected val BIGINT = Keyword("BIGINT")
61+
protected val BINARY = Keyword("BINARY")
62+
protected val DECIMAL = Keyword("DECIMAL")
63+
protected val DATE = Keyword("DATE")
64+
protected val TIMESTAMP = Keyword("TIMESTAMP")
65+
protected val VARCHAR = Keyword("VARCHAR")
66+
5267
protected val CREATE = Keyword("CREATE")
5368
protected val TEMPORARY = Keyword("TEMPORARY")
5469
protected val TABLE = Keyword("TABLE")
@@ -67,26 +82,129 @@ private[sql] class DDLParser extends StandardTokenParsers with PackratParsers wi
6782
protected lazy val ddl: Parser[LogicalPlan] = createTable
6883

6984
/**
70-
* CREATE TEMPORARY TABLE avroTable
85+
* `CREATE TEMPORARY TABLE avroTable
7186
* USING org.apache.spark.sql.avro
72-
* OPTIONS (path "../hive/src/test/resources/data/files/episodes.avro")
87+
* OPTIONS (path "../hive/src/test/resources/data/files/episodes.avro")`
88+
* or
89+
* `CREATE TEMPORARY TABLE avroTable(intField int, stringField string...)
90+
* USING org.apache.spark.sql.avro
91+
* OPTIONS (path "../hive/src/test/resources/data/files/episodes.avro")`
7392
*/
7493
protected lazy val createTable: Parser[LogicalPlan] =
75-
CREATE ~ TEMPORARY ~ TABLE ~> ident ~ (USING ~> className) ~ (OPTIONS ~> options) ^^ {
94+
( CREATE ~ TEMPORARY ~ TABLE ~> ident ~ (USING ~> className) ~ (OPTIONS ~> options) ^^ {
7695
case tableName ~ provider ~ opts =>
77-
CreateTableUsing(tableName, provider, opts)
96+
CreateTableUsing(tableName, Seq.empty, provider, opts)
97+
}
98+
|
99+
CREATE ~ TEMPORARY ~ TABLE ~> ident
100+
~ tableCols ~ (USING ~> className) ~ (OPTIONS ~> options) ^^ {
101+
case tableName ~ tableColumns ~ provider ~ opts =>
102+
CreateTableUsing(tableName, tableColumns, provider, opts)
78103
}
104+
)
105+
106+
protected lazy val metastoreTypes = new MetastoreTypes
107+
108+
protected lazy val tableCols: Parser[Seq[StructField]] = "(" ~> repsep(column, ",") <~ ")"
79109

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

83113
protected lazy val className: Parser[String] = repsep(ident, ".") ^^ { case s => s.mkString(".")}
84114

85115
protected lazy val pair: Parser[(String, String)] = ident ~ stringLit ^^ { case k ~ v => (k,v) }
116+
117+
protected lazy val column: Parser[StructField] =
118+
ident ~ ident ^^ { case name ~ typ =>
119+
StructField(name, metastoreTypes.toDataType(typ))
120+
}
121+
}
122+
123+
/**
124+
* :: DeveloperApi ::
125+
* Provides a parser for data types.
126+
*/
127+
@DeveloperApi
128+
private[sql] class MetastoreTypes extends RegexParsers {
129+
protected lazy val primitiveType: Parser[DataType] =
130+
"string" ^^^ StringType |
131+
"float" ^^^ FloatType |
132+
"int" ^^^ IntegerType |
133+
"tinyint" ^^^ ByteType |
134+
"smallint" ^^^ ShortType |
135+
"double" ^^^ DoubleType |
136+
"bigint" ^^^ LongType |
137+
"binary" ^^^ BinaryType |
138+
"boolean" ^^^ BooleanType |
139+
fixedDecimalType | // Hive 0.13+ decimal with precision/scale
140+
"decimal" ^^^ DecimalType.Unlimited | // Hive 0.12 decimal with no precision/scale
141+
"date" ^^^ DateType |
142+
"timestamp" ^^^ TimestampType |
143+
"varchar\\((\\d+)\\)".r ^^^ StringType
144+
145+
protected lazy val fixedDecimalType: Parser[DataType] =
146+
("decimal" ~> "(" ~> "\\d+".r) ~ ("," ~> "\\d+".r <~ ")") ^^ {
147+
case precision ~ scale =>
148+
DecimalType(precision.toInt, scale.toInt)
149+
}
150+
151+
protected lazy val arrayType: Parser[DataType] =
152+
"array" ~> "<" ~> dataType <~ ">" ^^ {
153+
case tpe => ArrayType(tpe)
154+
}
155+
156+
protected lazy val mapType: Parser[DataType] =
157+
"map" ~> "<" ~> dataType ~ "," ~ dataType <~ ">" ^^ {
158+
case t1 ~ _ ~ t2 => MapType(t1, t2)
159+
}
160+
161+
protected lazy val structField: Parser[StructField] =
162+
"[a-zA-Z0-9_]*".r ~ ":" ~ dataType ^^ {
163+
case name ~ _ ~ tpe => StructField(name, tpe, nullable = true)
164+
}
165+
166+
protected lazy val structType: Parser[DataType] =
167+
"struct" ~> "<" ~> repsep(structField,",") <~ ">" ^^ {
168+
case fields => new StructType(fields)
169+
}
170+
171+
private[sql] lazy val dataType: Parser[DataType] =
172+
arrayType |
173+
mapType |
174+
structType |
175+
primitiveType
176+
177+
def toDataType(metastoreType: String): DataType = parseAll(dataType, metastoreType) match {
178+
case Success(result, _) => result
179+
case failure: NoSuccess => sys.error(s"Unsupported dataType: $metastoreType")
180+
}
181+
182+
def toMetastoreType(dt: DataType): String = dt match {
183+
case ArrayType(elementType, _) => s"array<${toMetastoreType(elementType)}>"
184+
case StructType(fields) =>
185+
s"struct<${fields.map(f => s"${f.name}:${toMetastoreType(f.dataType)}").mkString(",")}>"
186+
case MapType(keyType, valueType, _) =>
187+
s"map<${toMetastoreType(keyType)},${toMetastoreType(valueType)}>"
188+
case StringType => "string"
189+
case FloatType => "float"
190+
case IntegerType => "int"
191+
case ByteType => "tinyint"
192+
case ShortType => "smallint"
193+
case DoubleType => "double"
194+
case LongType => "bigint"
195+
case BinaryType => "binary"
196+
case BooleanType => "boolean"
197+
case DateType => "date"
198+
case d: DecimalType => "decimal"
199+
case TimestampType => "timestamp"
200+
case NullType => "void"
201+
case udt: UserDefinedType[_] => toMetastoreType(udt.sqlType)
202+
}
86203
}
87204

88205
private[sql] case class CreateTableUsing(
89206
tableName: String,
207+
tableCols: Seq[StructField],
90208
provider: String,
91209
options: Map[String, String]) extends RunnableCommand {
92210

@@ -100,7 +218,8 @@ private[sql] case class CreateTableUsing(
100218
}
101219
}
102220
val dataSource = clazz.newInstance().asInstanceOf[org.apache.spark.sql.sources.RelationProvider]
103-
val relation = dataSource.createRelation(sqlContext, new CaseInsensitiveMap(options))
221+
val relation = dataSource.createRelation(
222+
sqlContext, new CaseInsensitiveMap(options), Some(StructType(tableCols)))
104223

105224
sqlContext.baseRelationToSchemaRDD(relation).registerTempTable(tableName)
106225
Seq.empty

sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -41,7 +41,10 @@ trait RelationProvider {
4141
* Note: the parameters' keywords are case insensitive and this insensitivity is enforced
4242
* by the Map that is passed to the function.
4343
*/
44-
def createRelation(sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation
44+
def createRelation(
45+
sqlContext: SQLContext,
46+
parameters: Map[String, String],
47+
schema: Option[StructType]): BaseRelation
4548
}
4649

4750
/**

0 commit comments

Comments
 (0)