[jira] [Commented] (GIRAPH-191) Random Walks on Graphs
[ https://issues.apache.org/jira/browse/GIRAPH-191?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13440167#comment-13440167 ] Gianmarco De Francisci Morales commented on GIRAPH-191: --- Forgot to mention, patch is available in RB now. https://reviews.apache.org/r/6653/ > Random Walks on Graphs > -- > > Key: GIRAPH-191 > URL: https://issues.apache.org/jira/browse/GIRAPH-191 > Project: Giraph > Issue Type: New Feature > Components: examples >Affects Versions: 0.2.0 >Reporter: Gianmarco De Francisci Morales >Assignee: Gianmarco De Francisci Morales > Attachments: GIRAPH-191-1.patch, GIRAPH-191.2.patch, > GIRAPH-191.3.patch, GIRAPH-191.patch > > > Implementing RWR on Giraph should be a very simple modification of the > SimplePageRankVertex code. > {code} > if ( myID == sourceID ) > DoubleWritable vertexValue = new DoubleWritable((0.15f + 0.85f * sum); > else > DoubleWritable vertexValue = new DoubleWritable(0.85f * sum); > {code} > It would be nice to make it as configurable as possible by using parametric > damping factors, preference vectors, strongly preferential, etc... > More or less along these lines: > http://law.dsi.unimi.it/software/docs/it/unimi/dsi/law/rank/PageRank.html -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (GIRAPH-301) InputSplit Reservations are clumping, leaving many workers asleep while other process too many splits and get overloaded.
[ https://issues.apache.org/jira/browse/GIRAPH-301?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13440509#comment-13440509 ] Eli Reisman commented on GIRAPH-301: Update: I've run this close to 40 times to completion on 2 different clusters with greatly varying # of workers, data loads etc. and I'm satisfied its safe and effective at this point. Take your time in reviewing, fresh eyes will be useful here, but the news on the testing is it officially went very well, input superstep is by far the fastest part of any job I run with this patch in place, and it scales very well. > InputSplit Reservations are clumping, leaving many workers asleep while other > process too many splits and get overloaded. > - > > Key: GIRAPH-301 > URL: https://issues.apache.org/jira/browse/GIRAPH-301 > Project: Giraph > Issue Type: Improvement > Components: bsp, graph, zookeeper >Affects Versions: 0.2.0 >Reporter: Eli Reisman >Assignee: Eli Reisman > Labels: patch > Fix For: 0.2.0 > > Attachments: GIRAPH-301-1.patch, GIRAPH-301-2.patch, > GIRAPH-301-3.patch, GIRAPH-301-4.patch, GIRAPH-301-5.patch, > GIRAPH-301-6.patch, GIRAPH-301-7.patch > > > With recent additions to the codebase, users here have noticed many workers > are able to load input splits extremely quickly, and this has altered the > behavior of Giraph during INPUT_SUPERSTEP when using the current algorithm > for split reservations. A few workers process multiple splits (often > overwhelming Netty and getting GC errors as they attempt to offload too much > data too quick) while many (often most) of the others just sleep through the > superstep, never successfully participating at all. > Essentially, the current algo is: > 1. scan input split list, skipping nodes that are marked "Finsihed" > 2. grab the first unfinished node in the list (reserved or not) and check its > reserved status. > 3. if not reserved, attempt to reserve & return it if successful. > 4. if the first one you check is already taken, sleep for way too long and > only wake up if another worker finishes a split, then contend with that > worker for another split, while the majority of the split list might sit > idle, not actually checked or claimed by anyone yet. > This does not work. By making a few simple changes (and acknowledging that ZK > reads are cheap, only writes are not) this patch is able to get every worker > involved, and keep them in the game, ensuring that the INPUT_SUPERSTEP passes > quickly and painlessly, and without overwhelming Netty by spreading the > memory load the split readers bear more evenly. If the giraph.splitmb and -w > options are set correctly, behavior is now exactly as one would expect it to > be. > This also results in INPUT_SUPERSTEP passing more quickly, and survive the > INPUT_SUPERSTEP for a given data load on less Hadoop memory slots. > -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (GIRAPH-313) Open Netty client and server on master
Maja Kabiljo created GIRAPH-313: --- Summary: Open Netty client and server on master Key: GIRAPH-313 URL: https://issues.apache.org/jira/browse/GIRAPH-313 Project: Giraph Issue Type: Improvement Reporter: Maja Kabiljo Assignee: Maja Kabiljo For GIRAPH-273 first thing we need is to open Netty communication on master, make connections to workers and make connections from workers to master. Since it's already significant amount of code I'm opening a separate issue for it. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (GIRAPH-313) Open Netty client and server on master
[ https://issues.apache.org/jira/browse/GIRAPH-313?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Maja Kabiljo updated GIRAPH-313: Attachment: GIRAPH-313.patch This doesn't send any messages for now, but I tested it with some dummy messages and it works. The patch adds all the client/server classes we had for worker for master, and does some redesign so common parts could be reused. Again there are some useNetty checks which we'll be able to remove soon. Passes mvn verify and pseudo distributed tests. > Open Netty client and server on master > -- > > Key: GIRAPH-313 > URL: https://issues.apache.org/jira/browse/GIRAPH-313 > Project: Giraph > Issue Type: Improvement >Reporter: Maja Kabiljo >Assignee: Maja Kabiljo > Attachments: GIRAPH-313.patch > > > For GIRAPH-273 first thing we need is to open Netty communication on master, > make connections to workers and make connections from workers to master. > Since it's already significant amount of code I'm opening a separate issue > for it. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
Review Request: Open Netty client and server on master
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/6753/ --- Review request for giraph. Description --- For GIRAPH-273 first thing we need is to open Netty communication on master, make connections to workers and make connections from workers to master. Since it's already significant amount of code I'm opening a separate issue for it. This doesn't send any messages for now, but I tested it with some dummy messages and it works. The patch adds all the client/server classes we had for worker for master, and does some redesign so common parts could be reused. Again there are some useNetty checks which we'll be able to remove soon. This addresses bug GIRAPH-313. https://issues.apache.org/jira/browse/GIRAPH-313 Diffs - http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/bsp/CentralizedServiceWorker.java 1375970 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/MasterClient.java PRE-CREATION http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/MasterClientServer.java PRE-CREATION http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/MasterRequest.java PRE-CREATION http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/MasterRequestServerHandler.java PRE-CREATION http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/MasterServer.java PRE-CREATION http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/NettyClient.java 1375970 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/NettyMasterClient.java PRE-CREATION http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/NettyMasterClientServer.java PRE-CREATION http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/NettyMasterServer.java PRE-CREATION http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/NettyServer.java 1375970 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/NettyWorkerClient.java 1375970 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/NettyWorkerServer.java 1375970 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/RequestDecoder.java 1375970 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/RequestEncoder.java 1375970 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/RequestInfo.java 1375970 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/RequestRegistry.java 1375970 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/RequestServerHandler.java 1375970 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/SendPartitionMessagesRequest.java 1375970 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/SendPartitionMutationsRequest.java 1375970 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/SendVertexRequest.java 1375970 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/WorkerRequest.java PRE-CREATION http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/WorkerRequestServerHandler.java PRE-CREATION http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/WritableRequest.java 1375970 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/graph/BspService.java 1375970 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/graph/BspServiceMaster.java 1375970 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/graph/BspServiceWorker.java 1375970 http://svn.apache.org/repos/asf/giraph/trunk/src/test/java/org/apache/giraph/comm/ConnectionTest.java 1375970 http://svn.apache.org/repos/asf/giraph/trunk/src/test/java/org/apache/giraph/comm/RequestFailureTest.java 1375970 http://svn.apache.org/repos/asf/giraph/trunk/src/test/java/org/apache/giraph/comm/RequestTest.java 1375970 Diff: https://reviews.apache.org/r/6753/diff/ Testing --- Passes mvn verify and pseudo distributed tests. Thanks, Maja Kabiljo
[jira] [Commented] (GIRAPH-313) Open Netty client and server on master
[ https://issues.apache.org/jira/browse/GIRAPH-313?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13440537#comment-13440537 ] Maja Kabiljo commented on GIRAPH-313: - https://reviews.apache.org/r/6753/ > Open Netty client and server on master > -- > > Key: GIRAPH-313 > URL: https://issues.apache.org/jira/browse/GIRAPH-313 > Project: Giraph > Issue Type: Improvement >Reporter: Maja Kabiljo >Assignee: Maja Kabiljo > Attachments: GIRAPH-313.patch > > > For GIRAPH-273 first thing we need is to open Netty communication on master, > make connections to workers and make connections from workers to master. > Since it's already significant amount of code I'm opening a separate issue > for it. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (GIRAPH-313) Open Netty client and server on master
[ https://issues.apache.org/jira/browse/GIRAPH-313?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13440553#comment-13440553 ] Maja Kabiljo commented on GIRAPH-313: - Also, we should create some packages in comm, it's becoming too big, I'll do it after this patch. > Open Netty client and server on master > -- > > Key: GIRAPH-313 > URL: https://issues.apache.org/jira/browse/GIRAPH-313 > Project: Giraph > Issue Type: Improvement >Reporter: Maja Kabiljo >Assignee: Maja Kabiljo > Attachments: GIRAPH-313.patch > > > For GIRAPH-273 first thing we need is to open Netty communication on master, > make connections to workers and make connections from workers to master. > Since it's already significant amount of code I'm opening a separate issue > for it. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (GIRAPH-273) Aggregators shouldn't use Zookeeper
[ https://issues.apache.org/jira/browse/GIRAPH-273?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13440637#comment-13440637 ] Eli Reisman commented on GIRAPH-273: I like the master connection, in-memory option. Avery is right, memory pressure on the master is low since it doesn't do much work during the super steps. This sounds like a great plan, and should be fairly quick in the super steps too. The aggregation tree is a great idea because we can piggyback aggregation connections on existing worker connections for many levels up the tree and hopefully only 2 workers will actually ever need to add extra connections to talk to the master. > Aggregators shouldn't use Zookeeper > --- > > Key: GIRAPH-273 > URL: https://issues.apache.org/jira/browse/GIRAPH-273 > Project: Giraph > Issue Type: Improvement >Reporter: Maja Kabiljo >Assignee: Maja Kabiljo > > We use Zookeeper znodes to transfer aggregated values from workers to master > and back. Zookeeper is supposed to be used for coordination, and it also has > a memory limit which prevents users from having aggregators with large value > objects. These are the reasons why we should implement aggregators gathering > and distribution in a different way. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (GIRAPH-314) Implement better message grouping to improve performance in SimpleTriangleClosingVertex
Eli Reisman created GIRAPH-314: -- Summary: Implement better message grouping to improve performance in SimpleTriangleClosingVertex Key: GIRAPH-314 URL: https://issues.apache.org/jira/browse/GIRAPH-314 Project: Giraph Issue Type: Improvement Components: examples Affects Versions: 0.2.0 Reporter: Eli Reisman Assignee: Eli Reisman Priority: Trivial Fix For: 0.2.0 After running SimpleTriangleClosingVertex at scale I'm thinking the sendMessageToAllEdges() is pretty in the code, but its not a good idea in practice since each vertex V sends degree(V)^2 messages right in the first superset in this algorithm. Could do something with a combiner etc. but just grouping messages by hand at the application level by using IntArrayListWritable again does the trick fine. Probably should have just done it this way before, but sendMessageToAllEdges() looked so nice. Sigh. Changed unit tests to reflect this new approach, passes mvn verify and cluster, etc. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (GIRAPH-314) Implement better message grouping to improve performance in SimpleTriangleClosingVertex
[ https://issues.apache.org/jira/browse/GIRAPH-314?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eli Reisman updated GIRAPH-314: --- Attachment: GIRAPH-314-1.patch > Implement better message grouping to improve performance in > SimpleTriangleClosingVertex > --- > > Key: GIRAPH-314 > URL: https://issues.apache.org/jira/browse/GIRAPH-314 > Project: Giraph > Issue Type: Improvement > Components: examples >Affects Versions: 0.2.0 >Reporter: Eli Reisman >Assignee: Eli Reisman >Priority: Trivial > Fix For: 0.2.0 > > Attachments: GIRAPH-314-1.patch > > > After running SimpleTriangleClosingVertex at scale I'm thinking the > sendMessageToAllEdges() is pretty in the code, but its not a good idea in > practice since each vertex V sends degree(V)^2 messages right in the first > superset in this algorithm. Could do something with a combiner etc. but just > grouping messages by hand at the application level by using > IntArrayListWritable again does the trick fine. > Probably should have just done it this way before, but > sendMessageToAllEdges() looked so nice. Sigh. Changed unit tests to reflect > this new approach, passes mvn verify and cluster, etc. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (GIRAPH-273) Aggregators shouldn't use Zookeeper
[ https://issues.apache.org/jira/browse/GIRAPH-273?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13440675#comment-13440675 ] Maja Kabiljo commented on GIRAPH-273: - We actually ended up with something better than aggregation tree. Say we have A aggregators and W workers. With the tree approach the whole aggregation would last for: A * (aggregation_time + transfer_time) * log W What we can do is perform aggregations in a completely distributed way. Each aggregator would have a worker which owns it and which does aggregation for it, so we would end up with about: A * (aggregation_time + transfer_time) After performing aggregations, all workers would send the final values to master, and after master.compute aggregators would go back the same way. In case of applications without master compute, we can even skip sending aggregated values to master all together. Is having all the workers connect to master an issue? Master will have the same number of connections as any other worker has, and in this approach we just send smaller amount of data through each of the connections, instead of having that same amount sent through just two. > Aggregators shouldn't use Zookeeper > --- > > Key: GIRAPH-273 > URL: https://issues.apache.org/jira/browse/GIRAPH-273 > Project: Giraph > Issue Type: Improvement >Reporter: Maja Kabiljo >Assignee: Maja Kabiljo > > We use Zookeeper znodes to transfer aggregated values from workers to master > and back. Zookeeper is supposed to be used for coordination, and it also has > a memory limit which prevents users from having aggregators with large value > objects. These are the reasons why we should implement aggregators gathering > and distribution in a different way. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira