[ 
https://issues.apache.org/jira/browse/SPARK-25879?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16668327#comment-16668327
 ] 

Hyukjin Kwon commented on SPARK-25879:
--------------------------------------

Shell we leave this resolved as a duplicate of SPARK-25407 since the PR is 
already open for that?

> Schema pruning fails when a nested field and top level field are selected
> -------------------------------------------------------------------------
>
>                 Key: SPARK-25879
>                 URL: https://issues.apache.org/jira/browse/SPARK-25879
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 2.4.0
>            Reporter: DB Tsai
>            Priority: Major
>
> If we select a nested field and a top level field, the schema pruning will 
> fail. Here is the reproducible test,
>  
> {code:java}
>   testSchemaPruning("select a single complex field and a top level field") {
>     val query = sql("select * from contacts")
>       .select("name.middle", "address")
>     query.explain(true)
>     query.printSchema()
>     query.show()
>     checkScan(query, "struct<name:struct<middle:string>,address:string>")
>   }
> {code}
> and the exception is
> {code:java}
> 23:16:05.864 ERROR org.apache.spark.executor.Executor: Exception in task 1.0 
> in stage 3.0 (TID 6)
> org.apache.spark.sql.execution.QueryExecutionException: Encounter error while 
> reading parquet files. One possible cause: Parquet column cannot be converted 
> in the corresponding files. Details: 
>       at 
> org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:193)
>       at 
> org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:101)
>       at 
> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown
>  Source)
>       at 
> org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
>       at 
> org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$11$$anon$1.hasNext(WholeStageCodegenExec.scala:674)
>       at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:255)
>       at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:247)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:850)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:850)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:325)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:289)
>       at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
>       at org.apache.spark.scheduler.Task.run(Task.scala:121)
>       at 
> org.apache.spark.executor.Executor$TaskRunner$$anonfun$11.apply(Executor.scala:419)
>       at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:425)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>       at java.lang.Thread.run(Thread.java:748)
> Caused by: org.apache.parquet.io.ParquetDecodingException: Can not read value 
> at 0 in block -1 in file 
> file:/private/var/folders/pr/4q3b9vkx36lbygjr5jhfmjcw0000gn/T/spark-a4fff68d-d51a-4c79-aa18-54cfd7f81a75/contacts/p=2/part-00000-8a4d9396-7be3-4fed-a55a-5580684ebda6-c000.snappy.parquet
>       at 
> org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:251)
>       at 
> org.apache.parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:207)
>       at 
> org.apache.spark.sql.execution.datasources.RecordReaderIterator.hasNext(RecordReaderIterator.scala:39)
>       at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:409)
>       at 
> org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:101)
>       at 
> org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:181)
>       ... 19 more
> Caused by: java.lang.IndexOutOfBoundsException: Index: 0, Size: 0
>       at java.util.ArrayList.rangeCheck(ArrayList.java:657)
>       at java.util.ArrayList.get(ArrayList.java:433)
>       at org.apache.parquet.io.GroupColumnIO.getFirst(GroupColumnIO.java:99)
>       at org.apache.parquet.io.GroupColumnIO.getFirst(GroupColumnIO.java:99)
>       at 
> org.apache.parquet.io.PrimitiveColumnIO.getFirst(PrimitiveColumnIO.java:97)
>       at 
> org.apache.parquet.io.PrimitiveColumnIO.isFirst(PrimitiveColumnIO.java:92)
>       at 
> org.apache.parquet.io.RecordReaderImplementation.<init>(RecordReaderImplementation.java:278)
>       at 
> org.apache.parquet.io.MessageColumnIO$1.visit(MessageColumnIO.java:147)
>       at 
> org.apache.parquet.io.MessageColumnIO$1.visit(MessageColumnIO.java:109)
>       at 
> org.apache.parquet.filter2.compat.FilterCompat$NoOpFilter.accept(FilterCompat.java:165)
>       at 
> org.apache.parquet.io.MessageColumnIO.getRecordReader(MessageColumnIO.java:109)
>       at 
> org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:137)
>       at 
> org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:222)
>       ... 24 more
> 23:16:05.896 WARN org.apache.spark.scheduler.TaskSetManager: Lost task 1.0 in 
> stage 3.0 (TID 6, localhost, executor driver): 
> org.apache.spark.sql.execution.QueryExecutionException: Encounter error while 
> reading parquet files. One possible cause: Parquet column cannot be converted 
> in the corresponding files. Details: 
>       at 
> org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:193)
>       at 
> org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:101)
>       at 
> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown
>  Source)
>       at 
> org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
>       at 
> org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$11$$anon$1.hasNext(WholeStageCodegenExec.scala:674)
>       at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:255)
>       at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:247)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:850)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:850)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:325)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:289)
>       at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
>       at org.apache.spark.scheduler.Task.run(Task.scala:121)
>       at 
> org.apache.spark.executor.Executor$TaskRunner$$anonfun$11.apply(Executor.scala:419)
>       at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:425)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>       at java.lang.Thread.run(Thread.java:748)
> Caused by: org.apache.parquet.io.ParquetDecodingException: Can not read value 
> at 0 in block -1 in file 
> file:/private/var/folders/pr/4q3b9vkx36lbygjr5jhfmjcw0000gn/T/spark-a4fff68d-d51a-4c79-aa18-54cfd7f81a75/contacts/p=2/part-00000-8a4d9396-7be3-4fed-a55a-5580684ebda6-c000.snappy.parquet
>       at 
> org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:251)
>       at 
> org.apache.parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:207)
>       at 
> org.apache.spark.sql.execution.datasources.RecordReaderIterator.hasNext(RecordReaderIterator.scala:39)
>       at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:409)
>       at 
> org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:101)
>       at 
> org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:181)
>       ... 19 more
> Caused by: java.lang.IndexOutOfBoundsException: Index: 0, Size: 0
>       at java.util.ArrayList.rangeCheck(ArrayList.java:657)
>       at java.util.ArrayList.get(ArrayList.java:433)
>       at org.apache.parquet.io.GroupColumnIO.getFirst(GroupColumnIO.java:99)
>       at org.apache.parquet.io.GroupColumnIO.getFirst(GroupColumnIO.java:99)
>       at 
> org.apache.parquet.io.PrimitiveColumnIO.getFirst(PrimitiveColumnIO.java:97)
>       at 
> org.apache.parquet.io.PrimitiveColumnIO.isFirst(PrimitiveColumnIO.java:92)
>       at 
> org.apache.parquet.io.RecordReaderImplementation.<init>(RecordReaderImplementation.java:278)
>       at 
> org.apache.parquet.io.MessageColumnIO$1.visit(MessageColumnIO.java:147)
>       at 
> org.apache.parquet.io.MessageColumnIO$1.visit(MessageColumnIO.java:109)
>       at 
> org.apache.parquet.filter2.compat.FilterCompat$NoOpFilter.accept(FilterCompat.java:165)
>       at 
> org.apache.parquet.io.MessageColumnIO.getRecordReader(MessageColumnIO.java:109)
>       at 
> org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:137)
>       at 
> org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:222)
>       ... 24 more
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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

Reply via email to