Skip to content

Commit

Permalink
When creating a SchemaRDD for a JSON dataset, users can apply an exis…
Browse files Browse the repository at this point in the history
…ting schema.
  • Loading branch information
yhuai committed Jul 11, 2014
1 parent 7a6a7e5 commit 949d6bb
Show file tree
Hide file tree
Showing 2 changed files with 31 additions and 8 deletions.
35 changes: 28 additions & 7 deletions sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -128,15 +128,23 @@ class SQLContext(@transient val sparkContext: SparkContext)
*
* @group userf
*/
def jsonFile(path: String): SchemaRDD = jsonFile(path, 1.0)
def jsonFile(path: String): SchemaRDD = jsonFile(path, 1.0, None)

/**
* Loads a JSON file (one object per line) and applies the given schema,
* returning the result as a [[SchemaRDD]].
*
* @group userf
*/
def jsonFile(path: String, schema: StructType): SchemaRDD = jsonFile(path, 1.0, Option(schema))

/**
* :: Experimental ::
*/
@Experimental
def jsonFile(path: String, samplingRatio: Double): SchemaRDD = {
def jsonFile(path: String, samplingRatio: Double, schema: Option[StructType]): SchemaRDD = {
val json = sparkContext.textFile(path)
jsonRDD(json, samplingRatio)
jsonRDD(json, samplingRatio, schema)
}

/**
Expand All @@ -146,15 +154,28 @@ class SQLContext(@transient val sparkContext: SparkContext)
*
* @group userf
*/
def jsonRDD(json: RDD[String]): SchemaRDD = jsonRDD(json, 1.0)
def jsonRDD(json: RDD[String]): SchemaRDD = jsonRDD(json, 1.0, None)

/**
* Loads an RDD[String] storing JSON objects (one object per record) and applies the given schema,
* returning the result as a [[SchemaRDD]].
*
* @group userf
*/
def jsonRDD(json: RDD[String], schema: StructType): SchemaRDD = jsonRDD(json, 1.0, Option(schema))

/**
* :: Experimental ::
*/
@Experimental
def jsonRDD(json: RDD[String], samplingRatio: Double): SchemaRDD = {
val schema = JsonRDD.nullTypeToStringType(JsonRDD.inferSchema(json, samplingRatio))
applySchemaToPartitions(json, schema, JsonRDD.jsonStringToRow(schema, _: Iterator[String]))
def jsonRDD(json: RDD[String], samplingRatio: Double, schema: Option[StructType]): SchemaRDD = {
val appliedSchema =
schema.getOrElse(JsonRDD.nullTypeToStringType(JsonRDD.inferSchema(json, samplingRatio)))

applySchemaToPartitions(
json,
appliedSchema,
JsonRDD.jsonStringToRow(appliedSchema, _: Iterator[String]))
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,9 @@ import org.apache.spark.sql.Logging

private[sql] object JsonRDD extends Logging {

private[sql] def jsonStringToRow(schema: StructType, jsonIter: Iterator[String]): Iterator[Row] = {
private[sql] def jsonStringToRow(
schema: StructType,
jsonIter: Iterator[String]): Iterator[Row] = {
parseJson(jsonIter).map(parsed => asRow(parsed, schema))
}

Expand Down

0 comments on commit 949d6bb

Please sign in to comment.