You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Nick Hryhoriev (Jira)" <ji...@apache.org> on 2021/04/15 17:16:00 UTC

[jira] [Created] (SPARK-35094) Spark from_json(JsonToStruct) function return wrong value in permissive mode in case best effort

Nick Hryhoriev created SPARK-35094:
--------------------------------------

             Summary: Spark from_json(JsonToStruct)  function return wrong value in permissive mode in case best effort
                 Key: SPARK-35094
                 URL: https://issues.apache.org/jira/browse/SPARK-35094
             Project: Spark
          Issue Type: Bug
          Components: Spark Core, SQL
    Affects Versions: 3.1.1, 3.0.2
            Reporter: Nick Hryhoriev


I use spark 3.1.1 and 3.0.2.
Function `from_json` return wrong result with Permissive mode.
In corner case:
1. Json message has complex nested structure
 \{sameNameField)damaged, nestedVal:{badSchemaNestedVal, sameNameField_WhichValueWillAppearInwrongPlace}}
2. Nested -> Nested Field: Schema is satisfy align with value in json.

scala code to reproduce:
{code:java}
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.functions.from_json
import org.apache.spark.sql.types.IntegerType
import org.apache.spark.sql.types.StringType
import org.apache.spark.sql.types.StructField
import org.apache.spark.sql.types.StructType

object Main {

  def main(args: Array[String]): Unit = {
    implicit val spark: SparkSession = SparkSession.builder().master("local[*]").getOrCreate()
    import spark.implicits._

    val schemaForFieldWhichWillHaveWrongValue = StructField("problematicName", StringType, nullable = true)
    val nestedFieldWhichNotSatisfyJsonMessage = StructField(
      "badNestedField",
      StructType(Seq(StructField("SomethingWhichNotInJsonMessage", IntegerType, nullable = true)))
    )
    val nestedFieldWithNestedFieldWhichNotSatisfyJsonMessage =
      StructField(
        "nestedField",
        StructType(Seq(nestedFieldWhichNotSatisfyJsonMessage, schemaForFieldWhichWillHaveWrongValue))
      )
    val customSchema = StructType(Seq(
      schemaForFieldWhichWillHaveWrongValue,
      nestedFieldWithNestedFieldWhichNotSatisfyJsonMessage
    ))

    val jsonStringToTest =
      """{"problematicName":"ThisValueWillBeOverwritten","nestedField":{"badNestedField":"14","problematicName":"thisValueInTwoPlaces"}}"""
    val df = List(jsonStringToTest)
      .toDF("json")
      // issue happen only in permissive mode during best effort
      .select(from_json($"json", customSchema).as("toBeFlatten"))
      .select("toBeFlatten.*")
    df.show(truncate = false)

    assert(
      df.select("problematicName").as[String].first() == "ThisValueWillBeOverwritten",
      "wrong value in root schema, parser take value from column with same name but in another nested elvel"
    )
  }

}

{code}
I was not able to debug this issue, to find the exact root cause.
But what I find in debug, that In `org.apache.spark.sql.catalyst.util.FailureSafeParser` in line 64. code block `e.partialResult()` already have a wrong value.

I hope this will help to fix the issue.

I do a DIRTY HACK to fix the issue.
I just fork this function and hardcode `None` -> `Iterator(toResultRow(None, e.record))`.
in my case, it's better to do not have any values in the row, than theoretically have a wrong value in some column.

 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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