You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Branden Smith (JIRA)" <ji...@apache.org> on 2019/01/28 00:40:00 UTC
[jira] [Created] (SPARK-26745) Non-parsing Dataset.count()
optimization causes inconsistent results for JSON inputs with empty lines
Branden Smith created SPARK-26745:
-------------------------------------
Summary: Non-parsing Dataset.count() optimization causes inconsistent results for JSON inputs with empty lines
Key: SPARK-26745
URL: https://issues.apache.org/jira/browse/SPARK-26745
Project: Spark
Issue Type: Bug
Components: SQL
Affects Versions: 2.4.0, 3.0.0
Reporter: Branden Smith
The optimization introduced by [SPARK-24959|https://issues.apache.org/jira/browse/SPARK-24959] (improving performance of {{{color:#0000FF}count(){color}}} for DataFrames read from non-multiline JSON in {{{color:#0000FF}PERMISSIVE{color}}} mode) appears to cause {{{color:#0000FF}count(){color}}} to erroneously include empty lines in its result total if run prior to JSON parsing taking place.
For the following input:
{code:json}
{ "a" : 1 , "b" : 2 , "c" : 3 }
{ "a" : 4 , "b" : 5 , "c" : 6 }
{ "a" : 7 , "b" : 8 , "c" : 9 }
{code}
*+Spark 2.3:+*
{code:scala}
scala> val df = spark.read.json("sql/core/src/test/resources/test-data/with-empty-line.json")
df: org.apache.spark.sql.DataFrame = [a: bigint, b: bigint ... 1 more field]
scala> df.count
res0: Long = 3
scala> df.cache.count
res3: Long = 3
{code}
*+Spark 2.4:+*
{code:scala}
scala> val df = spark.read.json("sql/core/src/test/resources/test-data/with-empty-line.json")
df: org.apache.spark.sql.DataFrame = [a: bigint, b: bigint ... 1 more field]
scala> df.count
res0: Long = 7
scala> df.cache.count
res1: Long = 3
{code}
Since the count is apparently updated and cached when the Jackson parser runs, the optimization also causes the count to appear to be unstable upon cache/persist operations, as shown above.
CSV inputs, also optimized via [SPARK-24959|https://issues.apache.org/jira/browse/SPARK-24959], do not appear to be impacted by this effect.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)
---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org