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

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

     [ https://issues.apache.org/jira/browse/SPARK-35094?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Apache Spark reassigned SPARK-35094:
------------------------------------

    Assignee:     (was: Apache Spark)

> 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.0.2, 3.1.1
>            Reporter: Nick Hryhoriev
>            Priority: Critical
>
> 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