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

[GitHub] spark pull request #14102: [SPARK-16434][SQL][WIP] Avoid record-per type dis...

GitHub user HyukjinKwon opened a pull request:

    https://github.com/apache/spark/pull/14102

    [SPARK-16434][SQL][WIP] Avoid record-per type dispatch in JSON when reading

    ## What changes were proposed in this pull request?
    
    (Please fill in changes proposed in this fix)
    
    
    ## How was this patch tested?
    
    (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
    
    
    (If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
    
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/HyukjinKwon/spark SPARK-16434

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/14102.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #14102
    
----
commit 74fa944209491b9884dbfc8b71e56e36b45e28a4
Author: hyukjinkwon <gu...@gmail.com>
Date:   2016-07-08T01:14:18Z

    Avoid record-per type dispatch in JSON when reading

----


---
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


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

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r71096802
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,306 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +private[sql] class JacksonParser(schema: StructType, options: JSONOptions) extends Logging {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedConversion(
    +      parser: JsonParser,
    +      dataType: DataType): Any = parser.getCurrentToken match {
    +    case null | VALUE_NULL =>
    +      null
    +
    +    case _ if parser.getTextLength < 1 =>
    +      // guard the non string type
    +      null
    +
    +    case token =>
    +    // We cannot parse this token based on the given data type. So, we throw a
    +    // SparkSQLJsonProcessingException and this exception will be caught by
    +    // parseJson method.
    +    throw new SparkSQLJsonProcessingException(
    +      s"Failed to parse a value for data type $dataType (current token: $token).")
    +  }
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(options.columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    -        // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      // SPARK-3308: support reading top level JSON arrays and take every element
    +      // in such an array as a row
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parser.getCurrentToken match {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +        case _ => failedConversion(parser, st)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case ArrayType(st: StructType, _) =>
    +      // the business end of SPARK-3308:
    +      // when an object is found but an array is requested just wrap it in a list
    --- End diff --
    
    also, this comment should be put at https://github.com/apache/spark/pull/14102/files#diff-8affe5ec7d691943a88e43eb30af656eR110, right?


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74219145
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,330 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedConversion(
    +      parser: JsonParser,
    +      dataType: DataType): Any = parser.getCurrentToken match {
    +    case VALUE_STRING if parser.getTextLength < 1 =>
    +      // If conversion is failed, this produces `null` rather than
    +      // returning empty string. This will protect the mismatch of types.
    +      null
    +
    +    case token =>
    +      // We cannot parse this token based on the given data type. So, we throw a
    +      // SparkSQLJsonProcessingException and this exception will be caught by
    +      // parseJson method.
    +      throw new SparkSQLJsonProcessingException(
    +        s"Failed to parse a value for data type $dataType (current token: $token).")
    +  }
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parser.getCurrentToken match {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +        case _ => failedConversion(parser, st)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parser.getCurrentToken match {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +        case _ => failedConversion(parser, st)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    -
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_TRUE => true
    +          case VALUE_FALSE => false
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ByteType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getByteValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case ShortType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getShortValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    +    case IntegerType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getIntValue
    +          case _ => failedConversion(parser, dataType)
             }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +      }
    +
    +    case LongType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getLongValue
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
    +
    +    case FloatType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +            parser.getFloatValue
    +
    +          case VALUE_STRING =>
    +            // Special case handling for NaN and Infinity.
    +            val value = parser.getText
    +            val lowerCaseValue = value.toLowerCase
    +            if (lowerCaseValue.equals("nan") ||
    +              lowerCaseValue.equals("infinity") ||
    +              lowerCaseValue.equals("-infinity") ||
    +              lowerCaseValue.equals("inf") ||
    +              lowerCaseValue.equals("-inf")) {
    +              value.toFloat
    +            } else {
    +              throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +            }
    +
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
    +
    +    case DoubleType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +            parser.getDoubleValue
    +
    +          case VALUE_STRING =>
    +            // Special case handling for NaN and Infinity.
    +            val value = parser.getText
    +            val lowerCaseValue = value.toLowerCase
    +            if (lowerCaseValue.equals("nan") ||
    +              lowerCaseValue.equals("infinity") ||
    +              lowerCaseValue.equals("-infinity") ||
    +              lowerCaseValue.equals("inf") ||
    +              lowerCaseValue.equals("-inf")) {
    +              value.toDouble
    +            } else {
    +              throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +            }
    +
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
    +
    +    case StringType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +          case _ =>
    +            val writer = new ByteArrayOutputStream()
    +            Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +              generator => generator.copyCurrentStructure(parser)
    +            }
    +            UTF8String.fromBytes(writer.toByteArray)
             }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case TimestampType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            // This one will lose microseconds parts.
    +            // See https://issues.apache.org/jira/browse/SPARK-10681.
    +            DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +          case VALUE_NUMBER_INT =>
    +            parser.getLongValue * 1000000L
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +    case DateType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            val stringValue = parser.getText
    +            if (stringValue.contains("-")) {
    +              // The format of this string will probably be "yyyy-mm-dd".
    +              DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +            } else {
    +              // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +              // So, we just convert it to Int.
    +              stringValue.toInt
    +            }
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
    +
    +    case BinaryType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING => parser.getBinaryValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +            Decimal(parser.getDecimalValue, dt.precision, dt.scale)
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +          case _ => failedConversion(parser, dt)
    +        }
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          case _ => failedConversion(parser, st)
    +        }
    +      }
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case START_ARRAY => convertArray(parser, elementConverter)
    +          case _ => failedConversion(parser, at)
    +        }
    +      }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case START_OBJECT => convertMap(parser, valueConverter)
    +          case _ => failedConversion(parser, mt)
    +        }
    +      }
     
    -      case (_, udt: UserDefinedType[_]) =>
    -        convertField(factory, parser, udt.sqlType)
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
     
    -      case (token, dataType) =>
    -        // We cannot parse this token based on the given data type. So, we throw a
    -        // SparkSQLJsonProcessingException and this exception will be caught by
    -        // parseJson method.
    -        throw new SparkSQLJsonProcessingException(
    -          s"Failed to parse a value for data type $dataType (current token: $token).")
    +    case _ =>
    +      (parser: JsonParser) =>
    +        failedConversion(parser, dataType)
    +  }
    +
    +  /**
    +   * This handles nulls ahead before trying to check the tokens, and applies
    +   * the conversion function. Also, it checks `FIELD_NAME` if exists and then skip.
    +   * If this is called after `START_OBJECT`, then, the next token can be
    +   * `FIELD_NAME`. Since the names are kept in `JacksonParser.convertObject`, this `FIELD_NAME`
    +   * token can be skipped as below. When this is called after `START_ARRAY`, the tokens become
    +   * ones about values until `END_ARRAY`. In this case, we don't have to skip.
    +   */
    +  private def handleNullAndFieldTokens (parser: JsonParser)(f: => Any): Any = {
    --- End diff --
    
    how about
    ```
    private def parseJsonToken(parser: JsonParser, dataType: DataType)(f: PartialFunction[JsonToken, Any]): Any = {
      parser.getCurrentToken match {
        case FIELD_NAME =>
          parser.nextToken()
          parseJsonToken(parser)(f)
    
        case null | VALUE_NULL => null
    
        case other => f. applyOrElse(other, failedConversion(parser, dataType))
      }
    }
    ```
    
    and the caller side:
    ```
    case IntegerType =>
      (parser: JsonParser) =>parseJsonToken(parser, dataType) {
        case VALUE_NUMBER_INT => parser.getIntValue
      }
    ```


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    @yhuai Thank you for your review! I will try to address all your comments first.


---
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


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

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r70029253
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,307 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +private[sql] class JacksonParser(schema: StructType, options: JSONOptions) extends Logging {
    --- End diff --
    
    Here, `JacksonParser` object was removed and `JacksonParser` class was added (to keep schema information).


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #63481 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/63481/consoleFull)** for PR 14102 at commit [`25c1423`](https://github.com/apache/spark/commit/25c142385a27595699c00e9aeb22ab83b73bc47b).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class JacksonParser(`


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Merged build finished. Test PASSed.


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74373952
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,289 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_TRUE => true
    +        case VALUE_FALSE => false
    +      }
     
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    -        }
    +    case ByteType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getByteValue
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ShortType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getShortValue
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case IntegerType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getIntValue
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    -        }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    -        }
    +    case LongType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getLongValue
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    -        }
    +    case FloatType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getFloatValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toFloat
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case DoubleType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getDoubleValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toDouble
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +    case StringType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +        case _ =>
    +          val writer = new ByteArrayOutputStream()
    +          Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +            generator => generator.copyCurrentStructure(parser)
    +          }
    +          UTF8String.fromBytes(writer.toByteArray)
    +      }
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +    case TimestampType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          // This one will lose microseconds parts.
    +          // See https://issues.apache.org/jira/browse/SPARK-10681.
    +          DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +        case VALUE_NUMBER_INT =>
    +          parser.getLongValue * 1000000L
    +      }
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +    case DateType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          val stringValue = parser.getText
    +          if (stringValue.contains("-")) {
    +            // The format of this string will probably be "yyyy-mm-dd".
    +            DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +          } else {
    +            // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +            // So, we just convert it to Int.
    +            stringValue.toInt
    +          }
    +      }
    +
    +    case BinaryType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING => parser.getBinaryValue
    +      }
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +          Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +      }
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertMap(parser, valueConverter)
    +      }
     
    -      case (_, udt: UserDefinedType[_]) =>
    -        convertField(factory, parser, udt.sqlType)
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
     
    -      case (token, dataType) =>
    -        // We cannot parse this token based on the given data type. So, we throw a
    -        // SparkSQLJsonProcessingException and this exception will be caught by
    -        // parseJson method.
    -        throw new SparkSQLJsonProcessingException(
    -          s"Failed to parse a value for data type $dataType (current token: $token).")
    +    case _ =>
    --- End diff --
    
    ah I missed NullType and CalendarIntervalType


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/63599/
    Test PASSed.


---
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


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

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r71096347
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,306 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +private[sql] class JacksonParser(schema: StructType, options: JSONOptions) extends Logging {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedConversion(
    +      parser: JsonParser,
    +      dataType: DataType): Any = parser.getCurrentToken match {
    +    case null | VALUE_NULL =>
    +      null
    +
    +    case _ if parser.getTextLength < 1 =>
    +      // guard the non string type
    +      null
    +
    +    case token =>
    +    // We cannot parse this token based on the given data type. So, we throw a
    +    // SparkSQLJsonProcessingException and this exception will be caught by
    +    // parseJson method.
    +    throw new SparkSQLJsonProcessingException(
    --- End diff --
    
    Format


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #63602 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/63602/consoleFull)** for PR 14102 at commit [`a8b6a04`](https://github.com/apache/spark/commit/a8b6a04aa753773aaf7d5e582cc4657f69bce9f9).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Merged build finished. Test PASSed.


---
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


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

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r71096401
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,306 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +private[sql] class JacksonParser(schema: StructType, options: JSONOptions) extends Logging {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedConversion(
    +      parser: JsonParser,
    +      dataType: DataType): Any = parser.getCurrentToken match {
    +    case null | VALUE_NULL =>
    +      null
    +
    +    case _ if parser.getTextLength < 1 =>
    +      // guard the non string type
    +      null
    --- End diff --
    
    We cannot call this method `failedConversion`, right?


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    @yhuai the commits I pushed include the changes below:
    
    - Reverts the changes in `JSONOptions` about `columnNameOfCorruptRecord` https://github.com/apache/spark/pull/14102#discussion_r71095725.
    - Changes  `skipFieldNameIfExists` to `convertField` with documentations https://github.com/apache/spark/pull/14102#discussion_r71096761.
    - Adds `convertValue` for `null` checking so, the weird dirty comparison https://github.com/apache/spark/pull/14102#discussion_r71097210 was removed.
    - Adds a example for https://github.com/apache/spark/pull/14102#discussion_r71096571
    - Corrects style nits.


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74193172
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,337 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedConversion(
    +      parser: JsonParser,
    +      dataType: DataType): Any = parser.getCurrentToken match {
    +    case _ if parser.getTextLength < 1 =>
    +      // If conversion is failed, this produces `null` rather than
    +      // returning empty string. This will protect the mismatch of types.
    +      null
    +
    +    case token =>
    +      // We cannot parse this token based on the given data type. So, we throw a
    +      // SparkSQLJsonProcessingException and this exception will be caught by
    +      // parseJson method.
    +      throw new SparkSQLJsonProcessingException(
    +        s"Failed to parse a value for data type $dataType (current token: $token).")
    +  }
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parser.getCurrentToken match {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +        case _ => failedConversion(parser, st)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parser.getCurrentToken match {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +        case _ => failedConversion(parser, st)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    -
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_TRUE => true
    +          case VALUE_FALSE => false
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ByteType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getByteValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case ShortType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getShortValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    +    case IntegerType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getIntValue
    +          case _ => failedConversion(parser, dataType)
             }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +      }
    +
    +    case LongType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getLongValue
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +    case FloatType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +            parser.getFloatValue
    +
    +          case VALUE_STRING =>
    +            // Special case handling for NaN and Infinity.
    +            val value = parser.getText
    +            val lowerCaseValue = value.toLowerCase
    +            if (lowerCaseValue.equals("nan") ||
    +              lowerCaseValue.equals("infinity") ||
    +              lowerCaseValue.equals("-infinity") ||
    +              lowerCaseValue.equals("inf") ||
    +              lowerCaseValue.equals("-inf")) {
    +              value.toFloat
    +            } else {
    +              throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +            }
    +
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case DoubleType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +            parser.getDoubleValue
    +
    +          case VALUE_STRING =>
    +            // Special case handling for NaN and Infinity.
    +            val value = parser.getText
    +            val lowerCaseValue = value.toLowerCase
    +            if (lowerCaseValue.equals("nan") ||
    +              lowerCaseValue.equals("infinity") ||
    +              lowerCaseValue.equals("-infinity") ||
    +              lowerCaseValue.equals("inf") ||
    +              lowerCaseValue.equals("-inf")) {
    +              value.toDouble
    +            } else {
    +              throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +            }
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +    case StringType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +          case _ =>
    +            val writer = new ByteArrayOutputStream()
    +            Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +              generator => generator.copyCurrentStructure(parser)
    +            }
    +            UTF8String.fromBytes(writer.toByteArray)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +    case TimestampType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            // This one will lose microseconds parts.
    +            // See https://issues.apache.org/jira/browse/SPARK-10681.
    +            DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +          case VALUE_NUMBER_INT =>
    +            parser.getLongValue * 1000000L
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case DateType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            val stringValue = parser.getText
    +            if (stringValue.contains("-")) {
    +              // The format of this string will probably be "yyyy-mm-dd".
    +              DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +            } else {
    +              // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +              // So, we just convert it to Int.
    +              stringValue.toInt
    +            }
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +    case BinaryType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING => parser.getBinaryValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
    +
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +            Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +
    +          case _ => failedConversion(parser, dt)
    +        }
    +      }
    +
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          case _ => failedConversion(parser, st)
    +        }
    +      }
    +
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case START_ARRAY => convertArray(parser, elementConverter)
    +          case _ => failedConversion(parser, at)
    +        }
    +      }
    +
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case START_OBJECT => convertMap(parser, valueConverter)
    +          case _ => failedConversion(parser, mt)
    +        }
    +      }
    +
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
    +
    +    case _ =>
    +      (parser: JsonParser) =>
    +        failedConversion(parser, dataType)
    +  }
    +
    +  /**
    +   * This converts a field. If this is called after `START_OBJECT`, then, the next token can be
    +   * `FIELD_NAME`. Since the names are kept in `JacksonParser.convertObject`, this `FIELD_NAME`
    +   * token can be skipped as below. When this is called after `START_ARRAY`, the tokens become
    +   * ones about values until `END_ARRAY`. In this case, we don't have to skip.
    +   */
    +  private def convertField(parser: JsonParser)(f: => Any): Any = {
    --- End diff --
    
    these 2 and https://github.com/apache/spark/pull/14102/files#r74034675 are just some common logics for all data types, can we put them together?


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74375786
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,289 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_TRUE => true
    +        case VALUE_FALSE => false
    +      }
     
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    -        }
    +    case ByteType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getByteValue
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ShortType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getShortValue
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case IntegerType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getIntValue
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    -        }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    -        }
    +    case LongType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getLongValue
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    -        }
    +    case FloatType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getFloatValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toFloat
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case DoubleType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getDoubleValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toDouble
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +    case StringType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +        case _ =>
    +          val writer = new ByteArrayOutputStream()
    +          Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +            generator => generator.copyCurrentStructure(parser)
    +          }
    +          UTF8String.fromBytes(writer.toByteArray)
    +      }
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +    case TimestampType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          // This one will lose microseconds parts.
    +          // See https://issues.apache.org/jira/browse/SPARK-10681.
    +          DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +        case VALUE_NUMBER_INT =>
    +          parser.getLongValue * 1000000L
    +      }
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +    case DateType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          val stringValue = parser.getText
    +          if (stringValue.contains("-")) {
    +            // The format of this string will probably be "yyyy-mm-dd".
    +            DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +          } else {
    +            // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +            // So, we just convert it to Int.
    +            stringValue.toInt
    +          }
    +      }
    +
    +    case BinaryType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING => parser.getBinaryValue
    +      }
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +          Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +      }
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertMap(parser, valueConverter)
    +      }
     
    -      case (_, udt: UserDefinedType[_]) =>
    -        convertField(factory, parser, udt.sqlType)
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
     
    -      case (token, dataType) =>
    -        // We cannot parse this token based on the given data type. So, we throw a
    -        // SparkSQLJsonProcessingException and this exception will be caught by
    -        // parseJson method.
    -        throw new SparkSQLJsonProcessingException(
    -          s"Failed to parse a value for data type $dataType (current token: $token).")
    +    case _ =>
    +      (parser: JsonParser) =>
    +        parseJsonToken(parser, dataType) {
    +          case token =>
    +            throw new SparkSQLJsonProcessingException(
    +              s"Failed to parse a value for data type $dataType (current token: $token).")
    +        }
    +  }
    +
    +  /**
    +   * This handles nulls ahead before trying to check the tokens, and applies the conversion
    +   * function and then checks failed the conversion afterward if it `f` fails to convert the value.
    +   *
    +   * In more details, it checks `FIELD_NAME` if exists and then skip. If this is called after
    +   * `START_OBJECT`, then, the next token can be `FIELD_NAME`. Since the names are kept in
    +   * `JacksonParser.convertObject`, this `FIELD_NAME` token can be skipped as below. When this
    +   * is called after `START_ARRAY`, the tokens become ones about values until `END_ARRAY`.
    +   * In this case, we don't have to skip.
    +   *
    +   * We check if the current token is null or not after that. Then, we apply `f` to convert
    +   * the value and then we check failed conversion afterward if it `f` fails to convert the value.
    +   */
    +  private def parseJsonToken(
    +      parser: JsonParser,
    +      dataType: DataType)(f: PartialFunction[JsonToken, Any]): Any = {
    +    parser.getCurrentToken match {
    +      case FIELD_NAME =>
    +        parser.nextToken()
    +        parseJsonToken(parser, dataType)(f)
    +
    +      case null | VALUE_NULL => null
    +
    +      case other => f.orElse {
    +        // We should specify the type of this `PartialFunction`. Otherwise this will
    --- End diff --
    
    we can make it a normal function, not inner, like what you did before.


---
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


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

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74375976
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,289 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_TRUE => true
    +        case VALUE_FALSE => false
    +      }
     
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    -        }
    +    case ByteType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getByteValue
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ShortType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getShortValue
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case IntegerType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getIntValue
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    -        }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    -        }
    +    case LongType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getLongValue
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    -        }
    +    case FloatType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getFloatValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toFloat
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case DoubleType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getDoubleValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toDouble
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +    case StringType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +        case _ =>
    +          val writer = new ByteArrayOutputStream()
    +          Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +            generator => generator.copyCurrentStructure(parser)
    +          }
    +          UTF8String.fromBytes(writer.toByteArray)
    +      }
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +    case TimestampType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          // This one will lose microseconds parts.
    +          // See https://issues.apache.org/jira/browse/SPARK-10681.
    +          DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +        case VALUE_NUMBER_INT =>
    +          parser.getLongValue * 1000000L
    +      }
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +    case DateType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          val stringValue = parser.getText
    +          if (stringValue.contains("-")) {
    +            // The format of this string will probably be "yyyy-mm-dd".
    +            DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +          } else {
    +            // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +            // So, we just convert it to Int.
    +            stringValue.toInt
    +          }
    +      }
    +
    +    case BinaryType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING => parser.getBinaryValue
    +      }
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +          Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +      }
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertMap(parser, valueConverter)
    +      }
     
    -      case (_, udt: UserDefinedType[_]) =>
    -        convertField(factory, parser, udt.sqlType)
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
     
    -      case (token, dataType) =>
    -        // We cannot parse this token based on the given data type. So, we throw a
    -        // SparkSQLJsonProcessingException and this exception will be caught by
    -        // parseJson method.
    -        throw new SparkSQLJsonProcessingException(
    -          s"Failed to parse a value for data type $dataType (current token: $token).")
    +    case _ =>
    +      (parser: JsonParser) =>
    +        parseJsonToken(parser, dataType) {
    +          case token =>
    +            throw new SparkSQLJsonProcessingException(
    +              s"Failed to parse a value for data type $dataType (current token: $token).")
    +        }
    +  }
    +
    +  /**
    +   * This handles nulls ahead before trying to check the tokens, and applies the conversion
    +   * function and then checks failed the conversion afterward if it `f` fails to convert the value.
    +   *
    +   * In more details, it checks `FIELD_NAME` if exists and then skip. If this is called after
    +   * `START_OBJECT`, then, the next token can be `FIELD_NAME`. Since the names are kept in
    +   * `JacksonParser.convertObject`, this `FIELD_NAME` token can be skipped as below. When this
    +   * is called after `START_ARRAY`, the tokens become ones about values until `END_ARRAY`.
    +   * In this case, we don't have to skip.
    +   *
    +   * We check if the current token is null or not after that. Then, we apply `f` to convert
    +   * the value and then we check failed conversion afterward if it `f` fails to convert the value.
    +   */
    +  private def parseJsonToken(
    +      parser: JsonParser,
    +      dataType: DataType)(f: PartialFunction[JsonToken, Any]): Any = {
    +    parser.getCurrentToken match {
    +      case FIELD_NAME =>
    +        parser.nextToken()
    +        parseJsonToken(parser, dataType)(f)
    +
    +      case null | VALUE_NULL => null
    +
    +      case other => f.orElse {
    +        // We should specify the type of this `PartialFunction`. Otherwise this will
    --- End diff --
    
    Thanks!


---
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


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

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74194642
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,337 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedConversion(
    +      parser: JsonParser,
    +      dataType: DataType): Any = parser.getCurrentToken match {
    +    case _ if parser.getTextLength < 1 =>
    +      // If conversion is failed, this produces `null` rather than
    +      // returning empty string. This will protect the mismatch of types.
    +      null
    +
    +    case token =>
    +      // We cannot parse this token based on the given data type. So, we throw a
    +      // SparkSQLJsonProcessingException and this exception will be caught by
    +      // parseJson method.
    +      throw new SparkSQLJsonProcessingException(
    +        s"Failed to parse a value for data type $dataType (current token: $token).")
    +  }
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parser.getCurrentToken match {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +        case _ => failedConversion(parser, st)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parser.getCurrentToken match {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +        case _ => failedConversion(parser, st)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    -
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_TRUE => true
    +          case VALUE_FALSE => false
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ByteType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getByteValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case ShortType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getShortValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    +    case IntegerType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getIntValue
    +          case _ => failedConversion(parser, dataType)
             }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +      }
    +
    +    case LongType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getLongValue
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +    case FloatType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +            parser.getFloatValue
    +
    +          case VALUE_STRING =>
    +            // Special case handling for NaN and Infinity.
    +            val value = parser.getText
    +            val lowerCaseValue = value.toLowerCase
    +            if (lowerCaseValue.equals("nan") ||
    +              lowerCaseValue.equals("infinity") ||
    +              lowerCaseValue.equals("-infinity") ||
    +              lowerCaseValue.equals("inf") ||
    +              lowerCaseValue.equals("-inf")) {
    +              value.toFloat
    +            } else {
    +              throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +            }
    +
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case DoubleType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +            parser.getDoubleValue
    +
    +          case VALUE_STRING =>
    +            // Special case handling for NaN and Infinity.
    +            val value = parser.getText
    +            val lowerCaseValue = value.toLowerCase
    +            if (lowerCaseValue.equals("nan") ||
    +              lowerCaseValue.equals("infinity") ||
    +              lowerCaseValue.equals("-infinity") ||
    +              lowerCaseValue.equals("inf") ||
    +              lowerCaseValue.equals("-inf")) {
    +              value.toDouble
    +            } else {
    +              throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +            }
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +    case StringType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +          case _ =>
    +            val writer = new ByteArrayOutputStream()
    +            Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +              generator => generator.copyCurrentStructure(parser)
    +            }
    +            UTF8String.fromBytes(writer.toByteArray)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +    case TimestampType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            // This one will lose microseconds parts.
    +            // See https://issues.apache.org/jira/browse/SPARK-10681.
    +            DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +          case VALUE_NUMBER_INT =>
    +            parser.getLongValue * 1000000L
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case DateType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            val stringValue = parser.getText
    +            if (stringValue.contains("-")) {
    +              // The format of this string will probably be "yyyy-mm-dd".
    +              DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +            } else {
    +              // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +              // So, we just convert it to Int.
    +              stringValue.toInt
    +            }
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +    case BinaryType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING => parser.getBinaryValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
    +
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +            Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +
    +          case _ => failedConversion(parser, dt)
    +        }
    +      }
    +
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          case _ => failedConversion(parser, st)
    +        }
    +      }
    +
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case START_ARRAY => convertArray(parser, elementConverter)
    +          case _ => failedConversion(parser, at)
    +        }
    +      }
    +
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case START_OBJECT => convertMap(parser, valueConverter)
    +          case _ => failedConversion(parser, mt)
    +        }
    +      }
    +
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
    +
    +    case _ =>
    +      (parser: JsonParser) =>
    +        failedConversion(parser, dataType)
    +  }
    +
    +  /**
    +   * This converts a field. If this is called after `START_OBJECT`, then, the next token can be
    +   * `FIELD_NAME`. Since the names are kept in `JacksonParser.convertObject`, this `FIELD_NAME`
    +   * token can be skipped as below. When this is called after `START_ARRAY`, the tokens become
    +   * ones about values until `END_ARRAY`. In this case, we don't have to skip.
    +   */
    +  private def convertField(parser: JsonParser)(f: => Any): Any = {
    --- End diff --
    
    Sure, I will merge both `convertValue` and `convertField` but additional checking should be added for merging  https://github.com/apache/spark/pull/14102/files#r74034675 to be called ahead, because in `StringType`, it does not call `failedConversion` when it fails. For `StringType`, it should produce empty string when `parser.getTextLength < 1`.
    
    This is also partly because I guess the failed cases are relatively low.



---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/63605/
    Test PASSed.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #62659 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62659/consoleFull)** for PR 14102 at commit [`4cf5d11`](https://github.com/apache/spark/commit/4cf5d11b0a24bab7f1baf9400511b377e33d90c9).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74380990
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,296 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
    +
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  /**
    +   * This function deals with the cases it fails to parse. This function will be called
    +   * when exceptions are caught during converting. This functions also deals with `mode` option.
    +   */
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * This function will be called afterward except the case for `StringType`. we
    +   * throw an exception when it is failed unless the value is null.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    +  private def failedConversion(
    --- End diff --
    
    can we move this method near `parseJsooToken`? It's only called there


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74193001
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,337 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedConversion(
    +      parser: JsonParser,
    +      dataType: DataType): Any = parser.getCurrentToken match {
    +    case _ if parser.getTextLength < 1 =>
    +      // If conversion is failed, this produces `null` rather than
    +      // returning empty string. This will protect the mismatch of types.
    +      null
    +
    +    case token =>
    +      // We cannot parse this token based on the given data type. So, we throw a
    +      // SparkSQLJsonProcessingException and this exception will be caught by
    +      // parseJson method.
    +      throw new SparkSQLJsonProcessingException(
    +        s"Failed to parse a value for data type $dataType (current token: $token).")
    +  }
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parser.getCurrentToken match {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +        case _ => failedConversion(parser, st)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parser.getCurrentToken match {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +        case _ => failedConversion(parser, st)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    -
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_TRUE => true
    +          case VALUE_FALSE => false
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ByteType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getByteValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case ShortType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getShortValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    +    case IntegerType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getIntValue
    +          case _ => failedConversion(parser, dataType)
             }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +      }
    +
    +    case LongType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getLongValue
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +    case FloatType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +            parser.getFloatValue
    +
    +          case VALUE_STRING =>
    +            // Special case handling for NaN and Infinity.
    +            val value = parser.getText
    +            val lowerCaseValue = value.toLowerCase
    +            if (lowerCaseValue.equals("nan") ||
    +              lowerCaseValue.equals("infinity") ||
    +              lowerCaseValue.equals("-infinity") ||
    +              lowerCaseValue.equals("inf") ||
    +              lowerCaseValue.equals("-inf")) {
    +              value.toFloat
    +            } else {
    +              throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +            }
    +
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case DoubleType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +            parser.getDoubleValue
    +
    +          case VALUE_STRING =>
    +            // Special case handling for NaN and Infinity.
    +            val value = parser.getText
    +            val lowerCaseValue = value.toLowerCase
    +            if (lowerCaseValue.equals("nan") ||
    +              lowerCaseValue.equals("infinity") ||
    +              lowerCaseValue.equals("-infinity") ||
    +              lowerCaseValue.equals("inf") ||
    +              lowerCaseValue.equals("-inf")) {
    +              value.toDouble
    +            } else {
    +              throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +            }
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +    case StringType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +          case _ =>
    +            val writer = new ByteArrayOutputStream()
    +            Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +              generator => generator.copyCurrentStructure(parser)
    +            }
    +            UTF8String.fromBytes(writer.toByteArray)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +    case TimestampType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            // This one will lose microseconds parts.
    +            // See https://issues.apache.org/jira/browse/SPARK-10681.
    +            DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +          case VALUE_NUMBER_INT =>
    +            parser.getLongValue * 1000000L
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case DateType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            val stringValue = parser.getText
    +            if (stringValue.contains("-")) {
    +              // The format of this string will probably be "yyyy-mm-dd".
    +              DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +            } else {
    +              // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +              // So, we just convert it to Int.
    +              stringValue.toInt
    +            }
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +    case BinaryType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING => parser.getBinaryValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
    +
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +            Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +
    +          case _ => failedConversion(parser, dt)
    +        }
    +      }
    +
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          case _ => failedConversion(parser, st)
    +        }
    +      }
    +
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case START_ARRAY => convertArray(parser, elementConverter)
    +          case _ => failedConversion(parser, at)
    +        }
    +      }
    +
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case START_OBJECT => convertMap(parser, valueConverter)
    +          case _ => failedConversion(parser, mt)
    +        }
    +      }
    +
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
    +
    +    case _ =>
    +      (parser: JsonParser) =>
    +        failedConversion(parser, dataType)
    +  }
    +
    +  /**
    +   * This converts a field. If this is called after `START_OBJECT`, then, the next token can be
    +   * `FIELD_NAME`. Since the names are kept in `JacksonParser.convertObject`, this `FIELD_NAME`
    +   * token can be skipped as below. When this is called after `START_ARRAY`, the tokens become
    +   * ones about values until `END_ARRAY`. In this case, we don't have to skip.
    +   */
    +  private def convertField(parser: JsonParser)(f: => Any): Any = {
    --- End diff --
    
    The name is misleading, it looks to me that this method just skips `FIELD_NAME` and handles null. Actually can't we merge this method and `convertValue`.


---
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


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

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r71095725
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONOptions.scala ---
    @@ -51,7 +53,8 @@ private[sql] class JSONOptions(
         parameters.get("allowBackslashEscapingAnyCharacter").map(_.toBoolean).getOrElse(false)
       val compressionCodec = parameters.get("compression").map(CompressionCodecs.getCodecClassName)
       private val parseMode = parameters.getOrElse("mode", "PERMISSIVE")
    -  val columnNameOfCorruptRecord = parameters.get("columnNameOfCorruptRecord")
    +  val columnNameOfCorruptRecord =
    +    parameters.getOrElse("columnNameOfCorruptRecord", sqlConf.columnNameOfCorruptRecord)
    --- End diff --
    
    I do not think we should pass in `sqlConf` just for this change.


---
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


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

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r71096761
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,306 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +private[sql] class JacksonParser(schema: StructType, options: JSONOptions) extends Logging {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedConversion(
    +      parser: JsonParser,
    +      dataType: DataType): Any = parser.getCurrentToken match {
    +    case null | VALUE_NULL =>
    +      null
    +
    +    case _ if parser.getTextLength < 1 =>
    +      // guard the non string type
    +      null
    +
    +    case token =>
    +    // We cannot parse this token based on the given data type. So, we throw a
    +    // SparkSQLJsonProcessingException and this exception will be caught by
    +    // parseJson method.
    +    throw new SparkSQLJsonProcessingException(
    +      s"Failed to parse a value for data type $dataType (current token: $token).")
    +  }
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(options.columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    -        // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      // SPARK-3308: support reading top level JSON arrays and take every element
    +      // in such an array as a row
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parser.getCurrentToken match {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +        case _ => failedConversion(parser, st)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case ArrayType(st: StructType, _) =>
    +      // the business end of SPARK-3308:
    +      // when an object is found but an array is requested just wrap it in a list
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parser.getCurrentToken match {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +        case _ => failedConversion(parser, st)
    +      }
    +
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    -
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_TRUE => true
    +          case VALUE_FALSE => false
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ByteType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getByteValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case ShortType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getShortValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    +    case IntegerType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getIntValue
    +          case _ => failedConversion(parser, dataType)
             }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +      }
    +
    +    case LongType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getLongValue
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
    +
    +    case FloatType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +            parser.getFloatValue
    +
    +          case VALUE_STRING =>
    +            // Special case handling for NaN and Infinity.
    +            val value = parser.getText
    +            val lowerCaseValue = value.toLowerCase
    +            if (lowerCaseValue.equals("nan") ||
    +              lowerCaseValue.equals("infinity") ||
    +              lowerCaseValue.equals("-infinity") ||
    +              lowerCaseValue.equals("inf") ||
    +              lowerCaseValue.equals("-inf")) {
    +              value.toFloat
    +            } else {
    +              throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +            }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case DoubleType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +            parser.getDoubleValue
    +
    +          case VALUE_STRING =>
    +            // Special case handling for NaN and Infinity.
    +            val value = parser.getText
    +            val lowerCaseValue = value.toLowerCase
    +            if (lowerCaseValue.equals("nan") ||
    +              lowerCaseValue.equals("infinity") ||
    +              lowerCaseValue.equals("-infinity") ||
    +              lowerCaseValue.equals("inf") ||
    +              lowerCaseValue.equals("-inf")) {
    +              value.toDouble
    +            } else {
    +              throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +            }
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +    case StringType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +          case token if token != VALUE_NULL =>
    +            val writer = new ByteArrayOutputStream()
    +            Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +              generator => generator.copyCurrentStructure(parser)
    +            }
    +            UTF8String.fromBytes(writer.toByteArray)
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +    case TimestampType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            // This one will lose microseconds parts.
    +            // See https://issues.apache.org/jira/browse/SPARK-10681.
    +            DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +          case VALUE_NUMBER_INT =>
    +            parser.getLongValue * 1000000L
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +    case DateType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            val stringValue = parser.getText
    +            if (stringValue.contains("-")) {
    +              // The format of this string will probably be "yyyy-mm-dd".
    +              DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +            } else {
    +              // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +              // So, we just convert it to Int.
    +              stringValue.toInt
    +            }
    +
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
    +
    +    case BinaryType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING => parser.getBinaryValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
    +
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +            Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +
    +          case _ => failedConversion(parser, dt)
    +        }
    +      }
    +
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          case _ => failedConversion(parser, st)
    +        }
    +      }
    +
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case START_ARRAY => convertArray(parser, elementConverter)
    +          case _ => failedConversion(parser, at)
    +        }
    +      }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case START_OBJECT => convertMap(parser, valueConverter)
    +          case _ => failedConversion(parser, mt)
    +        }
    +      }
     
    -      case (_, udt: UserDefinedType[_]) =>
    -        convertField(factory, parser, udt.sqlType)
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
     
    -      case (token, dataType) =>
    -        // We cannot parse this token based on the given data type. So, we throw a
    -        // SparkSQLJsonProcessingException and this exception will be caught by
    -        // parseJson method.
    -        throw new SparkSQLJsonProcessingException(
    -          s"Failed to parse a value for data type $dataType (current token: $token).")
    +    case _ =>
    +      (parser: JsonParser) =>
    +        failedConversion(parser, dataType)
    +  }
    +
    +  private def skipFieldNameTokenIfExists(parser: JsonParser)(f: => Any): Any = {
    --- End diff --
    
    doc 


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL][WIP] Avoid record-per type dispatch i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #61949 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/61949/consoleFull)** for PR 14102 at commit [`2d77f66`](https://github.com/apache/spark/commit/2d77f66f2c78bb139212011bfa1fa2efbf6b9d5b).


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74392578
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,303 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
    +
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  /**
    +   * This function deals with the cases it fails to parse. This function will be called
    +   * when exceptions are caught during converting. This functions also deals with `mode` option.
    +   */
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_TRUE => true
    +        case VALUE_FALSE => false
    +      }
     
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    -        }
    +    case ByteType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getByteValue
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ShortType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getShortValue
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case IntegerType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getIntValue
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    -        }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    -        }
    +    case LongType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getLongValue
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    -        }
    +    case FloatType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getFloatValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toFloat
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +          }
    +      }
    +
    +    case DoubleType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getDoubleValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toDouble
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +          }
    +      }
    +
    +    case StringType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          UTF8String.fromString(parser.getText)
    +
    +        case _ =>
    +          // Note that it always tries to convert the data as string without the case of failure.
    +          val writer = new ByteArrayOutputStream()
    +          Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +            generator => generator.copyCurrentStructure(parser)
    +          }
    +          UTF8String.fromBytes(writer.toByteArray)
    +      }
    +
    +    case TimestampType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          // This one will lose microseconds parts.
    +          // See https://issues.apache.org/jira/browse/SPARK-10681.
    +          DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +        case VALUE_NUMBER_INT =>
    +          parser.getLongValue * 1000000L
    +      }
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +    case DateType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          val stringValue = parser.getText
    +          if (stringValue.contains("-")) {
    +            // The format of this string will probably be "yyyy-mm-dd".
    +            DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +          } else {
    +            // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +            // So, we just convert it to Int.
    +            stringValue.toInt
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +    case BinaryType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING => parser.getBinaryValue
    +      }
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +          Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +      }
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +      }
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertMap(parser, valueConverter)
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +    case _ =>
    +      (parser: JsonParser) =>
    +        // Here, we pass empty `PartialFunction` so that this case can be
    +        // handled as a failed conversion. It will throw an exception as
    +        // long as the value is not null.
    +        parseJsonToken(parser, dataType)(PartialFunction.empty[JsonToken, Any])
    +  }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +  /**
    +   * This handles nulls ahead before trying to check the tokens, and applies the conversion
    +   * function and then checks failed the conversion afterward if the `f` fails to convert
    +   * the value.
    +   *
    +   * In more details, it checks `FIELD_NAME` if exists and then skip. If this is called after
    +   * `START_OBJECT`, then, the next token can be `FIELD_NAME`. Since the names are kept in
    +   * `JacksonParser.convertObject`, this `FIELD_NAME` token can be skipped as below. When this
    +   * is called after `START_ARRAY`, the tokens are consecutive tokens for values without
    +   * `FIELD_NAME` until `END_ARRAY`. In this case, we don't have to skip.
    +   *
    +   * For example, parsing ["a", "b", "c"] will produce the tokens as below:
    +   *
    +   *   [START_ARRAY, VALUE_STRING, VALUE_STRING, VALUE_STRING, END_ARRAY]
    +   *
    +   * Then, we check if the current token is null or not. Then, we apply `f` to convert
    +   * the value and then we check failed conversion afterward if the `f` fails to convert the value.
    +   */
    +  private def parseJsonToken(
    +      parser: JsonParser,
    +      dataType: DataType)(f: PartialFunction[JsonToken, Any]): Any = {
    +    parser.getCurrentToken match {
    +      case FIELD_NAME =>
    +        // Here, probably we are parsing consecutive pairs of a field name and a value
    +        // in a JSON object between `START_OBJECT` and `END_OBJECT` tokens.
    +        parser.nextToken()
    +        parseJsonToken(parser, dataType)(f)
     
    -      case (_, udt: UserDefinedType[_]) =>
    -        convertField(factory, parser, udt.sqlType)
    +      case null | VALUE_NULL => null
     
    -      case (token, dataType) =>
    -        // We cannot parse this token based on the given data type. So, we throw a
    -        // SparkSQLJsonProcessingException and this exception will be caught by
    -        // parseJson method.
    -        throw new SparkSQLJsonProcessingException(
    -          s"Failed to parse a value for data type $dataType (current token: $token).")
    +      case other => f.applyOrElse(other, failedConversion(parser, dataType))
         }
       }
     
       /**
    +   * This function will be called afterward except the case for `StringType`. we
    +   * throw an exception when it is failed unless the value is null.
    --- End diff --
    
    ```
    this function throws an exception for failed conversion, but will return null for empty string, to guard the non string types.
    ```


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid record-per type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/61961/
    Test PASSed.


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74380753
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,296 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
    +
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  /**
    +   * This function deals with the cases it fails to parse. This function will be called
    +   * when exceptions are caught during converting. This functions also deals with `mode` option.
    +   */
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * This function will be called afterward except the case for `StringType`. we
    +   * throw an exception when it is failed unless the value is null.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    +  private def failedConversion(
           parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +      dataType: DataType): PartialFunction[JsonToken, Any] = {
    +    case VALUE_STRING if parser.getTextLength < 1 =>
    +      // If conversion is failed, this produces `null` rather than
    +      // rather than throw exception. This will protect the mismatch of types.
    +      null
    +
    +    case token =>
    +      // We cannot parse this token based on the given data type. So, we throw a
    +      // SparkSQLJsonProcessingException and this exception will be caught by
    +      // `parse` method.
    +      throw new SparkSQLJsonProcessingException(
    +        s"Failed to parse a value for data type $dataType (current token: $token).")
    +  }
    +
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
    +   */
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    --- End diff --
    
    nvm, looks like we can't


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74378396
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,296 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
    +
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  /**
    +   * This function deals with the cases it fails to parse. This function will be called
    +   * when exceptions are caught during converting. This functions also deals with `mode` option.
    +   */
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * This function will be called afterward except the case for `StringType`. we
    +   * throw an exception when it is failed unless the value is null.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    +  private def failedConversion(
           parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +      dataType: DataType): PartialFunction[JsonToken, Any] = {
    +    case VALUE_STRING if parser.getTextLength < 1 =>
    +      // If conversion is failed, this produces `null` rather than
    +      // rather than throw exception. This will protect the mismatch of types.
    +      null
    +
    +    case token =>
    +      // We cannot parse this token based on the given data type. So, we throw a
    +      // SparkSQLJsonProcessingException and this exception will be caught by
    +      // `parse` method.
    +      throw new SparkSQLJsonProcessingException(
    +        s"Failed to parse a value for data type $dataType (current token: $token).")
    +  }
    +
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
    +   */
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_TRUE => true
    +        case VALUE_FALSE => false
    +      }
     
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    -        }
    +    case ByteType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getByteValue
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ShortType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getShortValue
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case IntegerType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getIntValue
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    -        }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    -        }
    +    case LongType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getLongValue
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    -        }
    +    case FloatType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getFloatValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toFloat
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +          }
    +      }
    +
    +    case DoubleType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getDoubleValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toDouble
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case StringType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +        case _ =>
    +          // Note that it always tries to convert the data as string without the case of failure.
    +          val writer = new ByteArrayOutputStream()
    +          Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +            generator => generator.copyCurrentStructure(parser)
    +          }
    +          UTF8String.fromBytes(writer.toByteArray)
    +      }
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +    case TimestampType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          // This one will lose microseconds parts.
    +          // See https://issues.apache.org/jira/browse/SPARK-10681.
    +          DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +        case VALUE_NUMBER_INT =>
    +          parser.getLongValue * 1000000L
    +      }
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +    case DateType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          val stringValue = parser.getText
    +          if (stringValue.contains("-")) {
    +            // The format of this string will probably be "yyyy-mm-dd".
    +            DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +          } else {
    +            // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +            // So, we just convert it to Int.
    +            stringValue.toInt
    +          }
    +      }
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +    case BinaryType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING => parser.getBinaryValue
    +      }
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +          Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +      }
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertMap(parser, valueConverter)
    +      }
    +
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
    +
    +    case _ =>
    +      (parser: JsonParser) =>
    +        // Here, we pass empty `PartialFunction` so that this case can be
    +        // handled as a failed conversion. It will throw an exception as
    +        // long as the value is not null.
    +        parseJsonToken(parser, dataType)(PartialFunction.empty[JsonToken, Any])
    +  }
    +
    +  /**
    +   * This handles nulls ahead before trying to check the tokens, and applies the conversion
    +   * function and then checks failed the conversion afterward if it `f` fails to convert the value.
    --- End diff --
    
    nit: `if the f`


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #63525 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/63525/consoleFull)** for PR 14102 at commit [`726aa7f`](https://github.com/apache/spark/commit/726aa7fcb1a438d0a3885e77d1004bb782bbf5ec).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


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

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r71099616
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,306 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +private[sql] class JacksonParser(schema: StructType, options: JSONOptions) extends Logging {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedConversion(
    +      parser: JsonParser,
    +      dataType: DataType): Any = parser.getCurrentToken match {
    +    case null | VALUE_NULL =>
    +      null
    +
    +    case _ if parser.getTextLength < 1 =>
    +      // guard the non string type
    +      null
    +
    +    case token =>
    +    // We cannot parse this token based on the given data type. So, we throw a
    +    // SparkSQLJsonProcessingException and this exception will be caught by
    +    // parseJson method.
    +    throw new SparkSQLJsonProcessingException(
    +      s"Failed to parse a value for data type $dataType (current token: $token).")
    +  }
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(options.columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    -        // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      // SPARK-3308: support reading top level JSON arrays and take every element
    +      // in such an array as a row
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parser.getCurrentToken match {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +        case _ => failedConversion(parser, st)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case ArrayType(st: StructType, _) =>
    +      // the business end of SPARK-3308:
    +      // when an object is found but an array is requested just wrap it in a list
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parser.getCurrentToken match {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +        case _ => failedConversion(parser, st)
    +      }
    +
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    -
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_TRUE => true
    +          case VALUE_FALSE => false
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ByteType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getByteValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case ShortType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getShortValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    +    case IntegerType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getIntValue
    +          case _ => failedConversion(parser, dataType)
             }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +      }
    +
    +    case LongType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getLongValue
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
    +
    +    case FloatType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +            parser.getFloatValue
    +
    +          case VALUE_STRING =>
    +            // Special case handling for NaN and Infinity.
    +            val value = parser.getText
    +            val lowerCaseValue = value.toLowerCase
    +            if (lowerCaseValue.equals("nan") ||
    +              lowerCaseValue.equals("infinity") ||
    +              lowerCaseValue.equals("-infinity") ||
    +              lowerCaseValue.equals("inf") ||
    +              lowerCaseValue.equals("-inf")) {
    +              value.toFloat
    +            } else {
    +              throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +            }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case DoubleType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +            parser.getDoubleValue
    +
    +          case VALUE_STRING =>
    +            // Special case handling for NaN and Infinity.
    +            val value = parser.getText
    +            val lowerCaseValue = value.toLowerCase
    +            if (lowerCaseValue.equals("nan") ||
    +              lowerCaseValue.equals("infinity") ||
    +              lowerCaseValue.equals("-infinity") ||
    +              lowerCaseValue.equals("inf") ||
    +              lowerCaseValue.equals("-inf")) {
    +              value.toDouble
    +            } else {
    +              throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +            }
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +    case StringType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +          case token if token != VALUE_NULL =>
    --- End diff --
    
    Oh, yes this was a bit dirty one and related your comment above, https://github.com/apache/spark/pull/14102#discussion_r71096388.
    
    It was also related with it was [here](https://github.com/apache/spark/blob/a95252823e09939b654dd425db38dadc4100bc87/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala#L83-L84) and [here](https://github.com/apache/spark/blob/a95252823e09939b654dd425db38dadc4100bc87/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala#L93-L95). `VALUE_NULL` was always being checked first so, it did not have to check `token != VALUE_NULL` before [here]((https://github.com/apache/spark/blob/a95252823e09939b654dd425db38dadc4100bc87/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala#L119-L124)).
    
    However, I move both `case null | VALUE_NULL =>` and `case _ if parser.getTextLength < 1 =>` to the last in `failedConversion` (they were being checked first). 
    
    Should I make another function for both cases and make this check them first like the original one? 
     



---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid record-per type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/61964/
    Test FAILed.


---
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


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

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74382183
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,296 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
    +
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  /**
    +   * This function deals with the cases it fails to parse. This function will be called
    +   * when exceptions are caught during converting. This functions also deals with `mode` option.
    +   */
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * This function will be called afterward except the case for `StringType`. we
    +   * throw an exception when it is failed unless the value is null.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    +  private def failedConversion(
           parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +      dataType: DataType): PartialFunction[JsonToken, Any] = {
    +    case VALUE_STRING if parser.getTextLength < 1 =>
    +      // If conversion is failed, this produces `null` rather than
    +      // rather than throw exception. This will protect the mismatch of types.
    +      null
    +
    +    case token =>
    +      // We cannot parse this token based on the given data type. So, we throw a
    +      // SparkSQLJsonProcessingException and this exception will be caught by
    +      // `parse` method.
    +      throw new SparkSQLJsonProcessingException(
    +        s"Failed to parse a value for data type $dataType (current token: $token).")
    +  }
    +
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
    +   */
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    --- End diff --
    
    Hm.. maybe I am not smart enough but this is what I think:
    
    As we are making converters according to the schema once, we can't know the cases for `(START_ARRAY, StructType)` and `(START_OBJECT, ArrayType)` ahead because we should check the tokens `START_OBJECT` or `START_ARRAY` together. 
    
    So, the root converter should have some logics to check the tokens during actual parsing as below.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid record-per type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Merged build finished. Test PASSed.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/63592/
    Test PASSed.


---
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


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

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74220035
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,330 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedConversion(
    +      parser: JsonParser,
    +      dataType: DataType): Any = parser.getCurrentToken match {
    +    case VALUE_STRING if parser.getTextLength < 1 =>
    +      // If conversion is failed, this produces `null` rather than
    +      // returning empty string. This will protect the mismatch of types.
    +      null
    +
    +    case token =>
    +      // We cannot parse this token based on the given data type. So, we throw a
    +      // SparkSQLJsonProcessingException and this exception will be caught by
    +      // parseJson method.
    +      throw new SparkSQLJsonProcessingException(
    +        s"Failed to parse a value for data type $dataType (current token: $token).")
    +  }
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parser.getCurrentToken match {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +        case _ => failedConversion(parser, st)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parser.getCurrentToken match {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +        case _ => failedConversion(parser, st)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    -
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_TRUE => true
    +          case VALUE_FALSE => false
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ByteType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getByteValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case ShortType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getShortValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    +    case IntegerType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getIntValue
    +          case _ => failedConversion(parser, dataType)
             }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +      }
    +
    +    case LongType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getLongValue
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
    +
    +    case FloatType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +            parser.getFloatValue
    +
    +          case VALUE_STRING =>
    +            // Special case handling for NaN and Infinity.
    +            val value = parser.getText
    +            val lowerCaseValue = value.toLowerCase
    +            if (lowerCaseValue.equals("nan") ||
    +              lowerCaseValue.equals("infinity") ||
    +              lowerCaseValue.equals("-infinity") ||
    +              lowerCaseValue.equals("inf") ||
    +              lowerCaseValue.equals("-inf")) {
    +              value.toFloat
    +            } else {
    +              throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +            }
    +
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
    +
    +    case DoubleType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +            parser.getDoubleValue
    +
    +          case VALUE_STRING =>
    +            // Special case handling for NaN and Infinity.
    +            val value = parser.getText
    +            val lowerCaseValue = value.toLowerCase
    +            if (lowerCaseValue.equals("nan") ||
    +              lowerCaseValue.equals("infinity") ||
    +              lowerCaseValue.equals("-infinity") ||
    +              lowerCaseValue.equals("inf") ||
    +              lowerCaseValue.equals("-inf")) {
    +              value.toDouble
    +            } else {
    +              throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +            }
    +
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
    +
    +    case StringType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +          case _ =>
    +            val writer = new ByteArrayOutputStream()
    +            Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +              generator => generator.copyCurrentStructure(parser)
    +            }
    +            UTF8String.fromBytes(writer.toByteArray)
             }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case TimestampType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            // This one will lose microseconds parts.
    +            // See https://issues.apache.org/jira/browse/SPARK-10681.
    +            DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +          case VALUE_NUMBER_INT =>
    +            parser.getLongValue * 1000000L
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +    case DateType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            val stringValue = parser.getText
    +            if (stringValue.contains("-")) {
    +              // The format of this string will probably be "yyyy-mm-dd".
    +              DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +            } else {
    +              // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +              // So, we just convert it to Int.
    +              stringValue.toInt
    +            }
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
    +
    +    case BinaryType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING => parser.getBinaryValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +            Decimal(parser.getDecimalValue, dt.precision, dt.scale)
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +          case _ => failedConversion(parser, dt)
    +        }
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          case _ => failedConversion(parser, st)
    +        }
    +      }
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case START_ARRAY => convertArray(parser, elementConverter)
    +          case _ => failedConversion(parser, at)
    +        }
    +      }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case START_OBJECT => convertMap(parser, valueConverter)
    +          case _ => failedConversion(parser, mt)
    +        }
    +      }
     
    -      case (_, udt: UserDefinedType[_]) =>
    -        convertField(factory, parser, udt.sqlType)
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
     
    -      case (token, dataType) =>
    -        // We cannot parse this token based on the given data type. So, we throw a
    -        // SparkSQLJsonProcessingException and this exception will be caught by
    -        // parseJson method.
    -        throw new SparkSQLJsonProcessingException(
    -          s"Failed to parse a value for data type $dataType (current token: $token).")
    +    case _ =>
    +      (parser: JsonParser) =>
    +        failedConversion(parser, dataType)
    +  }
    +
    +  /**
    +   * This handles nulls ahead before trying to check the tokens, and applies
    +   * the conversion function. Also, it checks `FIELD_NAME` if exists and then skip.
    +   * If this is called after `START_OBJECT`, then, the next token can be
    +   * `FIELD_NAME`. Since the names are kept in `JacksonParser.convertObject`, this `FIELD_NAME`
    +   * token can be skipped as below. When this is called after `START_ARRAY`, the tokens become
    +   * ones about values until `END_ARRAY`. In this case, we don't have to skip.
    +   */
    +  private def handleNullAndFieldTokens (parser: JsonParser)(f: => Any): Any = {
    --- End diff --
    
    Cool, I woll try do follow this suggestion tomorrow. Thanks!


---
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


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

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r71099344
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,306 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +private[sql] class JacksonParser(schema: StructType, options: JSONOptions) extends Logging {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedConversion(
    +      parser: JsonParser,
    +      dataType: DataType): Any = parser.getCurrentToken match {
    +    case null | VALUE_NULL =>
    +      null
    +
    +    case _ if parser.getTextLength < 1 =>
    +      // guard the non string type
    +      null
    --- End diff --
    
    These two cases are only the ones which do not care of `DataType`, it was [here](https://github.com/apache/spark/blob/a95252823e09939b654dd425db38dadc4100bc87/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala#L83-L84) and [here](https://github.com/apache/spark/blob/a95252823e09939b654dd425db38dadc4100bc87/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala#L93-L95).
    So, I put them here because all the `ValueConverter` should treat this case. I can move them into `ValueConverter` if it looks not great.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid record-per type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #61968 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/61968/consoleFull)** for PR 14102 at commit [`83a7ee9`](https://github.com/apache/spark/commit/83a7ee9bb5aca13d3eb51132ff220b9498aaa412).


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/63575/
    Test PASSed.


---
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


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

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r70029359
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONOptions.scala ---
    @@ -51,7 +53,8 @@ private[sql] class JSONOptions(
         parameters.get("allowBackslashEscapingAnyCharacter").map(_.toBoolean).getOrElse(false)
       val compressionCodec = parameters.get("compression").map(CompressionCodecs.getCodecClassName)
       private val parseMode = parameters.getOrElse("mode", "PERMISSIVE")
    -  val columnNameOfCorruptRecord = parameters.get("columnNameOfCorruptRecord")
    +  val columnNameOfCorruptRecord =
    +    parameters.getOrElse("columnNameOfCorruptRecord", sqlConf.columnNameOfCorruptRecord)
    --- End diff --
    
    Here, it resembles `ParquetOptions` (The duplicated logics about default values are put here together).


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74381204
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,296 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
    +
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  /**
    +   * This function deals with the cases it fails to parse. This function will be called
    +   * when exceptions are caught during converting. This functions also deals with `mode` option.
    +   */
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * This function will be called afterward except the case for `StringType`. we
    +   * throw an exception when it is failed unless the value is null.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    +  private def failedConversion(
           parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +      dataType: DataType): PartialFunction[JsonToken, Any] = {
    +    case VALUE_STRING if parser.getTextLength < 1 =>
    +      // If conversion is failed, this produces `null` rather than
    +      // rather than throw exception. This will protect the mismatch of types.
    +      null
    +
    +    case token =>
    +      // We cannot parse this token based on the given data type. So, we throw a
    +      // SparkSQLJsonProcessingException and this exception will be caught by
    +      // `parse` method.
    +      throw new SparkSQLJsonProcessingException(
    +        s"Failed to parse a value for data type $dataType (current token: $token).")
    +  }
    +
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
    +   */
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_TRUE => true
    +        case VALUE_FALSE => false
    +      }
     
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    -        }
    +    case ByteType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getByteValue
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ShortType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getShortValue
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case IntegerType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getIntValue
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    -        }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    -        }
    +    case LongType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getLongValue
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    -        }
    +    case FloatType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getFloatValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toFloat
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +          }
    +      }
    +
    +    case DoubleType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getDoubleValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toDouble
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case StringType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +        case _ =>
    +          // Note that it always tries to convert the data as string without the case of failure.
    +          val writer = new ByteArrayOutputStream()
    +          Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +            generator => generator.copyCurrentStructure(parser)
    +          }
    +          UTF8String.fromBytes(writer.toByteArray)
    +      }
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +    case TimestampType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          // This one will lose microseconds parts.
    +          // See https://issues.apache.org/jira/browse/SPARK-10681.
    +          DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +        case VALUE_NUMBER_INT =>
    +          parser.getLongValue * 1000000L
    +      }
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +    case DateType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          val stringValue = parser.getText
    +          if (stringValue.contains("-")) {
    +            // The format of this string will probably be "yyyy-mm-dd".
    +            DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +          } else {
    +            // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +            // So, we just convert it to Int.
    +            stringValue.toInt
    +          }
    +      }
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +    case BinaryType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING => parser.getBinaryValue
    +      }
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +          Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +      }
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertMap(parser, valueConverter)
    +      }
    +
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
    +
    +    case _ =>
    +      (parser: JsonParser) =>
    +        // Here, we pass empty `PartialFunction` so that this case can be
    +        // handled as a failed conversion. It will throw an exception as
    +        // long as the value is not null.
    +        parseJsonToken(parser, dataType)(PartialFunction.empty[JsonToken, Any])
    +  }
    +
    +  /**
    +   * This handles nulls ahead before trying to check the tokens, and applies the conversion
    +   * function and then checks failed the conversion afterward if it `f` fails to convert the value.
    +   *
    +   * In more details, it checks `FIELD_NAME` if exists and then skip. If this is called after
    +   * `START_OBJECT`, then, the next token can be `FIELD_NAME`. Since the names are kept in
    +   * `JacksonParser.convertObject`, this `FIELD_NAME` token can be skipped as below. When this
    +   * is called after `START_ARRAY`, the tokens become ones about values until `END_ARRAY`.
    +   * In this case, we don't have to skip.
    --- End diff --
    
    ```
    ... When this
     * is called after `START_ARRAY`, the tokens become ones about values until `END_ARRAY`.
     * In this case, we don't have to skip.
    ```
    what do you mean?


---
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


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

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74382623
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,296 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
    +
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  /**
    +   * This function deals with the cases it fails to parse. This function will be called
    +   * when exceptions are caught during converting. This functions also deals with `mode` option.
    +   */
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * This function will be called afterward except the case for `StringType`. we
    +   * throw an exception when it is failed unless the value is null.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    +  private def failedConversion(
           parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +      dataType: DataType): PartialFunction[JsonToken, Any] = {
    +    case VALUE_STRING if parser.getTextLength < 1 =>
    +      // If conversion is failed, this produces `null` rather than
    +      // rather than throw exception. This will protect the mismatch of types.
    +      null
    +
    +    case token =>
    +      // We cannot parse this token based on the given data type. So, we throw a
    +      // SparkSQLJsonProcessingException and this exception will be caught by
    +      // `parse` method.
    +      throw new SparkSQLJsonProcessingException(
    +        s"Failed to parse a value for data type $dataType (current token: $token).")
    +  }
    +
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
    +   */
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_TRUE => true
    +        case VALUE_FALSE => false
    +      }
     
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    -        }
    +    case ByteType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getByteValue
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ShortType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getShortValue
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case IntegerType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getIntValue
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    -        }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    -        }
    +    case LongType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getLongValue
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    -        }
    +    case FloatType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getFloatValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toFloat
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +          }
    +      }
    +
    +    case DoubleType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getDoubleValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toDouble
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case StringType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +        case _ =>
    +          // Note that it always tries to convert the data as string without the case of failure.
    +          val writer = new ByteArrayOutputStream()
    +          Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +            generator => generator.copyCurrentStructure(parser)
    +          }
    +          UTF8String.fromBytes(writer.toByteArray)
    +      }
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +    case TimestampType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          // This one will lose microseconds parts.
    +          // See https://issues.apache.org/jira/browse/SPARK-10681.
    +          DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +        case VALUE_NUMBER_INT =>
    +          parser.getLongValue * 1000000L
    +      }
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +    case DateType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          val stringValue = parser.getText
    +          if (stringValue.contains("-")) {
    +            // The format of this string will probably be "yyyy-mm-dd".
    +            DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +          } else {
    +            // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +            // So, we just convert it to Int.
    +            stringValue.toInt
    +          }
    +      }
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +    case BinaryType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING => parser.getBinaryValue
    +      }
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +          Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +      }
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertMap(parser, valueConverter)
    +      }
    +
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
    +
    +    case _ =>
    +      (parser: JsonParser) =>
    +        // Here, we pass empty `PartialFunction` so that this case can be
    +        // handled as a failed conversion. It will throw an exception as
    +        // long as the value is not null.
    +        parseJsonToken(parser, dataType)(PartialFunction.empty[JsonToken, Any])
    +  }
    +
    +  /**
    +   * This handles nulls ahead before trying to check the tokens, and applies the conversion
    +   * function and then checks failed the conversion afterward if it `f` fails to convert the value.
    +   *
    +   * In more details, it checks `FIELD_NAME` if exists and then skip. If this is called after
    +   * `START_OBJECT`, then, the next token can be `FIELD_NAME`. Since the names are kept in
    +   * `JacksonParser.convertObject`, this `FIELD_NAME` token can be skipped as below. When this
    +   * is called after `START_ARRAY`, the tokens become ones about values until `END_ARRAY`.
    +   * In this case, we don't have to skip.
    --- End diff --
    
    Ah, the tokens after `START_ARRAY` will be consecutive tokens until `END_ARRAY`. For example,
    
    ```scala
    parser.nextToken() // => START_ARRAY
    parser.nextToken() // => VALUE_STRING
    parser.nextToken() // => VALUE_STRING
    parser.nextToken() // => VALUE_STRING
    parser.nextToken() // => END_ARRAY
    ```
    
    In this case, we don't (can't) skip `FIELD_NAME` token. I will try to make the documentation cleaner.


---
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


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

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74372103
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,289 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_TRUE => true
    +        case VALUE_FALSE => false
    +      }
     
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    -        }
    +    case ByteType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getByteValue
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ShortType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getShortValue
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case IntegerType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getIntValue
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    -        }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    -        }
    +    case LongType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getLongValue
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    -        }
    +    case FloatType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getFloatValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toFloat
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case DoubleType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getDoubleValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toDouble
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +    case StringType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +        case _ =>
    +          val writer = new ByteArrayOutputStream()
    +          Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +            generator => generator.copyCurrentStructure(parser)
    +          }
    +          UTF8String.fromBytes(writer.toByteArray)
    +      }
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +    case TimestampType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          // This one will lose microseconds parts.
    +          // See https://issues.apache.org/jira/browse/SPARK-10681.
    +          DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +        case VALUE_NUMBER_INT =>
    +          parser.getLongValue * 1000000L
    +      }
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +    case DateType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          val stringValue = parser.getText
    +          if (stringValue.contains("-")) {
    +            // The format of this string will probably be "yyyy-mm-dd".
    +            DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +          } else {
    +            // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +            // So, we just convert it to Int.
    +            stringValue.toInt
    +          }
    +      }
    +
    +    case BinaryType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING => parser.getBinaryValue
    +      }
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +          Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +      }
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertMap(parser, valueConverter)
    +      }
     
    -      case (_, udt: UserDefinedType[_]) =>
    -        convertField(factory, parser, udt.sqlType)
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
     
    -      case (token, dataType) =>
    -        // We cannot parse this token based on the given data type. So, we throw a
    -        // SparkSQLJsonProcessingException and this exception will be caught by
    -        // parseJson method.
    -        throw new SparkSQLJsonProcessingException(
    -          s"Failed to parse a value for data type $dataType (current token: $token).")
    +    case _ =>
    --- End diff --
    
    Actually, `NullType` and `CalendarIntervalType` (maybe some more?) are missed here. As it is possible for user to specify the schema, I do favour to keep this.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #63602 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/63602/consoleFull)** for PR 14102 at commit [`a8b6a04`](https://github.com/apache/spark/commit/a8b6a04aa753773aaf7d5e582cc4657f69bce9f9).


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid record-per type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #61960 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/61960/consoleFull)** for PR 14102 at commit [`cd44ac5`](https://github.com/apache/spark/commit/cd44ac518645b17f2dc1397d09709f60fa20066d).


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL][WIP] Avoid record-per type dispatch i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #61948 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/61948/consoleFull)** for PR 14102 at commit [`74fa944`](https://github.com/apache/spark/commit/74fa944209491b9884dbfc8b71e56e36b45e28a4).


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Merged build finished. Test PASSed.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    LGTM except some comment suggestions, thanks for working on it!


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Thanks for reviewing this!


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid record-per type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #61961 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/61961/consoleFull)** for PR 14102 at commit [`c5cce08`](https://github.com/apache/spark/commit/c5cce08b615606185855b667f5ffe5c960e098b5).


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #63592 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/63592/consoleFull)** for PR 14102 at commit [`055e14b`](https://github.com/apache/spark/commit/055e14bb665c81979c8ee14b9b7619b2ea6b6d18).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #63481 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/63481/consoleFull)** for PR 14102 at commit [`25c1423`](https://github.com/apache/spark/commit/25c142385a27595699c00e9aeb22ab83b73bc47b).


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Merged build finished. Test PASSed.


---
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


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

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74041085
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,337 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +private[sql] class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedConversion(
    +      parser: JsonParser,
    +      dataType: DataType): Any = parser.getCurrentToken match {
    +    case _ if parser.getTextLength < 1 =>
    --- End diff --
    
    Sure! For `StringType`, it does not fail and call this function (https://github.com/apache/spark/pull/14102/files#diff-8affe5ec7d691943a88e43eb30af656eR231). So, for other types, when the actual data is empty, we should return `null` instead of empty string because it is guaranteed that it is not `StringType`.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid record-per type dispatch in JSO...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    cc @yhuai @liancheng Do you mind if I ask a quick look for this as well please?


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Merged build finished. Test PASSed.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    @cloud-fan Thanks! I think it is ready to be reviewed.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #63522 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/63522/consoleFull)** for PR 14102 at commit [`7077ee6`](https://github.com/apache/spark/commit/7077ee6254ea307afd2c4385c8c19a4fedac89a1).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #63599 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/63599/consoleFull)** for PR 14102 at commit [`aaac7db`](https://github.com/apache/spark/commit/aaac7db58c0b303aefa8eda98b47d2734e29633f).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


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

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74193768
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,337 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +private[sql] class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedConversion(
    +      parser: JsonParser,
    +      dataType: DataType): Any = parser.getCurrentToken match {
    +    case _ if parser.getTextLength < 1 =>
    --- End diff --
    
    Yes, it is.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid record-per type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #61961 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/61961/consoleFull)** for PR 14102 at commit [`c5cce08`](https://github.com/apache/spark/commit/c5cce08b615606185855b667f5ffe5c960e098b5).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Merged build finished. Test PASSed.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Merged build finished. Test PASSed.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #63605 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/63605/consoleFull)** for PR 14102 at commit [`80b2d3a`](https://github.com/apache/spark/commit/80b2d3a9d638eaeb87665a4db4c535954639b148).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/62501/
    Test PASSed.


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74373725
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,289 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_TRUE => true
    +        case VALUE_FALSE => false
    +      }
     
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    -        }
    +    case ByteType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getByteValue
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ShortType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getShortValue
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case IntegerType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getIntValue
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    -        }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    -        }
    +    case LongType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getLongValue
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    -        }
    +    case FloatType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getFloatValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toFloat
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case DoubleType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getDoubleValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toDouble
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +    case StringType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +        case _ =>
    +          val writer = new ByteArrayOutputStream()
    +          Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +            generator => generator.copyCurrentStructure(parser)
    +          }
    +          UTF8String.fromBytes(writer.toByteArray)
    +      }
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +    case TimestampType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          // This one will lose microseconds parts.
    +          // See https://issues.apache.org/jira/browse/SPARK-10681.
    +          DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +        case VALUE_NUMBER_INT =>
    +          parser.getLongValue * 1000000L
    +      }
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +    case DateType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          val stringValue = parser.getText
    +          if (stringValue.contains("-")) {
    +            // The format of this string will probably be "yyyy-mm-dd".
    +            DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +          } else {
    +            // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +            // So, we just convert it to Int.
    +            stringValue.toInt
    +          }
    +      }
    +
    +    case BinaryType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING => parser.getBinaryValue
    +      }
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +          Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +      }
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertMap(parser, valueConverter)
    +      }
     
    -      case (_, udt: UserDefinedType[_]) =>
    -        convertField(factory, parser, udt.sqlType)
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
     
    -      case (token, dataType) =>
    -        // We cannot parse this token based on the given data type. So, we throw a
    -        // SparkSQLJsonProcessingException and this exception will be caught by
    -        // parseJson method.
    -        throw new SparkSQLJsonProcessingException(
    -          s"Failed to parse a value for data type $dataType (current token: $token).")
    +    case _ =>
    +      (parser: JsonParser) =>
    +        parseJsonToken(parser, dataType) {
    +          case token =>
    +            throw new SparkSQLJsonProcessingException(
    +              s"Failed to parse a value for data type $dataType (current token: $token).")
    +        }
    +  }
    +
    +  /**
    +   * This handles nulls ahead before trying to check the tokens, and applies the conversion
    +   * function and then checks failed the conversion afterward if it `f` fails to convert the value.
    +   *
    +   * In more details, it checks `FIELD_NAME` if exists and then skip. If this is called after
    +   * `START_OBJECT`, then, the next token can be `FIELD_NAME`. Since the names are kept in
    +   * `JacksonParser.convertObject`, this `FIELD_NAME` token can be skipped as below. When this
    +   * is called after `START_ARRAY`, the tokens become ones about values until `END_ARRAY`.
    +   * In this case, we don't have to skip.
    +   *
    +   * We check if the current token is null or not after that. Then, we apply `f` to convert
    +   * the value and then we check failed conversion afterward if it `f` fails to convert the value.
    +   */
    +  private def parseJsonToken(
    +      parser: JsonParser,
    +      dataType: DataType)(f: PartialFunction[JsonToken, Any]): Any = {
    +    parser.getCurrentToken match {
    +      case FIELD_NAME =>
    +        parser.nextToken()
    +        parseJsonToken(parser, dataType)(f)
    +
    +      case null | VALUE_NULL => null
    +
    +      case other => f.orElse {
    +        // We should specify the type of this `PartialFunction`. Otherwise this will
    +        // throw a compilation error, "The argument types of an anonymous function
    +        // must be fully known. (SLS 8.5)".
    +        {
    +          case VALUE_STRING if parser.getTextLength < 1 =>
    +            // If conversion is failed, this produces `null` rather than
    +            // rather than throw exception. This will protect the mismatch of types.
    +            null
    +
    +          case token =>
    +            // We cannot parse this token based on the given data type. So, we throw a
    +            // SparkSQLJsonProcessingException and this exception will be caught by
    +            // parseJson method.
    +            throw new SparkSQLJsonProcessingException(
    +              s"Failed to parse a value for data type $dataType (current token: $token).")
    +        }: PartialFunction[JsonToken, Any]
    +      }.apply(other)
         }
       }
     
       /**
        * Parse an object from the token stream into a new Row representing the schema.
    -   *
        * Fields in the json that are not defined in the requested schema will be dropped.
        */
       private def convertObject(
    -      factory: JsonFactory,
           parser: JsonParser,
    -      schema: StructType): InternalRow = {
    -    val row = new GenericMutableRow(schema.length)
    +      currentSchema: StructType,
    --- End diff --
    
    what does `currentSchema` means? `schema` looks better


---
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


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

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r71096584
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,306 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +private[sql] class JacksonParser(schema: StructType, options: JSONOptions) extends Logging {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedConversion(
    +      parser: JsonParser,
    +      dataType: DataType): Any = parser.getCurrentToken match {
    +    case null | VALUE_NULL =>
    +      null
    +
    +    case _ if parser.getTextLength < 1 =>
    +      // guard the non string type
    +      null
    +
    +    case token =>
    +    // We cannot parse this token based on the given data type. So, we throw a
    +    // SparkSQLJsonProcessingException and this exception will be caught by
    +    // parseJson method.
    +    throw new SparkSQLJsonProcessingException(
    +      s"Failed to parse a value for data type $dataType (current token: $token).")
    +  }
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(options.columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    -        // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      // SPARK-3308: support reading top level JSON arrays and take every element
    +      // in such an array as a row
    --- End diff --
    
    Seems you want to put this comment at https://github.com/apache/spark/pull/14102/files#diff-8affe5ec7d691943a88e43eb30af656eR99.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL][WIP] Avoid record-per type dispatch i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/61948/
    Test FAILed.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    @yhuai I addressed the comments!


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid record-per type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Merged build finished. Test FAILed.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #63592 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/63592/consoleFull)** for PR 14102 at commit [`055e14b`](https://github.com/apache/spark/commit/055e14bb665c81979c8ee14b9b7619b2ea6b6d18).


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid record-per type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Merged build finished. Test PASSed.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #63575 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/63575/consoleFull)** for PR 14102 at commit [`bceda7b`](https://github.com/apache/spark/commit/bceda7ba4f06c0b6fd99f11ef2662f9f3a154af0).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/63522/
    Test PASSed.


---
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


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

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r71096571
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,306 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +private[sql] class JacksonParser(schema: StructType, options: JSONOptions) extends Logging {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedConversion(
    +      parser: JsonParser,
    +      dataType: DataType): Any = parser.getCurrentToken match {
    +    case null | VALUE_NULL =>
    +      null
    +
    +    case _ if parser.getTextLength < 1 =>
    +      // guard the non string type
    +      null
    +
    +    case token =>
    +    // We cannot parse this token based on the given data type. So, we throw a
    +    // SparkSQLJsonProcessingException and this exception will be caught by
    +    // parseJson method.
    +    throw new SparkSQLJsonProcessingException(
    +      s"Failed to parse a value for data type $dataType (current token: $token).")
    +  }
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(options.columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    -        // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      // SPARK-3308: support reading top level JSON arrays and take every element
    +      // in such an array as a row
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parser.getCurrentToken match {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +        case _ => failedConversion(parser, st)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case ArrayType(st: StructType, _) =>
    +      // the business end of SPARK-3308:
    +      // when an object is found but an array is requested just wrap it in a list
    --- End diff --
    
    Since we are at here, can you add an example?


---
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


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

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74214974
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,330 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedConversion(
    +      parser: JsonParser,
    +      dataType: DataType): Any = parser.getCurrentToken match {
    +    case VALUE_STRING if parser.getTextLength < 1 =>
    +      // If conversion is failed, this produces `null` rather than
    +      // returning empty string. This will protect the mismatch of types.
    +      null
    +
    +    case token =>
    +      // We cannot parse this token based on the given data type. So, we throw a
    +      // SparkSQLJsonProcessingException and this exception will be caught by
    +      // parseJson method.
    +      throw new SparkSQLJsonProcessingException(
    +        s"Failed to parse a value for data type $dataType (current token: $token).")
    +  }
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parser.getCurrentToken match {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +        case _ => failedConversion(parser, st)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parser.getCurrentToken match {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +        case _ => failedConversion(parser, st)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    -
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_TRUE => true
    +          case VALUE_FALSE => false
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ByteType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getByteValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case ShortType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getShortValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    +    case IntegerType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getIntValue
    +          case _ => failedConversion(parser, dataType)
             }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +      }
    +
    +    case LongType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getLongValue
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
    +
    +    case FloatType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +            parser.getFloatValue
    +
    +          case VALUE_STRING =>
    +            // Special case handling for NaN and Infinity.
    +            val value = parser.getText
    +            val lowerCaseValue = value.toLowerCase
    +            if (lowerCaseValue.equals("nan") ||
    +              lowerCaseValue.equals("infinity") ||
    +              lowerCaseValue.equals("-infinity") ||
    +              lowerCaseValue.equals("inf") ||
    +              lowerCaseValue.equals("-inf")) {
    +              value.toFloat
    +            } else {
    +              throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +            }
    +
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
    +
    +    case DoubleType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +            parser.getDoubleValue
    +
    +          case VALUE_STRING =>
    +            // Special case handling for NaN and Infinity.
    +            val value = parser.getText
    +            val lowerCaseValue = value.toLowerCase
    +            if (lowerCaseValue.equals("nan") ||
    +              lowerCaseValue.equals("infinity") ||
    +              lowerCaseValue.equals("-infinity") ||
    +              lowerCaseValue.equals("inf") ||
    +              lowerCaseValue.equals("-inf")) {
    +              value.toDouble
    +            } else {
    +              throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +            }
    +
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
    +
    +    case StringType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +          case _ =>
    +            val writer = new ByteArrayOutputStream()
    +            Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +              generator => generator.copyCurrentStructure(parser)
    +            }
    +            UTF8String.fromBytes(writer.toByteArray)
             }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case TimestampType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            // This one will lose microseconds parts.
    +            // See https://issues.apache.org/jira/browse/SPARK-10681.
    +            DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +          case VALUE_NUMBER_INT =>
    +            parser.getLongValue * 1000000L
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +    case DateType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            val stringValue = parser.getText
    +            if (stringValue.contains("-")) {
    +              // The format of this string will probably be "yyyy-mm-dd".
    +              DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +            } else {
    +              // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +              // So, we just convert it to Int.
    +              stringValue.toInt
    +            }
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
    +
    +    case BinaryType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING => parser.getBinaryValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +            Decimal(parser.getDecimalValue, dt.precision, dt.scale)
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +          case _ => failedConversion(parser, dt)
    +        }
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          case _ => failedConversion(parser, st)
    +        }
    +      }
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case START_ARRAY => convertArray(parser, elementConverter)
    +          case _ => failedConversion(parser, at)
    +        }
    +      }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case START_OBJECT => convertMap(parser, valueConverter)
    +          case _ => failedConversion(parser, mt)
    +        }
    +      }
     
    -      case (_, udt: UserDefinedType[_]) =>
    -        convertField(factory, parser, udt.sqlType)
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
     
    -      case (token, dataType) =>
    -        // We cannot parse this token based on the given data type. So, we throw a
    -        // SparkSQLJsonProcessingException and this exception will be caught by
    -        // parseJson method.
    -        throw new SparkSQLJsonProcessingException(
    -          s"Failed to parse a value for data type $dataType (current token: $token).")
    +    case _ =>
    +      (parser: JsonParser) =>
    +        failedConversion(parser, dataType)
    +  }
    +
    +  /**
    +   * This handles nulls ahead before trying to check the tokens, and applies
    +   * the conversion function. Also, it checks `FIELD_NAME` if exists and then skip.
    +   * If this is called after `START_OBJECT`, then, the next token can be
    +   * `FIELD_NAME`. Since the names are kept in `JacksonParser.convertObject`, this `FIELD_NAME`
    +   * token can be skipped as below. When this is called after `START_ARRAY`, the tokens become
    +   * ones about values until `END_ARRAY`. In this case, we don't have to skip.
    +   */
    +  private def handleNullAndFieldTokens (parser: JsonParser)(f: => Any): Any = {
    --- End diff --
    
    @cloud-fan I literally tried to come up with a good name for about 30 min but I couldn't. So, I just chose a verbose one..


---
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


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

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r71097210
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,306 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +private[sql] class JacksonParser(schema: StructType, options: JSONOptions) extends Logging {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedConversion(
    +      parser: JsonParser,
    +      dataType: DataType): Any = parser.getCurrentToken match {
    +    case null | VALUE_NULL =>
    +      null
    +
    +    case _ if parser.getTextLength < 1 =>
    +      // guard the non string type
    +      null
    +
    +    case token =>
    +    // We cannot parse this token based on the given data type. So, we throw a
    +    // SparkSQLJsonProcessingException and this exception will be caught by
    +    // parseJson method.
    +    throw new SparkSQLJsonProcessingException(
    +      s"Failed to parse a value for data type $dataType (current token: $token).")
    +  }
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(options.columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    -        // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      // SPARK-3308: support reading top level JSON arrays and take every element
    +      // in such an array as a row
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parser.getCurrentToken match {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +        case _ => failedConversion(parser, st)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case ArrayType(st: StructType, _) =>
    +      // the business end of SPARK-3308:
    +      // when an object is found but an array is requested just wrap it in a list
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parser.getCurrentToken match {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +        case _ => failedConversion(parser, st)
    +      }
    +
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    -
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_TRUE => true
    +          case VALUE_FALSE => false
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ByteType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getByteValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case ShortType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getShortValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    +    case IntegerType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getIntValue
    +          case _ => failedConversion(parser, dataType)
             }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +      }
    +
    +    case LongType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getLongValue
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
    +
    +    case FloatType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +            parser.getFloatValue
    +
    +          case VALUE_STRING =>
    +            // Special case handling for NaN and Infinity.
    +            val value = parser.getText
    +            val lowerCaseValue = value.toLowerCase
    +            if (lowerCaseValue.equals("nan") ||
    +              lowerCaseValue.equals("infinity") ||
    +              lowerCaseValue.equals("-infinity") ||
    +              lowerCaseValue.equals("inf") ||
    +              lowerCaseValue.equals("-inf")) {
    +              value.toFloat
    +            } else {
    +              throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +            }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case DoubleType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +            parser.getDoubleValue
    +
    +          case VALUE_STRING =>
    +            // Special case handling for NaN and Infinity.
    +            val value = parser.getText
    +            val lowerCaseValue = value.toLowerCase
    +            if (lowerCaseValue.equals("nan") ||
    +              lowerCaseValue.equals("infinity") ||
    +              lowerCaseValue.equals("-infinity") ||
    +              lowerCaseValue.equals("inf") ||
    +              lowerCaseValue.equals("-inf")) {
    +              value.toDouble
    +            } else {
    +              throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +            }
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +    case StringType =>
    +      (parser: JsonParser) => skipFieldNameTokenIfExists(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +          case token if token != VALUE_NULL =>
    --- End diff --
    
    explain this condition?


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL][WIP] Avoid record-per type dispatch i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #61956 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/61956/consoleFull)** for PR 14102 at commit [`9dbcec7`](https://github.com/apache/spark/commit/9dbcec74b2fe64e06c2d013da56d6c4a46fdabb7).


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74393606
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,303 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
    +
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  /**
    +   * This function deals with the cases it fails to parse. This function will be called
    +   * when exceptions are caught during converting. This functions also deals with `mode` option.
    +   */
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_TRUE => true
    +        case VALUE_FALSE => false
    +      }
     
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    -        }
    +    case ByteType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getByteValue
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ShortType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getShortValue
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case IntegerType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getIntValue
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    -        }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    -        }
    +    case LongType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getLongValue
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    -        }
    +    case FloatType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getFloatValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toFloat
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +          }
    +      }
    +
    +    case DoubleType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getDoubleValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toDouble
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +          }
    +      }
    +
    +    case StringType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          UTF8String.fromString(parser.getText)
    +
    +        case _ =>
    +          // Note that it always tries to convert the data as string without the case of failure.
    +          val writer = new ByteArrayOutputStream()
    +          Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +            generator => generator.copyCurrentStructure(parser)
    +          }
    +          UTF8String.fromBytes(writer.toByteArray)
    +      }
    +
    +    case TimestampType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          // This one will lose microseconds parts.
    +          // See https://issues.apache.org/jira/browse/SPARK-10681.
    +          DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +        case VALUE_NUMBER_INT =>
    +          parser.getLongValue * 1000000L
    +      }
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +    case DateType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          val stringValue = parser.getText
    +          if (stringValue.contains("-")) {
    +            // The format of this string will probably be "yyyy-mm-dd".
    +            DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +          } else {
    +            // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +            // So, we just convert it to Int.
    +            stringValue.toInt
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +    case BinaryType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING => parser.getBinaryValue
    +      }
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +          Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +      }
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +      }
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertMap(parser, valueConverter)
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +    case _ =>
    +      (parser: JsonParser) =>
    +        // Here, we pass empty `PartialFunction` so that this case can be
    +        // handled as a failed conversion. It will throw an exception as
    +        // long as the value is not null.
    +        parseJsonToken(parser, dataType)(PartialFunction.empty[JsonToken, Any])
    +  }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +  /**
    +   * This handles nulls ahead before trying to check the tokens, and applies the conversion
    +   * function and then checks failed the conversion afterward if the `f` fails to convert
    +   * the value.
    --- End diff --
    
    ```
    This method skips `FIELD_NAME`s at the beginning, and handles nulls ahead before trying to parse the JSON token using given function `f`. If the `f` failed to parse and convert the token, call `failedConversion` to handle the token.
    ```


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74393971
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,303 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
    +
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  /**
    +   * This function deals with the cases it fails to parse. This function will be called
    +   * when exceptions are caught during converting. This functions also deals with `mode` option.
    +   */
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_TRUE => true
    +        case VALUE_FALSE => false
    +      }
     
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    -        }
    +    case ByteType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getByteValue
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ShortType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getShortValue
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case IntegerType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getIntValue
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    -        }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    -        }
    +    case LongType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getLongValue
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    -        }
    +    case FloatType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getFloatValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toFloat
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +          }
    +      }
    +
    +    case DoubleType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getDoubleValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toDouble
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +          }
    +      }
    +
    +    case StringType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          UTF8String.fromString(parser.getText)
    +
    +        case _ =>
    +          // Note that it always tries to convert the data as string without the case of failure.
    +          val writer = new ByteArrayOutputStream()
    +          Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +            generator => generator.copyCurrentStructure(parser)
    +          }
    +          UTF8String.fromBytes(writer.toByteArray)
    +      }
    +
    +    case TimestampType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          // This one will lose microseconds parts.
    +          // See https://issues.apache.org/jira/browse/SPARK-10681.
    +          DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +        case VALUE_NUMBER_INT =>
    +          parser.getLongValue * 1000000L
    +      }
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +    case DateType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          val stringValue = parser.getText
    +          if (stringValue.contains("-")) {
    +            // The format of this string will probably be "yyyy-mm-dd".
    +            DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +          } else {
    +            // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +            // So, we just convert it to Int.
    +            stringValue.toInt
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +    case BinaryType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING => parser.getBinaryValue
    +      }
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +          Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +      }
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +      }
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertMap(parser, valueConverter)
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +    case _ =>
    +      (parser: JsonParser) =>
    +        // Here, we pass empty `PartialFunction` so that this case can be
    +        // handled as a failed conversion. It will throw an exception as
    +        // long as the value is not null.
    +        parseJsonToken(parser, dataType)(PartialFunction.empty[JsonToken, Any])
    +  }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +  /**
    +   * This handles nulls ahead before trying to check the tokens, and applies the conversion
    +   * function and then checks failed the conversion afterward if the `f` fails to convert
    +   * the value.
    +   *
    +   * In more details, it checks `FIELD_NAME` if exists and then skip. If this is called after
    +   * `START_OBJECT`, then, the next token can be `FIELD_NAME`. Since the names are kept in
    +   * `JacksonParser.convertObject`, this `FIELD_NAME` token can be skipped as below. When this
    +   * is called after `START_ARRAY`, the tokens are consecutive tokens for values without
    +   * `FIELD_NAME` until `END_ARRAY`. In this case, we don't have to skip.
    --- End diff --
    
    it's good to know this, but do we really need to put it in comments? It seems not that helpful for people to understand the code. We just need some comments `case FIELD_NAME =>` to explain why we need to skip, i.e. there are useless `FIELD_NAME`s between `START_OBJECT` and `END_OBJECT` tokens.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL][WIP] Avoid record-per type dispatch i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #61948 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/61948/consoleFull)** for PR 14102 at commit [`74fa944`](https://github.com/apache/spark/commit/74fa944209491b9884dbfc8b71e56e36b45e28a4).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Sorry for pinging here and there.. @yhuai @liancheng 


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid record-per type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #61968 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/61968/consoleFull)** for PR 14102 at commit [`83a7ee9`](https://github.com/apache/spark/commit/83a7ee9bb5aca13d3eb51132ff220b9498aaa412).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


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

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74374585
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,289 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_TRUE => true
    +        case VALUE_FALSE => false
    +      }
     
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    -        }
    +    case ByteType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getByteValue
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ShortType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getShortValue
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case IntegerType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getIntValue
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    -        }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    -        }
    +    case LongType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getLongValue
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    -        }
    +    case FloatType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getFloatValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toFloat
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case DoubleType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getDoubleValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toDouble
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +    case StringType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +        case _ =>
    +          val writer = new ByteArrayOutputStream()
    +          Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +            generator => generator.copyCurrentStructure(parser)
    +          }
    +          UTF8String.fromBytes(writer.toByteArray)
    +      }
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +    case TimestampType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          // This one will lose microseconds parts.
    +          // See https://issues.apache.org/jira/browse/SPARK-10681.
    +          DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +        case VALUE_NUMBER_INT =>
    +          parser.getLongValue * 1000000L
    +      }
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +    case DateType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          val stringValue = parser.getText
    +          if (stringValue.contains("-")) {
    +            // The format of this string will probably be "yyyy-mm-dd".
    +            DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +          } else {
    +            // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +            // So, we just convert it to Int.
    +            stringValue.toInt
    +          }
    +      }
    +
    +    case BinaryType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING => parser.getBinaryValue
    +      }
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +          Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +      }
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertMap(parser, valueConverter)
    +      }
     
    -      case (_, udt: UserDefinedType[_]) =>
    -        convertField(factory, parser, udt.sqlType)
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
     
    -      case (token, dataType) =>
    -        // We cannot parse this token based on the given data type. So, we throw a
    -        // SparkSQLJsonProcessingException and this exception will be caught by
    -        // parseJson method.
    -        throw new SparkSQLJsonProcessingException(
    -          s"Failed to parse a value for data type $dataType (current token: $token).")
    +    case _ =>
    +      (parser: JsonParser) =>
    +        parseJsonToken(parser, dataType) {
    +          case token =>
    +            throw new SparkSQLJsonProcessingException(
    +              s"Failed to parse a value for data type $dataType (current token: $token).")
    +        }
    +  }
    +
    +  /**
    +   * This handles nulls ahead before trying to check the tokens, and applies the conversion
    +   * function and then checks failed the conversion afterward if it `f` fails to convert the value.
    +   *
    +   * In more details, it checks `FIELD_NAME` if exists and then skip. If this is called after
    +   * `START_OBJECT`, then, the next token can be `FIELD_NAME`. Since the names are kept in
    +   * `JacksonParser.convertObject`, this `FIELD_NAME` token can be skipped as below. When this
    +   * is called after `START_ARRAY`, the tokens become ones about values until `END_ARRAY`.
    +   * In this case, we don't have to skip.
    +   *
    +   * We check if the current token is null or not after that. Then, we apply `f` to convert
    +   * the value and then we check failed conversion afterward if it `f` fails to convert the value.
    +   */
    +  private def parseJsonToken(
    +      parser: JsonParser,
    +      dataType: DataType)(f: PartialFunction[JsonToken, Any]): Any = {
    +    parser.getCurrentToken match {
    +      case FIELD_NAME =>
    +        parser.nextToken()
    +        parseJsonToken(parser, dataType)(f)
    +
    +      case null | VALUE_NULL => null
    +
    +      case other => f.orElse {
    +        // We should specify the type of this `PartialFunction`. Otherwise this will
    --- End diff --
    
    Cool!


---
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


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

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r71096388
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,306 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +private[sql] class JacksonParser(schema: StructType, options: JSONOptions) extends Logging {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedConversion(
    +      parser: JsonParser,
    +      dataType: DataType): Any = parser.getCurrentToken match {
    +    case null | VALUE_NULL =>
    +      null
    +
    +    case _ if parser.getTextLength < 1 =>
    +      // guard the non string type
    +      null
    --- End diff --
    
    Why do we put these two cases at here?


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/63481/
    Test PASSed.


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74192533
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,337 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +private[sql] class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedConversion(
    +      parser: JsonParser,
    +      dataType: DataType): Any = parser.getCurrentToken match {
    +    case _ if parser.getTextLength < 1 =>
    --- End diff --
    
    is it for this previous case?
    ```
    case (VALUE_STRING, _) if parser.getTextLength < 1 =>		
      // guard the non string type		
      null
    ```


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74378654
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,296 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
    +
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  /**
    +   * This function deals with the cases it fails to parse. This function will be called
    +   * when exceptions are caught during converting. This functions also deals with `mode` option.
    +   */
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * This function will be called afterward except the case for `StringType`. we
    +   * throw an exception when it is failed unless the value is null.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    +  private def failedConversion(
           parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +      dataType: DataType): PartialFunction[JsonToken, Any] = {
    +    case VALUE_STRING if parser.getTextLength < 1 =>
    +      // If conversion is failed, this produces `null` rather than
    +      // rather than throw exception. This will protect the mismatch of types.
    --- End diff --
    
    duplicated `rathe than`


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74216088
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,330 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedConversion(
    +      parser: JsonParser,
    +      dataType: DataType): Any = parser.getCurrentToken match {
    +    case VALUE_STRING if parser.getTextLength < 1 =>
    +      // If conversion is failed, this produces `null` rather than
    +      // returning empty string. This will protect the mismatch of types.
    --- End diff --
    
    `rather than throw exception`?


---
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


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

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74372641
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,289 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_TRUE => true
    +        case VALUE_FALSE => false
    +      }
     
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    -        }
    +    case ByteType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getByteValue
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ShortType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getShortValue
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case IntegerType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getIntValue
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    -        }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    -        }
    +    case LongType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getLongValue
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    -        }
    +    case FloatType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getFloatValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toFloat
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case DoubleType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getDoubleValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toDouble
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +    case StringType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +        case _ =>
    +          val writer = new ByteArrayOutputStream()
    +          Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +            generator => generator.copyCurrentStructure(parser)
    +          }
    +          UTF8String.fromBytes(writer.toByteArray)
    +      }
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +    case TimestampType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          // This one will lose microseconds parts.
    +          // See https://issues.apache.org/jira/browse/SPARK-10681.
    +          DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +        case VALUE_NUMBER_INT =>
    +          parser.getLongValue * 1000000L
    +      }
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +    case DateType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          val stringValue = parser.getText
    +          if (stringValue.contains("-")) {
    +            // The format of this string will probably be "yyyy-mm-dd".
    +            DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +          } else {
    +            // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +            // So, we just convert it to Int.
    +            stringValue.toInt
    +          }
    +      }
    +
    +    case BinaryType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING => parser.getBinaryValue
    +      }
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +          Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +      }
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertMap(parser, valueConverter)
    +      }
     
    -      case (_, udt: UserDefinedType[_]) =>
    -        convertField(factory, parser, udt.sqlType)
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
     
    -      case (token, dataType) =>
    -        // We cannot parse this token based on the given data type. So, we throw a
    -        // SparkSQLJsonProcessingException and this exception will be caught by
    -        // parseJson method.
    -        throw new SparkSQLJsonProcessingException(
    -          s"Failed to parse a value for data type $dataType (current token: $token).")
    +    case _ =>
    --- End diff --
    
    Or please let me correct this as below:
    
    ```
    parseJsonToken(parser, dataType)(PartialFunction.empty)
    ```
    
    So that they can be caught in `parseJsonToken` together with documentation.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid record-per type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #61964 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/61964/consoleFull)** for PR 14102 at commit [`5d30f2b`](https://github.com/apache/spark/commit/5d30f2b60aff2820cdff5d2a1b641570c550e174).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74196156
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,337 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedConversion(
    +      parser: JsonParser,
    +      dataType: DataType): Any = parser.getCurrentToken match {
    +    case _ if parser.getTextLength < 1 =>
    +      // If conversion is failed, this produces `null` rather than
    +      // returning empty string. This will protect the mismatch of types.
    +      null
    +
    +    case token =>
    +      // We cannot parse this token based on the given data type. So, we throw a
    +      // SparkSQLJsonProcessingException and this exception will be caught by
    +      // parseJson method.
    +      throw new SparkSQLJsonProcessingException(
    +        s"Failed to parse a value for data type $dataType (current token: $token).")
    +  }
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parser.getCurrentToken match {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +        case _ => failedConversion(parser, st)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parser.getCurrentToken match {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +        case _ => failedConversion(parser, st)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    -
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_TRUE => true
    +          case VALUE_FALSE => false
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ByteType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getByteValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case ShortType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getShortValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    +    case IntegerType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getIntValue
    +          case _ => failedConversion(parser, dataType)
             }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +      }
    +
    +    case LongType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getLongValue
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +    case FloatType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +            parser.getFloatValue
    +
    +          case VALUE_STRING =>
    +            // Special case handling for NaN and Infinity.
    +            val value = parser.getText
    +            val lowerCaseValue = value.toLowerCase
    +            if (lowerCaseValue.equals("nan") ||
    +              lowerCaseValue.equals("infinity") ||
    +              lowerCaseValue.equals("-infinity") ||
    +              lowerCaseValue.equals("inf") ||
    +              lowerCaseValue.equals("-inf")) {
    +              value.toFloat
    +            } else {
    +              throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +            }
    +
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case DoubleType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +            parser.getDoubleValue
    +
    +          case VALUE_STRING =>
    +            // Special case handling for NaN and Infinity.
    +            val value = parser.getText
    +            val lowerCaseValue = value.toLowerCase
    +            if (lowerCaseValue.equals("nan") ||
    +              lowerCaseValue.equals("infinity") ||
    +              lowerCaseValue.equals("-infinity") ||
    +              lowerCaseValue.equals("inf") ||
    +              lowerCaseValue.equals("-inf")) {
    +              value.toDouble
    +            } else {
    +              throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +            }
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +    case StringType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +          case _ =>
    +            val writer = new ByteArrayOutputStream()
    +            Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +              generator => generator.copyCurrentStructure(parser)
    +            }
    +            UTF8String.fromBytes(writer.toByteArray)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +    case TimestampType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            // This one will lose microseconds parts.
    +            // See https://issues.apache.org/jira/browse/SPARK-10681.
    +            DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +          case VALUE_NUMBER_INT =>
    +            parser.getLongValue * 1000000L
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case DateType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            val stringValue = parser.getText
    +            if (stringValue.contains("-")) {
    +              // The format of this string will probably be "yyyy-mm-dd".
    +              DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +            } else {
    +              // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +              // So, we just convert it to Int.
    +              stringValue.toInt
    +            }
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +    case BinaryType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING => parser.getBinaryValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
    +
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +            Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +
    +          case _ => failedConversion(parser, dt)
    +        }
    +      }
    +
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          case _ => failedConversion(parser, st)
    +        }
    +      }
    +
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case START_ARRAY => convertArray(parser, elementConverter)
    +          case _ => failedConversion(parser, at)
    +        }
    +      }
    +
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => convertField(parser) {
    +        parser.getCurrentToken match {
    +          case START_OBJECT => convertMap(parser, valueConverter)
    +          case _ => failedConversion(parser, mt)
    +        }
    +      }
    +
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
    +
    +    case _ =>
    +      (parser: JsonParser) =>
    +        failedConversion(parser, dataType)
    +  }
    +
    +  /**
    +   * This converts a field. If this is called after `START_OBJECT`, then, the next token can be
    +   * `FIELD_NAME`. Since the names are kept in `JacksonParser.convertObject`, this `FIELD_NAME`
    +   * token can be skipped as below. When this is called after `START_ARRAY`, the tokens become
    +   * ones about values until `END_ARRAY`. In this case, we don't have to skip.
    +   */
    +  private def convertField(parser: JsonParser)(f: => Any): Any = {
    --- End diff --
    
    ah i see. Yea we should leave it in `failedConversion `


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/63525/
    Test PASSed.


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74374015
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,289 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_TRUE => true
    +        case VALUE_FALSE => false
    +      }
     
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    -        }
    +    case ByteType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getByteValue
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ShortType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getShortValue
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case IntegerType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getIntValue
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    -        }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    -        }
    +    case LongType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getLongValue
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    -        }
    +    case FloatType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getFloatValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toFloat
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case DoubleType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getDoubleValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toDouble
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +    case StringType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +        case _ =>
    +          val writer = new ByteArrayOutputStream()
    +          Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +            generator => generator.copyCurrentStructure(parser)
    +          }
    +          UTF8String.fromBytes(writer.toByteArray)
    +      }
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +    case TimestampType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          // This one will lose microseconds parts.
    +          // See https://issues.apache.org/jira/browse/SPARK-10681.
    +          DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +        case VALUE_NUMBER_INT =>
    +          parser.getLongValue * 1000000L
    +      }
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +    case DateType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          val stringValue = parser.getText
    +          if (stringValue.contains("-")) {
    +            // The format of this string will probably be "yyyy-mm-dd".
    +            DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +          } else {
    +            // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +            // So, we just convert it to Int.
    +            stringValue.toInt
    +          }
    +      }
    +
    +    case BinaryType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING => parser.getBinaryValue
    +      }
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +          Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +      }
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertMap(parser, valueConverter)
    +      }
     
    -      case (_, udt: UserDefinedType[_]) =>
    -        convertField(factory, parser, udt.sqlType)
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
     
    -      case (token, dataType) =>
    -        // We cannot parse this token based on the given data type. So, we throw a
    -        // SparkSQLJsonProcessingException and this exception will be caught by
    -        // parseJson method.
    -        throw new SparkSQLJsonProcessingException(
    -          s"Failed to parse a value for data type $dataType (current token: $token).")
    +    case _ =>
    --- End diff --
    
    `PartialFunction.empty` looks good


---
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


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

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/14102


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74219825
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,330 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedConversion(
    +      parser: JsonParser,
    +      dataType: DataType): Any = parser.getCurrentToken match {
    +    case VALUE_STRING if parser.getTextLength < 1 =>
    +      // If conversion is failed, this produces `null` rather than
    +      // returning empty string. This will protect the mismatch of types.
    +      null
    +
    +    case token =>
    +      // We cannot parse this token based on the given data type. So, we throw a
    +      // SparkSQLJsonProcessingException and this exception will be caught by
    +      // parseJson method.
    +      throw new SparkSQLJsonProcessingException(
    +        s"Failed to parse a value for data type $dataType (current token: $token).")
    +  }
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parser.getCurrentToken match {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +        case _ => failedConversion(parser, st)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parser.getCurrentToken match {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +        case _ => failedConversion(parser, st)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    -
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_TRUE => true
    +          case VALUE_FALSE => false
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ByteType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getByteValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case ShortType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getShortValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    +    case IntegerType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getIntValue
    +          case _ => failedConversion(parser, dataType)
             }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +      }
    +
    +    case LongType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT => parser.getLongValue
    +          case _ => failedConversion(parser, dataType)
             }
    +      }
    +
    +    case FloatType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +            parser.getFloatValue
    +
    +          case VALUE_STRING =>
    +            // Special case handling for NaN and Infinity.
    +            val value = parser.getText
    +            val lowerCaseValue = value.toLowerCase
    +            if (lowerCaseValue.equals("nan") ||
    +              lowerCaseValue.equals("infinity") ||
    +              lowerCaseValue.equals("-infinity") ||
    +              lowerCaseValue.equals("inf") ||
    +              lowerCaseValue.equals("-inf")) {
    +              value.toFloat
    +            } else {
    +              throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +            }
    +
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
    +
    +    case DoubleType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +            parser.getDoubleValue
    +
    +          case VALUE_STRING =>
    +            // Special case handling for NaN and Infinity.
    +            val value = parser.getText
    +            val lowerCaseValue = value.toLowerCase
    +            if (lowerCaseValue.equals("nan") ||
    +              lowerCaseValue.equals("infinity") ||
    +              lowerCaseValue.equals("-infinity") ||
    +              lowerCaseValue.equals("inf") ||
    +              lowerCaseValue.equals("-inf")) {
    +              value.toDouble
    +            } else {
    +              throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +            }
    +
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
    +
    +    case StringType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +          case _ =>
    +            val writer = new ByteArrayOutputStream()
    +            Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +              generator => generator.copyCurrentStructure(parser)
    +            }
    +            UTF8String.fromBytes(writer.toByteArray)
             }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case TimestampType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            // This one will lose microseconds parts.
    +            // See https://issues.apache.org/jira/browse/SPARK-10681.
    +            DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +          case VALUE_NUMBER_INT =>
    +            parser.getLongValue * 1000000L
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +    case DateType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING =>
    +            val stringValue = parser.getText
    +            if (stringValue.contains("-")) {
    +              // The format of this string will probably be "yyyy-mm-dd".
    +              DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +            } else {
    +              // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +              // So, we just convert it to Int.
    +              stringValue.toInt
    +            }
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
    +
    +    case BinaryType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case VALUE_STRING => parser.getBinaryValue
    +          case _ => failedConversion(parser, dataType)
    +        }
    +      }
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +            Decimal(parser.getDecimalValue, dt.precision, dt.scale)
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +          case _ => failedConversion(parser, dt)
    +        }
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          case _ => failedConversion(parser, st)
    +        }
    +      }
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case START_ARRAY => convertArray(parser, elementConverter)
    +          case _ => failedConversion(parser, at)
    +        }
    +      }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => handleNullAndFieldTokens(parser) {
    +        parser.getCurrentToken match {
    +          case START_OBJECT => convertMap(parser, valueConverter)
    +          case _ => failedConversion(parser, mt)
    +        }
    +      }
     
    -      case (_, udt: UserDefinedType[_]) =>
    -        convertField(factory, parser, udt.sqlType)
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
     
    -      case (token, dataType) =>
    -        // We cannot parse this token based on the given data type. So, we throw a
    -        // SparkSQLJsonProcessingException and this exception will be caught by
    -        // parseJson method.
    -        throw new SparkSQLJsonProcessingException(
    -          s"Failed to parse a value for data type $dataType (current token: $token).")
    +    case _ =>
    +      (parser: JsonParser) =>
    +        failedConversion(parser, dataType)
    +  }
    +
    +  /**
    +   * This handles nulls ahead before trying to check the tokens, and applies
    +   * the conversion function. Also, it checks `FIELD_NAME` if exists and then skip.
    +   * If this is called after `START_OBJECT`, then, the next token can be
    +   * `FIELD_NAME`. Since the names are kept in `JacksonParser.convertObject`, this `FIELD_NAME`
    +   * token can be skipped as below. When this is called after `START_ARRAY`, the tokens become
    +   * ones about values until `END_ARRAY`. In this case, we don't have to skip.
    +   */
    +  private def handleNullAndFieldTokens (parser: JsonParser)(f: => Any): Any = {
    --- End diff --
    
    we can even inline `failedConversion` into it
    ```
    case other => f.orElse {
      case VALUE_STRING if parser.getTextLength < 1 => null
      case token => throw ...
    }.apply(other)
    ```


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid record-per type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Merged build finished. Test PASSed.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #63525 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/63525/consoleFull)** for PR 14102 at commit [`726aa7f`](https://github.com/apache/spark/commit/726aa7fcb1a438d0a3885e77d1004bb782bbf5ec).


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Merged build finished. Test PASSed.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    thanks,merging to master!


---
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


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

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r71261575
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/InferSchema.scala ---
    @@ -60,13 +60,13 @@ private[sql] object InferSchema {
               }
             } catch {
               case _: JsonParseException if shouldHandleCorruptRecord =>
    -            Some(StructType(Seq(StructField(columnNameOfCorruptRecords, StringType))))
    +            Some(StructType(Seq(StructField(columnNameOfCorruptRecord, StringType))))
               case _: JsonParseException =>
                 None
             }
           }
         }.fold(StructType(Seq()))(
    -      compatibleRootType(columnNameOfCorruptRecords, shouldHandleCorruptRecord))
    +      compatibleRootType(columnNameOfCorruptRecord, shouldHandleCorruptRecord))
    --- End diff --
    
    The name is `columnNameOfCorruptRecord` across JSON data source, not `columnNameOfCorruptRecords`.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #62501 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62501/consoleFull)** for PR 14102 at commit [`cfe6bed`](https://github.com/apache/spark/commit/cfe6beda1a1db64aab5d2f84a68a5ee1e2bdd905).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


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

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r71099098
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONOptions.scala ---
    @@ -51,7 +53,8 @@ private[sql] class JSONOptions(
         parameters.get("allowBackslashEscapingAnyCharacter").map(_.toBoolean).getOrElse(false)
       val compressionCodec = parameters.get("compression").map(CompressionCodecs.getCodecClassName)
       private val parseMode = parameters.getOrElse("mode", "PERMISSIVE")
    -  val columnNameOfCorruptRecord = parameters.get("columnNameOfCorruptRecord")
    +  val columnNameOfCorruptRecord =
    +    parameters.getOrElse("columnNameOfCorruptRecord", sqlConf.columnNameOfCorruptRecord)
    --- End diff --
    
    Oh, actually, the reason was not only to remove duplicated logics. Another reason is that we are passing `columnNameOfCorruptRecords` to `InferSchema.infer(...)` and `JacksonParser.parse(...)`
    
    
    ```scala
    def infer(
        json: RDD[String],
        columnNameOfCorruptRecords: String,
        configOptions: JSONOptions): StructType = {
      ...
    ```
    
    ```scala
    def parse(
        input: RDD[String],
        schema: StructType,
        columnNameOfCorruptRecords: String,
        configOptions: JSONOptions): RDD[InternalRow] = {
      ...
    ```
    
    although `columnNameOfCorruptRecords` is an option in `JSONOptions`. I would revert this change if it still does not sound good but I just wanted to let you know just in case.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL][WIP] Avoid record-per type dispatch i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/61949/
    Test FAILed.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid record-per type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Merged build finished. Test PASSed.


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74380391
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,296 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
    +
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  /**
    +   * This function deals with the cases it fails to parse. This function will be called
    +   * when exceptions are caught during converting. This functions also deals with `mode` option.
    +   */
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * This function will be called afterward except the case for `StringType`. we
    +   * throw an exception when it is failed unless the value is null.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    +  private def failedConversion(
           parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +      dataType: DataType): PartialFunction[JsonToken, Any] = {
    +    case VALUE_STRING if parser.getTextLength < 1 =>
    +      // If conversion is failed, this produces `null` rather than
    +      // rather than throw exception. This will protect the mismatch of types.
    +      null
    +
    +    case token =>
    +      // We cannot parse this token based on the given data type. So, we throw a
    +      // SparkSQLJsonProcessingException and this exception will be caught by
    +      // `parse` method.
    +      throw new SparkSQLJsonProcessingException(
    +        s"Failed to parse a value for data type $dataType (current token: $token).")
    +  }
    +
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
    +   */
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    --- End diff --
    
    the previous `convertRootField` only handles the special cases: (START_ARRAY, StructType) and (START_OBJECT, ArrayType). Can we follow that logic here?


---
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


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

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74047137
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,337 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +private[sql] class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedConversion(
    +      parser: JsonParser,
    +      dataType: DataType): Any = parser.getCurrentToken match {
    +    case _ if parser.getTextLength < 1 =>
    --- End diff --
    
    (For other types, they call this function when they fail to convert the values)


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74034073
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,337 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +private[sql] class JacksonParser(
    --- End diff --
    
    no need to add `private[sql]`, it's already under execution package and is meant to be private.


---
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


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

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74041126
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,337 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +private[sql] class JacksonParser(
    --- End diff --
    
    Thanks! I will fix up tomorrow.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Merged build finished. Test PASSed.


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74191412
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala ---
    @@ -1360,7 +1363,8 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData {
       }
     
       test("SPARK-8093 Erase empty structs") {
    -    val emptySchema = InferSchema.infer(emptyRecords, "", new JSONOptions(Map()))
    --- End diff --
    
    why change this? looks like they are same code?


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    @yhuai Could you please take another look?


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74034675
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,337 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +private[sql] class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedConversion(
    +      parser: JsonParser,
    +      dataType: DataType): Any = parser.getCurrentToken match {
    +    case _ if parser.getTextLength < 1 =>
    --- End diff --
    
    can you explain a bit more about this if condition?


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #62501 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62501/consoleFull)** for PR 14102 at commit [`cfe6bed`](https://github.com/apache/spark/commit/cfe6beda1a1db64aab5d2f84a68a5ee1e2bdd905).


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #62659 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62659/consoleFull)** for PR 14102 at commit [`4cf5d11`](https://github.com/apache/spark/commit/4cf5d11b0a24bab7f1baf9400511b377e33d90c9).


---
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


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

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74375624
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,289 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_TRUE => true
    +        case VALUE_FALSE => false
    +      }
     
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    -        }
    +    case ByteType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getByteValue
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ShortType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getShortValue
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case IntegerType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getIntValue
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    -        }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    -        }
    +    case LongType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getLongValue
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    -        }
    +    case FloatType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getFloatValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toFloat
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case DoubleType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getDoubleValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toDouble
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +    case StringType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +        case _ =>
    +          val writer = new ByteArrayOutputStream()
    +          Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +            generator => generator.copyCurrentStructure(parser)
    +          }
    +          UTF8String.fromBytes(writer.toByteArray)
    +      }
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +    case TimestampType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          // This one will lose microseconds parts.
    +          // See https://issues.apache.org/jira/browse/SPARK-10681.
    +          DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +        case VALUE_NUMBER_INT =>
    +          parser.getLongValue * 1000000L
    +      }
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +    case DateType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          val stringValue = parser.getText
    +          if (stringValue.contains("-")) {
    +            // The format of this string will probably be "yyyy-mm-dd".
    +            DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +          } else {
    +            // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +            // So, we just convert it to Int.
    +            stringValue.toInt
    +          }
    +      }
    +
    +    case BinaryType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING => parser.getBinaryValue
    +      }
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +          Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +      }
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertMap(parser, valueConverter)
    +      }
     
    -      case (_, udt: UserDefinedType[_]) =>
    -        convertField(factory, parser, udt.sqlType)
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
     
    -      case (token, dataType) =>
    -        // We cannot parse this token based on the given data type. So, we throw a
    -        // SparkSQLJsonProcessingException and this exception will be caught by
    -        // parseJson method.
    -        throw new SparkSQLJsonProcessingException(
    -          s"Failed to parse a value for data type $dataType (current token: $token).")
    +    case _ =>
    +      (parser: JsonParser) =>
    +        parseJsonToken(parser, dataType) {
    +          case token =>
    +            throw new SparkSQLJsonProcessingException(
    +              s"Failed to parse a value for data type $dataType (current token: $token).")
    +        }
    +  }
    +
    +  /**
    +   * This handles nulls ahead before trying to check the tokens, and applies the conversion
    +   * function and then checks failed the conversion afterward if it `f` fails to convert the value.
    +   *
    +   * In more details, it checks `FIELD_NAME` if exists and then skip. If this is called after
    +   * `START_OBJECT`, then, the next token can be `FIELD_NAME`. Since the names are kept in
    +   * `JacksonParser.convertObject`, this `FIELD_NAME` token can be skipped as below. When this
    +   * is called after `START_ARRAY`, the tokens become ones about values until `END_ARRAY`.
    +   * In this case, we don't have to skip.
    +   *
    +   * We check if the current token is null or not after that. Then, we apply `f` to convert
    +   * the value and then we check failed conversion afterward if it `f` fails to convert the value.
    +   */
    +  private def parseJsonToken(
    +      parser: JsonParser,
    +      dataType: DataType)(f: PartialFunction[JsonToken, Any]): Any = {
    +    parser.getCurrentToken match {
    +      case FIELD_NAME =>
    +        parser.nextToken()
    +        parseJsonToken(parser, dataType)(f)
    +
    +      case null | VALUE_NULL => null
    +
    +      case other => f.orElse {
    +        // We should specify the type of this `PartialFunction`. Otherwise this will
    --- End diff --
    
    Do you mind if I just define `failedConversion` as a inner function here?


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #63599 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/63599/consoleFull)** for PR 14102 at commit [`aaac7db`](https://github.com/apache/spark/commit/aaac7db58c0b303aefa8eda98b47d2734e29633f).


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #63575 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/63575/consoleFull)** for PR 14102 at commit [`bceda7b`](https://github.com/apache/spark/commit/bceda7ba4f06c0b6fd99f11ef2662f9f3a154af0).


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid record-per type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #61956 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/61956/consoleFull)** for PR 14102 at commit [`9dbcec7`](https://github.com/apache/spark/commit/9dbcec74b2fe64e06c2d013da56d6c4a46fdabb7).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #63522 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/63522/consoleFull)** for PR 14102 at commit [`7077ee6`](https://github.com/apache/spark/commit/7077ee6254ea307afd2c4385c8c19a4fedac89a1).


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL][WIP] Avoid record-per type dispatch i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Merged build finished. Test FAILed.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/63602/
    Test PASSed.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/62659/
    Test PASSed.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid record-per type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #61960 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/61960/consoleFull)** for PR 14102 at commit [`cd44ac5`](https://github.com/apache/spark/commit/cd44ac518645b17f2dc1397d09709f60fa20066d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74373524
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,289 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_TRUE => true
    +        case VALUE_FALSE => false
    +      }
     
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    -        }
    +    case ByteType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getByteValue
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ShortType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getShortValue
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case IntegerType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getIntValue
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    -        }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    -        }
    +    case LongType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getLongValue
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    -        }
    +    case FloatType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getFloatValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toFloat
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case DoubleType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getDoubleValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toDouble
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +    case StringType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +        case _ =>
    +          val writer = new ByteArrayOutputStream()
    +          Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +            generator => generator.copyCurrentStructure(parser)
    +          }
    +          UTF8String.fromBytes(writer.toByteArray)
    +      }
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +    case TimestampType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          // This one will lose microseconds parts.
    +          // See https://issues.apache.org/jira/browse/SPARK-10681.
    +          DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +        case VALUE_NUMBER_INT =>
    +          parser.getLongValue * 1000000L
    +      }
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +    case DateType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          val stringValue = parser.getText
    +          if (stringValue.contains("-")) {
    +            // The format of this string will probably be "yyyy-mm-dd".
    +            DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +          } else {
    +            // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +            // So, we just convert it to Int.
    +            stringValue.toInt
    +          }
    +      }
    +
    +    case BinaryType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING => parser.getBinaryValue
    +      }
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +          Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +      }
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertMap(parser, valueConverter)
    +      }
     
    -      case (_, udt: UserDefinedType[_]) =>
    -        convertField(factory, parser, udt.sqlType)
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
     
    -      case (token, dataType) =>
    -        // We cannot parse this token based on the given data type. So, we throw a
    -        // SparkSQLJsonProcessingException and this exception will be caught by
    -        // parseJson method.
    -        throw new SparkSQLJsonProcessingException(
    -          s"Failed to parse a value for data type $dataType (current token: $token).")
    +    case _ =>
    +      (parser: JsonParser) =>
    +        parseJsonToken(parser, dataType) {
    +          case token =>
    +            throw new SparkSQLJsonProcessingException(
    +              s"Failed to parse a value for data type $dataType (current token: $token).")
    +        }
    +  }
    +
    +  /**
    +   * This handles nulls ahead before trying to check the tokens, and applies the conversion
    +   * function and then checks failed the conversion afterward if it `f` fails to convert the value.
    +   *
    +   * In more details, it checks `FIELD_NAME` if exists and then skip. If this is called after
    +   * `START_OBJECT`, then, the next token can be `FIELD_NAME`. Since the names are kept in
    +   * `JacksonParser.convertObject`, this `FIELD_NAME` token can be skipped as below. When this
    +   * is called after `START_ARRAY`, the tokens become ones about values until `END_ARRAY`.
    +   * In this case, we don't have to skip.
    +   *
    +   * We check if the current token is null or not after that. Then, we apply `f` to convert
    +   * the value and then we check failed conversion afterward if it `f` fails to convert the value.
    +   */
    +  private def parseJsonToken(
    +      parser: JsonParser,
    +      dataType: DataType)(f: PartialFunction[JsonToken, Any]): Any = {
    +    parser.getCurrentToken match {
    +      case FIELD_NAME =>
    +        parser.nextToken()
    +        parseJsonToken(parser, dataType)(f)
    +
    +      case null | VALUE_NULL => null
    +
    +      case other => f.orElse {
    +        // We should specify the type of this `PartialFunction`. Otherwise this will
    --- End diff --
    
    instead of adding comments to explain this hack, I'd like to avoid it. We can still use the method `failedConversion`, and use `f.applyOrElse(failedConversion(...))` here


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL][WIP] Avoid record-per type dispatch i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #61949 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/61949/consoleFull)** for PR 14102 at commit [`2d77f66`](https://github.com/apache/spark/commit/2d77f66f2c78bb139212011bfa1fa2efbf6b9d5b).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid record-per type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/61956/
    Test PASSed.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #63605 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/63605/consoleFull)** for PR 14102 at commit [`80b2d3a`](https://github.com/apache/spark/commit/80b2d3a9d638eaeb87665a4db4c535954639b148).


---
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


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

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14102#discussion_r74371723
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala ---
    @@ -35,184 +34,289 @@ import org.apache.spark.util.Utils
     
     private[json] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)
     
    -object JacksonParser extends Logging {
    +class JacksonParser(
    +    schema: StructType,
    +    columnNameOfCorruptRecord: String,
    +    options: JSONOptions) extends Logging {
     
    -  def parse(
    -      input: RDD[String],
    -      schema: StructType,
    -      columnNameOfCorruptRecords: String,
    -      configOptions: JSONOptions): RDD[InternalRow] = {
    +  import com.fasterxml.jackson.core.JsonToken._
     
    -    input.mapPartitions { iter =>
    -      parseJson(iter, schema, columnNameOfCorruptRecords, configOptions)
    +  // A `ValueConverter` is responsible for converting a value from `JsonParser`
    +  // to a value in a field for `InternalRow`.
    +  private type ValueConverter = (JsonParser) => Any
    +
    +  // `ValueConverter`s for the root schema for all fields in the schema
    +  private val rootConverter: ValueConverter = makeRootConverter(schema)
    +
    +  private val factory = new JsonFactory()
    +  options.setJacksonOptions(factory)
    +
    +  private def failedRecord(record: String): Seq[InternalRow] = {
    +    // create a row even if no corrupt record column is present
    +    if (options.failFast) {
    +      throw new RuntimeException(s"Malformed line in FAILFAST mode: $record")
    +    }
    +    if (options.dropMalformed) {
    +      logWarning(s"Dropping malformed line: $record")
    +      Nil
    +    } else {
    +      val row = new GenericMutableRow(schema.length)
    +      for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecord)) {
    +        require(schema(corruptIndex).dataType == StringType)
    +        row.update(corruptIndex, UTF8String.fromString(record))
    +      }
    +      Seq(row)
         }
       }
     
       /**
    -   * Parse the current token (and related children) according to a desired schema
    -   * This is a wrapper for the method `convertField()` to handle a row wrapped
    -   * with an array.
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema. This is a wrapper for the method
    +   * `makeConverter()` to handle a row wrapped with an array.
        */
    -  def convertRootField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (START_ARRAY, st: StructType) =>
    -        // SPARK-3308: support reading top level JSON arrays and take every element
    -        // in such an array as a row
    -        convertArray(factory, parser, st)
    -
    -      case (START_OBJECT, ArrayType(st, _)) =>
    +  def makeRootConverter(dataType: DataType): ValueConverter = dataType match {
    +    case st: StructType =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +          // SPARK-3308: support reading top level JSON arrays and take every element
    +          // in such an array as a row
    +          //
    +          // For example, we support, the JSON data as below:
    +          //
    +          // [{"a":"str_a_1"}]
    +          // [{"a":"str_a_2"}, {"b":"str_b_3"}]
    +          //
    +          // resulting in:
    +          //
    +          // List([str_a_1,null])
    +          // List([str_a_2,null], [null,str_b_3])
    +          //
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
    +
    +    case ArrayType(st: StructType, _) =>
    +      val elementConverter = makeConverter(st)
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
             // the business end of SPARK-3308:
    -        // when an object is found but an array is requested just wrap it in a list
    -        convertField(factory, parser, st) :: Nil
    +        // when an object is found but an array is requested just wrap it in a list.
    +        // This is being wrapped in `JacksonParser.parse`.
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case _ =>
    -        convertField(factory, parser, schema)
    -    }
    +    case _ => makeConverter(dataType)
       }
     
    -  private def convertField(
    -      factory: JsonFactory,
    -      parser: JsonParser,
    -      schema: DataType): Any = {
    -    import com.fasterxml.jackson.core.JsonToken._
    -    (parser.getCurrentToken, schema) match {
    -      case (null | VALUE_NULL, _) =>
    -        null
    +  /**
    +   * Create a converter which converts the JSON documents held by the `JsonParser`
    +   * to a value according to a desired schema.
    +   */
    +  private def makeConverter(dataType: DataType): ValueConverter = dataType match {
    +    case BooleanType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_TRUE => true
    +        case VALUE_FALSE => false
    +      }
     
    -      case (FIELD_NAME, _) =>
    -        parser.nextToken()
    -        convertField(factory, parser, schema)
    -
    -      case (VALUE_STRING, StringType) =>
    -        UTF8String.fromString(parser.getText)
    -
    -      case (VALUE_STRING, _) if parser.getTextLength < 1 =>
    -        // guard the non string type
    -        null
    -
    -      case (VALUE_STRING, BinaryType) =>
    -        parser.getBinaryValue
    -
    -      case (VALUE_STRING, DateType) =>
    -        val stringValue = parser.getText
    -        if (stringValue.contains("-")) {
    -          // The format of this string will probably be "yyyy-mm-dd".
    -          DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    -        } else {
    -          // In Spark 1.5.0, we store the data as number of days since epoch in string.
    -          // So, we just convert it to Int.
    -          stringValue.toInt
    -        }
    +    case ByteType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getByteValue
    +      }
     
    -      case (VALUE_STRING, TimestampType) =>
    -        // This one will lose microseconds parts.
    -        // See https://issues.apache.org/jira/browse/SPARK-10681.
    -        DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
    +    case ShortType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getShortValue
    +      }
     
    -      case (VALUE_NUMBER_INT, TimestampType) =>
    -        parser.getLongValue * 1000000L
    +    case IntegerType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getIntValue
    +      }
     
    -      case (_, StringType) =>
    -        val writer = new ByteArrayOutputStream()
    -        Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    -          generator => generator.copyCurrentStructure(parser)
    -        }
    -        UTF8String.fromBytes(writer.toByteArray)
    -
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, FloatType) =>
    -        parser.getFloatValue
    -
    -      case (VALUE_STRING, FloatType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toFloat
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    -        }
    +    case LongType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT => parser.getLongValue
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, DoubleType) =>
    -        parser.getDoubleValue
    -
    -      case (VALUE_STRING, DoubleType) =>
    -        // Special case handling for NaN and Infinity.
    -        val value = parser.getText
    -        val lowerCaseValue = value.toLowerCase()
    -        if (lowerCaseValue.equals("nan") ||
    -          lowerCaseValue.equals("infinity") ||
    -          lowerCaseValue.equals("-infinity") ||
    -          lowerCaseValue.equals("inf") ||
    -          lowerCaseValue.equals("-inf")) {
    -          value.toDouble
    -        } else {
    -          throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    -        }
    +    case FloatType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getFloatValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toFloat
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT, dt: DecimalType) =>
    -        Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +    case DoubleType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT =>
    +          parser.getDoubleValue
    +
    +        case VALUE_STRING =>
    +          // Special case handling for NaN and Infinity.
    +          val value = parser.getText
    +          val lowerCaseValue = value.toLowerCase
    +          if (lowerCaseValue.equals("nan") ||
    +            lowerCaseValue.equals("infinity") ||
    +            lowerCaseValue.equals("-infinity") ||
    +            lowerCaseValue.equals("inf") ||
    +            lowerCaseValue.equals("-inf")) {
    +            value.toDouble
    +          } else {
    +            throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.")
    +          }
    +      }
     
    -      case (VALUE_NUMBER_INT, ByteType) =>
    -        parser.getByteValue
    +    case StringType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          UTF8String.fromString(parser.getText)
     
    -      case (VALUE_NUMBER_INT, ShortType) =>
    -        parser.getShortValue
    +        case _ =>
    +          val writer = new ByteArrayOutputStream()
    +          Utils.tryWithResource(factory.createGenerator(writer, JsonEncoding.UTF8)) {
    +            generator => generator.copyCurrentStructure(parser)
    +          }
    +          UTF8String.fromBytes(writer.toByteArray)
    +      }
     
    -      case (VALUE_NUMBER_INT, IntegerType) =>
    -        parser.getIntValue
    +    case TimestampType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          // This one will lose microseconds parts.
    +          // See https://issues.apache.org/jira/browse/SPARK-10681.
    +          DateTimeUtils.stringToTime(parser.getText).getTime * 1000L
     
    -      case (VALUE_NUMBER_INT, LongType) =>
    -        parser.getLongValue
    +        case VALUE_NUMBER_INT =>
    +          parser.getLongValue * 1000000L
    +      }
     
    -      case (VALUE_TRUE, BooleanType) =>
    -        true
    +    case DateType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING =>
    +          val stringValue = parser.getText
    +          if (stringValue.contains("-")) {
    +            // The format of this string will probably be "yyyy-mm-dd".
    +            DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)
    +          } else {
    +            // In Spark 1.5.0, we store the data as number of days since epoch in string.
    +            // So, we just convert it to Int.
    +            stringValue.toInt
    +          }
    +      }
    +
    +    case BinaryType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case VALUE_STRING => parser.getBinaryValue
    +      }
     
    -      case (VALUE_FALSE, BooleanType) =>
    -        false
    +    case dt: DecimalType =>
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) =>
    +          Decimal(parser.getDecimalValue, dt.precision, dt.scale)
    +      }
     
    -      case (START_OBJECT, st: StructType) =>
    -        convertObject(factory, parser, st)
    +    case st: StructType =>
    +      val fieldConverters = st.map(_.dataType).map(makeConverter)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertObject(parser, st, fieldConverters)
    +      }
     
    -      case (START_ARRAY, ArrayType(st, _)) =>
    -        convertArray(factory, parser, st)
    +    case at: ArrayType =>
    +      val elementConverter = makeConverter(at.elementType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_ARRAY => convertArray(parser, elementConverter)
    +      }
     
    -      case (START_OBJECT, MapType(StringType, kt, _)) =>
    -        convertMap(factory, parser, kt)
    +    case mt: MapType =>
    +      val valueConverter = makeConverter(mt.valueType)
    +      (parser: JsonParser) => parseJsonToken(parser, dataType) {
    +        case START_OBJECT => convertMap(parser, valueConverter)
    +      }
     
    -      case (_, udt: UserDefinedType[_]) =>
    -        convertField(factory, parser, udt.sqlType)
    +    case udt: UserDefinedType[_] =>
    +      makeConverter(udt.sqlType)
     
    -      case (token, dataType) =>
    -        // We cannot parse this token based on the given data type. So, we throw a
    -        // SparkSQLJsonProcessingException and this exception will be caught by
    -        // parseJson method.
    -        throw new SparkSQLJsonProcessingException(
    -          s"Failed to parse a value for data type $dataType (current token: $token).")
    +    case _ =>
    --- End diff --
    
    do we need this case? I think all the data types are covered.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid record-per type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/61960/
    Test PASSed.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid per-record type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Merged build finished. Test PASSed.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid record-per type dispatch in JSO...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    **[Test build #61964 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/61964/consoleFull)** for PR 14102 at commit [`5d30f2b`](https://github.com/apache/spark/commit/5d30f2b60aff2820cdff5d2a1b641570c550e174).


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL] Avoid record-per type dispatch in JSO...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/61968/
    Test PASSed.


---
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


[GitHub] spark issue #14102: [SPARK-16434][SQL][WIP] Avoid record-per type dispatch i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14102
  
    Merged build finished. Test FAILed.


---
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