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

Ayoub Benali edited comment on SPARK-5508 at 2/12/15 9:53 AM:
--------------------------------------------------------------

I narrowed down the problem, the issue seems to come from the "insert into 
table persisted_table select * from tmp_table" command.
By using scala code: saving the Schema RDD as a parquet file, reloading it, 
saving it in the hive meta store and querying the column of array type works 
just fine.

So When I do the insertion from the tmp_table to the persisted_table using Hive 
Context the data in column of array type seems to be inserted in the wrong way 
into parquet which breaks the query after words. 

I tried Spark SQL CLI and "insertIntoTable" method to do the insertion as well, 
but it lead to the same issue when querying the table. 


 


was (Author: ayoub):
I narrowed down the problem, the issue seems to come from the "insert into 
table persisted_table select * from tmp_table" command.
By using scala code: saving the Schema RDD as a parquet file, reloading it, 
saving it in the hive meta store and querying the column of array type works 
just fine.

So When I do the insertion from the tmp_table to the persisted_table using Hive 
Context the data in column of array type seems to be inserted in the wrong way 
into parquet which breaks the query after words. 

I tried Spark SQL CLI to do the insertion as well, but it lead to the same 
issue when querying the table. 


 

> [hive context] Unable to query array once saved as parquet
> ----------------------------------------------------------
>
>                 Key: SPARK-5508
>                 URL: https://issues.apache.org/jira/browse/SPARK-5508
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 1.2.1
>         Environment: mesos, cdh
>            Reporter: Ayoub Benali
>              Labels: hivecontext, parquet
>
> When the table is saved as parquet, we cannot query a field which is an array 
> of struct, like show bellow:  
> {noformat}
> scala> val data1="""{
>      |     "timestamp": 1422435598,
>      |     "data_array": [
>      |         {
>      |             "field1": 1,
>      |             "field2": 2
>      |         }
>      |     ]
>      | }"""
> scala> val data2="""{
>      |     "timestamp": 1422435598,
>      |     "data_array": [
>      |         {
>      |             "field1": 3,
>      |             "field2": 4
>      |         }
>      |     ]
> scala> val jsonRDD = sc.makeRDD(data1 :: data2 :: Nil)
> scala> val rdd = hiveContext.jsonRDD(jsonRDD)
> scala> rdd.printSchema
> root
>  |-- data_array: array (nullable = true)
>  |    |-- element: struct (containsNull = false)
>  |    |    |-- field1: integer (nullable = true)
>  |    |    |-- field2: integer (nullable = true)
>  |-- timestamp: integer (nullable = true)
> scala> rdd.registerTempTable("tmp_table")
> scala> hiveContext.sql("select data.field1 from tmp_table LATERAL VIEW 
> explode(data_array) nestedStuff AS data").collect
> res3: Array[org.apache.spark.sql.Row] = Array([1], [3])
> scala> hiveContext.sql("SET hive.exec.dynamic.partition = true")
> scala> hiveContext.sql("SET hive.exec.dynamic.partition.mode = nonstrict")
> scala> hiveContext.sql("set parquet.compression=GZIP")
> scala> hiveContext.setConf("spark.sql.parquet.binaryAsString", "true")
> scala> hiveContext.sql("create external table if not exists 
> persisted_table(data_array ARRAY <STRUCT<field1: INT, field2: INT>>, 
> timestamp INT) STORED AS PARQUET Location 'hdfs://****/test_table'")
> scala> hiveContext.sql("insert into table persisted_table select * from 
> tmp_table").collect
> scala> hiveContext.sql("select data.field1 from persisted_table LATERAL VIEW 
> explode(data_array) nestedStuff AS data").collect
> parquet.io.ParquetDecodingException: Can not read value at 0 in block -1 in 
> file hdfs://*****/test_table/part-00001
>   at 
> parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:213)
>   at 
> parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:204)
>   at org.apache.spark.rdd.NewHadoopRDD$$anon$1.hasNext(NewHadoopRDD.scala:145)
>   at 
> org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:39)
>   at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:327)
>   at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371)
>   at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:327)
>   at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:327)
>   at scala.collection.Iterator$class.foreach(Iterator.scala:727)
>   at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
>   at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48)
>   at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103)
>   at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47)
>   at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273)
>   at scala.collection.AbstractIterator.to(Iterator.scala:1157)
>   at 
> scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265)
>   at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157)
>   at scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252)
>   at scala.collection.AbstractIterator.toArray(Iterator.scala:1157)
>   at org.apache.spark.rdd.RDD$$anonfun$17.apply(RDD.scala:797)
>   at org.apache.spark.rdd.RDD$$anonfun$17.apply(RDD.scala:797)
>   at 
> org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1353)
>   at 
> org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1353)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)
>   at org.apache.spark.scheduler.Task.run(Task.scala:56)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:200)
>   at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>   at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>   at java.lang.Thread.run(Thread.java:744)
> Caused by: java.lang.IndexOutOfBoundsException: Index: 0, Size: 0
>   at java.util.ArrayList.rangeCheck(ArrayList.java:635)
>   at java.util.ArrayList.get(ArrayList.java:411)
>   at parquet.io.GroupColumnIO.getFirst(GroupColumnIO.java:99)
>   at parquet.io.GroupColumnIO.getFirst(GroupColumnIO.java:99)
>   at parquet.io.GroupColumnIO.getFirst(GroupColumnIO.java:99)
>   at parquet.io.PrimitiveColumnIO.getFirst(PrimitiveColumnIO.java:99)
>   at parquet.io.PrimitiveColumnIO.isFirst(PrimitiveColumnIO.java:94)
>   at 
> parquet.io.RecordReaderImplementation.<init>(RecordReaderImplementation.java:274)
>   at parquet.io.MessageColumnIO$1.visit(MessageColumnIO.java:131)
>   at parquet.io.MessageColumnIO$1.visit(MessageColumnIO.java:96)
>   at 
> parquet.filter2.compat.FilterCompat$NoOpFilter.accept(FilterCompat.java:136)
>   at parquet.io.MessageColumnIO.getRecordReader(MessageColumnIO.java:96)
>   at 
> parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:126)
>   at 
> parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:193)
>   ... 28 more
> 15/01/30 16:49:55 INFO TaskSetManager: Starting task 1.1 in stage 3.0 (TID 
> 12, ****, NODE_LOCAL, 1610 bytes)
> 15/01/30 16:49:55 INFO TaskSetManager: Lost task 1.1 in stage 3.0 (TID 12) on 
> executor ****: parquet.io.ParquetDecodingException (Can not read value at 0 
> in block -1 in file hdfs://*****/test_table/part-00001) [duplicate 1]
> 15/01/30 16:49:55 INFO TaskSetManager: Starting task 1.2 in stage 3.0 (TID 
> 13, ****, NODE_LOCAL, 1610 bytes)
> 15/01/30 16:49:55 WARN TaskSetManager: Lost task 2.0 in stage 3.0 (TID 11, 
> ****): parquet.io.ParquetDecodingException: Can not read value at 0 in block 
> -1 in file hdfs://*****/test_table/part-00002
>   at 
> parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:213)
>   at 
> parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:204)
>   at org.apache.spark.rdd.NewHadoopRDD$$anon$1.hasNext(NewHadoopRDD.scala:145)
>   at 
> org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:39)
>   at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:327)
>   at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371)
>   at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:327)
>   at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:327)
>   at scala.collection.Iterator$class.foreach(Iterator.scala:727)
>   at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
>   at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48)
>   at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103)
>   at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47)
>   at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273)
>   at scala.collection.AbstractIterator.to(Iterator.scala:1157)
>   at 
> scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265)
>   at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157)
>   at scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252)
>   at scala.collection.AbstractIterator.toArray(Iterator.scala:1157)
>   at org.apache.spark.rdd.RDD$$anonfun$17.apply(RDD.scala:797)
>   at org.apache.spark.rdd.RDD$$anonfun$17.apply(RDD.scala:797)
>   at 
> org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1353)
>   at 
> org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1353)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)
>   at org.apache.spark.scheduler.Task.run(Task.scala:56)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:200)
>   at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>   at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>   at java.lang.Thread.run(Thread.java:744)
> Caused by: java.lang.IndexOutOfBoundsException: Index: 0, Size: 0
>   at java.util.ArrayList.rangeCheck(ArrayList.java:635)
>   at java.util.ArrayList.get(ArrayList.java:411)
>   at parquet.io.GroupColumnIO.getFirst(GroupColumnIO.java:99)
>   at parquet.io.GroupColumnIO.getFirst(GroupColumnIO.java:99)
>   at parquet.io.GroupColumnIO.getFirst(GroupColumnIO.java:99)
>   at parquet.io.PrimitiveColumnIO.getFirst(PrimitiveColumnIO.java:99)
>   at parquet.io.PrimitiveColumnIO.isFirst(PrimitiveColumnIO.java:94)
>   at 
> parquet.io.RecordReaderImplementation.<init>(RecordReaderImplementation.java:274)
>   at parquet.io.MessageColumnIO$1.visit(MessageColumnIO.java:131)
>   at parquet.io.MessageColumnIO$1.visit(MessageColumnIO.java:96)
>   at 
> parquet.filter2.compat.FilterCompat$NoOpFilter.accept(FilterCompat.java:136)
>   at parquet.io.MessageColumnIO.getRecordReader(MessageColumnIO.java:96)
>   at 
> parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:126)
>   at 
> parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:193)
>   ... 28 more
> 15/01/30 16:49:55 INFO TaskSetManager: Starting task 2.1 in stage 3.0 (TID 
> 14, ****, NODE_LOCAL, 1610 bytes)
> 15/01/30 16:49:55 INFO TaskSetManager: Lost task 1.2 in stage 3.0 (TID 13) on 
> executor ****: parquet.io.ParquetDecodingException (Can not read value at 0 
> in block -1 in file hdfs://*****/test_table/part-00001) [duplicate 2]
> 15/01/30 16:49:55 INFO TaskSetManager: Starting task 1.3 in stage 3.0 (TID 
> 15, ****, NODE_LOCAL, 1610 bytes)
> 15/01/30 16:49:55 INFO TaskSetManager: Lost task 1.3 in stage 3.0 (TID 15) on 
> executor ****: parquet.io.ParquetDecodingException (Can not read value at 0 
> in block -1 in file hdfs://*****/test_table/part-00001) [duplicate 3]
> 15/01/30 16:49:55 ERROR TaskSetManager: Task 1 in stage 3.0 failed 4 times; 
> aborting job
> 15/01/30 16:49:55 INFO TaskSchedulerImpl: Cancelling stage 3
> 15/01/30 16:49:55 INFO TaskSchedulerImpl: Stage 3 was cancelled
> 15/01/30 16:49:55 INFO TaskSetManager: Lost task 2.1 in stage 3.0 (TID 14) on 
> executor ****: parquet.io.ParquetDecodingException (Can not read value at 0 
> in block -1 in file hdfs://*****/test_table/part-00002) [duplicate 1]
> 15/01/30 16:49:55 INFO TaskSchedulerImpl: Removed TaskSet 3.0, whose tasks 
> have all completed, from pool 
> 15/01/30 16:49:55 INFO DAGScheduler: Job 3 failed: collect at 
> SparkPlan.scala:84, took 1.259053 s
> org.apache.spark.SparkException: Job aborted due to stage failure: Task 1 in 
> stage 3.0 failed 4 times, most recent failure: Lost task 1.3 in stage 3.0 
> (TID 15, ****): parquet.io.ParquetDecodingException: Can not read value at 0 
> in block -1 in file hdfs://*****/test_table/part-00001
>   at 
> parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:213)
>   at 
> parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:204)
>   at org.apache.spark.rdd.NewHadoopRDD$$anon$1.hasNext(NewHadoopRDD.scala:145)
>   at 
> org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:39)
>   at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:327)
>   at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371)
>   at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:327)
>   at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:327)
>   at scala.collection.Iterator$class.foreach(Iterator.scala:727)
>   at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
>   at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48)
>   at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103)
>   at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47)
>   at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273)
>   at scala.collection.AbstractIterator.to(Iterator.scala:1157)
>   at 
> scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265)
>   at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157)
>   at scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252)
>   at scala.collection.AbstractIterator.toArray(Iterator.scala:1157)
>   at org.apache.spark.rdd.RDD$$anonfun$17.apply(RDD.scala:797)
>   at org.apache.spark.rdd.RDD$$anonfun$17.apply(RDD.scala:797)
>   at 
> org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1353)
>   at 
> org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1353)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)
>   at org.apache.spark.scheduler.Task.run(Task.scala:56)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:200)
>   at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>   at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>   at java.lang.Thread.run(Thread.java:744)
> Caused by: java.lang.IndexOutOfBoundsException: Index: 0, Size: 0
>   at java.util.ArrayList.rangeCheck(ArrayList.java:635)
>   at java.util.ArrayList.get(ArrayList.java:411)
>   at parquet.io.GroupColumnIO.getFirst(GroupColumnIO.java:99)
>   at parquet.io.GroupColumnIO.getFirst(GroupColumnIO.java:99)
>   at parquet.io.GroupColumnIO.getFirst(GroupColumnIO.java:99)
>   at parquet.io.PrimitiveColumnIO.getFirst(PrimitiveColumnIO.java:99)
>   at parquet.io.PrimitiveColumnIO.isFirst(PrimitiveColumnIO.java:94)
>   at 
> parquet.io.RecordReaderImplementation.<init>(RecordReaderImplementation.java:274)
>   at parquet.io.MessageColumnIO$1.visit(MessageColumnIO.java:131)
>   at parquet.io.MessageColumnIO$1.visit(MessageColumnIO.java:96)
>   at 
> parquet.filter2.compat.FilterCompat$NoOpFilter.accept(FilterCompat.java:136)
>   at parquet.io.MessageColumnIO.getRecordReader(MessageColumnIO.java:96)
>   at 
> parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:126)
>   at 
> parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:193)
>   ... 28 more
> Driver stacktrace:
>   at 
> org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1214)
>   at 
> org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1203)
>   at 
> org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1202)
>   at 
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
>   at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
>   at 
> org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1202)
>   at 
> org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:696)
>   at 
> org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:696)
>   at scala.Option.foreach(Option.scala:236)
>   at 
> org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:696)
>   at 
> org.apache.spark.scheduler.DAGSchedulerEventProcessActor$$anonfun$receive$2.applyOrElse(DAGScheduler.scala:1420)
>   at akka.actor.Actor$class.aroundReceive(Actor.scala:465)
>   at 
> org.apache.spark.scheduler.DAGSchedulerEventProcessActor.aroundReceive(DAGScheduler.scala:1375)
>   at akka.actor.ActorCell.receiveMessage(ActorCell.scala:516)
>   at akka.actor.ActorCell.invoke(ActorCell.scala:487)
>   at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:238)
>   at akka.dispatch.Mailbox.run(Mailbox.scala:220)
>   at 
> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:393)
>   at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
>   at 
> scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
>   at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
>   at 
> scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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

Reply via email to