You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by yhuai <gi...@git.apache.org> on 2016/07/07 16:08:21 UTC

[GitHub] spark pull request #14028: [SPARK-16351][SQL] Avoid per-record type dispatch...

Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14028#discussion_r69936163
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala ---
    @@ -17,74 +17,180 @@
     
     package org.apache.spark.sql.execution.datasources.json
     
    +import java.io.Writer
    +
     import com.fasterxml.jackson.core._
     
     import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.catalyst.expressions.SpecializedGetters
     import org.apache.spark.sql.catalyst.util.{ArrayData, DateTimeUtils, MapData}
     import org.apache.spark.sql.types._
     
    -private[sql] object JacksonGenerator {
    -  /** Transforms a single InternalRow to JSON using Jackson
    -   *
    -   * TODO: make the code shared with the other apply method.
    -   *
    -   * @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: InternalRow): Unit = {
    -    def valWriter: (DataType, Any) => Unit = {
    -      case (_, null) | (NullType, _) => gen.writeNull()
    -      case (StringType, v) => gen.writeString(v.toString)
    -      case (TimestampType, v: Long) => gen.writeString(DateTimeUtils.toJavaTimestamp(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: Decimal) => gen.writeNumber(v.toJavaBigDecimal)
    -      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: Int) => gen.writeString(DateTimeUtils.toJavaDate(v).toString)
    -      // For UDT values, they should be in the SQL type's corresponding value type.
    -      // We should not see values in the user-defined class at here.
    -      // For example, VectorUDT's SQL type is an array of double. So, we should expect that v is
    -      // an ArrayData at here, instead of a Vector.
    -      case (udt: UserDefinedType[_], v) => valWriter(udt.sqlType, v)
    -
    -      case (ArrayType(ty, _), v: ArrayData) =>
    -        gen.writeStartArray()
    -        v.foreach(ty, (_, value) => valWriter(ty, value))
    -        gen.writeEndArray()
    -
    -      case (MapType(kt, vt, _), v: MapData) =>
    -        gen.writeStartObject()
    -        v.foreach(kt, vt, { (k, v) =>
    -          gen.writeFieldName(k.toString)
    -          valWriter(vt, v)
    -        })
    -        gen.writeEndObject()
    -
    -      case (StructType(ty), v: InternalRow) =>
    -        gen.writeStartObject()
    -        var i = 0
    -        while (i < ty.length) {
    -          val field = ty(i)
    -          val value = v.get(i, field.dataType)
    -          if (value != null) {
    -            gen.writeFieldName(field.name)
    -            valWriter(field.dataType, value)
    -          }
    -          i += 1
    +private[sql] class JacksonGenerator(schema: StructType, writer: Writer) {
    +  // A `ValueWriter` is responsible for writing a field of an `InternalRow` to appropriate
    +  // JSON data. Here we are using `SpecializedGetters` rather than `InternalRow` so that
    +  // we can directly access data in `ArrayData` without the help of `SpecificMutableRow`.
    +  private type ValueWriter = (SpecializedGetters, Int) => Unit
    +
    +  // `ValueWriter`s for all fields of the schema
    +  private val rootFieldWriters: Seq[ValueWriter] = schema.map(_.dataType).map(makeWriter)
    +
    +  private val gen = new JsonFactory().createGenerator(writer).setRootValueSeparator(null)
    +
    +  private def makeWriter(dataType: DataType): ValueWriter = dataType match {
    +    case NullType =>
    +      (row: SpecializedGetters, ordinal: Int) =>
    +        gen.writeNull()
    +
    +    case BooleanType =>
    +      (row: SpecializedGetters, ordinal: Int) =>
    +        gen.writeBoolean(row.getBoolean(ordinal))
    +
    +    case ByteType =>
    +      (row: SpecializedGetters, ordinal: Int) =>
    +        gen.writeNumber(row.getByte(ordinal))
    +
    +    case ShortType =>
    +      (row: SpecializedGetters, ordinal: Int) =>
    +        gen.writeNumber(row.getShort(ordinal))
    +
    +    case IntegerType =>
    +      (row: SpecializedGetters, ordinal: Int) =>
    +        gen.writeNumber(row.getInt(ordinal))
    +
    +    case LongType =>
    +      (row: SpecializedGetters, ordinal: Int) =>
    +        gen.writeNumber(row.getLong(ordinal))
    +
    +    case FloatType =>
    +      (row: SpecializedGetters, ordinal: Int) =>
    +        gen.writeNumber(row.getFloat(ordinal))
    +
    +    case DoubleType =>
    +      (row: SpecializedGetters, ordinal: Int) =>
    +        gen.writeNumber(row.getDouble(ordinal))
    +
    +    case StringType =>
    +      (row: SpecializedGetters, ordinal: Int) =>
    +        gen.writeString(row.getUTF8String(ordinal).toString)
    +
    +    case TimestampType =>
    +      (row: SpecializedGetters, ordinal: Int) =>
    +        gen.writeString(DateTimeUtils.toJavaTimestamp(row.getLong(ordinal)).toString)
    +
    +    case DateType =>
    +      (row: SpecializedGetters, ordinal: Int) =>
    +        gen.writeString(DateTimeUtils.toJavaDate(row.getInt(ordinal)).toString)
    +
    +    case BinaryType =>
    +      (row: SpecializedGetters, ordinal: Int) =>
    +        gen.writeBinary(row.getBinary(ordinal))
    +
    +    case dt: DecimalType =>
    +      (row: SpecializedGetters, ordinal: Int) =>
    +        gen.writeNumber(row.getDecimal(ordinal, dt.precision, dt.scale).toJavaBigDecimal)
    +
    +    case st: StructType =>
    +      val fieldWriters = st.map(_.dataType).map(makeWriter)
    +      (row: SpecializedGetters, ordinal: Int) =>
    +        writeObject {
    +          writeFields(row.getStruct(ordinal, st.length), st, fieldWriters)
    +        }
    +
    +    case at: ArrayType =>
    +      val elementWriter = makeWriter(at.elementType)
    +      (row: SpecializedGetters, ordinal: Int) =>
    +        writeArray {
    +          writeArrayData(row.getArray(ordinal), elementWriter)
    --- End diff --
    
    Inline both writeArray and writeArrayData?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org