Skip to content

Commit

Permalink
Split up JsonRDD2 into multiple objects
Browse files Browse the repository at this point in the history
  • Loading branch information
Nathan Howell committed May 4, 2015
1 parent fa8234f commit 6822712
Show file tree
Hide file tree
Showing 9 changed files with 506 additions and 482 deletions.
4 changes: 2 additions & 2 deletions sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ import org.apache.spark.sql.catalyst.plans.{JoinType, Inner}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.execution.{EvaluatePython, ExplainCommand, LogicalRDD}
import org.apache.spark.sql.jdbc.JDBCWriteDetails
import org.apache.spark.sql.json.JsonRDD
import org.apache.spark.sql.json.{JacksonGenerator, JsonRDD}
import org.apache.spark.sql.types._
import org.apache.spark.sql.sources.{ResolvedDataSource, CreateTableUsingAsSelect}
import org.apache.spark.util.Utils
Expand Down Expand Up @@ -1369,7 +1369,7 @@ class DataFrame private[sql](
new Iterator[String] {
override def hasNext: Boolean = iter.hasNext
override def next(): String = {
JsonRDD.rowToJSON(rowSchema, gen)(iter.next())
JacksonGenerator(rowSchema, gen)(iter.next())
gen.flush()

val json = writer.toString
Expand Down
171 changes: 171 additions & 0 deletions sql/core/src/main/scala/org/apache/spark/sql/json/InferSchema.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,171 @@
/*
* 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.json

import com.fasterxml.jackson.core._

import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion
import org.apache.spark.sql.json.JacksonUtils.nextUntil
import org.apache.spark.sql.types._

private[sql] object InferSchema {
/**
* Infer the type of a collection of json records in three stages:
* 1. Infer the type of each record
* 2. Merge types by choosing the lowest type necessary to cover equal keys
* 3. Replace any remaining null fields with string, the top type
*/
def apply(
json: RDD[String],
samplingRatio: Double = 1.0,
columnNameOfCorruptRecords: String): StructType = {
require(samplingRatio > 0, s"samplingRatio ($samplingRatio) should be greater than 0")
val schemaData = if (samplingRatio > 0.99) {
json
} else {
json.sample(withReplacement = false, samplingRatio, 1)
}

// perform schema inference on each row and merge afterwards
schemaData.mapPartitions { iter =>
val factory = new JsonFactory()
iter.map { row =>
try {
val parser = factory.createParser(row)
parser.nextToken()
inferField(parser)
} catch {
case _: JsonParseException =>
StructType(Seq(StructField(columnNameOfCorruptRecords, StringType)))
}
}
}.treeAggregate[DataType](StructType(Seq()))(compatibleRootType, compatibleRootType) match {
case st: StructType => nullTypeToStringType(st)
}
}

/**
* Infer the type of a json document from the parser's token stream
*/
private def inferField(parser: JsonParser): DataType = {
import com.fasterxml.jackson.core.JsonToken._
parser.getCurrentToken match {
case null | VALUE_NULL => NullType

case FIELD_NAME =>
parser.nextToken()
inferField(parser)

case VALUE_STRING if parser.getTextLength < 1 =>
// Zero length strings and nulls have special handling to deal
// with JSON generators that do not distinguish between the two.
// To accurately infer types for empty strings that are really
// meant to represent nulls we assume that the two are isomorphic
// but will defer treating null fields as strings until all the
// record fields' types have been combined.
NullType

case VALUE_STRING => StringType
case START_OBJECT =>
val builder = Seq.newBuilder[StructField]
while (nextUntil(parser, END_OBJECT)) {
builder += StructField(parser.getCurrentName, inferField(parser), nullable = true)
}

StructType(builder.result().sortBy(_.name))

case START_ARRAY =>
// If this JSON array is empty, we use NullType as a placeholder.
// If this array is not empty in other JSON objects, we can resolve
// the type as we pass through all JSON objects.
var elementType: DataType = NullType
while (nextUntil(parser, END_ARRAY)) {
elementType = compatibleType(elementType, inferField(parser))
}

ArrayType(elementType)

case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
import JsonParser.NumberType._
parser.getNumberType match {
// For Integer values, use LongType by default.
case INT | LONG => LongType
// Since we do not have a data type backed by BigInteger,
// when we see a Java BigInteger, we use DecimalType.
case BIG_INTEGER | BIG_DECIMAL => DecimalType.Unlimited
case FLOAT | DOUBLE => DoubleType
}

case VALUE_TRUE | VALUE_FALSE => BooleanType
}
}

private def nullTypeToStringType(struct: StructType): StructType = {
val fields = struct.fields.map {
case StructField(fieldName, dataType, nullable, _) =>
val newType = dataType match {
case NullType => StringType
case ArrayType(NullType, containsNull) => ArrayType(StringType, containsNull)
case ArrayType(struct: StructType, containsNull) =>
ArrayType(nullTypeToStringType(struct), containsNull)
case struct: StructType =>nullTypeToStringType(struct)
case other: DataType => other
}

StructField(fieldName, newType, nullable)
}

StructType(fields)
}

/**
* Remove top-level ArrayType wrappers and merge the remaining schemas
*/
private def compatibleRootType: (DataType, DataType) => DataType = {
case (ArrayType(ty1, _), ty2) => compatibleRootType(ty1, ty2)
case (ty1, ArrayType(ty2, _)) => compatibleRootType(ty1, ty2)
case (ty1, ty2) => compatibleType(ty1, ty2)
}

/**
* Returns the most general data type for two given data types.
*/
private[json] def compatibleType(t1: DataType, t2: DataType): DataType = {
HiveTypeCoercion.findTightestCommonType(t1, t2).getOrElse {
// t1 or t2 is a StructType, ArrayType, or an unexpected type.
(t1, t2) match {
case (other: DataType, NullType) => other
case (NullType, other: DataType) => other
case (StructType(fields1), StructType(fields2)) =>
val newFields = (fields1 ++ fields2).groupBy(field => field.name).map {
case (name, fieldTypes) =>
val dataType = fieldTypes.view.map(_.dataType).reduce(compatibleType)
StructField(name, dataType, nullable = true)
}
StructType(newFields.toSeq.sortBy(_.name))

case (ArrayType(elementType1, containsNull1), ArrayType(elementType2, containsNull2)) =>
ArrayType(compatibleType(elementType1, elementType2), containsNull1 || containsNull2)

// strings and every string is a Json object.
case (_, _) => StringType
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -131,11 +131,10 @@ private[sql] class JSONRelation(

override lazy val schema = userSpecifiedSchema.getOrElse {
if (useJacksonStreamingAPI) {
JsonRDD2.nullTypeToStringType(
JsonRDD2.inferSchema(
baseRDD,
samplingRatio,
sqlContext.conf.columnNameOfCorruptRecord))
InferSchema(
baseRDD,
samplingRatio,
sqlContext.conf.columnNameOfCorruptRecord)
} else {
JsonRDD.nullTypeToStringType(
JsonRDD.inferSchema(
Expand All @@ -147,7 +146,7 @@ private[sql] class JSONRelation(

override def buildScan(): RDD[Row] = {
if (useJacksonStreamingAPI) {
JsonRDD2.jsonStringToRow(
JacksonParser(
baseRDD,
schema,
sqlContext.conf.columnNameOfCorruptRecord)
Expand All @@ -161,7 +160,7 @@ private[sql] class JSONRelation(

override def buildScan(requiredColumns: Seq[Attribute], filters: Seq[Expression]): RDD[Row] = {
if (useJacksonStreamingAPI) {
JsonRDD2.jsonStringToRow(
JacksonParser(
baseRDD,
StructType.fromAttributes(requiredColumns),
sqlContext.conf.columnNameOfCorruptRecord)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/*
* 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.json

import scala.collection.Map

import com.fasterxml.jackson.core._

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

private[sql] object JacksonGenerator {
/** Transforms a single Row to JSON using Jackson
*
* @param rowSchema the schema object used for conversion
* @param gen a JsonGenerator object
* @param row The row to convert
*/
def apply(rowSchema: StructType, gen: JsonGenerator)(row: Row): Unit = {
def valWriter: (DataType, Any) => Unit = {
case (_, null) | (NullType, _) => gen.writeNull()
case (StringType, v: String) => gen.writeString(v)
case (TimestampType, v: java.sql.Timestamp) => gen.writeString(v.toString)
case (IntegerType, v: Int) => gen.writeNumber(v)
case (ShortType, v: Short) => gen.writeNumber(v)
case (FloatType, v: Float) => gen.writeNumber(v)
case (DoubleType, v: Double) => gen.writeNumber(v)
case (LongType, v: Long) => gen.writeNumber(v)
case (DecimalType(), v: java.math.BigDecimal) => gen.writeNumber(v)
case (ByteType, v: Byte) => gen.writeNumber(v.toInt)
case (BinaryType, v: Array[Byte]) => gen.writeBinary(v)
case (BooleanType, v: Boolean) => gen.writeBoolean(v)
case (DateType, v) => gen.writeString(v.toString)
case (udt: UserDefinedType[_], v) => valWriter(udt.sqlType, udt.serialize(v))

case (ArrayType(ty, _), v: Seq[_] ) =>
gen.writeStartArray()
v.foreach(valWriter(ty,_))
gen.writeEndArray()

case (MapType(kv,vv, _), v: Map[_,_]) =>
gen.writeStartObject()
v.foreach { p =>
gen.writeFieldName(p._1.toString)
valWriter(vv,p._2)
}
gen.writeEndObject()

case (StructType(ty), v: Row) =>
gen.writeStartObject()
ty.zip(v.toSeq).foreach {
case (_, null) =>
case (field, v) =>
gen.writeFieldName(field.name)
valWriter(field.dataType, v)
}
gen.writeEndObject()
}

valWriter(rowSchema, row)
}
}
Loading

0 comments on commit 6822712

Please sign in to comment.