Hi Denis, Thanks for trying this out without the "isStaticGraph" option. I'll soon create a JIRA and fix this issue.
It is worth mentioning that the new OOC is designed in a way that it works even with a very low available memory. But, there are subtleties that should be considered. First, the Giraph itself (book-keeping, ZooKeeper, and other stuff not related to the graph structure or messages) would take about 2-3GB of space. Second, I strongly encourage you to use ParallelGC and set the OldGen size (tenured memory generation) to be about 90% of the available memory. Third, depending on the OldGen size, you would want to change some of the OOC parameters to have it working in a robust way. Let me give you an example. Let's say you have a 16GB-container (you may want to have about 10% more memory available on your system to accommodate JVM and off-heap message data). Let's say you set aside 2GB for the NewGen and 14GB for old gen. Assuming Giraph would take 3GB of data just for the non-graph-relevant book-keeping, that gives you 11GB of space only for graph structure. That means the job would fail if the amount of graph data in memory passes about 11GB or so! If you look at the ThresholdBasedOracle used as a default for the new OOC design, there is a flag called "giraph.memory.failPressure." You would want to set this flag to be 11/16 (or even slightly less) and scale the rest of the thresholds accordingly. So, in the example I just mentioned, here are the values you should set the flags: giraph.memory.failPressure=0.65 giraph.memory.emergencyPressure=0.625 giraph.memory.highPressure=0.6 giraph.memory.optimalPressure=0.55 giraph.memory.lowPressure=0.5 You only need to tune these flags once for a given amount of memory in a container. Once tuned, the same set of thresholds should work for various applications. I hope it helps. Best, Hassan On Tue, Nov 8, 2016 at 4:25 AM, Denis Dudinski <denis.dudin...@gmail.com> wrote: > Hi Hassan, > > Thank you for really quick response! > > I changed "giraph.isStaticGraph" to false and the error disappeared. > As expected iteration became slow and wrote to disk edges once again > in superstep 1. > > However, the computation failed at superstep 2 with error > "java.lang.OutOfMemoryError: GC overhead limit exceeded". It seems to > be unrelated to "isStaticGraph" issue, but I think it worth mentioning > to see the picture as a whole. > > Are there any other tests/information I am able to execute/check to > help to pinpoint "isStaticGraph" problem? > > Best Regards, > Denis Dudinski > > > 2016-11-07 20:00 GMT+03:00 Hassan Eslami <hsn.esl...@gmail.com>: > > Hi Denis, > > > > Thanks for bringing up the issue. In the previous conversation thread, > the > > similar problem is reported even with a simpler example connected > component > > calculation. Although, back then, we were developing other > > performance-critical components of OOC. > > > > Let's debug this issue together to make the new OOC more stable. I > suspect > > the problem is with "giraph.isStaticGraph=true" (as this is only an > > optimization and most of our end-to-end testing was on cases where the > graph > > could change). Let's get rid of it for now and see if the problem still > > exists. > > > > Best, > > Hassan > > > > On Mon, Nov 7, 2016 at 6:24 AM, Denis Dudinski <denis.dudin...@gmail.com > > > > wrote: > >> > >> Hello, > >> > >> We are trying to calculate PageRank on huge graph, which does not fit > >> into memory. For calculation to succeed we tried to turn on OutOfCore > >> feature of Giraph, but every launch we tried resulted in > >> com.esotericsoftware.kryo.KryoException: Buffer underflow. > >> Each time it happens on different servers but exactly right after > >> start of superstep 1. > >> > >> We are using Giraph 1.2.0 on hadoop 2.7.3 (our prod version, can't > >> back-step to Giraph's officially supported version and had to patch > >> Giraph a little) placed on 11 servers + 3 master servers (namenodes > >> etc.) with separate ZooKeeper cluster deployment. > >> > >> Our launch command: > >> > >> hadoop jar /opt/giraph-1.2.0/pr-job-jar-with-dependencies.jar > >> org.apache.giraph.GiraphRunner com.prototype.di.pr.PageRankComputation > >> \ > >> -mc com.prototype.di.pr.PageRankMasterCompute \ > >> -yj pr-job-jar-with-dependencies.jar \ > >> -vif com.belprime.di.pr.input.HBLongVertexInputFormat \ > >> -vof org.apache.giraph.io.formats.IdWithValueTextOutputFormat \ > >> -op /user/hadoop/output/pr_test \ > >> -w 10 \ > >> -c com.prototype.di.pr.PRDoubleCombiner \ > >> -wc com.prototype.di.pr.PageRankWorkerContext \ > >> -ca hbase.rootdir=hdfs://namenode1.webmeup.com:8020/hbase \ > >> -ca giraph.logLevel=info \ > >> -ca hbase.mapreduce.inputtable=di_test \ > >> -ca hbase.mapreduce.scan.columns=di:n \ > >> -ca hbase.defaults.for.version.skip=true \ > >> -ca hbase.table.row.textkey=false \ > >> -ca giraph.yarn.task.heap.mb=48000 \ > >> -ca giraph.isStaticGraph=true \ > >> -ca giraph.SplitMasterWorker=false \ > >> -ca giraph.oneToAllMsgSending=true \ > >> -ca giraph.metrics.enable=true \ > >> -ca giraph.jmap.histo.enable=true \ > >> -ca giraph.vertexIdClass=com.prototype.di.pr.DomainPartAwareLongWritable > \ > >> -ca giraph.outgoingMessageValueClass=org.apache.hadoop.io.DoubleWritable > \ > >> -ca giraph.inputOutEdgesClass=org.apache.giraph.edge.LongNullArrayEdges > \ > >> -ca giraph.useOutOfCoreGraph=true \ > >> -ca giraph.waitForPerWorkerRequests=true \ > >> -ca giraph.maxNumberOfUnsentRequests=1000 \ > >> -ca > >> giraph.vertexInputFilterClass=com.prototype.di.pr.input. > PagesFromSameDomainLimiter > >> \ > >> -ca giraph.useInputSplitLocality=true \ > >> -ca hbase.mapreduce.scan.cachedrows=10000 \ > >> -ca giraph.minPartitionsPerComputeThread=60 \ > >> -ca > >> giraph.graphPartitionerFactoryClass=com.prototype.di.pr. > DomainAwareGraphPartitionerFactory > >> \ > >> -ca giraph.numInputThreads=1 \ > >> -ca giraph.inputSplitSamplePercent=20 \ > >> -ca giraph.pr.maxNeighborsPerVertex=50 \ > >> -ca giraph.partitionClass=org.apache.giraph.partition.ByteArrayPartition > \ > >> -ca giraph.vertexClass=org.apache.giraph.graph.ByteValueVertex \ > >> -ca > >> giraph.partitionsDirectory=/disk1/_bsp/_partitions,/disk2/ > _bsp/_partitions > >> > >> Logs excerpt: > >> > >> 16/11/06 15:47:15 INFO pr.PageRankWorkerContext: Pre superstep in worker > >> context > >> 16/11/06 15:47:15 INFO graph.GraphTaskManager: execute: 60 partitions > >> to process with 1 compute thread(s), originally 1 thread(s) on > >> superstep 1 > >> 16/11/06 15:47:15 INFO ooc.OutOfCoreEngine: startIteration: with 60 > >> partitions in memory and 1 active threads > >> 16/11/06 15:47:15 INFO pr.PageRankComputation: Pre superstep1 in PR > >> computation > >> 16/11/06 15:47:15 INFO policy.ThresholdBasedOracle: getNextIOActions: > >> usedMemoryFraction = 0.75 > >> 16/11/06 15:47:16 INFO ooc.OutOfCoreEngine: > >> updateActiveThreadsFraction: updating the number of active threads to > >> 1 > >> 16/11/06 15:47:16 INFO policy.ThresholdBasedOracle: > >> updateRequestsCredit: updating the credit to 20 > >> 16/11/06 15:47:17 INFO graph.GraphTaskManager: installGCMonitoring: > >> name = PS Scavenge, action = end of minor GC, cause = Allocation > >> Failure, duration = 937ms > >> 16/11/06 15:47:17 INFO policy.ThresholdBasedOracle: getNextIOActions: > >> usedMemoryFraction = 0.72 > >> 16/11/06 15:47:18 INFO policy.ThresholdBasedOracle: getNextIOActions: > >> usedMemoryFraction = 0.74 > >> 16/11/06 15:47:18 INFO ooc.OutOfCoreEngine: > >> updateActiveThreadsFraction: updating the number of active threads to > >> 1 > >> 16/11/06 15:47:18 INFO policy.ThresholdBasedOracle: > >> updateRequestsCredit: updating the credit to 20 > >> 16/11/06 15:47:19 INFO policy.ThresholdBasedOracle: getNextIOActions: > >> usedMemoryFraction = 0.76 > >> 16/11/06 15:47:19 INFO ooc.OutOfCoreEngine: doneProcessingPartition: > >> processing partition 234 is done! > >> 16/11/06 15:47:20 INFO policy.ThresholdBasedOracle: getNextIOActions: > >> usedMemoryFraction = 0.79 > >> 16/11/06 15:47:21 INFO ooc.OutOfCoreEngine: > >> updateActiveThreadsFraction: updating the number of active threads to > >> 1 > >> 16/11/06 15:47:21 INFO policy.ThresholdBasedOracle: > >> updateRequestsCredit: updating the credit to 18 > >> 16/11/06 15:47:21 INFO handler.RequestDecoder: decode: Server window > >> metrics MBytes/sec received = 1.0994, MBytesReceived = 33.0459, ave > >> received req MBytes = 0.0138, secs waited = 30.058 > >> 16/11/06 15:47:21 INFO policy.ThresholdBasedOracle: getNextIOActions: > >> usedMemoryFraction = 0.82 > >> 16/11/06 15:47:21 INFO ooc.OutOfCoreIOCallable: call: thread 0's next > >> IO command is: StorePartitionIOCommand: (partitionId = 234) > >> 16/11/06 15:47:21 INFO ooc.OutOfCoreIOCallable: call: thread 0's > >> command StorePartitionIOCommand: (partitionId = 234) completed: bytes= > >> 64419740, duration=351, bandwidth=175.03, bandwidth (excluding GC > >> time)=175.03 > >> 16/11/06 15:47:21 INFO policy.ThresholdBasedOracle: getNextIOActions: > >> usedMemoryFraction = 0.83 > >> 16/11/06 15:47:21 INFO ooc.OutOfCoreIOCallable: call: thread 0's next > >> IO command is: StoreIncomingMessageIOCommand: (partitionId = 234) > >> 16/11/06 15:47:21 INFO ooc.OutOfCoreIOCallable: call: thread 0's > >> command StoreIncomingMessageIOCommand: (partitionId = 234) completed: > >> bytes= 0, duration=0, bandwidth=NaN, bandwidth (excluding GC time)=NaN > >> 16/11/06 15:47:21 INFO policy.ThresholdBasedOracle: getNextIOActions: > >> usedMemoryFraction = 0.83 > >> 16/11/06 15:47:40 INFO graph.GraphTaskManager: installGCMonitoring: > >> name = PS Scavenge, action = end of minor GC, cause = Allocation > >> Failure, duration = 3107ms > >> 16/11/06 15:47:40 INFO graph.GraphTaskManager: installGCMonitoring: > >> name = PS MarkSweep, action = end of major GC, cause = Ergonomics, > >> duration = 15064ms > >> 16/11/06 15:47:40 INFO ooc.OutOfCoreEngine: > >> updateActiveThreadsFraction: updating the number of active threads to > >> 1 > >> 16/11/06 15:47:40 INFO policy.ThresholdBasedOracle: > >> updateRequestsCredit: updating the credit to 20 > >> 16/11/06 15:47:40 INFO policy.ThresholdBasedOracle: getNextIOActions: > >> usedMemoryFraction = 0.71 > >> 16/11/06 15:47:40 INFO ooc.OutOfCoreIOCallable: call: thread 0's next > >> IO command is: LoadPartitionIOCommand: (partitionId = 234, superstep = > >> 2) > >> JMap histo dump at Sun Nov 06 15:47:41 CET 2016 > >> 16/11/06 15:47:41 INFO ooc.OutOfCoreEngine: doneProcessingPartition: > >> processing partition 364 is done! > >> 16/11/06 15:47:48 INFO ooc.OutOfCoreEngine: > >> updateActiveThreadsFraction: updating the number of active threads to > >> 1 > >> 16/11/06 15:47:48 INFO policy.ThresholdBasedOracle: > >> updateRequestsCredit: updating the credit to 20 > >> -- > >> -- num #instances #bytes class name > >> -- ---------------------------------------------- > >> -- 1: 224004229 10752202992 > >> java.util.concurrent.ConcurrentHashMap$Node > >> -- 2: 19751666 6645730528 [B > >> -- 3: 222135985 5331263640 > >> com.belprime.di.pr.DomainPartAwareLongWritable > >> -- 4: 214686483 5152475592 org.apache.hadoop.io. > DoubleWritable > >> -- 5: 353 4357261784 > >> [Ljava.util.concurrent.ConcurrentHashMap$Node; > >> -- 6: 486266 204484688 [I > >> -- 7: 6017652 192564864 > >> org.apache.giraph.utils.UnsafeByteArrayOutputStream > >> -- 8: 3986203 159448120 > >> org.apache.giraph.utils.UnsafeByteArrayInputStream > >> -- 9: 2064182 148621104 > >> org.apache.giraph.graph.ByteValueVertex > >> -- 10: 2064182 82567280 > >> org.apache.giraph.edge.ByteArrayEdges > >> -- 11: 1886875 45285000 java.lang.Integer > >> -- 12: 349409 30747992 > >> java.util.concurrent.ConcurrentHashMap$TreeNode > >> -- 13: 916970 29343040 java.util.Collections$1 > >> -- 14: 916971 22007304 java.util.Collections$ > SingletonSet > >> -- 15: 47270 3781600 > >> java.util.concurrent.ConcurrentHashMap$TreeBin > >> -- 16: 26201 2590912 [C > >> -- 17: 34175 1367000 > >> org.apache.giraph.edge.ByteArrayEdges$ByteArrayEdgeIterator > >> -- 18: 6143 1067704 java.lang.Class > >> -- 19: 25953 830496 java.lang.String > >> -- 20: 34175 820200 org.apache.giraph.edge. > EdgeNoValue > >> -- 21: 4488 703400 [Ljava.lang.Object; > >> -- 22: 70 395424 [Ljava.nio.channels.SelectionKey; > >> -- 23: 2052 328320 java.lang.reflect.Method > >> -- 24: 6600 316800 > >> org.apache.giraph.utils.ByteArrayVertexIdMessages > >> -- 25: 5781 277488 java.util.HashMap$Node > >> -- 26: 5651 271248 java.util.Hashtable$Entry > >> -- 27: 6604 211328 > >> org.apache.giraph.factories.DefaultMessageValueFactory > >> 16/11/06 15:47:49 ERROR utils.LogStacktraceCallable: Execution of > >> callable failed > >> java.lang.RuntimeException: call: execution of IO command > >> LoadPartitionIOCommand: (partitionId = 234, superstep = 2) failed! > >> at > >> org.apache.giraph.ooc.OutOfCoreIOCallable.call( > OutOfCoreIOCallable.java:115) > >> at > >> org.apache.giraph.ooc.OutOfCoreIOCallable.call( > OutOfCoreIOCallable.java:36) > >> at > >> org.apache.giraph.utils.LogStacktraceCallable.call( > LogStacktraceCallable.java:67) > >> at java.util.concurrent.FutureTask.run(FutureTask.java:266) > >> at > >> java.util.concurrent.ThreadPoolExecutor.runWorker( > ThreadPoolExecutor.java:1142) > >> at > >> java.util.concurrent.ThreadPoolExecutor$Worker.run( > ThreadPoolExecutor.java:617) > >> at java.lang.Thread.run(Thread.java:745) > >> Caused by: com.esotericsoftware.kryo.KryoException: Buffer underflow. > >> at com.esotericsoftware.kryo.io.Input.require(Input.java:199) > >> at com.esotericsoftware.kryo.io.UnsafeInput.readLong( > UnsafeInput.java:112) > >> at > >> com.esotericsoftware.kryo.io.KryoDataInput.readLong( > KryoDataInput.java:91) > >> at org.apache.hadoop.io.LongWritable.readFields(LongWritable.java:47) > >> at > >> org.apache.giraph.ooc.data.DiskBackedPartitionStore.readOutEdges( > DiskBackedPartitionStore.java:245) > >> at > >> org.apache.giraph.ooc.data.DiskBackedPartitionStore. > loadInMemoryPartitionData(DiskBackedPartitionStore.java:278) > >> at > >> org.apache.giraph.ooc.data.DiskBackedDataStore.loadPartitionDataProxy( > DiskBackedDataStore.java:234) > >> at > >> org.apache.giraph.ooc.data.DiskBackedPartitionStore.loadPartitionData( > DiskBackedPartitionStore.java:311) > >> at > >> org.apache.giraph.ooc.command.LoadPartitionIOCommand.execute( > LoadPartitionIOCommand.java:66) > >> at > >> org.apache.giraph.ooc.OutOfCoreIOCallable.call( > OutOfCoreIOCallable.java:99) > >> ... 6 more > >> 16/11/06 15:47:49 FATAL graph.GraphTaskManager: uncaughtException: > >> OverrideExceptionHandler on thread ooc-io-0, msg = call: execution of > >> IO command LoadPartitionIOCommand: (partitionId = 234, superstep = 2) > >> failed!, exiting... > >> java.lang.RuntimeException: call: execution of IO command > >> LoadPartitionIOCommand: (partitionId = 234, superstep = 2) failed! > >> at > >> org.apache.giraph.ooc.OutOfCoreIOCallable.call( > OutOfCoreIOCallable.java:115) > >> at > >> org.apache.giraph.ooc.OutOfCoreIOCallable.call( > OutOfCoreIOCallable.java:36) > >> at > >> org.apache.giraph.utils.LogStacktraceCallable.call( > LogStacktraceCallable.java:67) > >> at java.util.concurrent.FutureTask.run(FutureTask.java:266) > >> at > >> java.util.concurrent.ThreadPoolExecutor.runWorker( > ThreadPoolExecutor.java:1142) > >> at > >> java.util.concurrent.ThreadPoolExecutor$Worker.run( > ThreadPoolExecutor.java:617) > >> at java.lang.Thread.run(Thread.java:745) > >> Caused by: com.esotericsoftware.kryo.KryoException: Buffer underflow. > >> at com.esotericsoftware.kryo.io.Input.require(Input.java:199) > >> at com.esotericsoftware.kryo.io.UnsafeInput.readLong( > UnsafeInput.java:112) > >> at > >> com.esotericsoftware.kryo.io.KryoDataInput.readLong( > KryoDataInput.java:91) > >> at org.apache.hadoop.io.LongWritable.readFields(LongWritable.java:47) > >> at > >> org.apache.giraph.ooc.data.DiskBackedPartitionStore.readOutEdges( > DiskBackedPartitionStore.java:245) > >> at > >> org.apache.giraph.ooc.data.DiskBackedPartitionStore. > loadInMemoryPartitionData(DiskBackedPartitionStore.java:278) > >> at > >> org.apache.giraph.ooc.data.DiskBackedDataStore.loadPartitionDataProxy( > DiskBackedDataStore.java:234) > >> at > >> org.apache.giraph.ooc.data.DiskBackedPartitionStore.loadPartitionData( > DiskBackedPartitionStore.java:311) > >> at > >> org.apache.giraph.ooc.command.LoadPartitionIOCommand.execute( > LoadPartitionIOCommand.java:66) > >> at > >> org.apache.giraph.ooc.OutOfCoreIOCallable.call( > OutOfCoreIOCallable.java:99) > >> ... 6 more > >> 16/11/06 15:47:49 ERROR worker.BspServiceWorker: unregisterHealth: Got > >> failure, unregistering health on > >> > >> /_hadoopBsp/giraph_yarn_application_1478342673283_ > 0009/_applicationAttemptsDir/0/_superstepDir/1/_ > workerHealthyDir/datanode6.webmeup.com_5 > >> on superstep 1 > >> > >> We looked into one thread > >> > >> http://mail-archives.apache.org/mod_mbox/giraph-user/201607.mbox/% > 3CCAECWHa3MOqubf8--wMVhzqOYwwZ0ZuP6_iiqTE_xT% > 3DoLJAAPQw%40mail.gmail.com%3E > >> but it is rather old and at that time the answer was "do not use it > >> yet". > >> (see reply > >> http://mail-archives.apache.org/mod_mbox/giraph-user/201607.mbox/% > 3CCAH1LQfdbpbZuaKsu1b7TCwOzGMxi_vf9vYi6Xg_Bp8o43H7u%2Bw% > 40mail.gmail.com%3E). > >> Does it hold today? We would like to use new advanced adaptive OOC > >> approach if possible... > >> > >> Thank you in advance, any help or hint would be really appreciated. > >> > >> Best Regards, > >> Denis Dudinski > > > > >