I checked several other slave machines.
Basically the map task is waiting on this trace:

"main" prio=10 tid=0x00000000098ed000 nid=0x7beb in Object.wait()
[0x00000000413e7000]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        - waiting on <0x0000000400108530> (a
java.util.concurrent.ConcurrentHashMap)
        at
org.apache.giraph.comm.netty.NettyClient.waitSomeRequests(NettyClient.java:690)
        - locked <0x0000000400108530> (a
java.util.concurrent.ConcurrentHashMap)
        at
org.apache.giraph.comm.netty.NettyClient.waitAllRequests(NettyClient.java:666)
        at
org.apache.giraph.comm.netty.NettyWorkerClient.waitAllRequests(NettyWorkerClient.java:149)
        at
org.apache.giraph.worker.BspServiceWorker.waitForRequestsToFinish(BspServiceWorker.java:829)
        at
org.apache.giraph.worker.BspServiceWorker.finishSuperstep(BspServiceWorker.java:743)
        at
org.apache.giraph.graph.GraphTaskManager.completeSuperstepAndCollectStats(GraphTaskManager.java:387)
        at
org.apache.giraph.graph.GraphTaskManager.execute(GraphTaskManager.java:276)
        at org.apache.giraph.graph.GraphMapper.run(GraphMapper.java:92)
        at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:763)
        at org.apache.hadoop.mapred.MapTask.run(MapTask.java:369)
        at org.apache.hadoop.mapred.Child$4.run(Child.java:259)
        at java.security.AccessController.doPrivileged(Native Method)
        at javax.security.auth.Subject.doAs(Subject.java:415)
        at
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1059)
        at org.apache.hadoop.mapred.Child.main(Child.java:253)

Is it because I miss some setting?

Yingyi



On Thu, Sep 26, 2013 at 3:16 PM, Yingyi Bu <buyin...@gmail.com> wrote:

> I have 61 slave machines. Each slave machine has 16GB memory and 4 cores.
>
> I tried two configurations:
> 1.   Let mapred.map.child.java.opts to be -Xmx4g, and run the job with 4
> workers per machine on average (-w 240, try to use all the cores).
> 2.   Let mapred.map.child.java.opts to be -Xmx16g, and run the job with 1
> worker per machine on average (-w 60).
>
> I used the combiner.
> Here are the behaviors of the two configurations:
> 1. Configuration 1 fails with OutOfMemoryError--GC limit exceeds during
> superstep -1.
> 2. Configuration 2 can finish superstep -1 but hang at superstep 0 for a
> long time (more than 40 minutes).  The status of each map task is
> "startSuperstep: WORKER_ONLY - Attempt=0, Superstep=0".  I checked several
> slave machines -- the CPU is not used.  Attached is the dumped stacktrace.
> Does any one have experience with similar situations?
>
> Another question is: how can I effectively use all the cores in slave
> machines?   Does each worker do multi-threading?
> Thanks a lot!
>
> Yingyi
>
>
>
> On Thu, Sep 26, 2013 at 1:08 PM, Avery Ching <ach...@apache.org> wrote:
>
>>  Hopefully you are using combiners and also re-using objects.  This can
>> keep memory usage much lower.  Also implementing your own OutEdges can make
>> it much more efficient.
>>
>> How much memory do you have?
>>
>> Avery
>>
>>
>> On 9/26/13 12:51 PM, Yingyi Bu wrote:
>>
>> >> I think you may have added the same vertex 2x?
>> I ran the job over roughly half of the graph and saw this.  However the
>> input is not a connected components such that there might be target vertex
>> ids which do not exist.
>> When I ran the job over the entire graph,  I cannot see this but the job
>> fails with exceeding GC limit (trying out-of-core now).
>>
>>  Yingyi
>>
>>
>>
>> On Thu, Sep 26, 2013 at 12:05 PM, Avery Ching <ach...@apache.org> wrote:
>>
>>>  I think you may have added the same vertex 2x?  That being said, I
>>> don't see why the code is this way.  It should be fine.  We should file a
>>> JIRA.
>>>
>>>
>>> On 9/26/13 11:02 AM, Yingyi Bu wrote:
>>>
>>>  Thanks, Lukas!
>>>  I think the reason of this exception is that I run the job over part of
>>> the graph where some target ids do not exist.
>>>
>>>  Yingyi
>>>
>>>
>>> On Thu, Sep 26, 2013 at 1:13 AM, Lukas Nalezenec <
>>> lukas.naleze...@firma.seznam.cz> wrote:
>>>
>>>>  Hi,
>>>> Do you use partition balancing ?
>>>>  Lukas
>>>>
>>>>
>>>>
>>>> On 09/26/13 05:16, Yingyi Bu wrote:
>>>>
>>>>  Hi,
>>>>
>>>> I got this exception when I ran a Giraph-1.0.0 PageRank job over a 60 
>>>> machine cluster with 28GB input data.  But I got this exception:
>>>>
>>>> java.lang.IllegalStateException: run: Caught an unrecoverable exception 
>>>> resolveMutations: Already has missing vertex on this worker for 20464109
>>>>    at org.apache.giraph.graph.GraphMapper.run(GraphMapper.java:102)
>>>>    at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:763)
>>>>    at org.apache.hadoop.mapred.MapTask.run(MapTask.java:369)
>>>>    at org.apache.hadoop.mapred.Child$4.run(Child.java:259)
>>>>    at java.security.AccessController.doPrivileged(Native Method)
>>>>    at javax.security.auth.Subject.doAs(Subject.java:415)
>>>>    at 
>>>> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1059)
>>>>    at org.apache.hadoop.mapred.Child.main(Child.java:253)
>>>> Caused by: java.lang.IllegalStateException: resolveMutations: Already has 
>>>> missing vertex on this worker for 20464109
>>>>    at 
>>>> org.apache.giraph.comm.netty.NettyWorkerServer.resolveMutations(NettyWorkerServer.java:184)
>>>>    at 
>>>> org.apache.giraph.comm.netty.NettyWorkerServer.prepareSuperstep(NettyWorkerServer.java:152)
>>>>    at 
>>>> org.apache.giraph.worker.BspServiceWorker.startSuperstep(BspServiceWorker.java:677)
>>>>    at 
>>>> org.apache.giraph.graph.GraphTaskManager.execute(GraphTaskManager.java:249)
>>>>    at org.apache.giraph.graph.GraphMapper.run(GraphMapper.java:92)
>>>>    ... 7 more
>>>>
>>>>
>>>>
>>>> Does anyone know what is the possible cause of this exception?
>>>>
>>>> Thanks!
>>>>
>>>>
>>>> Yingyi
>>>>
>>>>
>>>>
>>>
>>>
>>
>>
>

Reply via email to