You might also want to check the zookeeper memory options.
Some of our production jobs use parameters such as
-Xmx5g -XX:ParallelGCThreads=4 -XX:+UseConcMarkSweepGC
-XX:CMSInitiatingOccupancyFraction=70 -XX:MaxGCPauseMillis=100
Since the master doesn't use much memory letting zk have more is reasonable.
On 5/27/14, 9:25 AM, Praveen kumar s.k wrote:
Hi All,
I am getting several errors consistently while processing large graph.
The code works when the size of the graph is in terms of GB's.
we have implemented compression and removing the dead end nodes in de
Bruijn graph
My cluster settings are
Cores Workers RAM/Core Graphsize AggregateRAM
252 250 10.5 GB 2.3 TB 2.6 TB
Below are the type of errors I am getting.
1. I believe that this error occurred because of zookeeper session
expired. To address this I changed the parameter minSessionTimeout in
configuration to large value. However some workers still throw this
error.
2014-05-27 00:19:55,187 FATAL org.apache.giraph.graph.GraphMapper:
uncaughtException: OverrideExceptionHandler on thread
org.apache.giraph.master.MasterThread, msg = java.lang.Il$
java.lang.IllegalStateException: java.lang.IllegalStateException:
Failed to create job state path due to KeeperException
at org.apache.giraph.master.MasterThread.run(MasterThread.java:185)
Caused by: java.lang.IllegalStateException: Failed to create job state
path due to KeeperException
at org.apache.giraph.bsp.BspService.getJobState(BspService.java:679)
at
org.apache.giraph.master.BspServiceMaster.becomeMaster(BspServiceMaster.java:843)
at org.apache.giraph.master.MasterThread.run(MasterThread.java:98)
Caused by: org.apache.zookeeper.KeeperException$SessionExpiredException:
KeeperErrorCode = Session expired for
/_hadoopBsp/job_201405262302_0003/_masterJobState
at
org.apache.zookeeper.KeeperException.create(KeeperException.java:118)
at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
at org.apache.zookeeper.ZooKeeper.create(ZooKeeper.java:637)
at org.apache.giraph.zk.ZooKeeperExt.createExt(ZooKeeperExt.java:152)
at org.apache.giraph.bsp.BspService.getJobState(BspService.java:670)
... 2 more
2. I dont know why this below error is thrown. My guess is that,
master worker is failing for some reason
2014-05-27 00:19:55,184 ERROR org.apache.giraph.master.MasterThread:
masterThread: Master algorithm failed with IllegalStateException
java.lang.IllegalStateException: Failed to create job state path due
to KeeperException
at org.apache.giraph.bsp.BspService.getJobState(BspService.java:679)
at
org.apache.giraph.master.BspServiceMaster.becomeMaster(BspServiceMaster.java:843)
at org.apache.giraph.master.MasterThread.run(MasterThread.java:98)
Caused by: org.apache.zookeeper.KeeperException$SessionExpiredException:
KeeperErrorCode = Session expired for
/_hadoopBsp/job_201405262302_0003/_masterJobState
at
org.apache.zookeeper.KeeperException.create(KeeperException.java:118)
at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
at org.apache.zookeeper.ZooKeeper.create(ZooKeeper.java:637)
at org.apache.giraph.zk.ZooKeeperExt.createExt(ZooKeeperExt.java:152)
at org.apache.giraph.bsp.BspService.getJobState(BspService.java:670)
... 2 more
3. Below is one more type of error
java.lang.IllegalStateException: Failed to create job state path due
to KeeperException
at org.apache.giraph.bsp.BspService.getJobState(BspService.java:679)
at
org.apache.giraph.master.BspServiceMaster.becomeMaster(BspServiceMaster.java:843)
at org.apache.giraph.master.MasterThread.run(MasterThread.java:98)
Caused by: org.apache.zookeeper.KeeperException$SessionExpiredException:
KeeperErrorCode = Session expired for
/_hadoopBsp/job_201405261249_0008/_masterJobState
at
org.apache.zookeeper.KeeperException.create(KeeperException.java:118)
at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
at org.apache.zookeeper.ZooKeeper.create(ZooKeeper.java:637)
at org.apache.giraph.zk.ZooKeeperExt.createExt(ZooKeeperExt.java:152)
at org.apache.giraph.bsp.BspService.getJobState(BspService.java:670)
... 2 more
2014-05-26 18:19:54,269 FATAL org.apache.giraph.graph.GraphMapper:
uncaughtException: OverrideExceptionHandler on thread
org.apache.giraph.master.MasterThread, msg = java.lang.Il$
java.lang.IllegalStateException: java.lang.IllegalStateException:
Failed to create job state path due to KeeperException
at org.apache.giraph.master.MasterThread.run(MasterThread.java:185)
Caused by: java.lang.IllegalStateException: Failed to create job state
path due to KeeperException
at org.apache.giraph.bsp.BspService.getJobState(BspService.java:679)
at
org.apache.giraph.master.BspServiceMaster.becomeMaster(BspServiceMaster.java:843)
at org.apache.giraph.master.MasterThread.run(MasterThread.java:98)
Caused by: org.apache.zookeeper.KeeperException$SessionExpiredException:
KeeperErrorCode = Session expired for
/_hadoopBsp/job_201405261249_0008/_masterJobState
at
org.apache.zookeeper.KeeperException.create(KeeperException.java:118)
at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
at org.apache.zookeeper.ZooKeeper.create(ZooKeeper.java:637)
at org.apache.giraph.zk.ZooKeeperExt.createExt(ZooKeeperExt.java:152)
at org.apache.giraph.bsp.BspService.getJobState(BspService.java:670)
4. Sometimes I get GC overhead limit exceed error. I have no clue to
address this
Caused by: java.util.concurrent.ExecutionException:
java.lang.OutOfMemoryError: GC overhead limit exceeded
at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:262)
at java.util.concurrent.FutureTask.get(FutureTask.java:119)
at
org.apache.giraph.utils.ProgressableUtils$FutureWaitable.waitFor(ProgressableUtils.java:300)
at org.apache.giraph.utils.ProgressableUtils.waitFor(ProgressableUtils.java:173)
... 16 more
Caused by: java.lang.OutOfMemoryError: GC overhead limit exceeded
at java.nio.ByteBuffer.allocate(ByteBuffer.java:329)
at java.nio.charset.CharsetEncoder.encode(CharsetEncoder.java:777)
at org.apache.hadoop.io.Text.encode(Text.java:388)
at org.apache.hadoop.io.Text.set(Text.java:178)
at org.apache.hadoop.io.Text.<init>(Text.java:81)
at
contrail.GraphTextInputFormat$LongDoubleDoubleDoubleVertexReader.getCurrentVertex(GraphTextInputFormat.java:70)
at
org.apache.giraph.io.internal.WrappedVertexReader.getCurrentVertex(WrappedVertexReader.java:89)
at
org.apache.giraph.worker.VertexInputSplitsCallable.readInputSplit(VertexInputSplitsCallable.java:148)
at
org.apache.giraph.worker.InputSplitsCallable.loadInputSplit(InputSplitsCallable.java:267)
at
org.apache.giraph.worker.InputSplitsCallable.call(InputSplitsCallable.java:211)
at
org.apache.giraph.worker.InputSplitsCallable.call(InputSplitsCallable.java:60)
at
org.apache.giraph.utils.LogStacktraceCallable.call(LogStacktraceCallable.java:51)
at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
at java.util.concurrent.FutureTask.run(FutureTask.java:166)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
at java.lang.Thread.run(Thread.java:679)
5. Some time some of the workers complete successfully and few of the
workers fail because of this entire job fails.
any help would be greatly appreciated.
Thanks in Advance,
Praveenkumar