Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import java.io._
import scala.util.parsing.combinator.RegexParsers

import com.fasterxml.jackson.core._
import com.fasterxml.jackson.core.json.JsonReadFeature

import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
Expand Down Expand Up @@ -99,10 +100,10 @@ private[this] object JsonPathParser extends RegexParsers {
}

private[this] object SharedFactory {
val jsonFactory = new JsonFactory()

// Enabled for Hive compatibility
jsonFactory.enable(JsonParser.Feature.ALLOW_UNQUOTED_CONTROL_CHARS)
val jsonFactory = new JsonFactoryBuilder()
// Enabled for Hive compatibility
.enable(JsonReadFeature.ALLOW_UNESCAPED_CONTROL_CHARS)
.build()
}

/**
Expand Down Expand Up @@ -756,11 +757,7 @@ case class SchemaOfJson(
private lazy val jsonOptions = new JSONOptions(options, "UTC")

@transient
private lazy val jsonFactory = {
val factory = new JsonFactory()
jsonOptions.setJacksonOptions(factory)
factory
}
private lazy val jsonFactory = jsonOptions.buildJsonFactory()

@transient
private lazy val jsonInferSchema = new JsonInferSchema(jsonOptions)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,8 @@ import java.nio.charset.{Charset, StandardCharsets}
import java.time.ZoneId
import java.util.Locale

import com.fasterxml.jackson.core.{JsonFactory, JsonParser}
import com.fasterxml.jackson.core.{JsonFactory, JsonFactoryBuilder}
import com.fasterxml.jackson.core.json.JsonReadFeature

import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.util._
Expand All @@ -30,7 +31,7 @@ import org.apache.spark.sql.internal.SQLConf
/**
* Options for parsing JSON data into Spark SQL rows.
*
* Most of these map directly to Jackson's internal options, specified in [[JsonParser.Feature]].
* Most of these map directly to Jackson's internal options, specified in [[JsonReadFeature]].
*/
private[sql] class JSONOptions(
@transient val parameters: CaseInsensitiveMap[String],
Expand Down Expand Up @@ -129,16 +130,19 @@ private[sql] class JSONOptions(
*/
val inferTimestamp: Boolean = parameters.get("inferTimestamp").map(_.toBoolean).getOrElse(true)

/** Sets config options on a Jackson [[JsonFactory]]. */
def setJacksonOptions(factory: JsonFactory): Unit = {
factory.configure(JsonParser.Feature.ALLOW_COMMENTS, allowComments)
factory.configure(JsonParser.Feature.ALLOW_UNQUOTED_FIELD_NAMES, allowUnquotedFieldNames)
factory.configure(JsonParser.Feature.ALLOW_SINGLE_QUOTES, allowSingleQuotes)
factory.configure(JsonParser.Feature.ALLOW_NUMERIC_LEADING_ZEROS, allowNumericLeadingZeros)
factory.configure(JsonParser.Feature.ALLOW_NON_NUMERIC_NUMBERS, allowNonNumericNumbers)
factory.configure(JsonParser.Feature.ALLOW_BACKSLASH_ESCAPING_ANY_CHARACTER,
allowBackslashEscapingAnyCharacter)
factory.configure(JsonParser.Feature.ALLOW_UNQUOTED_CONTROL_CHARS, allowUnquotedControlChars)
/** Build a Jackson [[JsonFactory]] using JSON options. */
def buildJsonFactory(): JsonFactory = {
new JsonFactoryBuilder()
.configure(JsonReadFeature.ALLOW_JAVA_COMMENTS, allowComments)
.configure(JsonReadFeature.ALLOW_UNQUOTED_FIELD_NAMES, allowUnquotedFieldNames)
.configure(JsonReadFeature.ALLOW_SINGLE_QUOTES, allowSingleQuotes)
.configure(JsonReadFeature.ALLOW_LEADING_ZEROS_FOR_NUMBERS, allowNumericLeadingZeros)
.configure(JsonReadFeature.ALLOW_NON_NUMERIC_NUMBERS, allowNonNumericNumbers)
.configure(
JsonReadFeature.ALLOW_BACKSLASH_ESCAPING_ANY_CHARACTER,
allowBackslashEscapingAnyCharacter)
.configure(JsonReadFeature.ALLOW_UNESCAPED_CONTROL_CHARS, allowUnquotedControlChars)
.build()
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,8 +52,7 @@ class JacksonParser(
// `ValueConverter`s for the root schema for all fields in the schema
private val rootConverter = makeRootConverter(schema)

private val factory = new JsonFactory()
options.setJacksonOptions(factory)
private val factory = options.buildJsonFactory()

private val timestampFormatter = TimestampFormatter(
options.timestampFormat,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,8 +57,7 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
// In each RDD partition, perform schema inference on each row and merge afterwards.
val typeMerger = JsonInferSchema.compatibleRootType(columnNameOfCorruptRecord, parseMode)
val mergedTypesFromPartitions = json.mapPartitions { iter =>
val factory = new JsonFactory()
options.setJacksonOptions(factory)
val factory = options.buildJsonFactory()
iter.flatMap { row =>
try {
Utils.tryWithResource(createParser(factory, row)) { parser =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,8 +28,7 @@ class JsonInferSchemaSuite extends SparkFunSuite with SQLHelper {
def checkType(options: Map[String, String], json: String, dt: DataType): Unit = {
val jsonOptions = new JSONOptions(options, "UTC", "")
val inferSchema = new JsonInferSchema(jsonOptions)
val factory = new JsonFactory()
jsonOptions.setJacksonOptions(factory)
val factory = jsonOptions.buildJsonFactory()
val parser = CreateJacksonParser.string(factory, json)
parser.nextToken()
val expectedType = StructType(Seq(StructField("a", dt, true)))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -103,7 +103,7 @@ class JsonParsingOptionsSuite extends QueryTest with SharedSparkSession {
}

// The following two tests are not really working - need to look into Jackson's
// JsonParser.Feature.ALLOW_NON_NUMERIC_NUMBERS.
// JsonReadFeature.ALLOW_NON_NUMERIC_NUMBERS.
ignore("allowNonNumericNumbers off") {
val str = """{"age": NaN}"""
val df = spark.read.json(Seq(str).toDS())
Expand Down