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

Maximilian Michels commented on FLINK-2235:
-------------------------------------------

I managed to get my hands on some logs:

{noformat}
15:21:43,802 INFO  org.apache.flink.api.java.ExecutionEnvironment               
 - The job has 0 registered types and 0 default Kryo serializers
15:21:44,605 INFO  akka.event.slf4j.Slf4jLogger                                 
 - Slf4jLogger started
15:21:44,648 INFO  org.apache.flink.runtime.blob.BlobServer                     
 - Created BLOB server storage directory 
/var/folders/v7/kbtg426s74x7d45n8t6d9mndcy9g9w/T/blobStore-1c50a0b6-d71d-4f87-a849-c5b4cd88fb97
15:21:44,658 INFO  org.apache.flink.runtime.blob.BlobServer                     
 - Started BLOB server at 0.0.0.0:51747 - max concurrent requests: 50 - max 
backlog: 1000
15:21:44,703 INFO  org.apache.flink.runtime.jobmanager.JobManager               
 - Starting JobManager at akka://flink/user/jobmanager#-557612298.
15:21:44,714 INFO  org.apache.flink.runtime.taskmanager.TaskManager             
 - Messages between TaskManager and JobManager have a max timeout of 100000 
milliseconds
15:21:44,725 INFO  org.apache.flink.runtime.taskmanager.TaskManager             
 - Temporary file directory '/var/folders/v7/kbtg426s74x7d45n8t6d9mndcy9g9w/T': 
total 232 GB, usable 101 GB (43.53% usable)
15:21:44,824 INFO  org.apache.flink.runtime.io.network.buffer.NetworkBufferPool 
 - Allocated 64 MB for network buffer pool (number of memory segments: 2048, 
bytes per segment: 32768).
15:21:44,826 INFO  org.apache.flink.runtime.taskmanager.TaskManager             
 - Using 17592186044414 MB for Flink managed memory.
Exception in thread "main" java.lang.IllegalArgumentException: Size of total 
memory must be positive.
    at 
org.apache.flink.runtime.memorymanager.DefaultMemoryManager.<init>(DefaultMemoryManager.java:110)
    at 
org.apache.flink.runtime.taskmanager.TaskManager$.startTaskManagerComponentsAndActor(TaskManager.scala:1423)
    at 
org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster.startTaskManager(LocalFlinkMiniCluster.scala:121)
    at 
org.apache.flink.runtime.minicluster.FlinkMiniCluster$$anonfun$1.apply(FlinkMiniCluster.scala:86)
    at 
org.apache.flink.runtime.minicluster.FlinkMiniCluster$$anonfun$1.apply(FlinkMiniCluster.scala:79)
    at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
    at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
    at scala.collection.immutable.Range.foreach(Range.scala:141)
    at scala.collection.TraversableLike$class.map(TraversableLike.scala:244)
    at scala.collection.AbstractTraversable.map(Traversable.scala:105)
    at 
org.apache.flink.runtime.minicluster.FlinkMiniCluster.<init>(FlinkMiniCluster.scala:79)
    at 
org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster.<init>(LocalFlinkMiniCluster.scala:45)
    at 
org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster.<init>(LocalFlinkMiniCluster.scala:52)
    at org.apache.flink.client.LocalExecutor.start(LocalExecutor.java:112)
    at org.apache.flink.client.LocalExecutor.executePlan(LocalExecutor.java:165)
    at 
org.apache.flink.api.java.LocalEnvironment.execute(LocalEnvironment.java:54)
    at 
org.apache.flink.api.java.ExecutionEnvironment.execute(ExecutionEnvironment.java:789)
    at org.apache.flink.api.java.DataSet.collect(DataSet.java:408)
    at org.apache.flink.api.java.DataSet.print(DataSet.java:1346)
    at org.apache.flink.quickstart.WordCount.main(WordCount.java:67)
{noformat}

> Local Flink cluster allocates too much memory
> ---------------------------------------------
>
>                 Key: FLINK-2235
>                 URL: https://issues.apache.org/jira/browse/FLINK-2235
>             Project: Flink
>          Issue Type: Bug
>          Components: Local Runtime, TaskManager
>    Affects Versions: 0.9
>         Environment: Oracle JDK: 1.6.0_65-b14-462
> Eclipse
>            Reporter: Maximilian Michels
>            Priority: Minor
>
> When executing a Flink job locally, the task manager gets initialized with an 
> insane amount of memory. After a quick look in the code it seems that the 
> call to {{EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag()}} 
> returns a wrong estimate of the heap memory size.
> Moreover, the same user switched to Oracle JDK 1.8 and that made the error 
> disappear. So I'm guessing this is some Java 1.6 quirk.



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

Reply via email to