giraph-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Lukas Nalezenec <lukas.naleze...@firma.seznam.cz>
Subject Re: Zookeeper Problems when running Giraph (PageRankVertex) on large graphs.
Date Wed, 24 Sep 2014 12:50:17 GMT
Hi,
I had similar problems. Files that Giraph wrote to zookeeper were over 
limit so zookeeper crushed. In my case i had too many partitions with 
too long input and checkpoint paths.
You can try to get logs from the Zookeeper process.

Lukas

On 24.9.2014 14:31, Fontana, Peter C. wrote:
> Hello,
>
> I am trying to run the PageRankVertex code on a large graph. I 
> successfully got it to run on smaller examples, but when I try to run 
> it on a large example (100M nodes, 10B edges, 300GB space), it does 
> not finish. I get the following error.
>
> java.lang.IllegalStateException: run: Caught an unrecoverable 
> exception waitFor: ExecutionException occurred while waiting for 
> org.apache.giraph.utils.ProgressableUtils$FutureWaitable@2ffecaeb
> at org.apache.giraph.graph.GraphMapper.run(GraphMapper.java:102)
> at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:672)
> at org.apache.hadoop.mapred.MapTask.run(MapTask.java:330)
> at org.apache.hadoop.mapred.Child$4.run(Child.java:268)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:396)
> at 
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1408)
> at org.apache.hadoop.mapred.Child.main(Child.java:262)
> Caused by: java.lang.IllegalStateException: waitFor: 
> ExecutionException occurred while waiting for 
> org.apache.giraph.utils.ProgressableUtils$FutureWaitable@2ffecaeb
> at 
> org.apache.giraph.utils.ProgressableUtils.waitFor(ProgressableUtils.java:151)
> at org.apache.giraph.utils.ProgressableUtils.waitForever(ProgressableU
> -------
> java.lang.Throwable: Child Error
> at org.apache.hadoop.mapred.TaskRunner.run(TaskRunner.java:250)
> Caused by: java.io.IOException: Task process exit with nonzero status 
> of 65.
> at org.apache.hadoop.mapred.TaskRunner.run(TaskRunner.java:237)
>
>
> From looking at the logs, I think the following line is the cause of 
> the error:
>
> date hr:19:56,576 INFO org.apache.giraph.utils.ProgressableUtils: 
> waitFor: Waiting for 
> org.apache.giraph.utils.ProgressableUtils$FutureWaitable@2ffecaeb
> date hr:19:56,577 INFO org.apache.zookeeper.ClientCnxn: Client session 
> timed out, have not heard from server in 52735ms for sessionid 
> 0x24889e30b710063, closing socket connection and attempting reconnect
>
> After the signature is the syntax of the command and the full error 
> log of the failing nodes (with machine names, pathnames, and ip 
> addresses replaced with generic names).
>
> I looked at the following thread on the giraph mailing list: 
> http://mail-archives.apache.org/mod_mbox/giraph-user/201310.mbox/%3cCAEv8GwXmg9YPTYoR6QAtwqqcWAgT8PbMaqFjKz=PN1+w51Mc4w@mail.gmail.com%3e

> but that change did not solve the problem.
>
> I have tried using an out of core graph, but that did not solve the 
> problem. I have also tried enabling checkpoints with 
> -Dgiraph.checkpointFrequency=1 but that does not solve the problem. I 
> get similar errors. I have also both increased and decreased the 
> number of workers, but that did not solve the problem.
>
> Does anybody have any thoughts? Is it a memory issue or is it 
> something else? I am using Giraph 1.0.0 built from the 1.0.0 branch of 
> the github repository. PageRankVertex (and all the other classes) are 
> example classes that are bundled with the graph source 
> (giraph-examples), so I am using pre-built code rather than compiling 
> my own Giraph code. I get a similar error using PageRankBenchmark.
>
> Thank you for your time.
>
> Best Wishes,
> Peter
>
>
> Command:
> /usr/local/giraph$ hadoop jar giraph-examples.jar 
> org.apache.giraph.GiraphRunner 
>  -Dgiraph.zkList=node1.loc:port,node2.loc:port,node3.loc:port 
> -Dmapred.child.java-opts="-Xmx64g -Xms64g XX:+UseConcMarkSweepGC 
> -XX:-UseGCOverheadLimit" -Dgiraph.zkJavaOpts="-Xmx64g 
> -XX:ParallelGCThreads=4 -XX:+UseConcMarkSweepGC 
> -XX:CMSInitiatingOccupancyFraction=70 -XX:MaxGCPauseMillis=100 
> -XX:-UseGCOverheadLimit" -Dgiraph.useSuperstepCounters=false 
>  -Dgiraph.zkSessionMsecTimeout=6000000 
>  -Dgiraph.useInputSplitLocality=false  -Dgiraph.zkServerCount=1 
> org.apache.giraph.examples.PageRankVertex -vif 
> org.apache.giraph.examples.LongDoubleNullTextInputFormat -ca 
> org.apache.giraph.examples.RandomWalkVertex.teleportationProbability=0.10f 
> -ca org.apache.giraph.examples.RandomWalkVertex.maxSupersteps=2 -vip 
> inputDir -of 
> org.apache.giraph.examples.VertexWithDoubleValueNullEdgeTextOutputFormat 
> -op outputDir -mc 
> org.apache.giraph.examples.RandomWalkVertexMasterCompute -wc 
> org.apache.giraph.examples.RandomWalkWorkerContext -w 23
>
> Log of Failed Node:
>
> date hr:17:59,966 WARN mapreduce.Counters: Group 
> org.apache.hadoop.mapred.Task$Counter is deprecated. Use 
> org.apache.hadoop.mapreduce.TaskCounter instead
> date hr:18:00,462 WARN org.apache.hadoop.conf.Configuration: 
> session.id is deprecated. Instead, use dfs.metrics.session-id
> date hr:18:00,463 INFO org.apache.hadoop.metrics.jvm.JvmMetrics: 
> Initializing JVM Metrics with processName=MAP, sessionId=
> date hr:18:00,546 WARN org.apache.hadoop.conf.Configuration: 
> slave.host.name is deprecated. Instead, use dfs.datanode.hostname
> date hr:18:01,265 INFO org.apache.hadoop.util.ProcessTree: setsid 
> exited with exit code 0
> date hr:18:01,274 INFO org.apache.hadoop.mapred.Task:  Using 
> ResourceCalculatorPlugin : 
> org.apache.hadoop.util.LinuxResourceCalculatorPlugin@1532021a
> date hr:18:01,600 INFO org.apache.hadoop.mapred.MapTask: Processing 
> split: 'org.apache.giraph.bsp.BspInputSplit, index=-1, num=-1
> date hr:18:01,613 INFO org.apache.giraph.graph.GraphTaskManager: 
> setup: Log level remains at info
> date hr:18:01,659 INFO org.apache.giraph.graph.GraphTaskManager: 
> Distributed cache is empty. Assuming fatjar.
> date hr:18:01,660 INFO org.apache.giraph.graph.GraphTaskManager: 
> setup: classpath @ 
> /data6/hadoop/mapred/local/taskTracker/peter/jobcache/job_num/jars/job.jar 
> for job Giraph: org.apache.giraph.examples.PageRankVertex
> date hr:18:01,660 INFO org.apache.giraph.graph.GraphTaskManager: 
> setup: Starting up BspServiceWorker...
> date hr:18:01,673 INFO org.apache.giraph.bsp.BspService: BspService: 
> Connecting to ZooKeeper with job job_num, 12 on 
> node1.loc:port,node2.loc:port,node3.loc:port
> date hr:18:01,682 INFO org.apache.zookeeper.ZooKeeper: Client 
> environment:zookeeper.version=3.4.5-cdh4.5.0--1, built on 11/20/2013 
> 22:27 GMT
> date hr:18:01,682 INFO org.apache.zookeeper.ZooKeeper: Client 
> environment:host.name=host.loc
> date hr:18:01,682 INFO org.apache.zookeeper.ZooKeeper: Client 
> environment:java.version=1.6.0_37
> date hr:18:01,682 INFO org.apache.zookeeper.ZooKeeper: Client 
> environment:java.vendor=Sun Microsystems Inc.
> date hr:18:01,682 INFO org.apache.zookeeper.ZooKeeper: Client 
> environment:java.home=/usr/java/jdk1.6.0_37/jre
> date hr:18:01,682 INFO org.apache.zookeeper.ZooKeeper: Client 
> environment:java.class.path=<<classpath>>
> date hr:18:01,682 INFO org.apache.zookeeper.ZooKeeper: Client 
> environment:java.library.path=<<libpath>>
> date hr:18:01,682 INFO org.apache.zookeeper.ZooKeeper: Client 
> environment:java.io.tmpdir=<<tmpdir>>
> date hr:18:01,682 INFO org.apache.zookeeper.ZooKeeper: Client 
> environment:java.compiler=<NA>
> date hr:18:01,682 INFO org.apache.zookeeper.ZooKeeper: Client 
> environment:os.name=Linux
> date hr:18:01,682 INFO org.apache.zookeeper.ZooKeeper: Client 
> environment:os.arch=amd64
> date hr:18:01,682 INFO org.apache.zookeeper.ZooKeeper: Client 
> environment:os.version=2.6.32-431.11.2.el6.x86_64
> date hr:18:01,682 INFO org.apache.zookeeper.ZooKeeper: Client 
> environment:user.name=mapred
> date hr:18:01,682 INFO org.apache.zookeeper.ZooKeeper: Client 
> environment:user.home=/usr/lib/hadoop-0.20-mapreduce
> date hr:18:01,682 INFO org.apache.zookeeper.ZooKeeper: Client 
> environment:user.dir=<<userdir>>
> date hr:18:01,683 INFO org.apache.zookeeper.ZooKeeper: Initiating 
> client connection, 
> connectString=node1.loc:port,node2.loc:port,node3.loc:port 
> sessionTimeout=6000000 
> watcher=org.apache.giraph.worker.BspServiceWorker@5e89c116
> date hr:18:01,705 INFO org.apache.zookeeper.ClientCnxn: Opening socket 
> connection to server node2.loc/ipLoc:port. Will not attempt to 
> authenticate using SASL (java.lang.SecurityException: Unable to locate 
> a login configuration)
> date hr:18:01,705 INFO org.apache.zookeeper.ClientCnxn: Socket 
> connection established to node2.loc/ipLoc:port, initiating session
> date hr:18:01,714 INFO org.apache.zookeeper.ClientCnxn: Session 
> establishment complete on server node2.loc/ipLoc:port, sessionid = 
> 0x24889e30b710063, negotiated timeout = 40000
> date hr:18:01,715 INFO org.apache.giraph.bsp.BspService: process: 
> Asynchronous connection complete.
> date hr:18:01,722 INFO org.apache.giraph.comm.netty.NettyWorkerServer: 
> createMessageStoreFactory: Using ByteArrayMessagesPerVertexStore since 
> there is no combiner
> date hr:18:01,839 INFO org.apache.giraph.comm.netty.NettyServer: 
> NettyServer: Using execution handler with 8 threads after 
> requestFrameDecoder.
> date hr:18:01,914 INFO org.apache.giraph.comm.netty.NettyServer: 
> start: Started server communication server: host.loc/ipLocB:30012 with 
> up to 16 threads on bind attempt 0 with sendBufferSize = 32768 
> receiveBufferSize = 524288 backlog = 23
> date hr:18:01,924 INFO org.apache.giraph.comm.netty.NettyClient: 
> NettyClient: Using execution handler with 8 threads after requestEncoder.
> date hr:18:01,947 INFO org.apache.giraph.graph.GraphTaskManager: 
> setup: Registering health of this worker...
> date hr:18:01,987 INFO org.apache.giraph.bsp.BspService: getJobState: 
> Job state already exists (/_hadoopBsp/job_num/_masterJobState)
> date hr:18:02,001 INFO org.apache.giraph.bsp.BspService: 
> getApplicationAttempt: Node 
> /_hadoopBsp/job_num/_applicationAttemptsDir already exists!
> date hr:18:02,007 INFO org.apache.giraph.bsp.BspService: 
> getApplicationAttempt: Node 
> /_hadoopBsp/job_num/_applicationAttemptsDir already exists!
> date hr:18:02,017 INFO org.apache.giraph.worker.BspServiceWorker: 
> registerHealth: Created my health node for attempt=0, superstep=-1 
> with 
> /_hadoopBsp/job_num/_applicationAttemptsDir/0/_superstepDir/-1/_workerHealthyDir/host.loc_12

> and workerInfo= Worker(hostname=host.loc, MRtaskID=12, port=30012)
> date hr:18:18,116 INFO org.apache.giraph.comm.netty.NettyServer: 
> start: Using Netty without authentication.
> date hr:18:18,171 INFO org.apache.giraph.bsp.BspService: process: 
> partitionAssignmentsReadyChanged (partitions are assigned)
> date hr:18:18,194 INFO org.apache.giraph.worker.BspServiceWorker: 
> startSuperstep: Master(hostname=hn.loc, MRtaskID=0, port=30000)
> date hr:18:18,194 INFO org.apache.giraph.worker.BspServiceWorker: 
> startSuperstep: Ready for computation on superstep -1 since worker 
> selection and vertex range assignments are done in 
> /_hadoopBsp/job_num/_applicationAttemptsDir/0/_superstepDir/-1/_addressesAndPartitions
> date hr:18:18,198 INFO org.apache.giraph.comm.netty.NettyClient: Using 
> Netty without authentication.
> date hr:18:18,213 INFO org.apache.giraph.comm.netty.NettyClient: Using 
> Netty without authentication.
> date hr:18:18,216 INFO org.apache.giraph.comm.netty.NettyClient: Using 
> Netty without authentication.
> date hr:18:18,220 INFO org.apache.giraph.comm.netty.NettyClient: Using 
> Netty without authentication.
> date hr:18:18,221 INFO org.apache.giraph.comm.netty.NettyServer: 
> start: Using Netty without authentication.
> date hr:18:18,224 INFO org.apache.giraph.comm.netty.NettyClient: Using 
> Netty without authentication.
> date hr:18:18,226 INFO org.apache.giraph.comm.netty.NettyClient: Using 
> Netty without authentication.
> date hr:18:18,227 INFO org.apache.giraph.comm.netty.NettyServer: 
> start: Using Netty without authentication.
> date hr:18:18,230 INFO org.apache.giraph.comm.netty.NettyClient: Using 
> Netty without authentication.
> date hr:18:18,231 INFO org.apache.giraph.comm.netty.NettyClient: Using 
> Netty without authentication.
> date hr:18:18,232 INFO org.apache.giraph.comm.netty.NettyServer: 
> start: Using Netty without authentication.
> date hr:18:18,234 INFO org.apache.giraph.comm.netty.NettyClient: Using 
> Netty without authentication.
> date hr:18:18,234 INFO org.apache.giraph.comm.netty.NettyServer: 
> start: Using Netty without authentication.
> date hr:18:18,235 INFO org.apache.giraph.comm.netty.NettyClient: Using 
> Netty without authentication.
> date hr:18:18,235 INFO org.apache.giraph.comm.netty.NettyServer: 
> start: Using Netty without authentication.
> date hr:18:18,237 INFO org.apache.giraph.comm.netty.NettyClient: Using 
> Netty without authentication.
> date hr:18:18,237 INFO org.apache.giraph.comm.netty.NettyServer: 
> start: Using Netty without authentication.
> date hr:18:18,239 INFO org.apache.giraph.comm.netty.NettyServer: 
> start: Using Netty without authentication.
> date hr:18:18,239 INFO org.apache.giraph.comm.netty.NettyClient: Using 
> Netty without authentication.
> date hr:18:18,241 INFO org.apache.giraph.comm.netty.NettyServer: 
> start: Using Netty without authentication.
> date hr:18:18,241 INFO org.apache.giraph.comm.netty.NettyClient: Using 
> Netty without authentication.
> date hr:18:18,242 INFO org.apache.giraph.comm.netty.NettyClient: Using 
> Netty without authentication.
> date hr:18:18,242 INFO org.apache.giraph.comm.netty.NettyServer: 
> start: Using Netty without authentication.
> date hr:18:18,244 INFO org.apache.giraph.comm.netty.NettyClient: Using 
> Netty without authentication.
> date hr:18:18,244 INFO org.apache.giraph.comm.netty.NettyServer: 
> start: Using Netty without authentication.
> date hr:18:18,244 INFO org.apache.giraph.comm.netty.NettyClient: Using 
> Netty without authentication.
> date hr:18:18,246 INFO org.apache.giraph.comm.netty.NettyServer: 
> start: Using Netty without authentication.
> date hr:18:18,246 INFO org.apache.giraph.comm.netty.NettyClient: Using 
> Netty without authentication.
> date hr:18:18,247 INFO org.apache.giraph.comm.netty.NettyServer: 
> start: Using Netty without authentication.
> date hr:18:18,248 INFO org.apache.giraph.comm.netty.NettyClient: Using 
> Netty without authentication.
> date hr:18:18,249 INFO org.apache.giraph.comm.netty.NettyServer: 
> start: Using Netty without authentication.
> date hr:18:18,249 INFO org.apache.giraph.comm.netty.NettyClient: Using 
> Netty without authentication.
> date hr:18:18,251 INFO org.apache.giraph.comm.netty.NettyServer: 
> start: Using Netty without authentication.
> date hr:18:18,269 INFO org.apache.giraph.comm.netty.NettyClient: Using 
> Netty without authentication.
> date hr:18:18,271 INFO org.apache.giraph.comm.netty.NettyClient: Using 
> Netty without authentication.
> date hr:18:18,271 INFO org.apache.giraph.comm.netty.NettyServer: 
> start: Using Netty without authentication.
> date hr:18:18,274 INFO org.apache.giraph.comm.netty.NettyClient: Using 
> Netty without authentication.
> date hr:18:18,274 INFO org.apache.giraph.comm.netty.NettyServer: 
> start: Using Netty without authentication.
> date hr:18:18,278 INFO org.apache.giraph.comm.netty.NettyClient: Using 
> Netty without authentication.
> date hr:18:18,278 INFO org.apache.giraph.comm.netty.NettyServer: 
> start: Using Netty without authentication.
> date hr:18:18,282 INFO org.apache.giraph.comm.netty.NettyServer: 
> start: Using Netty without authentication.
> date hr:18:18,284 INFO org.apache.giraph.comm.netty.NettyServer: 
> start: Using Netty without authentication.
> date hr:18:18,285 INFO org.apache.giraph.comm.netty.NettyServer: 
> start: Using Netty without authentication.
> date hr:18:18,286 INFO org.apache.giraph.comm.netty.NettyServer: 
> start: Using Netty without authentication.
> date hr:18:18,290 INFO org.apache.giraph.comm.netty.NettyClient: 
> connectAllAddresses: Successfully added 23 connections, (23 total 
> connected) 0 failed, 0 failures total.
> date hr:18:18,337 INFO org.apache.giraph.worker.BspServiceWorker: 
> loadInputSplits: Using 1 thread(s), originally 1 threads(s) for 1920 
> total splits.
> date hr:18:18,346 INFO org.apache.giraph.comm.SendPartitionCache: 
> SendPartitionCache: maxVerticesPerTransfer = 10000
> date hr:18:18,346 INFO org.apache.giraph.comm.SendPartitionCache: 
> SendPartitionCache: maxEdgesPerTransfer = 80000
> date hr:18:18,402 INFO org.apache.giraph.worker.InputSplitsHandler: 
> reserveInputSplit: Reserved input split path 
> /_hadoopBsp/job_num/_vertexInputSplitDir/900, overall roughly 0.0% 
> input splits reserved
> date hr:18:18,405 INFO org.apache.giraph.worker.InputSplitsCallable: 
> getInputSplit: Reserved /_hadoopBsp/job_num/_vertexInputSplitDir/900 
> from ZooKeeper and got input split 
> 'hdfs://xd-namenode.loc:8020/inputDir/part-r-00060:0+67108864'
> date hr:18:19,550 INFO org.apache.giraph.comm.netty.NettyServer: 
> start: Using Netty without authentication.
> date hr:18:21,319 INFO org.apache.giraph.worker.InputSplitsCallable: 
> loadFromInputSplit: Finished loading 
> /_hadoopBsp/job_num/_vertexInputSplitDir/900 (v=63590, e=6761739)
> date hr:18:21,329 INFO org.apache.giraph.worker.InputSplitsHandler: 
> reserveInputSplit: Reserved input split path 
> /_hadoopBsp/job_num/_vertexInputSplitDir/786, overall roughly 
> 0.052083332% input splits reserved
> date hr:18:21,329 INFO org.apache.giraph.worker.InputSplitsCallable: 
> getInputSplit: Reserved /_hadoopBsp/job_num/_vertexInputSplitDir/786 
> from ZooKeeper and got input split 
> 'hdfs://xd-namenode.loc:8020/inputDir/part-r-00052:402653184+67108864'
> date hr:18:23,864 INFO org.apache.giraph.worker.InputSplitsCallable: 
> loadFromInputSplit: Finished loading 
> /_hadoopBsp/job_num/_vertexInputSplitDir/786 (v=59916, e=6058611)
> date hr:18:23,873 INFO org.apache.giraph.worker.InputSplitsHandler: 
> reserveInputSplit: Reserved input split path 
> /_hadoopBsp/job_num/_vertexInputSplitDir/1660, overall roughly 
> 0.104166664% input splits reserved
> date hr:18:23,873 INFO org.apache.giraph.worker.InputSplitsCallable: 
> getInputSplit: Reserved /_hadoopBsp/job_num/_vertexInputSplitDir/1660 
> from ZooKeeper and got input split 
> 'hdfs://xd-namenode.loc:8020/inputDir/part-r-00110:671088640+67108864'
> date hr:18:26,067 INFO org.apache.giraph.worker.InputSplitsCallable: 
> loadFromInputSplit: Finished loading 
> /_hadoopBsp/job_num/_vertexInputSplitDir/1660 (v=57679, e=6057188)
> date hr:18:26,079 INFO org.apache.giraph.worker.InputSplitsHandler: 
> reserveInputSplit: Reserved input split path 
> /_hadoopBsp/job_num/_vertexInputSplitDir/1412, overall roughly 
> 0.20833333% input splits reserved
> date hr:18:26,080 INFO org.apache.giraph.worker.InputSplitsCallable: 
> getInputSplit: Reserved /_hadoopBsp/job_num/_vertexInputSplitDir/1412 
> from ZooKeeper and got input split 
> 'hdfs://xd-namenode.loc:8020/inputDir/part-r-00094:134217728+67108864'
> date hr:18:26,102 INFO 
> org.apache.giraph.comm.netty.handler.RequestDecoder: decode: Server 
> window metrics MBytes/sec sent = 0, MBytes/sec received = 0.0254, 
> MBytesSent = 0, MBytesReceived = 0.6186, ave sent req MBytes = 0, ave 
> received req MBytes = 0.2062, secs waited = 24.326
> date hr:18:28,725 INFO org.apache.giraph.worker.InputSplitsCallable: 
> loadFromInputSplit: Finished loading 
> /_hadoopBsp/job_num/_vertexInputSplitDir/1412 (v=55743, e=6675371)
> date hr:18:28,738 INFO org.apache.giraph.worker.InputSplitsHandler: 
> reserveInputSplit: Reserved input split path 
> /_hadoopBsp/job_num/_vertexInputSplitDir/354, overall roughly 
> 0.26041666% input splits reserved
> date hr:18:28,739 INFO org.apache.giraph.worker.InputSplitsCallable: 
> getInputSplit: Reserved /_hadoopBsp/job_num/_vertexInputSplitDir/354 
> from ZooKeeper and got input split 
> 'hdfs://xd-namenode.loc:8020/inputDir/part-r-00023:603979776+67108864'
> date hr:18:31,559 INFO org.apache.giraph.worker.InputSplitsCallable: 
> loadFromInputSplit: Finished loading 
> /_hadoopBsp/job_num/_vertexInputSplitDir/354 (v=64903, e=6049318)
> date hr:18:31,569 INFO org.apache.giraph.worker.InputSplitsHandler: 
> reserveInputSplit: Reserved input split path 
> /_hadoopBsp/job_num/_vertexInputSplitDir/27, overall roughly 0.3125% 
> input splits reserved
> date hr:18:31,570 INFO org.apache.giraph.worker.InputSplitsCallable: 
> getInputSplit: Reserved /_hadoopBsp/job_num/_vertexInputSplitDir/27 
> from ZooKeeper and got input split 
> 'hdfs://xd-namenode.loc:8020/inputDir/part-r-00001:805306368+67108864'
> date hr:18:34,257 INFO org.apache.giraph.worker.InputSplitsCallable: 
> loadFromInputSplit: Finished loading 
> /_hadoopBsp/job_num/_vertexInputSplitDir/27 (v=38975, e=6071655)
> date hr:18:34,269 INFO org.apache.giraph.worker.InputSplitsHandler: 
> reserveInputSplit: Reserved input split path 
> /_hadoopBsp/job_num/_vertexInputSplitDir/237, overall roughly 
> 0.36458334% input splits reserved
> date hr:18:34,270 INFO org.apache.giraph.worker.InputSplitsCallable: 
> getInputSplit: Reserved /_hadoopBsp/job_num/_vertexInputSplitDir/237 
> from ZooKeeper and got input split 
> 'hdfs://xd-namenode.loc:8020/inputDir/part-r-00015:805306368+67108864'
> date hr:18:37,307 INFO org.apache.giraph.worker.InputSplitsCallable: 
> loadFromInputSplit: Finished loading 
> /_hadoopBsp/job_num/_vertexInputSplitDir/237 (v=34028, e=6076095)
> date hr:18:37,317 INFO org.apache.giraph.worker.InputSplitsHandler: 
> reserveInputSplit: Reserved input split path 
> /_hadoopBsp/job_num/_vertexInputSplitDir/1146, overall roughly 
> 0.41666666% input splits reserved
> date hr:18:37,318 INFO org.apache.giraph.worker.InputSplitsCallable: 
> getInputSplit: Reserved /_hadoopBsp/job_num/_vertexInputSplitDir/1146 
> from ZooKeeper and got input split 
> 'hdfs://xd-namenode.loc:8020/inputDir/part-r-00076:402653184+67108864'
> date hr:18:39,766 INFO org.apache.giraph.worker.InputSplitsCallable: 
> loadFromInputSplit: Finished loading 
> /_hadoopBsp/job_num/_vertexInputSplitDir/1146 (v=59946, e=6056027)
> date hr:18:39,776 INFO org.apache.giraph.worker.InputSplitsHandler: 
> reserveInputSplit: Reserved input split path 
> /_hadoopBsp/job_num/_vertexInputSplitDir/489, overall roughly 0.46875% 
> input splits reserved
> date hr:18:39,777 INFO org.apache.giraph.worker.InputSplitsCallable: 
> getInputSplit: Reserved /_hadoopBsp/job_num/_vertexInputSplitDir/489 
> from ZooKeeper and got input split 
> 'hdfs://xd-namenode.loc:8020/inputDir/part-r-00032:603979776+67108864'
> date hr:18:42,161 INFO org.apache.giraph.worker.InputSplitsCallable: 
> loadFromInputSplit: Finished loading 
> /_hadoopBsp/job_num/_vertexInputSplitDir/489 (v=65922, e=6049457)
> date hr:18:42,176 INFO org.apache.giraph.worker.InputSplitsHandler: 
> reserveInputSplit: Reserved input split path 
> /_hadoopBsp/job_num/_vertexInputSplitDir/333, overall roughly 
> 0.5208333% input splits reserved
> date hr:18:42,177 INFO org.apache.giraph.worker.InputSplitsCallable: 
> getInputSplit: Reserved /_hadoopBsp/job_num/_vertexInputSplitDir/333 
> from ZooKeeper and got input split 
> 'hdfs://xd-namenode.loc:8020/inputDir/part-r-00022:201326592+67108864'
> date hr:18:44,768 INFO org.apache.giraph.worker.InputSplitsCallable: 
> loadFromInputSplit: Finished loading 
> /_hadoopBsp/job_num/_vertexInputSplitDir/333 (v=38606, e=6660726)
> date hr:18:44,782 INFO org.apache.giraph.worker.InputSplitsHandler: 
> reserveInputSplit: Reserved input split path 
> /_hadoopBsp/job_num/_vertexInputSplitDir/243, overall roughly 
> 0.5729167% input splits reserved
> date hr:18:44,783 INFO org.apache.giraph.worker.InputSplitsCallable: 
> getInputSplit: Reserved /_hadoopBsp/job_num/_vertexInputSplitDir/243 
> from ZooKeeper and got input split 
> 'hdfs://xd-namenode.loc:8020/inputDir/part-r-00016:201326592+67108864'
> date hr:18:47,361 INFO org.apache.giraph.worker.InputSplitsCallable: 
> loadFromInputSplit: Finished loading 
> /_hadoopBsp/job_num/_vertexInputSplitDir/243 (v=36872, e=6665114)
> date hr:18:47,373 INFO org.apache.giraph.worker.InputSplitsHandler: 
> reserveInputSplit: Reserved input split path 
> /_hadoopBsp/job_num/_vertexInputSplitDir/190, overall roughly 0.625% 
> input splits reserved
> date hr:18:47,373 INFO org.apache.giraph.worker.InputSplitsCallable: 
> getInputSplit: Reserved /_hadoopBsp/job_num/_vertexInputSplitDir/190 
> from ZooKeeper and got input split 
> 'hdfs://xd-namenode.loc:8020/inputDir/part-r-00012:671088640+67108864'
> date hr:18:50,483 INFO org.apache.giraph.worker.InputSplitsCallable: 
> loadFromInputSplit: Finished loading 
> /_hadoopBsp/job_num/_vertexInputSplitDir/190 (v=57933, e=6056243)
> date hr:18:50,503 INFO org.apache.giraph.worker.InputSplitsHandler: 
> reserveInputSplit: Reserved input split path 
> /_hadoopBsp/job_num/_vertexInputSplitDir/473, overall roughly 
> 0.6770833% input splits reserved
> date hr:18:50,504 INFO org.apache.giraph.worker.InputSplitsCallable: 
> getInputSplit: Reserved /_hadoopBsp/job_num/_vertexInputSplitDir/473 
> from ZooKeeper and got input split 
> 'hdfs://xd-namenode.loc:8020/inputDir/part-r-00031:536870912+67108864'
> date hr:18:56,225 INFO 
> org.apache.giraph.comm.netty.handler.RequestDecoder: decode: Server 
> window metrics MBytes/sec sent = 0.0002, MBytes/sec received = 
> 10.5257, MBytesSent = 0.0062, MBytesReceived = 317.0765, ave sent req 
> MBytes = 0, ave received req MBytes = 0.0465, secs waited = 30.123
> date hr:19:56,575 INFO org.apache.giraph.utils.ProgressableUtils: 
> waitFor: Future result not ready yet 
> java.util.concurrent.FutureTask@45e33bb8
> date hr:19:56,576 INFO org.apache.giraph.utils.ProgressableUtils: 
> waitFor: Waiting for 
> org.apache.giraph.utils.ProgressableUtils$FutureWaitable@2ffecaeb
> date hr:19:56,577 INFO org.apache.zookeeper.ClientCnxn: Client session 
> timed out, have not heard from server in 52735ms for sessionid 
> 0x24889e30b710063, closing socket connection and attempting reconnect
> date hr:19:56,578 INFO 
> org.apache.giraph.comm.netty.handler.RequestDecoder: decode: Server 
> window metrics MBytes/sec sent = 0, MBytes/sec received = 2.3878, 
> MBytesSent = 0.0027, MBytesReceived = 142.6861, ave sent req MBytes = 
> 0, ave received req MBytes = 0.0616, secs waited = 60.353
> date hr:19:56,679 WARN org.apache.giraph.bsp.BspService: process: 
> Disconnected from ZooKeeper (will automatically try to recover) 
> WatchedEvent state:Disconnected type:None path:null
> date hr:19:56,679 WARN org.apache.giraph.worker.InputSplitsHandler: 
> process: Problem with zookeeper, got event with path null, state 
> Disconnected, event type None
> date hr:19:56,949 INFO org.apache.zookeeper.ClientCnxn: Opening socket 
> connection to server node3.loc/ip:port. Will not attempt to 
> authenticate using SASL (java.lang.SecurityException: Unable to locate 
> a login configuration)
> date hr:19:56,949 INFO org.apache.zookeeper.ClientCnxn: Socket 
> connection established to node3.loc/ip:port, initiating session
> date hr:19:56,953 WARN org.apache.giraph.bsp.BspService: process: Got 
> unknown null path event WatchedEvent state:Expired type:None path:null
> date hr:19:56,953 INFO org.apache.zookeeper.ClientCnxn: Unable to 
> reconnect to ZooKeeper service, session 0x24889e30b710063 has expired, 
> closing socket connection
> date hr:19:56,953 WARN org.apache.giraph.worker.InputSplitsHandler: 
> process: Problem with zookeeper, got event with path null, state 
> Expired, event type None
> date hr:19:56,953 INFO org.apache.zookeeper.ClientCnxn: EventThread 
> shut down
> date hr:19:57,731 INFO org.apache.giraph.worker.InputSplitsCallable: 
> loadFromInputSplit: Finished loading 
> /_hadoopBsp/job_num/_vertexInputSplitDir/473 (v=68319, e=6046591)
> date hr:19:57,733 ERROR org.apache.giraph.utils.LogStacktraceCallable: 
> Execution of callable failed
> java.lang.IllegalStateException: markInputSplitPathFinished: 
> KeeperException on 
> /_hadoopBsp/job_num/_vertexInputSplitDir/473/_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:886)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> at java.lang.Thread.run(Thread.java:662)
> Caused by: 
> org.apache.zookeeper.KeeperException$SessionExpiredException: 
> KeeperErrorCode = Session expired for 
> /_hadoopBsp/job_num/_vertexInputSplitDir/473/_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
> date hr:19:57,736 ERROR org.apache.giraph.worker.BspServiceWorker: 
> unregisterHealth: Got failure, unregistering health on 
> /_hadoopBsp/job_num/_applicationAttemptsDir/0/_superstepDir/-1/_workerHealthyDir/host.loc_12

> on superstep -1
> date hr:19:57,738 ERROR org.apache.giraph.graph.GraphTaskManager: run: 
> Worker failure failed on another RuntimeException, original expection 
> will be rethrown
> java.lang.IllegalStateException: unregisterHealth: KeeperException - 
> Couldn't delete 
> /_hadoopBsp/job_num/_applicationAttemptsDir/0/_superstepDir/-1/_workerHealthyDir/host.loc_12
> at 
> org.apache.giraph.worker.BspServiceWorker.unregisterHealth(BspServiceWorker.java:656)
> at 
> org.apache.giraph.worker.BspServiceWorker.failureCleanup(BspServiceWorker.java:664)
> at 
> org.apache.giraph.graph.GraphTaskManager.workerFailureCleanup(GraphTaskManager.java:908)
> at org.apache.giraph.graph.GraphMapper.run(GraphMapper.java:101)
> at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:672)
> at org.apache.hadoop.mapred.MapTask.run(MapTask.java:330)
> at org.apache.hadoop.mapred.Child$4.run(Child.java:268)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:396)
> at 
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1408)
> at org.apache.hadoop.mapred.Child.main(Child.java:262)
> Caused by: 
> org.apache.zookeeper.KeeperException$SessionExpiredException: 
> KeeperErrorCode = Session expired for 
> /_hadoopBsp/job_num/_applicationAttemptsDir/0/_superstepDir/-1/_workerHealthyDir/host.loc_12
> at org.apache.zookeeper.KeeperException.create(KeeperException.java:127)
> at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
> at org.apache.zookeeper.ZooKeeper.delete(ZooKeeper.java:873)
> at org.apache.giraph.zk.ZooKeeperExt.deleteExt(ZooKeeperExt.java:302)
> at 
> org.apache.giraph.worker.BspServiceWorker.unregisterHealth(BspServiceWorker.java:650)
> ... 10 more
> date hr:19:57,745 INFO org.apache.hadoop.mapred.TaskLogsTruncater: 
> Initializing logs' truncater with mapRetainSize=-1 and reduceRetainSize=-1
> date hr:19:57,748 WARN org.apache.hadoop.mapred.Child: Error running child
> java.lang.IllegalStateException: run: Caught an unrecoverable 
> exception waitFor: ExecutionException occurred while waiting for 
> org.apache.giraph.utils.ProgressableUtils$FutureWaitable@2ffecaeb
> at org.apache.giraph.graph.GraphMapper.run(GraphMapper.java:102)
> at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:672)
> at org.apache.hadoop.mapred.MapTask.run(MapTask.java:330)
> at org.apache.hadoop.mapred.Child$4.run(Child.java:268)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:396)
> at 
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1408)
> at org.apache.hadoop.mapred.Child.main(Child.java:262)
> Caused by: java.lang.IllegalStateException: waitFor: 
> ExecutionException occurred while waiting for 
> org.apache.giraph.utils.ProgressableUtils$FutureWaitable@2ffecaeb
> at 
> org.apache.giraph.utils.ProgressableUtils.waitFor(ProgressableUtils.java:151)
> at 
> org.apache.giraph.utils.ProgressableUtils.waitForever(ProgressableUtils.java:111)
> at 
> org.apache.giraph.utils.ProgressableUtils.getFutureResult(ProgressableUtils.java:73)
> at 
> org.apache.giraph.utils.ProgressableUtils.getResultsWithNCallables(ProgressableUtils.java:192)
> at 
> org.apache.giraph.worker.BspServiceWorker.loadInputSplits(BspServiceWorker.java:276)
> at 
> org.apache.giraph.worker.BspServiceWorker.loadVertices(BspServiceWorker.java:323)
> at 
> org.apache.giraph.worker.BspServiceWorker.setup(BspServiceWorker.java:506)
> at 
> org.apache.giraph.graph.GraphTaskManager.execute(GraphTaskManager.java:230)
> at org.apache.giraph.graph.GraphMapper.run(GraphMapper.java:92)
> ... 7 more
> Caused by: java.util.concurrent.ExecutionException: 
> java.lang.IllegalStateException: markInputSplitPathFinished: 
> KeeperException on 
> /_hadoopBsp/job_num/_vertexInputSplitDir/473/_vertexInputSplitFinished
> at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:232)
> at java.util.concurrent.FutureTask.get(FutureTask.java:91)
> at 
> org.apache.giraph.utils.ProgressableUtils$FutureWaitable.waitFor(ProgressableUtils.java:271)
> at 
> org.apache.giraph.utils.ProgressableUtils.waitFor(ProgressableUtils.java:143)
> ... 15 more
> Caused by: java.lang.IllegalStateException: 
> markInputSplitPathFinished: KeeperException on 
> /_hadoopBsp/job_num/_vertexInputSplitDir/473/_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:886)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> at java.lang.Thread.run(Thread.java:662)
> Caused by: 
> org.apache.zookeeper.KeeperException$SessionExpiredException: 
> KeeperErrorCode = Session expired for 
> /_hadoopBsp/job_num/_vertexInputSplitDir/473/_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
> date hr:19:57,753 INFO org.apache.hadoop.mapred.Task: Runnning cleanup 
> for the task
>


Mime
View raw message