You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "dmtran (JIRA)" <ji...@apache.org> on 2017/08/02 08:57:00 UTC

[jira] [Created] (SPARK-21610) Corrupt records are not handled properly when creating a dataframe from a file

dmtran created SPARK-21610:
------------------------------

             Summary: Corrupt records are not handled properly when creating a dataframe from a file
                 Key: SPARK-21610
                 URL: https://issues.apache.org/jira/browse/SPARK-21610
             Project: Spark
          Issue Type: Bug
          Components: SQL
    Affects Versions: 2.2.0, 2.0.2
         Environment: macOs Sierra 10.12.5


            Reporter: dmtran


Consider a jsonl file with 3 records. The third record has a value of type string, instead of int.
{code}
echo '{"field": 1}
{"field": 2}
{"field": "3"}' >/tmp/sample.json
{code}

Create a dataframe from this file, with a schema that contains "_corrupt_record" so that corrupt records are kept.

{code}
import org.apache.spark.sql.types._

val schema = new StructType()
  .add("field", ByteType)
  .add("_corrupt_record", StringType)

val file = "/tmp/sample.json"

val dfFromFile = spark.read.schema(schema).json(file)
{code}

Run the following lines from a spark-shell:
{code}
scala> dfFromFile.show(false)
+-----+---------------+
|field|_corrupt_record|
+-----+---------------+
|1    |null           |
|2    |null           |
|null |{"field": "3"} |
+-----+---------------+

scala> dfFromFile.filter($"_corrupt_record".isNotNull).count()
res1: Long = 0

scala> dfFromFile.filter($"_corrupt_record".isNull).count()
res2: Long = 3
{code}
The expected result is 1 corrupt record and 2 valid records, but the actual one is 0 corrupt record and 3 valid records.

The bug is not reproduced if we create a dataframe from a RDD:
{code}
scala> val rdd = sc.textFile(file)
rdd: org.apache.spark.rdd.RDD[String] = /tmp/sample.json MapPartitionsRDD[92] at textFile at <console>:28

scala> val dfFromRdd = spark.read.schema(schema).json(rdd)
dfFromRdd: org.apache.spark.sql.DataFrame = [field: tinyint, _corrupt_record: string]

scala> dfFromRdd.show(false)
+-----+---------------+
|field|_corrupt_record|
+-----+---------------+
|1    |null           |
|2    |null           |
|null |{"field": "3"} |
+-----+---------------+

scala> dfFromRdd.filter($"_corrupt_record".isNotNull).count()
res5: Long = 1

scala> dfFromRdd.filter($"_corrupt_record".isNull).count()
res6: Long = 2
{code}




--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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