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

Cristian Opris commented on SPARK-3633:
---------------------------------------

This looks like a memory leak in ConnectionManager where responses 
(BufferMessage) are retained by the TimerTask waiting for ACK even after the 
Future completes with Success, please see the 

Possibly related to 
https://github.com/apache/spark/commit/76fa0eaf515fd6771cdd69422b1259485debcae5 

+------------------------------------------------------+--------------+----------------------+-------------------------+
|                        Class                         |   Objects    |     
Shallow Size     |      Retained Size      |
+------------------------------------------------------+--------------+----------------------+-------------------------+
|  java.util.TaskQueue                                 |    1    0 %  |         
  24    0 %  |     885,048,168  100 %  |
|  java.util.TimerTask[]                               |    1    0 %  |        
2,064    0 %  |     885,048,144   99 %  |
|  org.apache.spark.network.ConnectionManager$$anon$5  |  286    5 %  |       
13,728    0 %  |  ~  885,046,080   99 %  |
|  org.apache.spark.network.BufferMessage              |  572   10 %  |       
36,608    0 %  |  ~  885,018,624   99 %  |
|  scala.concurrent.impl.Promise$DefaultPromise        |  286    5 %  |        
4,576    0 %  |  ~  884,968,288   99 %  |
|  scala.util.Success                                  |  286    5 %  |        
4,576    0 %  |  ~  884,963,712   99 %  |
|  scala.collection.mutable.ArrayBuffer                |  572   10 %  |       
13,728    0 %  |  ~  884,915,768   99 %  |
|  java.lang.Object[]                                  |  572   10 %  |       
45,760    0 %  |  ~  884,902,040   99 %  |
|  java.nio.HeapByteBuffer                             |  286    5 %  |       
13,728    0 %  |  ~  884,856,280   99 %  |
|  byte[]                                              |  286    5 %  |  
884,842,552   99 %  |  ~  884,842,552   99 %  |
|  java.net.InetSocketAddress                          |  572   10 %  |        
9,152    0 %  |       ~  66,248    0 %  |
|  java.net.InetSocketAddress$InetSocketAddressHolder  |  572   10 %  |       
13,728    0 %  |       ~  57,096    0 %  |
|  java.net.Inet4Address                               |  286    5 %  |        
6,864    0 %  |       ~  43,368    0 %  |
|  java.net.InetAddress$InetAddressHolder              |  286    5 %  |        
6,864    0 %  |       ~  36,504    0 %  |
|  java.lang.String                                    |  285    5 %  |        
6,840    0 %  |       ~  29,640    0 %  |
|  char[]                                              |  285    5 %  |       
22,800    0 %  |       ~  22,800    0 %  |
|  java.lang.Object                                    |  286    5 %  |        
4,576    0 %  |        ~  4,576    0 %  |
+------------------------------------------------------+--------------+----------------------+-------------------------+

Generated by YourKit Java Profiler 2014 build 14110 12-Nov-2014 17:44:32


> Fetches failure observed after SPARK-2711
> -----------------------------------------
>
>                 Key: SPARK-3633
>                 URL: https://issues.apache.org/jira/browse/SPARK-3633
>             Project: Spark
>          Issue Type: Bug
>          Components: Block Manager
>    Affects Versions: 1.1.0
>            Reporter: Nishkam Ravi
>            Priority: Critical
>
> Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. 
> Recently upgraded to Spark 1.1. The workload fails with the following error 
> message(s):
> {code}
> 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, 
> c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, 
> c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120)
> 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages
> {code}
> In order to identify the problem, I carried out change set analysis. As I go 
> back in time, the error message changes to:
> {code}
> 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, 
> c1706.halxg.cloudera.com): java.io.FileNotFoundException: 
> /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034
>  (Too many open files)
>         java.io.FileOutputStream.open(Native Method)
>         java.io.FileOutputStream.<init>(FileOutputStream.java:221)
>         
> org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117)
>         
> org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185)
>         
> org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197)
>         
> org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145)
>         org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58)
>         
> org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51)
>         
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68)
>         
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
>         org.apache.spark.scheduler.Task.run(Task.scala:54)
>         org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199)
>         
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>         
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>         java.lang.Thread.run(Thread.java:745)
> {code}
> All the way until Aug 4th. Turns out the problem changeset is 4fde28c. 



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