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

Davies Liu reassigned SPARK-19500:
----------------------------------

    Assignee: Davies Liu

> Fail to spill the aggregated hash map when radix sort is used
> -------------------------------------------------------------
>
>                 Key: SPARK-19500
>                 URL: https://issues.apache.org/jira/browse/SPARK-19500
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 2.1.0
>            Reporter: Davies Liu
>            Assignee: Davies Liu
>
> Radix sort requires that only half of the array could be occupied. But the 
> aggregated hash map have a off-by-1 bug that could have 1 more item than half 
> of the array, when this happen, the spilling will fail as:
> {code}
> org.apache.spark.SparkException: Job aborted due to stage failure: Task 171 
> in stage 10.0 failed 4 times, most recent failure: Lost task 171.3 in stage 
> 10.0 (TID 23899, 10.145.253.180, executor 24): 
> java.lang.IllegalStateException: There is no space for new record 
> at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.insertRecord(UnsafeInMemorySorter.java:227)
>  
> at 
> org.apache.spark.sql.execution.UnsafeKVExternalSorter.<init>(UnsafeKVExternalSorter.java:130)
>  
> at 
> org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap.destructAndCreateExternalSorter(UnsafeFixedWidthAggregationMap.java:250)
>  
> at 
> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.agg_doAggregateWithKeys$(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$10$$anon$2.hasNext(WholeStageCodegenExec.scala:396)
>  
> at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408) 
> at 
> org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:166)
>  
> at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96) 
> at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53) 
> at org.apache.spark.scheduler.Task.run(Task.scala:99) 
> at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:322) 
> 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:1435)
>  
> at 
> org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1423)
>  
> at 
> org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1422)
>  
> 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:1422) 
> at 
> org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:802)
>  
> at 
> org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:802)
>  
> at scala.Option.foreach(Option.scala:257) 
> at 
> org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:802)
>  
> at 
> org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1650)
>  
> at 
> org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1605)
>  
> at 
> org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1594)
>  
> at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48) 
> at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:628) 
> at org.apache.spark.SparkContext.runJob(SparkContext.scala:1918) 
> at org.apache.spark.SparkContext.runJob(SparkContext.scala:1931) 
> at org.apache.spark.SparkContext.runJob(SparkContext.scala:1951) 
> at 
> org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply$mcV$sp(FileFormatWriter.scala:137)
>  
> ... 32 more 
> Caused by: java.lang.IllegalStateException: There is no space for new record 
> at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.insertRecord(UnsafeInMemorySorter.java:227)
>  
> at 
> org.apache.spark.sql.execution.UnsafeKVExternalSorter.<init>(UnsafeKVExternalSorter.java:130)
>  
> at 
> org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap.destructAndCreateExternalSorter(UnsafeFixedWidthAggregationMap.java:250)
>  
> at 
> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.agg_doAggregateWithKeys$(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$10$$anon$2.hasNext(WholeStageCodegenExec.scala:396)
>  
> at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408) 
> at 
> org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:166)
>  
> at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96) 
> at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53) 
> at org.apache.spark.scheduler.Task.run(Task.scala:99) 
> at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:322) 
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  
> ... 1 more
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

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

Reply via email to