spark-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Yin Huai (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (SPARK-10733) TungstenAggregation cannot acquire page after switching to sort-based
Date Wed, 23 Sep 2015 19:24:04 GMT

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

Yin Huai edited comment on SPARK-10733 at 9/23/15 7:23 PM:
-----------------------------------------------------------

[~jameszhouyi] Another two places for logging are {{UnsafeExternalSorter.acquireNewPage}}
and {{ShuffleMemoryManager.tryToAcquire}}. In {{UnsafeExternalSorter.acquireNewPage}}, we
log an entry to say we are trying to acquire some memory space. In {{ShuffleMemoryManager.tryToAcquire}},
we log the size of memory that we want to acquire, the size of memory that has already acquired
for this task ({{curMem}}), {{maxToGrant}}, {{maxMemory}}, {{maxMemory / (2 * numActiveTasks)}},
and {{maxMemory / numActiveTasks}}. These information will be very helpful for debugging.


was (Author: yhuai):
[~jameszhouyi] Another two places for logging are {{UnsafeExternalSorter.acquireNewPage}}
and {{ShuffleMemoryManager.tryToAcquire}}. In {{UnsafeExternalSorter.acquireNewPage}}, we
log an entry to say we are trying to acquire some memory space. In {{ShuffleMemoryManager.tryToAcquire}},
we log the size of memory that we want to acquire, the size of memory that has already acquired
for this task ({{curMem}}), {{maxToGrant}}, {{maxMemory / (2 * numActiveTasks)}}, and {{maxMemory
/ numActiveTasks}}. These information will be very helpful for debugging.

> TungstenAggregation cannot acquire page after switching to sort-based
> ---------------------------------------------------------------------
>
>                 Key: SPARK-10733
>                 URL: https://issues.apache.org/jira/browse/SPARK-10733
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 1.5.0
>            Reporter: Andrew Or
>            Assignee: Andrew Or
>            Priority: Blocker
>
> This is uncovered after fixing SPARK-10474. Stack trace:
> {code}
> 15/09/21 12:51:46 WARN scheduler.TaskSetManager: Lost task 115.0 in stage 152.0 (TID
1736, bb-node2): java.io.IOException: Unable to acquire 16777216 bytes of memory
> at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPage(UnsafeExternalSorter.java:378)
> at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPageIfNecessary(UnsafeExternalSorter.java:359)
> at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.insertKVRecord(UnsafeExternalSorter.java:488)
> at org.apache.spark.sql.execution.UnsafeKVExternalSorter.insertKV(UnsafeKVExternalSorter.java:144)
> at org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.switchToSortBasedAggregation(TungstenAggregationIterator.scala:465)
> 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}



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

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


Mime
View raw message