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

Shixiong Zhu commented on SPARK-24472:
--------------------------------------

cc [~cloud_fan]

> Orc RecordReaderFactory throws IndexOutOfBoundsException
> --------------------------------------------------------
>
>                 Key: SPARK-24472
>                 URL: https://issues.apache.org/jira/browse/SPARK-24472
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 2.3.0
>            Reporter: Shixiong Zhu
>            Priority: Major
>
> When the column number of the underlying file schema is greater than the 
> column number of the table schema, Orc RecordReaderFactory will throw 
> IndexOutOfBoundsException. "spark.sql.hive.convertMetastoreOrc" should be 
> turned off to use HiveTableScanExec. Here is a reproducer:
> {code}
> scala> :paste
> // Entering paste mode (ctrl-D to finish)
> Seq(("abc", 123, 123L)).toDF("s", "i", 
> "l").write.partitionBy("i").format("orc").mode("append").save("/tmp/orctest")
> spark.sql("""
> CREATE EXTERNAL TABLE orctest(s string)
> PARTITIONED BY (i int)
> ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.orc.OrcSerde'
> WITH SERDEPROPERTIES (
>   'serialization.format' = '1'
> )
> STORED AS
>   INPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcInputFormat'
>   OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat'
> LOCATION '/tmp/orctest'
> """)
> spark.sql("msck repair table orctest")
> spark.sql("set spark.sql.hive.convertMetastoreOrc=false")
> // Exiting paste mode, now interpreting.
> 18/06/05 15:34:52 WARN ObjectStore: Failed to get database global_temp, 
> returning NoSuchObjectException
> res0: org.apache.spark.sql.DataFrame = [key: string, value: string]
> scala> spark.read.format("orc").load("/tmp/orctest").show()
> +---+---+---+
> |  s|  l|  i|
> +---+---+---+
> |abc|123|123|
> +---+---+---+
> scala> spark.sql("select * from orctest").show()
> 18/06/05 15:34:59 ERROR Executor: Exception in task 0.0 in stage 2.0 (TID 2)
> java.lang.IndexOutOfBoundsException: toIndex = 2
>       at java.util.ArrayList.subListRangeCheck(ArrayList.java:1004)
>       at java.util.ArrayList.subList(ArrayList.java:996)
>       at 
> org.apache.hadoop.hive.ql.io.orc.RecordReaderFactory.getSchemaOnRead(RecordReaderFactory.java:161)
>       at 
> org.apache.hadoop.hive.ql.io.orc.RecordReaderFactory.createTreeReader(RecordReaderFactory.java:66)
>       at 
> org.apache.hadoop.hive.ql.io.orc.RecordReaderImpl.<init>(RecordReaderImpl.java:202)
>       at 
> org.apache.hadoop.hive.ql.io.orc.ReaderImpl.rowsOptions(ReaderImpl.java:539)
>       at 
> org.apache.hadoop.hive.ql.io.orc.OrcRawRecordMerger$ReaderPair.<init>(OrcRawRecordMerger.java:183)
>       at 
> org.apache.hadoop.hive.ql.io.orc.OrcRawRecordMerger$OriginalReaderPair.<init>(OrcRawRecordMerger.java:226)
>       at 
> org.apache.hadoop.hive.ql.io.orc.OrcRawRecordMerger.<init>(OrcRawRecordMerger.java:437)
>       at 
> org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getReader(OrcInputFormat.java:1215)
>       at 
> org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getRecordReader(OrcInputFormat.java:1113)
>       at 
> org.apache.spark.rdd.HadoopRDD$$anon$1.liftedTree1$1(HadoopRDD.scala:267)
>       at org.apache.spark.rdd.HadoopRDD$$anon$1.<init>(HadoopRDD.scala:266)
>       at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:224)
>       at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:95)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>       at org.apache.spark.rdd.UnionRDD.compute(UnionRDD.scala:105)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>       at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
>       at org.apache.spark.scheduler.Task.run(Task.scala:109)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:367)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> 18/06/05 15:34:59 WARN TaskSetManager: Lost task 0.0 in stage 2.0 (TID 2, 
> localhost, executor driver): java.lang.IndexOutOfBoundsException: toIndex = 2
>       at java.util.ArrayList.subListRangeCheck(ArrayList.java:1004)
>       at java.util.ArrayList.subList(ArrayList.java:996)
>       at 
> org.apache.hadoop.hive.ql.io.orc.RecordReaderFactory.getSchemaOnRead(RecordReaderFactory.java:161)
>       at 
> org.apache.hadoop.hive.ql.io.orc.RecordReaderFactory.createTreeReader(RecordReaderFactory.java:66)
>       at 
> org.apache.hadoop.hive.ql.io.orc.RecordReaderImpl.<init>(RecordReaderImpl.java:202)
>       at 
> org.apache.hadoop.hive.ql.io.orc.ReaderImpl.rowsOptions(ReaderImpl.java:539)
>       at 
> org.apache.hadoop.hive.ql.io.orc.OrcRawRecordMerger$ReaderPair.<init>(OrcRawRecordMerger.java:183)
>       at 
> org.apache.hadoop.hive.ql.io.orc.OrcRawRecordMerger$OriginalReaderPair.<init>(OrcRawRecordMerger.java:226)
>       at 
> org.apache.hadoop.hive.ql.io.orc.OrcRawRecordMerger.<init>(OrcRawRecordMerger.java:437)
>       at 
> org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getReader(OrcInputFormat.java:1215)
>       at 
> org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getRecordReader(OrcInputFormat.java:1113)
>       at 
> org.apache.spark.rdd.HadoopRDD$$anon$1.liftedTree1$1(HadoopRDD.scala:267)
>       at org.apache.spark.rdd.HadoopRDD$$anon$1.<init>(HadoopRDD.scala:266)
>       at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:224)
>       at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:95)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>       at org.apache.spark.rdd.UnionRDD.compute(UnionRDD.scala:105)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>       at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
>       at org.apache.spark.scheduler.Task.run(Task.scala:109)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:367)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> 18/06/05 15:34:59 ERROR TaskSetManager: Task 0 in stage 2.0 failed 1 times; 
> aborting job
> org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in 
> stage 2.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2.0 
> (TID 2, localhost, executor driver): java.lang.IndexOutOfBoundsException: 
> toIndex = 2
>       at java.util.ArrayList.subListRangeCheck(ArrayList.java:1004)
>       at java.util.ArrayList.subList(ArrayList.java:996)
>       at 
> org.apache.hadoop.hive.ql.io.orc.RecordReaderFactory.getSchemaOnRead(RecordReaderFactory.java:161)
>       at 
> org.apache.hadoop.hive.ql.io.orc.RecordReaderFactory.createTreeReader(RecordReaderFactory.java:66)
>       at 
> org.apache.hadoop.hive.ql.io.orc.RecordReaderImpl.<init>(RecordReaderImpl.java:202)
>       at 
> org.apache.hadoop.hive.ql.io.orc.ReaderImpl.rowsOptions(ReaderImpl.java:539)
>       at 
> org.apache.hadoop.hive.ql.io.orc.OrcRawRecordMerger$ReaderPair.<init>(OrcRawRecordMerger.java:183)
>       at 
> org.apache.hadoop.hive.ql.io.orc.OrcRawRecordMerger$OriginalReaderPair.<init>(OrcRawRecordMerger.java:226)
>       at 
> org.apache.hadoop.hive.ql.io.orc.OrcRawRecordMerger.<init>(OrcRawRecordMerger.java:437)
>       at 
> org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getReader(OrcInputFormat.java:1215)
>       at 
> org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getRecordReader(OrcInputFormat.java:1113)
>       at 
> org.apache.spark.rdd.HadoopRDD$$anon$1.liftedTree1$1(HadoopRDD.scala:267)
>       at org.apache.spark.rdd.HadoopRDD$$anon$1.<init>(HadoopRDD.scala:266)
>       at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:224)
>       at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:95)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>       at org.apache.spark.rdd.UnionRDD.compute(UnionRDD.scala:105)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>       at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
>       at org.apache.spark.scheduler.Task.run(Task.scala:109)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:367)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> Driver stacktrace:
>   at 
> org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1600)
>   at 
> org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1588)
>   at 
> org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1587)
>   at 
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
>   at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
>   at 
> org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1587)
>   at 
> org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:831)
>   at 
> org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:831)
>   at scala.Option.foreach(Option.scala:257)
>   at 
> org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:831)
>   at 
> org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1821)
>   at 
> org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1770)
>   at 
> org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1759)
>   at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)
>   at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:642)
>   at org.apache.spark.SparkContext.runJob(SparkContext.scala:2030)
>   at org.apache.spark.SparkContext.runJob(SparkContext.scala:2051)
>   at org.apache.spark.SparkContext.runJob(SparkContext.scala:2070)
>   at org.apache.spark.sql.execution.SparkPlan.executeTake(SparkPlan.scala:363)
>   at 
> org.apache.spark.sql.execution.CollectLimitExec.executeCollect(limit.scala:38)
>   at 
> org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$collectFromPlan(Dataset.scala:3285)
>   at org.apache.spark.sql.Dataset$$anonfun$head$1.apply(Dataset.scala:2496)
>   at org.apache.spark.sql.Dataset$$anonfun$head$1.apply(Dataset.scala:2496)
>   at org.apache.spark.sql.Dataset$$anonfun$52.apply(Dataset.scala:3266)
>   at 
> org.apache.spark.sql.execution.SQLExecution$$anonfun$withNewExecutionId$1.apply(SQLExecution.scala:78)
>   at 
> org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:125)
>   at 
> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:73)
>   at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3265)
>   at org.apache.spark.sql.Dataset.head(Dataset.scala:2496)
>   at org.apache.spark.sql.Dataset.take(Dataset.scala:2710)
>   at org.apache.spark.sql.Dataset.showString(Dataset.scala:254)
>   at org.apache.spark.sql.Dataset.show(Dataset.scala:733)
>   at org.apache.spark.sql.Dataset.show(Dataset.scala:692)
>   at org.apache.spark.sql.Dataset.show(Dataset.scala:701)
>   ... 51 elided
> Caused by: java.lang.IndexOutOfBoundsException: toIndex = 2
>   at java.util.ArrayList.subListRangeCheck(ArrayList.java:1004)
>   at java.util.ArrayList.subList(ArrayList.java:996)
>   at 
> org.apache.hadoop.hive.ql.io.orc.RecordReaderFactory.getSchemaOnRead(RecordReaderFactory.java:161)
>   at 
> org.apache.hadoop.hive.ql.io.orc.RecordReaderFactory.createTreeReader(RecordReaderFactory.java:66)
>   at 
> org.apache.hadoop.hive.ql.io.orc.RecordReaderImpl.<init>(RecordReaderImpl.java:202)
>   at 
> org.apache.hadoop.hive.ql.io.orc.ReaderImpl.rowsOptions(ReaderImpl.java:539)
>   at 
> org.apache.hadoop.hive.ql.io.orc.OrcRawRecordMerger$ReaderPair.<init>(OrcRawRecordMerger.java:183)
>   at 
> org.apache.hadoop.hive.ql.io.orc.OrcRawRecordMerger$OriginalReaderPair.<init>(OrcRawRecordMerger.java:226)
>   at 
> org.apache.hadoop.hive.ql.io.orc.OrcRawRecordMerger.<init>(OrcRawRecordMerger.java:437)
>   at 
> org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getReader(OrcInputFormat.java:1215)
>   at 
> org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getRecordReader(OrcInputFormat.java:1113)
>   at org.apache.spark.rdd.HadoopRDD$$anon$1.liftedTree1$1(HadoopRDD.scala:267)
>   at org.apache.spark.rdd.HadoopRDD$$anon$1.<init>(HadoopRDD.scala:266)
>   at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:224)
>   at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:95)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>   at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>   at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>   at org.apache.spark.rdd.UnionRDD.compute(UnionRDD.scala:105)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>   at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>   at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>   at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>   at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
>   at org.apache.spark.scheduler.Task.run(Task.scala:109)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:367)
>   at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>   at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>   at java.lang.Thread.run(Thread.java:745)
> {code}
> The issue is 
> [HiveTableScanExec|https://github.com/apache/spark/blob/93df3cd03503fca7745141fbd2676b8bf70fe92f/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala#L138]
>  sets "serdeConstants.LIST_COLUMNS" and trigger [this line in 
> RecordReaderFactory|https://github.com/apache/hive/blob/release-1.2.1/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderFactory.java#L161]



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