zookeeper connection issue while running for second time

2013-09-26 Thread Jyotirmoy Sundi
Hi ,

   I got the connected component working for 1B nodes, but when I run
the job again, it fails with the below error. Aprt form this in
zookeeper the data is not cleared in the data directory. For
successful jobs the data in zookeper from giraph is cleared.

The following errors seems to be coming because the node tries to
connect to the zookeeper with a session id which is cleared as seens
in

"Client session timed out, have not heard from server in 68845ms for
sessionid 0x3415cc6ce930059, closing socket connection and attempting
reconnect" , Any idea if increasing the session time out will be good
?

2013-09-27 00:57:11,748 WARN org.apache.giraph.bsp.BspService:
process: Got unknown null path event WatchedEvent state:Expired
type:None path:null
2013-09-27 00:57:11,748 INFO org.apache.zookeeper.ClientCnxn: Unable
to reconnect to ZooKeeper service, session 0x3415cc6ce930059 has
expired, closing socket connection
2013-09-27 00:57:11,748 WARN
org.apache.giraph.worker.InputSplitsHandler: process: Problem with
zookeeper, got event with path null, state Expired, event type None
2013-09-27 00:57:11,748 INFO org.apache.zookeeper.ClientCnxn:
EventThread shut down
2013-09-27 00:57:11,925 INFO
org.apache.giraph.worker.InputSplitsCallable: loadFromInputSplit:
Finished loading
/_hadoopBsp/job_201309260044_0116/_vertexInputSplitDir/89 (v=258127,
e=1792906)
2013-09-27 00:57:11,926 ERROR
org.apache.giraph.utils.LogStacktraceCallable: Execution of callable
failed
java.lang.IllegalStateException: markInputSplitPathFinished:
KeeperException on
/_hadoopBsp/job_201309260044_0116/_vertexInputSplitDir/89/_vertexInputSplitFinished
at 
org.apache.giraph.worker.InputSplitsHandler.markInputSplitPathFinished(InputSplitsHandler.java:168)
at 
org.apache.giraph.worker.InputSplitsCallable.loadInputSplit(InputSplitsCallable.java:226)
at 
org.apache.giraph.worker.InputSplitsCallable.call(InputSplitsCallable.java:161)
at 
org.apache.giraph.worker.InputSplitsCallable.call(InputSplitsCallable.java:58)
at 
org.apache.giraph.utils.LogStacktraceCallable.call(LogStacktraceCallable.java:51)
at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
at java.util.concurrent.FutureTask.run(FutureTask.java:138)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:895)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918)
at java.lang.Thread.run(Thread.java:662)
Caused by: org.apache.zookeeper.KeeperException$SessionExpiredException:
KeeperErrorCode = Session expired for
/_hadoopBsp/job_201309260044_0116/_vertexInputSplitDir/89/_vertexInputSplitFinished
at org.apache.zookeeper.KeeperException.create(KeeperException.java:127)
at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
at org.apache.zookeeper.ZooKeeper.create(ZooKeeper.java:783)
at org.apache.giraph.zk.ZooKeeperExt.createExt(ZooKeeperExt.java:152)
at 
org.apache.giraph.worker.InputSplitsHandler.markInputSplitPathFinished(InputSplitsHandler.java:159)
... 9 more


-- 
Best Regards,
Jyotirmoy Sundi
Data Engineer,
Admobius

San Francisco, CA 94158


Re: Exception "Already has missing vertex on this worker"

2013-09-26 Thread Yingyi Bu
The job eventually fails because of out-of-hearbeat from tasks:

Task attempt_201309262131_0001_m_58_0 failed to report status for
602 seconds. Killing!

Any suggestions?

Yingyi



On Thu, Sep 26, 2013 at 3:27 PM, Yingyi Bu  wrote:

> I checked several other slave machines.
> Basically the map task is waiting on this trace:
>
> "main" prio=10 tid=0x098ed000 nid=0x7beb in Object.wait()
> [0x413e7000]
>java.lang.Thread.State: TIMED_WAITING (on object monitor)
> at java.lang.Object.wait(Native Method)
> - waiting on <0x000400108530> (a
> java.util.concurrent.ConcurrentHashMap)
> at
> org.apache.giraph.comm.netty.NettyClient.waitSomeRequests(NettyClient.java:690)
> - locked <0x000400108530> (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  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  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  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.gr

Re: Exception "Already has missing vertex on this worker"

2013-09-26 Thread Yingyi Bu
I checked several other slave machines.
Basically the map task is waiting on this trace:

"main" prio=10 tid=0x098ed000 nid=0x7beb in Object.wait()
[0x413e7000]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)
at java.lang.Object.wait(Native Method)
- waiting on <0x000400108530> (a
java.util.concurrent.ConcurrentHashMap)
at
org.apache.giraph.comm.netty.NettyClient.waitSomeRequests(NettyClient.java:690)
- locked <0x000400108530> (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  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  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  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.apa

IOException

2013-09-26 Thread Jyotirmoy Sundi
Hi,

   Below is the error trace, any idea to resolve it ?

2013-09-26 20:48:43,558 ERROR
org.apache.giraph.utils.LogStacktraceCallable: Execution of callable
failed
java.lang.IllegalStateException: call: IOException
at 
org.apache.giraph.worker.InputSplitsCallable.call(InputSplitsCallable.java:172)
at 
org.apache.giraph.worker.InputSplitsCallable.call(InputSplitsCallable.java:58)
at 
org.apache.giraph.utils.LogStacktraceCallable.call(LogStacktraceCallable.java:51)
at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
at java.util.concurrent.FutureTask.run(FutureTask.java:138)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:895)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918)
at java.lang.Thread.run(Thread.java:662)
Caused by: java.io.IOException: Filesystem closed
at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:565)
at 
org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(DFSInputStream.java:736)
at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:794)
at java.io.DataInputStream.read(DataInputStream.java:83)
at 
org.apache.hadoop.util.LineReader.readDefaultLine(LineReader.java:209)
at org.apache.hadoop.util.LineReader.readLine(LineReader.java:173)
at 
org.apache.hadoop.mapreduce.lib.input.LineRecordReader.nextKeyValue(LineRecordReader.java:147)
at 
org.apache.giraph.io.formats.TextVertexInputFormat$TextVertexReaderFromEachLineProcessedHandlingExceptions.nextVertex(TextVertexInputFormat.java:304)
at 
org.apache.giraph.worker.VertexInputSplitsCallable.readInputSplit(VertexInputSplitsCallable.java:124)
at 
org.apache.giraph.worker.InputSplitsCallable.loadInputSplit(InputSplitsCallable.java:220)
at 
org.apache.giraph.worker.InputSplitsCallable.call(InputSplitsCallable.java:161)
... 7 more
2013-09-26 20:48:43,580 ERROR
org.apache.giraph.worker.BspServiceWorker: unregisterHealth: Got
failure, unregistering health on
/_hadoopBsp/job_201309260044_0086/_applicationAttemptsDir/0/_superstepDir/-1/_workerHealthyDir/had13.rsk.admobius.com_3
on superstep -1

--

-- 
Best Regards,
Jyotirmoy Sundi
Data Engineer,
Admobius

San Francisco, CA 94158


Re: Exception "Already has missing vertex on this worker"

2013-09-26 Thread Avery Ching
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 > 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
mailto: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










Re: Exception "Already has missing vertex on this worker"

2013-09-26 Thread Yingyi Bu
>> 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  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
>>
>>
>>
>
>


Re: Exception "Already has missing vertex on this worker"

2013-09-26 Thread Avery Ching
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 
> 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







Re: Exception "Already has missing vertex on this worker"

2013-09-26 Thread Yingyi Bu
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
>
>
>


Giraph not loading all vertices

2013-09-26 Thread Jyotirmoy Sundi
Hi,
I am trying to do a the connected component example. I have modified
it, the Writables a bit to fit my example. Some of the vertex indexes are
negative (long hashed of strings), I tried with a sample example with
around 10 vertices and it works properly. But when I load the whole set
around 1.5B vertices , all of them do not get loaded and the final output
misses a lot of the pairs. Any idea what might be the reason ? is it
required to have the vertex indexes as positive only, I tried with -ve with
a sample set and it works, any other place I should look into or put debug
statements ?

I am running with 118 workers.
Cluster Config :
Running Map TasksRunning Reduce TasksTotal SubmissionsNodesOccupied Map
SlotsOccupied Reduce SlotsReserved Map SlotsReserved Reduce SlotsMap Task
CapacityReduce Task CapacityAvg. Tasks/NodeBlacklisted NodesExcluded Nodes 0
0387 16884
36.000 0

-- 
Best Regards,
Jyotirmoy Sundi
Data Engineer,
Admobius

San Francisco, CA 94158


Re: Exception "Already has missing vertex on this worker"

2013-09-26 Thread Lukas Nalezenec

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