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

Naden Franciscus edited comment on SPARK-10474 at 10/1/15 1:33 PM:
-------------------------------------------------------------------

We are executing 20 Spark SQL jobs in parallel using Spark Job Server and 
hitting this issue pretty routinely.

40GB heaps x 6 nodes. Have tried adjusting shuffle.memoryFraction from 0.2 -> 
0.1 with no difference. I am using the HEAD of the Spark 1.5 branch so it 
definitely includes all the commits above.

.16): org.apache.spark.SparkException: Task failed while writing rows.
        at 
org.apache.spark.sql.execution.datasources.DefaultWriterContainer.writeRows(WriterContainer.scala:250)
        at 
org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelation$$anonfun$run$1$$anonfun$apply$mcV$sp$3.apply(InsertIntoHadoopFsRelation.scala:150)
        at 
org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelation$$anonfun$run$1$$anonfun$apply$mcV$sp$3.apply(InsertIntoHadoopFsRelation.scala:150)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
        at org.apache.spark.scheduler.Task.run(Task.scala:88)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
        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)
Caused by: java.io.IOException: Unable to acquire 16777216 bytes of memory
        at 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPage(UnsafeExternalSorter.java:351)
        at 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.<init>(UnsafeExternalSorter.java:138)
        at 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.create(UnsafeExternalSorter.java:106)
        at 
org.apache.spark.sql.execution.UnsafeExternalRowSorter.<init>(UnsafeExternalRowSorter.java:68)
        at 
org.apache.spark.sql.execution.TungstenSort.org$apache$spark$sql$execution$TungstenSort$$preparePartition$1(sort.scala:146)
        at 
org.apache.spark.sql.execution.TungstenSort$$anonfun$doExecute$3.apply(sort.scala:169)
        at 
org.apache.spark.sql.execution.TungstenSort$$anonfun$doExecute$3.apply(sort.scala:169)
        at 
org.apache.spark.rdd.MapPartitionsWithPreparationRDD.prepare(MapPartitionsWithPreparationRDD.scala:50)
        at 
org.apache.spark.rdd.ZippedPartitionsBaseRDD$$anonfun$tryPrepareParents$1.applyOrElse(ZippedPartitionsRDD.scala:83)
        at 
org.apache.spark.rdd.ZippedPartitionsBaseRDD$$anonfun$tryPrepareParents$1.applyOrElse(ZippedPartitionsRDD.scala:82)
        at 
scala.runtime.AbstractPartialFunction.apply(AbstractPartialFunction.scala:33)
        at 
scala.collection.TraversableLike$$anonfun$collect$1.apply(TraversableLike.scala:278)
        at scala.collection.immutable.List.foreach(List.scala:318)
        at 
scala.collection.TraversableLike$class.collect(TraversableLike.scala:278)
        at scala.collection.AbstractTraversable.collect(Traversable.scala:105)
        at 
org.apache.spark.rdd.ZippedPartitionsBaseRDD.tryPrepareParents(ZippedPartitionsRDD.scala:82)
        at 
org.apache.spark.rdd.ZippedPartitionsRDD2.compute(ZippedPartitionsRDD.scala:97)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)



was (Author: nadenf):
We are executing 20 Spark SQL jobs in parallel using Spark Job Server and 
hitting this issue pretty routinely.

40GB heaps x 6 nodes. Have tried adjusting shuffle.memoryFraction from 0.2 -> 
0.1 with no difference. I am using the HEAD of the Spark 1.5 branch so it 
definitely includes all the commits above.

Caused by: java.io.IOException: Unable to acquire 16777216 bytes of memory
        at 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPage(UnsafeExternalSorter.java:351)
        at 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.<init>(UnsafeExternalSorter.java:138)
        at 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.create(UnsafeExternalSorter.java:106)
        at 
org.apache.spark.sql.execution.UnsafeExternalRowSorter.<init>(UnsafeExternalRowSorter.java:68)
        at 
org.apache.spark.sql.execution.TungstenSort.org$apache$spark$sql$execution$TungstenSort$$preparePartition$1(sort.scala:146)
        at 
org.apache.spark.sql.execution.TungstenSort$$anonfun$doExecute$3.apply(sort.scala:169)
        at 
org.apache.spark.sql.execution.TungstenSort$$anonfun$doExecute$3.apply(sort.scala:169)
        at 
org.apache.spark.rdd.MapPartitionsWithPreparationRDD.prepare(MapPartitionsWithPreparationRDD.scala:50)
        at 
org.apache.spark.rdd.ZippedPartitionsBaseRDD$$anonfun$tryPrepareParents$1.applyOrElse(ZippedPartitionsRDD.scala:83)
        at 
org.apache.spark.rdd.ZippedPartitionsBaseRDD$$anonfun$tryPrepareParents$1.applyOrElse(ZippedPartitionsRDD.scala:82)
        at 
scala.runtime.AbstractPartialFunction.apply(AbstractPartialFunction.scala:33)
        at 
scala.collection.TraversableLike$$anonfun$collect$1.apply(TraversableLike.scala:278)
        at scala.collection.immutable.List.foreach(List.scala:318)
        at 
scala.collection.TraversableLike$class.collect(TraversableLike.scala:278)
        at scala.collection.AbstractTraversable.collect(Traversable.scala:105)
        at 
org.apache.spark.rdd.ZippedPartitionsBaseRDD.tryPrepareParents(ZippedPartitionsRDD.scala:82)


> TungstenAggregation cannot acquire memory for pointer array after switching 
> to sort-based
> -----------------------------------------------------------------------------------------
>
>                 Key: SPARK-10474
>                 URL: https://issues.apache.org/jira/browse/SPARK-10474
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 1.5.0
>            Reporter: Yi Zhou
>            Assignee: Andrew Or
>            Priority: Blocker
>             Fix For: 1.5.1, 1.6.0
>
>
> In aggregation case, a  Lost task happened with below error.
> {code}
>  java.io.IOException: Could not acquire 65536 bytes of memory
>         at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.initializeForWriting(UnsafeExternalSorter.java:169)
>         at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.spill(UnsafeExternalSorter.java:220)
>         at 
> org.apache.spark.sql.execution.UnsafeKVExternalSorter.<init>(UnsafeKVExternalSorter.java:126)
>         at 
> org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap.destructAndCreateExternalSorter(UnsafeFixedWidthAggregationMap.java:257)
>         at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.switchToSortBasedAggregation(TungstenAggregationIterator.scala:435)
>         at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.processInputs(TungstenAggregationIterator.scala:379)
>         at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.start(TungstenAggregationIterator.scala:622)
>         at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1.org$apache$spark$sql$execution$aggregate$TungstenAggregate$$anonfun$$executePartition$1(TungstenAggregate.scala:110)
>         at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1$$anonfun$2.apply(TungstenAggregate.scala:119)
>         at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1$$anonfun$2.apply(TungstenAggregate.scala:119)
>         at 
> org.apache.spark.rdd.MapPartitionsWithPreparationRDD.compute(MapPartitionsWithPreparationRDD.scala:64)
>         at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
>         at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
>         at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>         at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
>         at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
>         at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73)
>         at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
>         at org.apache.spark.scheduler.Task.run(Task.scala:88)
>         at 
> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
>         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:745)
> {code}
> Key SQL Query
> {code:sql}
> INSERT INTO TABLE test_table
> SELECT
>   ss.ss_customer_sk AS cid,
>   count(CASE WHEN i.i_class_id=1  THEN 1 ELSE NULL END) AS id1,
>   count(CASE WHEN i.i_class_id=3  THEN 1 ELSE NULL END) AS id3,
>   count(CASE WHEN i.i_class_id=5  THEN 1 ELSE NULL END) AS id5,
>   count(CASE WHEN i.i_class_id=7  THEN 1 ELSE NULL END) AS id7,
>   count(CASE WHEN i.i_class_id=9  THEN 1 ELSE NULL END) AS id9,
>   count(CASE WHEN i.i_class_id=11 THEN 1 ELSE NULL END) AS id11,
>   count(CASE WHEN i.i_class_id=13 THEN 1 ELSE NULL END) AS id13,
>   count(CASE WHEN i.i_class_id=15 THEN 1 ELSE NULL END) AS id15,
>   count(CASE WHEN i.i_class_id=2  THEN 1 ELSE NULL END) AS id2,
>   count(CASE WHEN i.i_class_id=4  THEN 1 ELSE NULL END) AS id4,
>   count(CASE WHEN i.i_class_id=6  THEN 1 ELSE NULL END) AS id6,
>   count(CASE WHEN i.i_class_id=8  THEN 1 ELSE NULL END) AS id8,
>   count(CASE WHEN i.i_class_id=10 THEN 1 ELSE NULL END) AS id10,
>   count(CASE WHEN i.i_class_id=14 THEN 1 ELSE NULL END) AS id14,
>   count(CASE WHEN i.i_class_id=16 THEN 1 ELSE NULL END) AS id16
> FROM store_sales ss
> INNER JOIN item i ON ss.ss_item_sk = i.i_item_sk
> WHERE i.i_category IN ('Books')
> AND ss.ss_customer_sk IS NOT NULL
> GROUP BY ss.ss_customer_sk
> HAVING count(ss.ss_item_sk) > 5
> {code}
> Note:
> the store_sales is a big fact table and item is a small dimension table.



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