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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org