You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "nirav patel (JIRA)" <ji...@apache.org> on 2019/03/18 20:53:00 UTC

[jira] [Updated] (SPARK-26844) Parquet Reader exception - ArrayIndexOutOfBound should give more information to user

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

nirav patel updated SPARK-26844:
--------------------------------
    Description: 
I get following error while reading parquet file which has primitive datatypes (INT32, binary)

 Parquet file is potentially corrupt. It has newline character in some field value.

 

spark.read.format("parquet").load(path).show() // error happens here

 

Caused by: java.lang.ArrayIndexOutOfBoundsException

at java.lang.System.arraycopy(Native Method)

at org.apache.spark.sql.execution.vectorized.OnHeapColumnVector.putBytes(OnHeapColumnVector.java:163)

at org.apache.spark.sql.execution.vectorized.ColumnVector.appendBytes(ColumnVector.java:733)

at org.apache.spark.sql.execution.vectorized.OnHeapColumnVector.putByteArray(OnHeapColumnVector.java:410)

at org.apache.spark.sql.execution.datasources.parquet.VectorizedPlainValuesReader.readBinary(VectorizedPlainValuesReader.java:167)

at org.apache.spark.sql.execution.datasources.parquet.VectorizedRleValuesReader.readBinarys(VectorizedRleValuesReader.java:402)

at org.apache.spark.sql.execution.datasources.parquet.VectorizedColumnReader.readBinaryBatch(VectorizedColumnReader.java:419)

at org.apache.spark.sql.execution.datasources.parquet.VectorizedColumnReader.readBatch(VectorizedColumnReader.java:203)

at org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader.nextBatch(VectorizedParquetRecordReader.java:230)

at org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader.nextKeyValue(VectorizedParquetRecordReader.java:137)

at org.apache.spark.sql.execution.datasources.RecordReaderIterator.hasNext(RecordReaderIterator.scala:39)

at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:105)

at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:177)

at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:105)

at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.scan_nextBatch$(Unknown Source)

at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)

at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)

at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:395)

at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:234)

at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:228)

at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:827)

at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:827)

at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)

at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)

at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)

at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)

at org.apache.spark.scheduler.Task.run(Task.scala:108)

at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:338)

at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)

at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)

 

 

 

Point if  ArrayIndexOutOfBoundsException raised on a column/field spark should say what particular column/field it is. it helps in troubleshoot.

 

e.g. I get following error while reading same file using Drill reader.

org.apache.drill.common.exceptions.UserRemoteException: DATA_READ ERROR: Error reading page data File: /.../../part-00016-00000-m-00016.parquet *Column: GROUP_NAME* Row Group Start: 5539 Fragment 0:0 

I also get more specific information in Drillbit.log

  was:
I get following error while reading parquet file which has primitive datatypes (INT32, binary)

 

 

spark.read.format("parquet").load(path).show() // error happens here

 

Caused by: java.lang.ArrayIndexOutOfBoundsException

at java.lang.System.arraycopy(Native Method)

at org.apache.spark.sql.execution.vectorized.OnHeapColumnVector.putBytes(OnHeapColumnVector.java:163)

at org.apache.spark.sql.execution.vectorized.ColumnVector.appendBytes(ColumnVector.java:733)

at org.apache.spark.sql.execution.vectorized.OnHeapColumnVector.putByteArray(OnHeapColumnVector.java:410)

at org.apache.spark.sql.execution.datasources.parquet.VectorizedPlainValuesReader.readBinary(VectorizedPlainValuesReader.java:167)

at org.apache.spark.sql.execution.datasources.parquet.VectorizedRleValuesReader.readBinarys(VectorizedRleValuesReader.java:402)

at org.apache.spark.sql.execution.datasources.parquet.VectorizedColumnReader.readBinaryBatch(VectorizedColumnReader.java:419)

at org.apache.spark.sql.execution.datasources.parquet.VectorizedColumnReader.readBatch(VectorizedColumnReader.java:203)

at org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader.nextBatch(VectorizedParquetRecordReader.java:230)

at org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader.nextKeyValue(VectorizedParquetRecordReader.java:137)

at org.apache.spark.sql.execution.datasources.RecordReaderIterator.hasNext(RecordReaderIterator.scala:39)

at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:105)

at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:177)

at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:105)

at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.scan_nextBatch$(Unknown Source)

at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)

at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)

at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:395)

at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:234)

at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:228)

at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:827)

at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:827)

at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)

at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)

at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)

at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)

at org.apache.spark.scheduler.Task.run(Task.scala:108)

at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:338)

at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)

at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)

 

 

 

Point if  ArrayIndexOutOfBoundsException raised on a column/field spark should say what particular column/field it is. it helps in troubleshoot.

 

e.g. I get following error while reading same file using Drill reader.

org.apache.drill.common.exceptions.UserRemoteException: DATA_READ ERROR: Error reading page data File: /.../../part-00016-00000-m-00016.parquet *Column: GROUP_NAME* Row Group Start: 5539 Fragment 0:0 

I also get more specific information in Drillbit.log


> Parquet Reader exception - ArrayIndexOutOfBound should give more information to user
> ------------------------------------------------------------------------------------
>
>                 Key: SPARK-26844
>                 URL: https://issues.apache.org/jira/browse/SPARK-26844
>             Project: Spark
>          Issue Type: Improvement
>          Components: SQL
>    Affects Versions: 2.2.1, 2.3.1
>            Reporter: nirav patel
>            Priority: Minor
>
> I get following error while reading parquet file which has primitive datatypes (INT32, binary)
>  Parquet file is potentially corrupt. It has newline character in some field value.
>  
> spark.read.format("parquet").load(path).show() // error happens here
>  
> Caused by: java.lang.ArrayIndexOutOfBoundsException
> at java.lang.System.arraycopy(Native Method)
> at org.apache.spark.sql.execution.vectorized.OnHeapColumnVector.putBytes(OnHeapColumnVector.java:163)
> at org.apache.spark.sql.execution.vectorized.ColumnVector.appendBytes(ColumnVector.java:733)
> at org.apache.spark.sql.execution.vectorized.OnHeapColumnVector.putByteArray(OnHeapColumnVector.java:410)
> at org.apache.spark.sql.execution.datasources.parquet.VectorizedPlainValuesReader.readBinary(VectorizedPlainValuesReader.java:167)
> at org.apache.spark.sql.execution.datasources.parquet.VectorizedRleValuesReader.readBinarys(VectorizedRleValuesReader.java:402)
> at org.apache.spark.sql.execution.datasources.parquet.VectorizedColumnReader.readBinaryBatch(VectorizedColumnReader.java:419)
> at org.apache.spark.sql.execution.datasources.parquet.VectorizedColumnReader.readBatch(VectorizedColumnReader.java:203)
> at org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader.nextBatch(VectorizedParquetRecordReader.java:230)
> at org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader.nextKeyValue(VectorizedParquetRecordReader.java:137)
> at org.apache.spark.sql.execution.datasources.RecordReaderIterator.hasNext(RecordReaderIterator.scala:39)
> at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:105)
> at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:177)
> at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:105)
> at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.scan_nextBatch$(Unknown Source)
> at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)
> at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
> at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:395)
> at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:234)
> at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:228)
> at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:827)
> at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:827)
> at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
> at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
> at org.apache.spark.scheduler.Task.run(Task.scala:108)
> at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:338)
> at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  
>  
>  
> Point if  ArrayIndexOutOfBoundsException raised on a column/field spark should say what particular column/field it is. it helps in troubleshoot.
>  
> e.g. I get following error while reading same file using Drill reader.
> org.apache.drill.common.exceptions.UserRemoteException: DATA_READ ERROR: Error reading page data File: /.../../part-00016-00000-m-00016.parquet *Column: GROUP_NAME* Row Group Start: 5539 Fragment 0:0 
> I also get more specific information in Drillbit.log



--
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