Re: Review Request: GIRAPH-20 Move temporary test files from the project directory
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/5077/#review7756 --- Overall, looks great. Can you address the questions/comments and then I'll re-review? trunk/src/main/java/org/apache/giraph/graph/TextAggregatorWriter.java https://reviews.apache.org/r/5077/#comment17058 Just out of curiosity, why this change? trunk/src/main/java/org/apache/giraph/utils/FileUtils.java https://reviews.apache.org/r/5077/#comment17059 Why delete it? trunk/src/test/java/org/apache/giraph/BspCase.java https://reviews.apache.org/r/5077/#comment17060 Empty params and return. trunk/src/test/java/org/apache/giraph/BspCase.java https://reviews.apache.org/r/5077/#comment17061 Empty params and return. trunk/src/test/java/org/apache/giraph/BspCase.java https://reviews.apache.org/r/5077/#comment17062 Empty params and return. trunk/src/test/java/org/apache/giraph/BspCase.java https://reviews.apache.org/r/5077/#comment17063 Empty params and return. trunk/src/test/java/org/apache/giraph/BspCase.java https://reviews.apache.org/r/5077/#comment17064 @return trunk/src/test/java/org/apache/giraph/BspCase.java https://reviews.apache.org/r/5077/#comment17065 @return trunk/src/test/java/org/apache/giraph/TestBspBasic.java https://reviews.apache.org/r/5077/#comment17066 shouldn't it be 49 not 491? - Avery On 2012-05-09 11:37:47, Sebastian Schelter wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/5077/ --- (Updated 2012-05-09 11:37:47) Review request for giraph. Summary --- All temporary files that the tests generate are now written to /tmp/_giraphTests including zooKeeper files, checkpoints etc. This behavior will be automatically configured whenever InternalVertexRunner.run() or BspCase.prepareJob() is used. Usually I can't stop myself once I have my refactoring hat on, therefore I also tidied up a lot of minor stuff, removed code duplications etc. This addresses bug GIRAPH-20. https://issues.apache.org/jira/browse/GIRAPH-20 Diffs - trunk/src/test/java/org/apache/giraph/TestZooKeeperExt.java 1332106 trunk/src/test/java/org/apache/giraph/graph/TestEdgeListVertex.java 1332106 trunk/src/test/java/org/apache/giraph/TestJsonBase64Format.java 1332106 trunk/src/test/java/org/apache/giraph/TestManualCheckpoint.java 1332106 trunk/src/test/java/org/apache/giraph/TestMutateGraphVertex.java 1332106 trunk/src/test/java/org/apache/giraph/TestNotEnoughMapTasks.java 1332106 trunk/src/test/java/org/apache/giraph/TestGraphPartitioner.java 1332106 trunk/src/test/java/org/apache/giraph/TestAutoCheckpoint.java 1332106 trunk/src/test/java/org/apache/giraph/TestBspBasic.java 1332106 trunk/src/test/java/org/apache/giraph/BspCase.java 1332106 trunk/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java 1332106 trunk/src/main/java/org/apache/giraph/examples/SimplePageRankVertex.java 1332106 trunk/src/main/java/org/apache/giraph/graph/GraphMapper.java 1332106 trunk/src/main/java/org/apache/giraph/graph/TextAggregatorWriter.java 1332106 trunk/src/main/java/org/apache/giraph/utils/FileUtils.java PRE-CREATION Diff: https://reviews.apache.org/r/5077/diff Testing --- successfully passed local and pseudo-distributed tests with Hadoop 0.20.203 Thanks, Sebastian
Re: Review Request: GIRAPH-20 Move temporary test files from the project directory
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/5077/#review7772 --- Ship it! +1, thanks Sebastian! I'm assuming it also passes 'mvn verify', but please double check before you commit. - Avery On 2012-05-10 09:32:10, Sebastian Schelter wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/5077/ --- (Updated 2012-05-10 09:32:10) Review request for giraph. Summary --- All temporary files that the tests generate are now written to /tmp/_giraphTests including zooKeeper files, checkpoints etc. This behavior will be automatically configured whenever InternalVertexRunner.run() or BspCase.prepareJob() is used. Usually I can't stop myself once I have my refactoring hat on, therefore I also tidied up a lot of minor stuff, removed code duplications etc. This addresses bug GIRAPH-20. https://issues.apache.org/jira/browse/GIRAPH-20 Diffs - trunk/src/main/java/org/apache/giraph/examples/SimplePageRankVertex.java 1336504 trunk/src/main/java/org/apache/giraph/graph/GraphMapper.java 1336504 trunk/src/main/java/org/apache/giraph/graph/TextAggregatorWriter.java 1336504 trunk/src/main/java/org/apache/giraph/utils/FileUtils.java PRE-CREATION trunk/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java 1336504 trunk/src/test/java/org/apache/giraph/BspCase.java 1336504 trunk/src/test/java/org/apache/giraph/TestAutoCheckpoint.java 1336506 trunk/src/test/java/org/apache/giraph/TestBspBasic.java 1336504 trunk/src/test/java/org/apache/giraph/TestGraphPartitioner.java 1336504 trunk/src/test/java/org/apache/giraph/TestJsonBase64Format.java 1336504 trunk/src/test/java/org/apache/giraph/TestManualCheckpoint.java 1336506 trunk/src/test/java/org/apache/giraph/TestMutateGraphVertex.java 1336504 trunk/src/test/java/org/apache/giraph/TestNotEnoughMapTasks.java 1336504 trunk/src/test/java/org/apache/giraph/TestZooKeeperExt.java 1336504 trunk/src/test/java/org/apache/giraph/graph/TestEdgeListVertex.java 1336504 Diff: https://reviews.apache.org/r/5077/diff Testing --- successfully passed local and pseudo-distributed tests with Hadoop 0.20.203 Thanks, Sebastian
[jira] [Commented] (GIRAPH-37) Implement Netty-backed rpc solution
[ https://issues.apache.org/jira/browse/GIRAPH-37?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13271245#comment-13271245 ] Avery Ching commented on GIRAPH-37: --- Thanks Claudio. Here are more results with a scaled up 10 worker setup: Hadoop RPC: hadoop jar ~/giraph-0.2-SNAPSHOT-jar-with-dependencies.jar org.apache.giraph.benchmark.PageRankBenchmark -Dgiraph.useNetty=false -w 10 -V 1000 -s 5 -e 2 -v 12/05/09 02:32:05 INFO mapred.JobClient: Giraph Timers 12/05/09 02:32:05 INFO mapred.JobClient: Total (milliseconds)=149880 12/05/09 02:32:05 INFO mapred.JobClient: Superstep 3 (milliseconds)=21575 12/05/09 02:32:05 INFO mapred.JobClient: Setup (milliseconds)=7428 12/05/09 02:32:05 INFO mapred.JobClient: Shutdown (milliseconds)=174 12/05/09 02:32:05 INFO mapred.JobClient: Vertex input superstep (milliseconds)=39558 12/05/09 02:32:05 INFO mapred.JobClient: Superstep 0 (milliseconds)=16887 12/05/09 02:32:05 INFO mapred.JobClient: Superstep 4 (milliseconds)=18613 12/05/09 02:32:05 INFO mapred.JobClient: Superstep 5 (milliseconds)=3292 12/05/09 02:32:05 INFO mapred.JobClient: Superstep 2 (milliseconds)=21313 12/05/09 02:32:05 INFO mapred.JobClient: Superstep 1 (milliseconds)=21035 Netty: hadoop jar ~/giraph-0.2-SNAPSHOT-jar-with-dependencies.jar org.apache.giraph.benchmark.PageRankBenchmark -Dgiraph.useNetty=true -w 10 -V 1000 -s 5 -e 2 -v 12/05/09 02:35:06 INFO mapred.JobClient: Giraph Timers 12/05/09 02:35:06 INFO mapred.JobClient: Total (milliseconds)=59270 12/05/09 02:35:06 INFO mapred.JobClient: Superstep 3 (milliseconds)=11827 12/05/09 02:35:06 INFO mapred.JobClient: Setup (milliseconds)=3196 12/05/09 02:35:06 INFO mapred.JobClient: Shutdown (milliseconds)=124 12/05/09 02:35:06 INFO mapred.JobClient: Vertex input superstep (milliseconds)=13130 12/05/09 02:35:06 INFO mapred.JobClient: Superstep 0 (milliseconds)=8564 12/05/09 02:35:06 INFO mapred.JobClient: Superstep 4 (milliseconds)=5540 12/05/09 02:35:06 INFO mapred.JobClient: Superstep 5 (milliseconds)=2012 12/05/09 02:35:06 INFO mapred.JobClient: Superstep 2 (milliseconds)=8601 12/05/09 02:35:06 INFO mapred.JobClient: Superstep 1 (milliseconds)=6271 These results are fairly similar to the first set (even though there are more workers). I'm pretty sure we can squeeze more performance from Netty in the future in future patches (i.e. local send optimization is missing, tuning TCP parameters, exposing more knobs to the user, etc.). Implement Netty-backed rpc solution --- Key: GIRAPH-37 URL: https://issues.apache.org/jira/browse/GIRAPH-37 Project: Giraph Issue Type: New Feature Reporter: Jakob Homan Assignee: Jakob Homan Attachments: GIRAPH-37-wip.patch, GIRAPH-37.patch GIRAPH-12 considered replacing the current Hadoop based rpc method with Netty, but didn't went in another direction. I think there is still value in this approach, and will also look at Finagle. -- 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] [Resolved] (GIRAPH-37) Implement Netty-backed IPC
[ https://issues.apache.org/jira/browse/GIRAPH-37?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Avery Ching resolved GIRAPH-37. --- Resolution: Fixed Hudson is successful, closing. Implement Netty-backed IPC -- Key: GIRAPH-37 URL: https://issues.apache.org/jira/browse/GIRAPH-37 Project: Giraph Issue Type: New Feature Reporter: Jakob Homan Assignee: Avery Ching Attachments: GIRAPH-37-wip.patch, GIRAPH-37.patch GIRAPH-12 considered replacing the current Hadoop based rpc method with Netty, but didn't went in another direction. I think there is still value in this approach, and will also look at Finagle. -- 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-189) Synchronization on Map values should be in a thread safe object
Avery Ching created GIRAPH-189: -- Summary: Synchronization on Map values should be in a thread safe object Key: GIRAPH-189 URL: https://issues.apache.org/jira/browse/GIRAPH-189 Project: Giraph Issue Type: Improvement Reporter: Avery Ching See https://reviews.apache.org/r/5074/ for reasoning -- 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
Re: Review Request: GIRAPH-20 Move temporary test files from the project directory
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/5077/#review7745 --- Hey, Sebastian, overall looks good. If no one else gets to it, I'll finish this review tonight. - Avery On 2012-05-09 11:37:47, Sebastian Schelter wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/5077/ --- (Updated 2012-05-09 11:37:47) Review request for giraph. Summary --- All temporary files that the tests generate are now written to /tmp/_giraphTests including zooKeeper files, checkpoints etc. This behavior will be automatically configured whenever InternalVertexRunner.run() or BspCase.prepareJob() is used. Usually I can't stop myself once I have my refactoring hat on, therefore I also tidied up a lot of minor stuff, removed code duplications etc. This addresses bug GIRAPH-20. https://issues.apache.org/jira/browse/GIRAPH-20 Diffs - trunk/src/test/java/org/apache/giraph/TestZooKeeperExt.java 1332106 trunk/src/test/java/org/apache/giraph/graph/TestEdgeListVertex.java 1332106 trunk/src/test/java/org/apache/giraph/TestJsonBase64Format.java 1332106 trunk/src/test/java/org/apache/giraph/TestManualCheckpoint.java 1332106 trunk/src/test/java/org/apache/giraph/TestMutateGraphVertex.java 1332106 trunk/src/test/java/org/apache/giraph/TestNotEnoughMapTasks.java 1332106 trunk/src/test/java/org/apache/giraph/TestGraphPartitioner.java 1332106 trunk/src/test/java/org/apache/giraph/TestAutoCheckpoint.java 1332106 trunk/src/test/java/org/apache/giraph/TestBspBasic.java 1332106 trunk/src/test/java/org/apache/giraph/BspCase.java 1332106 trunk/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java 1332106 trunk/src/main/java/org/apache/giraph/examples/SimplePageRankVertex.java 1332106 trunk/src/main/java/org/apache/giraph/graph/GraphMapper.java 1332106 trunk/src/main/java/org/apache/giraph/graph/TextAggregatorWriter.java 1332106 trunk/src/main/java/org/apache/giraph/utils/FileUtils.java PRE-CREATION Diff: https://reviews.apache.org/r/5077/diff Testing --- successfully passed local and pseudo-distributed tests with Hadoop 0.20.203 Thanks, Sebastian
[jira] [Created] (GIRAPH-190) Create GiraphConf extends Configuration
Avery Ching created GIRAPH-190: -- Summary: Create GiraphConf extends Configuration Key: GIRAPH-190 URL: https://issues.apache.org/jira/browse/GIRAPH-190 Project: Giraph Issue Type: Improvement Reporter: Avery Ching Priority: Minor Currently all the options in Giraph are in the GiraphJob. It would be cleaner to do configuration as part of a special GiraphConf (analagous to HiveConf) and would simplify code elsewhere as well. -- 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
Re: [DISCUSS] Giraph graduation resolution
+1, sounds good to me. I also would like to have a rotating PMC chair. Avery On 5/4/12 10:26 AM, Jakob Homan wrote: Both suggestions sound reasonable to me. +1 on the current resolution. On Fri, May 4, 2012 at 10:23 AM, Owen O'Malleyomal...@apache.org wrote: On Fri, May 4, 2012 at 10:09 AM, Jakob Homanjgho...@gmail.com wrote: Looks good. I might suggest adding language to rotate the PMC chair annually to spread the responsibility around a bit and increase our Bus Number. Also, I was hoping to have seen Christian a bit more during incubation... Since this resolution is a one-off, I would suggest putting the anual rotation in the bylaws that will be part of the project's permanent website. That will make it more visible and easier for the project to change it itself. I agree that Christian hasn't been involved while it is in the incubator. On the other hand, he was heavily involved before it came to Apache. Looking at the svn logs, the number of commits (not contributions) per user is: 148 aching 47 jghoman 37 ckunz 11 claudio 7 exg 5 ssc 4 hyunsik 3 omalley 1 kunzchr 1 jmannix 1 ekoontz 1 asuresh So Christian only has 1 commit at Apache, but he has 37 prior. Given that level of involvement, I'd rather put him on the Giraph PMC and let him go emeritis in a few months. What do others think? -- Owen
[jira] [Commented] (GIRAPH-153) HBase/Accumulo Input and Output formats
[ https://issues.apache.org/jira/browse/GIRAPH-153?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13265155#comment-13265155 ] Avery Ching commented on GIRAPH-153: I'll take a look, sorry for the delay. HBase/Accumulo Input and Output formats --- Key: GIRAPH-153 URL: https://issues.apache.org/jira/browse/GIRAPH-153 Project: Giraph Issue Type: New Feature Components: bsp Affects Versions: 0.1.0 Environment: Single host OSX 10.6.8 2.2Ghz Intel i7, 8GB Reporter: Brian Femiano Attachments: GIRAPH-153.1.patch, GIRAPH-153.patch Four abstract classes that wrap their respective delegate input/output formats for easy hooks into vertex input format subclasses. I've included some sample programs that show two very simple graph algorithms. I have a graph generator that builds out a very simple directed structure, starting with a few 'root' nodes. Root nodes are defined as nodes which are not listed as a child anywhere in the graph. Algorithm 1) AccumuloRootMarker.java -- Accumulo as read/write source. Every vertex starts thinking it's a root. At superstep 0, send a message down to each child as a non-root notification. After superstep 1, only root nodes will have never been messaged. Algorithm 2) TableRootMarker -- HBase as read/write source. Expands on A1 by bundling the notification logic followed by root node propagation. Once we've marked the appropriate nodes as roots, tell every child which roots it can be traced back to via one or more spanning trees. This will take N + 2 supersteps where N is the maximum number of hops from any root to any leaf, plus 2 supersteps for the initial root flagging. I've included all relevant code plus DistributedCacheHelper.java for recursive cache file and archive searches. It is more hadoop centric than giraph, but these jobs use it so I figured why not commit here. These have been tested through local JobRunner, pseudo-distributed on the aforementioned hardware, and full distributed on EC2. More details in the comments. -- 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-153) HBase/Accumulo Input and Output formats
[ https://issues.apache.org/jira/browse/GIRAPH-153?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13265205#comment-13265205 ] Avery Ching commented on GIRAPH-153: Is this a fresh checkout? We shouldn't have to answer any questions like Reversed (or previously applied) patch detected! Assume -R. HBase/Accumulo Input and Output formats --- Key: GIRAPH-153 URL: https://issues.apache.org/jira/browse/GIRAPH-153 Project: Giraph Issue Type: New Feature Components: bsp Affects Versions: 0.1.0 Environment: Single host OSX 10.6.8 2.2Ghz Intel i7, 8GB Reporter: Brian Femiano Attachments: GIRAPH-153.1.patch, GIRAPH-153.patch Four abstract classes that wrap their respective delegate input/output formats for easy hooks into vertex input format subclasses. I've included some sample programs that show two very simple graph algorithms. I have a graph generator that builds out a very simple directed structure, starting with a few 'root' nodes. Root nodes are defined as nodes which are not listed as a child anywhere in the graph. Algorithm 1) AccumuloRootMarker.java -- Accumulo as read/write source. Every vertex starts thinking it's a root. At superstep 0, send a message down to each child as a non-root notification. After superstep 1, only root nodes will have never been messaged. Algorithm 2) TableRootMarker -- HBase as read/write source. Expands on A1 by bundling the notification logic followed by root node propagation. Once we've marked the appropriate nodes as roots, tell every child which roots it can be traced back to via one or more spanning trees. This will take N + 2 supersteps where N is the maximum number of hops from any root to any leaf, plus 2 supersteps for the initial root flagging. I've included all relevant code plus DistributedCacheHelper.java for recursive cache file and archive searches. It is more hadoop centric than giraph, but these jobs use it so I figured why not commit here. These have been tested through local JobRunner, pseudo-distributed on the aforementioned hardware, and full distributed on EC2. More details in the comments. -- 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
Re: Review Request: HBase/Accumulo Input and Output formats (on behalf of Brian)
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/4801/ --- (Updated 2012-04-30 23:54:55.758151) Review request for giraph. Changes --- Update of Brian's 153.2. Summary --- Brian's patch for GIRAPH-153. This addresses bug GIRAPH-153. https://issues.apache.org/jira/browse/GIRAPH-153 Diffs (updated) - http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/LICENSE.txt PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/license-header.txt PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/pom.xml PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/assembly/compile.xml PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/accumulo/AccumuloVertexInputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/accumulo/AccumuloVertexOutputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/accumulo/package-info.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/hbase/HBaseVertexInputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/hbase/HBaseVertexOutputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/hbase/package-info.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/BspCase.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/format/accumulo/TestAccumuloVertexFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/format/accumulo/edgemarker/AccumuloEdgeInputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/format/accumulo/edgemarker/AccumuloEdgeOutputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/format/hbase/TestHBaseRootMarkerVertextFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/format/hbase/edgemarker/TableEdgeInputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/format/hbase/edgemarker/TableEdgeOutputFormat.java PRE-CREATION Diff: https://reviews.apache.org/r/4801/diff Testing --- Thanks, Avery
Re: Review Request: HBase/Accumulo Input and Output formats (on behalf of Brian)
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/4801/#review7404 --- Hi Brian, the patch applies nicely, but it filled with duplicates. Also there are some javadoc indentation fixes to make. I just gave a couple of examples. Can you please fix this and resubmit? Thanks! http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/LICENSE.txt https://reviews.apache.org/r/4801/#comment16328 This license is duplicated several times. http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/license-header.txt https://reviews.apache.org/r/4801/#comment16329 This license is duplicated several times. http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/assembly/compile.xml https://reviews.apache.org/r/4801/#comment16330 More duplication. http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/accumulo/AccumuloVertexInputFormat.java https://reviews.apache.org/r/4801/#comment16331 Please fix indentation. http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/accumulo/AccumuloVertexInputFormat.java https://reviews.apache.org/r/4801/#comment16332 Please fix indentation. http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/accumulo/AccumuloVertexInputFormat.java https://reviews.apache.org/r/4801/#comment16333 Please fix indentation. http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/accumulo/AccumuloVertexInputFormat.java https://reviews.apache.org/r/4801/#comment16334 Please fix indentation. http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/accumulo/AccumuloVertexInputFormat.java https://reviews.apache.org/r/4801/#comment16335 Code duplication. http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/accumulo/AccumuloVertexInputFormat.java https://reviews.apache.org/r/4801/#comment16336 Extra * - Avery On 2012-04-30 23:54:55, Avery Ching wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/4801/ --- (Updated 2012-04-30 23:54:55) Review request for giraph. Summary --- Brian's patch for GIRAPH-153. This addresses bug GIRAPH-153. https://issues.apache.org/jira/browse/GIRAPH-153 Diffs - http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/LICENSE.txt PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/license-header.txt PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/pom.xml PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/assembly/compile.xml PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/accumulo/AccumuloVertexInputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/accumulo/AccumuloVertexOutputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/accumulo/package-info.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/hbase/HBaseVertexInputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/hbase/HBaseVertexOutputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/hbase/package-info.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/BspCase.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/format/accumulo/TestAccumuloVertexFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/format/accumulo/edgemarker/AccumuloEdgeInputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src
[jira] [Commented] (GIRAPH-185) Improve concurrency of putMsg / putMsgList
[ https://issues.apache.org/jira/browse/GIRAPH-185?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13261906#comment-13261906 ] Avery Ching commented on GIRAPH-185: I agree that a benchmark should be done, although I expect the impact to be very small. We should at least show it's not slower. =) Improve concurrency of putMsg / putMsgList -- Key: GIRAPH-185 URL: https://issues.apache.org/jira/browse/GIRAPH-185 Project: Giraph Issue Type: Improvement Components: graph Affects Versions: 0.2.0 Reporter: Bo Wang Assignee: Bo Wang Fix For: 0.2.0 Attachments: GIRAPH-185.patch, GIRAPH-185.patch Original Estimate: 2h Remaining Estimate: 2h Currently in putMsg / putMsgList, a synchronized closure is used to protect the whole transientInMessages when adding the new message. This lock prevents other concurrent calls to putMsg/putMsgList and increases the response time. We should use fine-grain locks to allow high concurrency in message communication. -- 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
Re: Review Request: Improve concurrency of putMsg / putMsgList
On 2012-04-24 20:53:33, Avery Ching wrote: http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/comm/BasicRPCCommunications.java, lines 776-777 https://reviews.apache.org/r/4852/diff/1/?file=104059#file104059line776 Bo, I'm a little leery about converting the List and ArrayList to LinkedList and ConcurrentLinkedList. I believe that linked list's will use more memory than the array list due to the double links (forward and backward). Also, is ConcurrentLinkedList supposted to outperform a synchronized ArrayList? I haven't seen much on that. The concurrenthashmap changes look good. Bo Wang wrote: Avery, thanks for the comments. I just measured the sizes of these classes and below are an estimation. java.util.ArrayList: 149 bytes java.util.LinkedList: 101 bytes java.util.concurrent.ConcurrentLinkedQueue: 118 bytes The tool I was using is a program from the link below. http://www.javapractices.com/topic/TopicAction.do?Id=83 In terms of performance, here is a benchmark. http://www.javacodegeeks.com/2010/09/java-best-practices-queue-battle-and.html In its test #1 (adding element), ConcurrentLinkedQueue performed slightly better than LinkedList. In test #3 (iterator), LinkedList outperformed ConcurrentLinkedQueue. I think the most time consuming part is add, while iteration is also heavily used but no concurrent accesses. Thanks for the response Bo. Those numbers are for the empty data structures I'm assuming. I was referring to the incremental cost of adding elements (messages) to the data structures. The performance isn't a a concern to me (unless we call size() somewhere). - Avery --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/4852/#review7185 --- On 2012-04-24 06:11:38, Bo Wang wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/4852/ --- (Updated 2012-04-24 06:11:38) Review request for giraph. Summary --- Use ConcurrentHashMap and ConcurrentLinkedQueue to allow concurrent assess to message map. The concurrencyLevel of ConcurrentHashMap uses the default value. There may be some performance gain by tuning this value. This addresses bug GIRAPH-185. https://issues.apache.org/jira/browse/GIRAPH-185 Diffs - http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/comm/BasicRPCCommunications.java 1328747 Diff: https://reviews.apache.org/r/4852/diff Testing --- Thanks, Bo
Re: Review Request: Improve concurrency of putMsg / putMsgList
On 2012-04-24 20:53:33, Avery Ching wrote: http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/comm/BasicRPCCommunications.java, lines 776-777 https://reviews.apache.org/r/4852/diff/1/?file=104059#file104059line776 Bo, I'm a little leery about converting the List and ArrayList to LinkedList and ConcurrentLinkedList. I believe that linked list's will use more memory than the array list due to the double links (forward and backward). Also, is ConcurrentLinkedList supposted to outperform a synchronized ArrayList? I haven't seen much on that. The concurrenthashmap changes look good. Bo Wang wrote: Avery, thanks for the comments. I just measured the sizes of these classes and below are an estimation. java.util.ArrayList: 149 bytes java.util.LinkedList: 101 bytes java.util.concurrent.ConcurrentLinkedQueue: 118 bytes The tool I was using is a program from the link below. http://www.javapractices.com/topic/TopicAction.do?Id=83 In terms of performance, here is a benchmark. http://www.javacodegeeks.com/2010/09/java-best-practices-queue-battle-and.html In its test #1 (adding element), ConcurrentLinkedQueue performed slightly better than LinkedList. In test #3 (iterator), LinkedList outperformed ConcurrentLinkedQueue. I think the most time consuming part is add, while iteration is also heavily used but no concurrent accesses. Avery Ching wrote: Thanks for the response Bo. Those numbers are for the empty data structures I'm assuming. I was referring to the incremental cost of adding elements (messages) to the data structures. The performance isn't a a concern to me (unless we call size() somewhere). By the incremental cost, I mean the memory cost, sorry. - Avery --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/4852/#review7185 --- On 2012-04-24 06:11:38, Bo Wang wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/4852/ --- (Updated 2012-04-24 06:11:38) Review request for giraph. Summary --- Use ConcurrentHashMap and ConcurrentLinkedQueue to allow concurrent assess to message map. The concurrencyLevel of ConcurrentHashMap uses the default value. There may be some performance gain by tuning this value. This addresses bug GIRAPH-185. https://issues.apache.org/jira/browse/GIRAPH-185 Diffs - http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/comm/BasicRPCCommunications.java 1328747 Diff: https://reviews.apache.org/r/4852/diff Testing --- Thanks, Bo
[jira] [Commented] (GIRAPH-185) Improve concurrency of putMsg / putMsgList
[ https://issues.apache.org/jira/browse/GIRAPH-185?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13259412#comment-13259412 ] Avery Ching commented on GIRAPH-185: Thanks for looking at this Bo. I have a couple of questions/comments. 1) Do you have any idea what kind of performance gain there is? Can you run a few experiments? 2) The one downside is memory related. By pre-allocating a list for every vertex, we are going to use memory, whether the vertex will receive a message or not. I thought you might be looking into a higher level concurrency by using ConcurrentHashMap or something like that for transientInMessages? Improve concurrency of putMsg / putMsgList -- Key: GIRAPH-185 URL: https://issues.apache.org/jira/browse/GIRAPH-185 Project: Giraph Issue Type: Improvement Components: graph Affects Versions: 0.2.0 Reporter: Bo Wang Assignee: Bo Wang Fix For: 0.2.0 Attachments: GIRAPH-185.patch Original Estimate: 2h Remaining Estimate: 2h Currently in putMsg / putMsgList, a synchronized closure is used to protect the whole transientInMessages when adding the new message. This lock prevents other concurrent calls to putMsg/putMsgList and increases the response time. We should use fine-grain locks to allow high concurrency in message communication. -- 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-185) Improve concurrency of putMsg / putMsgList
[ https://issues.apache.org/jira/browse/GIRAPH-185?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13259841#comment-13259841 ] Avery Ching commented on GIRAPH-185: Since we only allocate one ConcurrentHashMap per worker, the empty overhead isn't a concern. If however, the per element memory cost of a entry into the concurrent hash map is much more expensive then I would definitely be worried. We can also tune the concurrency level (default 16) to a reasonable tradeoff. Improve concurrency of putMsg / putMsgList -- Key: GIRAPH-185 URL: https://issues.apache.org/jira/browse/GIRAPH-185 Project: Giraph Issue Type: Improvement Components: graph Affects Versions: 0.2.0 Reporter: Bo Wang Assignee: Bo Wang Fix For: 0.2.0 Attachments: GIRAPH-185.patch Original Estimate: 2h Remaining Estimate: 2h Currently in putMsg / putMsgList, a synchronized closure is used to protect the whole transientInMessages when adding the new message. This lock prevents other concurrent calls to putMsg/putMsgList and increases the response time. We should use fine-grain locks to allow high concurrency in message communication. -- 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: HBase/Accumulo Input and Output formats (on behalf of Brian)
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/4801/ --- Review request for giraph. Summary --- Brian's patch for GIRAPH-153. This addresses bug GIRAPH-153. https://issues.apache.org/jira/browse/GIRAPH-153 Diffs - http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/LICENSE.txt PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/license-header.txt PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/pom.xml PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/assembly/compile.xml PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/accumulo/AccumuloVertexInputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/accumulo/AccumuloVertexOutputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/hbase/HBaseVertexInputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/hbase/HBaseVertexOutputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/BspCase.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/format/accumulo/TestAccumuloVertexFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/format/accumulo/edgemarker/AccumuloEdgeInputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/format/accumulo/edgemarker/AccumuloEdgeOutputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/format/hbase/TestHBaseRootMarkerVertextFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/format/hbase/edgemarker/TableEdgeInputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/format/hbase/edgemarker/TableEdgeOutputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/pom.xml 1327637 Diff: https://reviews.apache.org/r/4801/diff Testing --- Thanks, Avery
Re: Review Request: HBase/Accumulo Input and Output formats (on behalf of Brian)
that we don't forget any licenses here. - Avery On 2012-04-19 07:27:14, Avery Ching wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/4801/ --- (Updated 2012-04-19 07:27:14) Review request for giraph. Summary --- Brian's patch for GIRAPH-153. This addresses bug GIRAPH-153. https://issues.apache.org/jira/browse/GIRAPH-153 Diffs - http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/LICENSE.txt PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/license-header.txt PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/pom.xml PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/assembly/compile.xml PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/accumulo/AccumuloVertexInputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/accumulo/AccumuloVertexOutputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/hbase/HBaseVertexInputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/format/hbase/HBaseVertexOutputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/BspCase.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/format/accumulo/TestAccumuloVertexFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/format/accumulo/edgemarker/AccumuloEdgeInputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/format/accumulo/edgemarker/AccumuloEdgeOutputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/format/hbase/TestHBaseRootMarkerVertextFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/format/hbase/edgemarker/TableEdgeInputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/format/hbase/edgemarker/TableEdgeOutputFormat.java PRE-CREATION http://svn.apache.org/repos/asf/incubator/giraph/trunk/pom.xml 1327637 Diff: https://reviews.apache.org/r/4801/diff Testing --- Thanks, Avery
[jira] [Commented] (GIRAPH-153) HBase/Accumulo Input and Output formats
[ https://issues.apache.org/jira/browse/GIRAPH-153?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13257343#comment-13257343 ] Avery Ching commented on GIRAPH-153: Okay, so it's still tonight (even though it is 12:44 AM). =) Brian, I've done an initial look at the code on reviewboard https://reviews.apache.org/r/4801/. Please take a look. Thanks. HBase/Accumulo Input and Output formats --- Key: GIRAPH-153 URL: https://issues.apache.org/jira/browse/GIRAPH-153 Project: Giraph Issue Type: New Feature Components: bsp Affects Versions: 0.1.0 Environment: Single host OSX 10.6.8 2.2Ghz Intel i7, 8GB Reporter: Brian Femiano Attachments: GIRAPH-153.patch Four abstract classes that wrap their respective delegate input/output formats for easy hooks into vertex input format subclasses. I've included some sample programs that show two very simple graph algorithms. I have a graph generator that builds out a very simple directed structure, starting with a few 'root' nodes. Root nodes are defined as nodes which are not listed as a child anywhere in the graph. Algorithm 1) AccumuloRootMarker.java -- Accumulo as read/write source. Every vertex starts thinking it's a root. At superstep 0, send a message down to each child as a non-root notification. After superstep 1, only root nodes will have never been messaged. Algorithm 2) TableRootMarker -- HBase as read/write source. Expands on A1 by bundling the notification logic followed by root node propagation. Once we've marked the appropriate nodes as roots, tell every child which roots it can be traced back to via one or more spanning trees. This will take N + 2 supersteps where N is the maximum number of hops from any root to any leaf, plus 2 supersteps for the initial root flagging. I've included all relevant code plus DistributedCacheHelper.java for recursive cache file and archive searches. It is more hadoop centric than giraph, but these jobs use it so I figured why not commit here. These have been tested through local JobRunner, pseudo-distributed on the aforementioned hardware, and full distributed on EC2. More details in the comments. -- 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-180) Publish SNAPSHOTs and released artifacts in the Maven repository
[ https://issues.apache.org/jira/browse/GIRAPH-180?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13256246#comment-13256246 ] Avery Ching commented on GIRAPH-180: This is a good idea. The only question I would have though is would we publish different jars for every version of hadoop? Publish SNAPSHOTs and released artifacts in the Maven repository Key: GIRAPH-180 URL: https://issues.apache.org/jira/browse/GIRAPH-180 Project: Giraph Issue Type: Improvement Components: build Affects Versions: 0.1.0 Reporter: Paolo Castagna Priority: Minor Original Estimate: 4h Remaining Estimate: 4h Currently Giraph uses Maven to drive its build. However, no Maven artifacts nor SNAPSHOTs are published in the Apache Maven repository or Maven central. It would be useful to have Apache Giraph artifacts and SNAPSHOTs published and enable people to use Giraph without recompiling themselves. Right now users can checkout Giraph, mvn install it and use this for their dependency: dependency groupIdorg.apache.giraph/groupId artifactIdgiraph/artifactId version0.2-SNAPSHOT/version /dependency So, it's not that bad, but it can be better. :-) -- 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-153) HBase/Accumulo Input and Output formats
[ https://issues.apache.org/jira/browse/GIRAPH-153?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13256767#comment-13256767 ] Avery Ching commented on GIRAPH-153: I think hosting the submodule on github would produce one more barrier to entry. I prefer to have it with Giraph directly. Anyone else? HBase/Accumulo Input and Output formats --- Key: GIRAPH-153 URL: https://issues.apache.org/jira/browse/GIRAPH-153 Project: Giraph Issue Type: New Feature Components: bsp Affects Versions: 0.1.0 Environment: Single host OSX 10.6.8 2.2Ghz Intel i7, 8GB Reporter: Brian Femiano Attachments: GIRAPH-153.patch Four abstract classes that wrap their respective delegate input/output formats for easy hooks into vertex input format subclasses. I've included some sample programs that show two very simple graph algorithms. I have a graph generator that builds out a very simple directed structure, starting with a few 'root' nodes. Root nodes are defined as nodes which are not listed as a child anywhere in the graph. Algorithm 1) AccumuloRootMarker.java -- Accumulo as read/write source. Every vertex starts thinking it's a root. At superstep 0, send a message down to each child as a non-root notification. After superstep 1, only root nodes will have never been messaged. Algorithm 2) TableRootMarker -- HBase as read/write source. Expands on A1 by bundling the notification logic followed by root node propagation. Once we've marked the appropriate nodes as roots, tell every child which roots it can be traced back to via one or more spanning trees. This will take N + 2 supersteps where N is the maximum number of hops from any root to any leaf, plus 2 supersteps for the initial root flagging. I've included all relevant code plus DistributedCacheHelper.java for recursive cache file and archive searches. It is more hadoop centric than giraph, but these jobs use it so I figured why not commit here. These have been tested through local JobRunner, pseudo-distributed on the aforementioned hardware, and full distributed on EC2. More details in the comments. -- 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-153) HBase/Accumulo Input and Output formats
[ https://issues.apache.org/jira/browse/GIRAPH-153?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13256856#comment-13256856 ] Avery Ching commented on GIRAPH-153: I'll take a look at this patch tonight Brian. =) HBase/Accumulo Input and Output formats --- Key: GIRAPH-153 URL: https://issues.apache.org/jira/browse/GIRAPH-153 Project: Giraph Issue Type: New Feature Components: bsp Affects Versions: 0.1.0 Environment: Single host OSX 10.6.8 2.2Ghz Intel i7, 8GB Reporter: Brian Femiano Attachments: GIRAPH-153.patch Four abstract classes that wrap their respective delegate input/output formats for easy hooks into vertex input format subclasses. I've included some sample programs that show two very simple graph algorithms. I have a graph generator that builds out a very simple directed structure, starting with a few 'root' nodes. Root nodes are defined as nodes which are not listed as a child anywhere in the graph. Algorithm 1) AccumuloRootMarker.java -- Accumulo as read/write source. Every vertex starts thinking it's a root. At superstep 0, send a message down to each child as a non-root notification. After superstep 1, only root nodes will have never been messaged. Algorithm 2) TableRootMarker -- HBase as read/write source. Expands on A1 by bundling the notification logic followed by root node propagation. Once we've marked the appropriate nodes as roots, tell every child which roots it can be traced back to via one or more spanning trees. This will take N + 2 supersteps where N is the maximum number of hops from any root to any leaf, plus 2 supersteps for the initial root flagging. I've included all relevant code plus DistributedCacheHelper.java for recursive cache file and archive searches. It is more hadoop centric than giraph, but these jobs use it so I figured why not commit here. These have been tested through local JobRunner, pseudo-distributed on the aforementioned hardware, and full distributed on EC2. More details in the comments. -- 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] [Assigned] (GIRAPH-185) Improve concurrency of putMsg / putMsgList
[ https://issues.apache.org/jira/browse/GIRAPH-185?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Avery Ching reassigned GIRAPH-185: -- Assignee: Bo Wang Improve concurrency of putMsg / putMsgList -- Key: GIRAPH-185 URL: https://issues.apache.org/jira/browse/GIRAPH-185 Project: Giraph Issue Type: Improvement Components: graph Affects Versions: 0.2.0 Reporter: Bo Wang Assignee: Bo Wang Fix For: 0.2.0 Original Estimate: 2h Remaining Estimate: 2h Currently in putMsg / putMsgList, a synchronized closure is used to protect the whole transientInMessages when adding the new message. This lock prevents other concurrent calls to putMsg/putMsgList and increases the response time. We should use fine-grain locks to allow high concurrency in message communication. -- 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] [Assigned] (GIRAPH-186) Improve concurrency of putVertexList
[ https://issues.apache.org/jira/browse/GIRAPH-186?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Avery Ching reassigned GIRAPH-186: -- Assignee: Bo Wang Go for it Bo! Improve concurrency of putVertexList Key: GIRAPH-186 URL: https://issues.apache.org/jira/browse/GIRAPH-186 Project: Giraph Issue Type: Improvement Components: graph Affects Versions: 0.2.0 Reporter: Bo Wang Assignee: Bo Wang Fix For: 0.2.0 Original Estimate: 2h Remaining Estimate: 2h It's pretty similar to GIRAPH-185. The whole inPartitionVertexMap is locked when there is a call to it. We should allow multiple calls adding different partitions to the same worker at the same time. -- 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-181) Add Hadoop 1.0 profile to pom.xml
[ https://issues.apache.org/jira/browse/GIRAPH-181?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13254890#comment-13254890 ] Avery Ching commented on GIRAPH-181: +1, committed. Add Hadoop 1.0 profile to pom.xml - Key: GIRAPH-181 URL: https://issues.apache.org/jira/browse/GIRAPH-181 Project: Giraph Issue Type: Improvement Components: build Affects Versions: 0.2.0 Reporter: Eugene Koontz Assignee: Eugene Koontz Fix For: 0.2.0 Attachments: GIRAPH-181.patch, GIRAPH-181.patch Hadoop 1.0.x is now considered the current stable version of Hadoop, according to http://hadoop.apache.org/common/releases.html#Download . This JIRA is to add support within Giraph's maven profile for the 1.0.x Hadoop release. -- 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-184) Upgrade to junit4
[ https://issues.apache.org/jira/browse/GIRAPH-184?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13254152#comment-13254152 ] Avery Ching commented on GIRAPH-184: Thanks! Upgrade to junit4 - Key: GIRAPH-184 URL: https://issues.apache.org/jira/browse/GIRAPH-184 Project: Giraph Issue Type: Bug Reporter: Devaraj K Assignee: Devaraj K Presently Giraph uses JUnit 3.8.1. We can upgrade to JUnit 4 -- 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-183) Add Claudio's FOSDEM presentation (slides and video) to the site
[ https://issues.apache.org/jira/browse/GIRAPH-183?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13253488#comment-13253488 ] Avery Ching commented on GIRAPH-183: +1. This is great stuff Claudio. Add Claudio's FOSDEM presentation (slides and video) to the site Key: GIRAPH-183 URL: https://issues.apache.org/jira/browse/GIRAPH-183 Project: Giraph Issue Type: Improvement Components: site Reporter: Claudio Martella Assignee: Claudio Martella Priority: Trivial Labels: newbie Attachments: GIRAPH-183.diff Presentation: http://prezi.com/9ake_klzwrga/apache-giraph-distributed-graph-processing-in-the-cloud/ Video: http://www.youtube.com/watch?v=3ZrqPEIPRe4, http://www.youtube.com/watch?v=BmRaejKGeDM -- 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-183) Add Claudio's FOSDEM presentation (slides and video) to the site
[ https://issues.apache.org/jira/browse/GIRAPH-183?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13253655#comment-13253655 ] Avery Ching commented on GIRAPH-183: Are the problems related to GIRAPH-168? Add Claudio's FOSDEM presentation (slides and video) to the site Key: GIRAPH-183 URL: https://issues.apache.org/jira/browse/GIRAPH-183 Project: Giraph Issue Type: Improvement Components: site Reporter: Claudio Martella Assignee: Claudio Martella Priority: Trivial Labels: newbie Attachments: GIRAPH-183.diff Presentation: http://prezi.com/9ake_klzwrga/apache-giraph-distributed-graph-processing-in-the-cloud/ Video: http://www.youtube.com/watch?v=3ZrqPEIPRe4, http://www.youtube.com/watch?v=BmRaejKGeDM -- 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
Re: [DISCUSS] Giraph Graduation (was Re: Giraph status (Was: [Incubator Wiki] Update of April2012 by OwenOmalley))
Sounds good to me. Avery On 4/12/12 11:06 AM, Jakob Homan wrote: +1. It's time to fly the nest. On Wed, Apr 11, 2012 at 11:05 PM, Mattmann, Chris A (388J) chris.a.mattm...@jpl.nasa.gov wrote: [moving to giraph-dev@ and dropping general@incubator] Hey Owen, On Apr 11, 2012, at 9:44 PM, Owen O'Malley wrote: Looking at the report and recent project activity it looks like Giraph is doing pretty well. In fact I can't spot any obvious graduation blockers. Anything I'm missing? If not, have you already started preparing for graduation? We haven't started yet, but I was thinking along similar lines. Should I start a vote on the dev list? +1 on this line of thinking too. How about we: 1. have a [DISCUSS] thread, wait ~48-72 hours, let it settle, if all well; then 2. have a [VOTE] thread on giraph-dev, wait ~48-72 hours, let it settle, if all well; then 3. have a [VOTE] on general@, pasting the discussion thus far, and tallying the VOTEs During this time, we'll need to: a. Draft a resolution for Giraph's graduation. b. Nominate a chair. If there are no objections, I nominate either Avery, or Jakob. Both have really stepped it up along the way and would make a fine chair. Cheers, Chris ++ Chris Mattmann, Ph.D. Senior Computer Scientist NASA Jet Propulsion Laboratory Pasadena, CA 91109 USA Office: 171-266B, Mailstop: 171-246 Email: chris.a.mattm...@nasa.gov WWW: http://sunset.usc.edu/~mattmann/ ++ Adjunct Assistant Professor, Computer Science Department University of Southern California, Los Angeles, CA 90089 USA ++
[jira] [Commented] (GIRAPH-168) Simplify munge directive usage with new munge flag HADOOP_SECURE (rather than HADOOP_FACEBOOK) and remove usage of HADOOP
[ https://issues.apache.org/jira/browse/GIRAPH-168?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13250476#comment-13250476 ] Avery Ching commented on GIRAPH-168: Eugene, I committed your patch, which passed 'mvn verify', however, seems to have changed the way the Junit test report somehow. Here's the result after your patch (99) Recording test results No test report files were found. Configuration error? Build step 'Publish JUnit test result report' changed build result to FAILURE Updating GIRAPH-168 Finished: FAILURE https://builds.apache.org/job/Giraph-trunk-Commit/99/ The last commit seemed to have the JUnit test result reports just fine (https://builds.apache.org/job/Giraph-trunk-Commit/98/). Can you please take a look? Simplify munge directive usage with new munge flag HADOOP_SECURE (rather than HADOOP_FACEBOOK) and remove usage of HADOOP - Key: GIRAPH-168 URL: https://issues.apache.org/jira/browse/GIRAPH-168 Project: Giraph Issue Type: Improvement Affects Versions: 0.2.0 Reporter: Eugene Koontz Assignee: Eugene Koontz Attachments: GIRAPH-168.patch, GIRAPH-168.patch, GIRAPH-168.patch, GIRAPH-168.patch, GIRAPH-168.patch, GIRAPH-168.patch This JIRA relates to the mail thread here: http://mail-archives.apache.org/mod_mbox/incubator-giraph-dev/201203.mbox/browser Currently we check for the munge flags HADOOP, HADOOP_FACEBOOK and HADOOP_NON_SECURE when using munge in a few places. Hopefully we can eliminate usage of munge in the future, but until then, we can mitigate the complexity by consolidating the number of flags checked. This JIRA renames HADOOP_FACEBOOK to HADOOP_SECURE, and removes usages of HADOOP, to handle the same conditional compilation requirements. It also makes it easier to add more maven profiles so that we can easily increase our hadoop version coverage. This patch modifies the existing hadoop_facebook profile to use the new HADOOP_SECURE munge flag, rather than HADOOP_FACEBOOK. It also adds a new hadoop maven profile, hadoop_trunk, which also sets HADOOP_SECURE. Finally, it adds a default profile, hadoop_0.20.203. This is needed so that we can specify its dependencies separately from hadoop_trunk, because the hadoop dependencies have changed between trunk and 0.205.0 - the former requires hadoop-common, hadoop-mapreduce-client-core, and hadoop-mapreduce-client-common, whereas the latter requires hadoop-core. With this patch, the following passes: {code} mvn clean verify mvn -Phadoop_trunk clean verify mvn -Phadoop_0.20.203 clean verify {code} Current problems: * I left in place the usage of HADOOP_NON_SECURE, but note that the profile that uses this is hadoop_non_secure, which fails to compile on trunk: https://issues.apache.org/jira/browse/GIRAPH-167 . * I couldn't get -Phadoop_facebook to work; does this work outside of Facebook? -- 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-168) Simplify munge directive usage with new munge flag HADOOP_SECURE (rather than HADOOP_FACEBOOK) and remove usage of HADOOP
[ https://issues.apache.org/jira/browse/GIRAPH-168?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13250874#comment-13250874 ] Avery Ching commented on GIRAPH-168: I can modify Hudson to do execute the commands you used above. Any thoughts/comments? Simplify munge directive usage with new munge flag HADOOP_SECURE (rather than HADOOP_FACEBOOK) and remove usage of HADOOP - Key: GIRAPH-168 URL: https://issues.apache.org/jira/browse/GIRAPH-168 Project: Giraph Issue Type: Improvement Affects Versions: 0.2.0 Reporter: Eugene Koontz Assignee: Eugene Koontz Attachments: GIRAPH-168.patch, GIRAPH-168.patch, GIRAPH-168.patch, GIRAPH-168.patch, GIRAPH-168.patch, GIRAPH-168.patch This JIRA relates to the mail thread here: http://mail-archives.apache.org/mod_mbox/incubator-giraph-dev/201203.mbox/browser Currently we check for the munge flags HADOOP, HADOOP_FACEBOOK and HADOOP_NON_SECURE when using munge in a few places. Hopefully we can eliminate usage of munge in the future, but until then, we can mitigate the complexity by consolidating the number of flags checked. This JIRA renames HADOOP_FACEBOOK to HADOOP_SECURE, and removes usages of HADOOP, to handle the same conditional compilation requirements. It also makes it easier to add more maven profiles so that we can easily increase our hadoop version coverage. This patch modifies the existing hadoop_facebook profile to use the new HADOOP_SECURE munge flag, rather than HADOOP_FACEBOOK. It also adds a new hadoop maven profile, hadoop_trunk, which also sets HADOOP_SECURE. Finally, it adds a default profile, hadoop_0.20.203. This is needed so that we can specify its dependencies separately from hadoop_trunk, because the hadoop dependencies have changed between trunk and 0.205.0 - the former requires hadoop-common, hadoop-mapreduce-client-core, and hadoop-mapreduce-client-common, whereas the latter requires hadoop-core. With this patch, the following passes: {code} mvn clean verify mvn -Phadoop_trunk clean verify mvn -Phadoop_0.20.203 clean verify {code} Current problems: * I left in place the usage of HADOOP_NON_SECURE, but note that the profile that uses this is hadoop_non_secure, which fails to compile on trunk: https://issues.apache.org/jira/browse/GIRAPH-167 . * I couldn't get -Phadoop_facebook to work; does this work outside of Facebook? -- 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-168) Simplify munge directive usage with new munge flag HADOOP_SECURE (rather than HADOOP_FACEBOOK) and remove usage of HADOOP
[ https://issues.apache.org/jira/browse/GIRAPH-168?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13250947#comment-13250947 ] Avery Ching commented on GIRAPH-168: I would ignore the facebook one for now (we can add it later), but I can try mvn -Phadoop_non_secure clean verify mvn -Phadoop_0.20.203 clean verify mvn clean verify mvn -Phadoop_0.23 clean verify mvn -Phadoop_trunk clean verify Simplify munge directive usage with new munge flag HADOOP_SECURE (rather than HADOOP_FACEBOOK) and remove usage of HADOOP - Key: GIRAPH-168 URL: https://issues.apache.org/jira/browse/GIRAPH-168 Project: Giraph Issue Type: Improvement Affects Versions: 0.2.0 Reporter: Eugene Koontz Assignee: Eugene Koontz Attachments: GIRAPH-168.patch, GIRAPH-168.patch, GIRAPH-168.patch, GIRAPH-168.patch, GIRAPH-168.patch, GIRAPH-168.patch This JIRA relates to the mail thread here: http://mail-archives.apache.org/mod_mbox/incubator-giraph-dev/201203.mbox/browser Currently we check for the munge flags HADOOP, HADOOP_FACEBOOK and HADOOP_NON_SECURE when using munge in a few places. Hopefully we can eliminate usage of munge in the future, but until then, we can mitigate the complexity by consolidating the number of flags checked. This JIRA renames HADOOP_FACEBOOK to HADOOP_SECURE, and removes usages of HADOOP, to handle the same conditional compilation requirements. It also makes it easier to add more maven profiles so that we can easily increase our hadoop version coverage. This patch modifies the existing hadoop_facebook profile to use the new HADOOP_SECURE munge flag, rather than HADOOP_FACEBOOK. It also adds a new hadoop maven profile, hadoop_trunk, which also sets HADOOP_SECURE. Finally, it adds a default profile, hadoop_0.20.203. This is needed so that we can specify its dependencies separately from hadoop_trunk, because the hadoop dependencies have changed between trunk and 0.205.0 - the former requires hadoop-common, hadoop-mapreduce-client-core, and hadoop-mapreduce-client-common, whereas the latter requires hadoop-core. With this patch, the following passes: {code} mvn clean verify mvn -Phadoop_trunk clean verify mvn -Phadoop_0.20.203 clean verify {code} Current problems: * I left in place the usage of HADOOP_NON_SECURE, but note that the profile that uses this is hadoop_non_secure, which fails to compile on trunk: https://issues.apache.org/jira/browse/GIRAPH-167 . * I couldn't get -Phadoop_facebook to work; does this work outside of Facebook? -- 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-182) Provide SequenceFileVertexOutputFormat as an available OutputFormat
[ https://issues.apache.org/jira/browse/GIRAPH-182?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13251180#comment-13251180 ] Avery Ching commented on GIRAPH-182: Agreed, would you like to work on it Pradeep? Provide SequenceFileVertexOutputFormat as an available OutputFormat --- Key: GIRAPH-182 URL: https://issues.apache.org/jira/browse/GIRAPH-182 Project: Giraph Issue Type: New Feature Components: lib Reporter: Pradeep Gollakota Priority: Minor SequenceFile's are heavily used in Hadoop. We should provide SequenceFileVertexOutputFormat. Since SequenceFileVertexInputFormat is already provided, it makes sense to also provide a mirroring OutputFormat -- 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] [Assigned] (GIRAPH-182) Provide SequenceFileVertexOutputFormat as an available OutputFormat
[ https://issues.apache.org/jira/browse/GIRAPH-182?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Avery Ching reassigned GIRAPH-182: -- Assignee: Pradeep Gollakota Great! Provide SequenceFileVertexOutputFormat as an available OutputFormat --- Key: GIRAPH-182 URL: https://issues.apache.org/jira/browse/GIRAPH-182 Project: Giraph Issue Type: New Feature Components: lib Reporter: Pradeep Gollakota Assignee: Pradeep Gollakota Priority: Minor SequenceFile's are heavily used in Hadoop. We should provide SequenceFileVertexOutputFormat. Since SequenceFileVertexInputFormat is already provided, it makes sense to also provide a mirroring OutputFormat -- 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-168) Simplify munge directive usage with new munge flag HADOOP_SECURE (rather than HADOOP_FACEBOOK) and remove usage of HADOOP
[ https://issues.apache.org/jira/browse/GIRAPH-168?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13249702#comment-13249702 ] Avery Ching commented on GIRAPH-168: Nice that you got it working with all the versions! One question though, why is the line below needed in pom.xml? org.apache.hadoop.giraph.zkJargiraph-0.2-SNAPSHOT-jar-with-dependencies.jar/org.apache.hadoop.giraph.zkJar Simplify munge directive usage with new munge flag HADOOP_SECURE (rather than HADOOP_FACEBOOK) and remove usage of HADOOP - Key: GIRAPH-168 URL: https://issues.apache.org/jira/browse/GIRAPH-168 Project: Giraph Issue Type: Improvement Affects Versions: 0.2.0 Reporter: Eugene Koontz Assignee: Eugene Koontz Attachments: GIRAPH-168.patch, GIRAPH-168.patch, GIRAPH-168.patch, GIRAPH-168.patch, GIRAPH-168.patch This JIRA relates to the mail thread here: http://mail-archives.apache.org/mod_mbox/incubator-giraph-dev/201203.mbox/browser Currently we check for the munge flags HADOOP, HADOOP_FACEBOOK and HADOOP_NON_SECURE when using munge in a few places. Hopefully we can eliminate usage of munge in the future, but until then, we can mitigate the complexity by consolidating the number of flags checked. This JIRA renames HADOOP_FACEBOOK to HADOOP_SECURE, and removes usages of HADOOP, to handle the same conditional compilation requirements. It also makes it easier to add more maven profiles so that we can easily increase our hadoop version coverage. This patch modifies the existing hadoop_facebook profile to use the new HADOOP_SECURE munge flag, rather than HADOOP_FACEBOOK. It also adds a new hadoop maven profile, hadoop_trunk, which also sets HADOOP_SECURE. Finally, it adds a default profile, hadoop_0.20.203. This is needed so that we can specify its dependencies separately from hadoop_trunk, because the hadoop dependencies have changed between trunk and 0.205.0 - the former requires hadoop-common, hadoop-mapreduce-client-core, and hadoop-mapreduce-client-common, whereas the latter requires hadoop-core. With this patch, the following passes: {code} mvn clean verify mvn -Phadoop_trunk clean verify mvn -Phadoop_0.20.203 clean verify {code} Current problems: * I left in place the usage of HADOOP_NON_SECURE, but note that the profile that uses this is hadoop_non_secure, which fails to compile on trunk: https://issues.apache.org/jira/browse/GIRAPH-167 . * I couldn't get -Phadoop_facebook to work; does this work outside of Facebook? -- 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-168) Simplify munge directive usage with new munge flag HADOOP_SECURE (rather than HADOOP_FACEBOOK) and remove usage of HADOOP
[ https://issues.apache.org/jira/browse/GIRAPH-168?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13250224#comment-13250224 ] Avery Ching commented on GIRAPH-168: +1. Given this is a somewhat large change, I'll wait until tonight to see if anyone opposes it. If not, I'll commit. Simplify munge directive usage with new munge flag HADOOP_SECURE (rather than HADOOP_FACEBOOK) and remove usage of HADOOP - Key: GIRAPH-168 URL: https://issues.apache.org/jira/browse/GIRAPH-168 Project: Giraph Issue Type: Improvement Affects Versions: 0.2.0 Reporter: Eugene Koontz Assignee: Eugene Koontz Attachments: GIRAPH-168.patch, GIRAPH-168.patch, GIRAPH-168.patch, GIRAPH-168.patch, GIRAPH-168.patch, GIRAPH-168.patch This JIRA relates to the mail thread here: http://mail-archives.apache.org/mod_mbox/incubator-giraph-dev/201203.mbox/browser Currently we check for the munge flags HADOOP, HADOOP_FACEBOOK and HADOOP_NON_SECURE when using munge in a few places. Hopefully we can eliminate usage of munge in the future, but until then, we can mitigate the complexity by consolidating the number of flags checked. This JIRA renames HADOOP_FACEBOOK to HADOOP_SECURE, and removes usages of HADOOP, to handle the same conditional compilation requirements. It also makes it easier to add more maven profiles so that we can easily increase our hadoop version coverage. This patch modifies the existing hadoop_facebook profile to use the new HADOOP_SECURE munge flag, rather than HADOOP_FACEBOOK. It also adds a new hadoop maven profile, hadoop_trunk, which also sets HADOOP_SECURE. Finally, it adds a default profile, hadoop_0.20.203. This is needed so that we can specify its dependencies separately from hadoop_trunk, because the hadoop dependencies have changed between trunk and 0.205.0 - the former requires hadoop-common, hadoop-mapreduce-client-core, and hadoop-mapreduce-client-common, whereas the latter requires hadoop-core. With this patch, the following passes: {code} mvn clean verify mvn -Phadoop_trunk clean verify mvn -Phadoop_0.20.203 clean verify {code} Current problems: * I left in place the usage of HADOOP_NON_SECURE, but note that the profile that uses this is hadoop_non_secure, which fails to compile on trunk: https://issues.apache.org/jira/browse/GIRAPH-167 . * I couldn't get -Phadoop_facebook to work; does this work outside of Facebook? -- 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-171) total time in MasterThread.run() is calculated incorrectly
[ https://issues.apache.org/jira/browse/GIRAPH-171?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13248175#comment-13248175 ] Avery Ching commented on GIRAPH-171: +1 Argh, it is inconsistent with the counter, GIRAPH_TIMERS_COUNTER_GROUP_NAME. Thanks for the fix Eugene! total time in MasterThread.run() is calculated incorrectly -- Key: GIRAPH-171 URL: https://issues.apache.org/jira/browse/GIRAPH-171 Project: Giraph Issue Type: Bug Reporter: Eugene Koontz Assignee: Eugene Koontz Attachments: GIRAPH-171.patch While running PageMarkBenchMark, I was seeing in the output: {{graph.MasterThread(172): total: Took 1.3336739262910001E9 seconds.}} This was because currently, in {{MasterThread.run()}}, we have: {code} LOG.info(total: Took + ((System.currentTimeMillis() / 1000.0d) - setupSecs) + seconds.); {code} but it should be: {code} LOG.info(total: Took + ((System.currentTimeMillis() - startMillis) / 1000.0d) + seconds.); {code} -- 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
Re: On helping new contributors pitch in quickly...
Dan, you're definitely right that this has been mentioned a few times. The multigraph issue is one part of it, but a helper VertexInputFormat (and maybe VertexOutputFormat) would certainly still help as you mention. Can you please open a JIRA (and help if you have time)? Avery On 4/5/12 1:49 AM, Dan Brickley wrote: On 5 April 2012 05:49, Jakob Homanjgho...@gmail.com wrote: Ack!, I suck. Sorry. I hadn't realized we'd gone through most of them, which itself is a good thing. I'll get some new ones added first thing in the morning. Sorry. Do we have something around document a workflow to get RDF graph data into Giraph?. A few of us have been talking about it here or there, and I've heard various strategies mentioned (e.g. Ntriples as it's a simple line-oriented format; piggybacking on HBase or other storage that Giraph already has adaptors for; integrating Apache Jena; ...). I can't find much in JIRA but https://issues.apache.org/jira/browse/GIRAPH-141 touches on the issue (since we can't currently easily represent fully general RDF graphs since two nodes might be connected by more than one typed edge). Even without multigraphs it ought to be possible to bring RDF-sourced data into Giraph, e.g. perhaps some app is only interested in say the Movies + People subset of a big RDF collection. And so perhaps most of the work is in preprocessing for now - e.g. via Ntriples + Pig; but still it would be great to have a clear HOWTO. As an interested party on the periphery, a JIRA for this would give a natural place to monitor, read up, maybe even help. And I'm sure I'm not alone... cheers, Dan
Re: On helping new contributors pitch in quickly...
Here is a related JIRA https://issues.apache.org/jira/browse/GIRAPH-155 Avery On 4/5/12 9:45 AM, Paolo Castagna wrote: Hi Dan, I have not an answer to your questions/observations yet. However, I suspect N-Triples | N-Quads might not be the best option for something like Giraph. Something more like an adjacency list might be better. So, my intuition, is that if you start with RDF in N-Triples format, the first step would be a simple MapReduce job to group RDF statements by subject (eventually filtering out certain properties): Input: s1 --p1-- o1 s1 --p2-- o2 s1 --p2-- o3 s2 ... Output (adjacency list): s1 (p1 o1) (p2 o2) (p2 o3) s2 ... But, as I said, is it too early for me to say definitely this is the best approach. Paolo Dan Brickley wrote: On 5 April 2012 05:49, Jakob Homanjgho...@gmail.com wrote: Ack!, I suck. Sorry. I hadn't realized we'd gone through most of them, which itself is a good thing. I'll get some new ones added first thing in the morning. Sorry. Do we have something around document a workflow to get RDF graph data into Giraph?. A few of us have been talking about it here or there, and I've heard various strategies mentioned (e.g. Ntriples as it's a simple line-oriented format; piggybacking on HBase or other storage that Giraph already has adaptors for; integrating Apache Jena; ...). I can't find much in JIRA but https://issues.apache.org/jira/browse/GIRAPH-141 touches on the issue (since we can't currently easily represent fully general RDF graphs since two nodes might be connected by more than one typed edge). Even without multigraphs it ought to be possible to bring RDF-sourced data into Giraph, e.g. perhaps some app is only interested in say the Movies + People subset of a big RDF collection. And so perhaps most of the work is in preprocessing for now - e.g. via Ntriples + Pig; but still it would be great to have a clear HOWTO. As an interested party on the periphery, a JIRA for this would give a natural place to monitor, read up, maybe even help. And I'm sure I'm not alone... cheers, Dan
[jira] [Commented] (GIRAPH-77) Coordinator should expose a web interface with progress, vertex region assignments, etc.
[ https://issues.apache.org/jira/browse/GIRAPH-77?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13246561#comment-13246561 ] Avery Ching commented on GIRAPH-77: --- Paolo, would you be interested in working on this? =) Coordinator should expose a web interface with progress, vertex region assignments, etc. Key: GIRAPH-77 URL: https://issues.apache.org/jira/browse/GIRAPH-77 Project: Giraph Issue Type: New Feature Reporter: Jakob Homan It would be nice if the coordinator worker had a web interface that showed progress, splits, etc. during job execution. Right now it would duplicate information currently being exposed through task status, but with the move to YARN, it will be a necessity. It would be great if we could do this in a modern way to avoid the screen-scraping, etc. currently used to get information from most other Hadoop project's web interfaces. The coordinator could announce its address at the beginning or via status updates. -- 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
Re: Giraph as Whirr service, see WHIRR-530
I don't use Whirr...I haven't heard it mentioned on this forum yet. Anyone? Avery On 4/4/12 9:30 PM, Paolo Castagna wrote: Hi, seen this? WHIRR-530 - Add Giraph as a service https://issues.apache.org/jira/browse/WHIRR-530 This could be quite useful for users who want to give Giraph a spin on cloud infrastructure, just for testing or to run a few small experiments. My experience with Whirr an small 10-20 nodes clusters has be quite positive. Less so for larger clusters, but it more a problem/limit with the cloud provider rather than Whirr itself. I think. Whirr makes extremely easy and pleasant deploy stuff on-demand. ... and Whirr already supports YARN: https://issues.apache.org/jira/browse/WHIRR-391 Is any Giraph developers/users here also a Whirr user? Paolo
[jira] [Commented] (GIRAPH-153) HBase/Accumulo Input and Output formats
[ https://issues.apache.org/jira/browse/GIRAPH-153?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13245480#comment-13245480 ] Avery Ching commented on GIRAPH-153: From what you've described, sounds good to me. In the worst case, we can change it to a submodule if that makes more sense in the future. I would like to use a similar approach for https://issues.apache.org/jira/browse/GIRAPH-93, as Jakob mentioned. HBase/Accumulo Input and Output formats --- Key: GIRAPH-153 URL: https://issues.apache.org/jira/browse/GIRAPH-153 Project: Giraph Issue Type: New Feature Components: bsp Affects Versions: 0.1.0 Environment: Single host OSX 10.6.8 2.2Ghz Intel i7, 8GB Reporter: Brian Femiano Four abstract classes that wrap their respective delegate input/output formats for easy hooks into vertex input format subclasses. I've included some sample programs that show two very simple graph algorithms. I have a graph generator that builds out a very simple directed structure, starting with a few 'root' nodes. Root nodes are defined as nodes which are not listed as a child anywhere in the graph. Algorithm 1) AccumuloRootMarker.java -- Accumulo as read/write source. Every vertex starts thinking it's a root. At superstep 0, send a message down to each child as a non-root notification. After superstep 1, only root nodes will have never been messaged. Algorithm 2) TableRootMarker -- HBase as read/write source. Expands on A1 by bundling the notification logic followed by root node propagation. Once we've marked the appropriate nodes as roots, tell every child which roots it can be traced back to via one or more spanning trees. This will take N + 2 supersteps where N is the maximum number of hops from any root to any leaf, plus 2 supersteps for the initial root flagging. I've included all relevant code plus DistributedCacheHelper.java for recursive cache file and archive searches. It is more hadoop centric than giraph, but these jobs use it so I figured why not commit here. These have been tested through local JobRunner, pseudo-distributed on the aforementioned hardware, and full distributed on EC2. More details in the comments. -- 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-141) mulitgraph support in giraph
[ https://issues.apache.org/jira/browse/GIRAPH-141?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13245484#comment-13245484 ] Avery Ching commented on GIRAPH-141: Yes, I also think this is an important feature. Anyone want to work on it? =) mulitgraph support in giraph Key: GIRAPH-141 URL: https://issues.apache.org/jira/browse/GIRAPH-141 Project: Giraph Issue Type: Improvement Components: graph Reporter: André Kelpe The current vertex API only supports simple graphs, meaning that there can only ever be one edge between two vertices. Many graphs like the road network are in fact multigraphs, where many edges can connect two vertices at the same time. Support for this could be added by introducing an IteratorEdgeWritable getEdgeValue() or a similar construct. Maybe introducing a slim object like a Connector between the edge and the vertex is also a good idea, so that you could do something like: {code} for (final ConnectorEdgeWritable, VertexWritable conn: getEdgeValues(){ final EdgeWritable edge = conn.getEdge(); final VertexWritable otherVertex = conn.getOther(); doInterestingStuff(otherVertex); doMoreInterestingStuff(edge); } {code} -- 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-169) How to close all child when a job finished?
[ https://issues.apache.org/jira/browse/GIRAPH-169?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13245488#comment-13245488 ] Avery Ching commented on GIRAPH-169: This is a simple case. I'll try and see if I can replicate it sometime this week. Feel free to bug me if I forget. =) How to close all child when a job finished? --- Key: GIRAPH-169 URL: https://issues.apache.org/jira/browse/GIRAPH-169 Project: Giraph Issue Type: Improvement Components: mapreduce Affects Versions: 0.2.0 Environment: sles 11 x64,jdk 1.6,hadoop 0.20.205.0,1 Master and 8 slaves, Reporter: Jianfeng Qian Priority: Minor I ran pagerank at hadoop 0.20.205.0. When the job finished,the child in slaves didn't quit immediately and sometimes they never quit and I have to kill them. -- 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
Re: Status report
Thanks Jakob. Avery On 4/2/12 4:31 PM, Jakob Homan wrote: I'll do it tonight. On Mon, Apr 2, 2012 at 4:14 PM, Owen O'Malleyomal...@apache.org wrote: All, We need a status report for the last quarter by Wednesday. Anyone want to take the first shot at it? -- Owen
Re: Status report
Looks good to me as well. Avery On 4/2/12 10:17 PM, Owen O'Malley wrote: That looks great, Jakob. I've put that into the wiki for now until we have further edits. -- Owen
[jira] [Commented] (GIRAPH-169) How to close all child when a job finished?
[ https://issues.apache.org/jira/browse/GIRAPH-169?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13240234#comment-13240234 ] Avery Ching commented on GIRAPH-169: Looks like the worker log got cut off? Also, what version of Hadoop is this? Does it work with different numbers of workers? How to close all child when a job finished? --- Key: GIRAPH-169 URL: https://issues.apache.org/jira/browse/GIRAPH-169 Project: Giraph Issue Type: Improvement Components: mapreduce Affects Versions: 0.2.0 Environment: sles 11 x64,jdk 1.6,hadoop 0.20.205.0,1 Master and 8 slaves, Reporter: Jianfeng Qian Priority: Minor I ran pagerank at hadoop 0.20.205.0. When the job finished,the child in slaves didn't quit immediately and sometimes they never quit and I have to kill them. -- 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
Re: Propagation of exception message in GraphMapper
Dionysis, I'm a little confused. The below message seems to explain the problem with Input path does not exist: hdfs://localhost:9000/x Am I missing something? Avery On 3/27/12 4:04 AM, Dionysis Logothetis wrote: Hi all, I was running one of the example apps and I accidentally entered the wrong input, so I ended up getting an exception from GraphMapper. But I think the actual cause of the exception (directory missing) is not propagated properly, which makes debugging a bit more difficult. Perhaps the generateInputSplits() method should include the type of exception in the message? Below I'm pasting the messages printed: 2012-03-27 12:50:03,925 FATAL org.apache.giraph.graph.GraphMapper: uncaughtException: OverrideExceptionHandler on thread org.apache.giraph.graph.MasterThread, msg = generateInputSplits: Got IOException, exiting... java.lang.IllegalStateException: generateInputSplits: Got IOException at org.apache.giraph.graph.BspServiceMaster.generateInputSplits(BspServiceMaster.java:249) at org.apache.giraph.graph.BspServiceMaster.createInputSplits(BspServiceMaster.java:506) at org.apache.giraph.graph.MasterThread.run(MasterThread.java:98) Caused by: org.apache.hadoop.mapreduce.lib.input.InvalidInputException: Input path does not exist: hdfs://localhost:9000/x at org.apache.hadoop.mapreduce.lib.input.FileInputFormat.listStatus(FileInputFormat.java:231) at org.apache.hadoop.mapreduce.lib.input.FileInputFormat.getSplits(FileInputFormat.java:248) at org.apache.giraph.lib.TextVertexInputFormat.getSplits(TextVertexInputFormat.java:120) at org.apache.giraph.graph.BspServiceMaster.generateInputSplits(BspServiceMaster.java:229) ... 2 more 2012-03-27 12:50:03,941 WARN org.apache.giraph.zk.ZooKeeperManager: onlineZooKeeperServers: Forced a shutdown hook kill of the ZooKeeper process.
[jira] [Resolved] (GIRAPH-144) GiraphJob should not extend Job (users should not be able to call Job methods like waitForCompletion or setMapper..etc)
[ https://issues.apache.org/jira/browse/GIRAPH-144?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Avery Ching resolved GIRAPH-144. Resolution: Fixed Hudson says success. Rebasing this one was a bit harder than usual =). Resolving. GiraphJob should not extend Job (users should not be able to call Job methods like waitForCompletion or setMapper..etc) Key: GIRAPH-144 URL: https://issues.apache.org/jira/browse/GIRAPH-144 Project: Giraph Issue Type: Bug Reporter: Dave Assignee: Avery Ching Attachments: GIRAPH-144.patch Original Estimate: 24h Remaining Estimate: 24h -- 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-169) How to close all child when a job finished?
[ https://issues.apache.org/jira/browse/GIRAPH-169?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13240112#comment-13240112 ] Avery Ching commented on GIRAPH-169: How many task trackers do you have? Are you seeing any errors? Is the job completing successfully? I'm guessing that the job isn't completing successfully, since everything should be cleaned up. How to close all child when a job finished? --- Key: GIRAPH-169 URL: https://issues.apache.org/jira/browse/GIRAPH-169 Project: Giraph Issue Type: Improvement Components: mapreduce Affects Versions: 0.2.0 Environment: sles 11 x64,jdk 1.6,hadoop 0.20.205.0,1 Master and 8 slaves, Reporter: Jianfeng Qian Priority: Minor I ran pagerank at hadoop 0.20.205.0. When the job finished,the child in slaves didn't quit immediately and sometimes they never quit and I have to kill them. -- 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-169) How to close all child when a job finished?
[ https://issues.apache.org/jira/browse/GIRAPH-169?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13240161#comment-13240161 ] Avery Ching commented on GIRAPH-169: Do you have the logs of the workers? I'd like to see why they can't exit. How to close all child when a job finished? --- Key: GIRAPH-169 URL: https://issues.apache.org/jira/browse/GIRAPH-169 Project: Giraph Issue Type: Improvement Components: mapreduce Affects Versions: 0.2.0 Environment: sles 11 x64,jdk 1.6,hadoop 0.20.205.0,1 Master and 8 slaves, Reporter: Jianfeng Qian Priority: Minor I ran pagerank at hadoop 0.20.205.0. When the job finished,the child in slaves didn't quit immediately and sometimes they never quit and I have to kill them. -- 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-159) Case insensitive file/directory name matching will produce errors on M/R jar unpack.
[ https://issues.apache.org/jira/browse/GIRAPH-159?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13237909#comment-13237909 ] Avery Ching commented on GIRAPH-159: +1. I left out your GiraphRunner changes since they were fixed by earlier JIRAs, but verified both the problem and the solution you proposed. Looks good! Thanks for the fix! Committing. Case insensitive file/directory name matching will produce errors on M/R jar unpack. - Key: GIRAPH-159 URL: https://issues.apache.org/jira/browse/GIRAPH-159 Project: Giraph Issue Type: Bug Components: build Affects Versions: 0.2.0 Environment: OSX 10.6.8 Reporter: Brian Femiano Attachments: GIRAPH-159.patch, compile.xml This only seems to affect platforms where there can be a file/directory naming conflicts from case insensitive matches. I was able to reproduce running the pseudo-distributed unit tests within OSX. This has affected other projects: https://issues.apache.org/jira/browse/MAHOUT-780 I've been able to reproduce this on my local OSX install with the following error: https://groups.google.com/a/cloudera.org/group/cdh-user/browse_thread/thread/a201218000e956d3/cc6eca3ef9f80ff8 Since LICENSE.txt contains the same content as the file LICENSE, I propose we exclude any LICENSE matches found in the unpacked dependency jars when the maven assembly phase hits 'jar-with-dependencies'. I have a patch which moves the 'jar-with-dependencies' descriptor to an external compile.xml file which has the proper excludes. This might also come in handy down the road should any additional tweaks be needed to the compile phase. -- 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-153) HBase/Accumulo Input and Output formats
[ https://issues.apache.org/jira/browse/GIRAPH-153?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13237951#comment-13237951 ] Avery Ching commented on GIRAPH-153: 34 MB is huge. Can we do something like make the dependency scope provided and then use the distributed cache for unittests? HBase/Accumulo Input and Output formats --- Key: GIRAPH-153 URL: https://issues.apache.org/jira/browse/GIRAPH-153 Project: Giraph Issue Type: New Feature Components: bsp Affects Versions: 0.1.0 Environment: Single host OSX 10.6.8 2.2Ghz Intel i7, 8GB Reporter: Brian Femiano Four abstract classes that wrap their respective delegate input/output formats for easy hooks into vertex input format subclasses. I've included some sample programs that show two very simple graph algorithms. I have a graph generator that builds out a very simple directed structure, starting with a few 'root' nodes. Root nodes are defined as nodes which are not listed as a child anywhere in the graph. Algorithm 1) AccumuloRootMarker.java -- Accumulo as read/write source. Every vertex starts thinking it's a root. At superstep 0, send a message down to each child as a non-root notification. After superstep 1, only root nodes will have never been messaged. Algorithm 2) TableRootMarker -- HBase as read/write source. Expands on A1 by bundling the notification logic followed by root node propagation. Once we've marked the appropriate nodes as roots, tell every child which roots it can be traced back to via one or more spanning trees. This will take N + 2 supersteps where N is the maximum number of hops from any root to any leaf, plus 2 supersteps for the initial root flagging. I've included all relevant code plus DistributedCacheHelper.java for recursive cache file and archive searches. It is more hadoop centric than giraph, but these jobs use it so I figured why not commit here. These have been tested through local JobRunner, pseudo-distributed on the aforementioned hardware, and full distributed on EC2. More details in the comments. -- 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-153) HBase/Accumulo Input and Output formats
[ https://issues.apache.org/jira/browse/GIRAPH-153?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13237468#comment-13237468 ] Avery Ching commented on GIRAPH-153: Brian, could you make it a single patch for us to take a look at? I'm excited to see this work. HBase/Accumulo Input and Output formats --- Key: GIRAPH-153 URL: https://issues.apache.org/jira/browse/GIRAPH-153 Project: Giraph Issue Type: New Feature Components: bsp Affects Versions: 0.1.0 Environment: Single host OSX 10.6.8 2.2Ghz Intel i7, 8GB Reporter: Brian Femiano Four abstract classes that wrap their respective delegate input/output formats for easy hooks into vertex input format subclasses. I've included some sample programs that show two very simple graph algorithms. I have a graph generator that builds out a very simple directed structure, starting with a few 'root' nodes. Root nodes are defined as nodes which are not listed as a child anywhere in the graph. Algorithm 1) AccumuloRootMarker.java -- Accumulo as read/write source. Every vertex starts thinking it's a root. At superstep 0, send a message down to each child as a non-root notification. After superstep 1, only root nodes will have never been messaged. Algorithm 2) TableRootMarker -- HBase as read/write source. Expands on A1 by bundling the notification logic followed by root node propagation. Once we've marked the appropriate nodes as roots, tell every child which roots it can be traced back to via one or more spanning trees. This will take N + 2 supersteps where N is the maximum number of hops from any root to any leaf, plus 2 supersteps for the initial root flagging. I've included all relevant code plus DistributedCacheHelper.java for recursive cache file and archive searches. It is more hadoop centric than giraph, but these jobs use it so I figured why not commit here. These have been tested through local JobRunner, pseudo-distributed on the aforementioned hardware, and full distributed on EC2. More details in the comments. -- 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-144) GiraphJob should not extend Job (users should not be able to call Job methods like waitForCompletion or setMapper..etc)
[ https://issues.apache.org/jira/browse/GIRAPH-144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13237597#comment-13237597 ] Avery Ching commented on GIRAPH-144: Ping, anyone? I'd like to close this out, one way or another. GiraphJob should not extend Job (users should not be able to call Job methods like waitForCompletion or setMapper..etc) Key: GIRAPH-144 URL: https://issues.apache.org/jira/browse/GIRAPH-144 Project: Giraph Issue Type: Bug Reporter: Dave Assignee: Avery Ching Attachments: GIRAPH-144.patch Original Estimate: 24h Remaining Estimate: 24h -- 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-167) mvn -Phadoop_non_secure clean verify fails
[ https://issues.apache.org/jira/browse/GIRAPH-167?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13236414#comment-13236414 ] Avery Ching commented on GIRAPH-167: +1, Commited, thanks for fixing this. mvn -Phadoop_non_secure clean verify fails -- Key: GIRAPH-167 URL: https://issues.apache.org/jira/browse/GIRAPH-167 Project: Giraph Issue Type: Bug Affects Versions: 0.2.0 Reporter: Eugene Koontz Assignee: Eugene Koontz Labels: build, hadoop Attachments: GIRAPH-167.patch The {{hadoop_non_secure}} profile, which uses hadoop 0.20.2, is failing to compile: {code} [ERROR] COMPILATION ERROR : [INFO] - [ERROR] /Users/ekoontz/giraph/target/munged/main/org/apache/giraph/comm/RPCCommunications.java:[184,48] cannot find symbol symbol : variable versionID location: class org.apache.giraph.comm.RPCCommunicationsI,V,E,M [INFO] 1 error {code} -- 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-161) Handling null messages and edges when initializing IntIntNullIntVertex
[ https://issues.apache.org/jira/browse/GIRAPH-161?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13234139#comment-13234139 ] Avery Ching commented on GIRAPH-161: +1. There are 5 checkstyle violations from GIRAPH-156, but this isn't the cause. Committing, thanks Dionysios! Handling null messages and edges when initializing IntIntNullIntVertex -- Key: GIRAPH-161 URL: https://issues.apache.org/jira/browse/GIRAPH-161 Project: Giraph Issue Type: Bug Components: graph Affects Versions: 0.1.0 Reporter: Dionysios Logothetis Attachments: GIRAPH-161.patch The initialize() method in org.apache.giraph.graph.IntIntNullIntVertex should handle null messages or null edges. Especially initializing with null messages is a common case. -- 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-162) BspCase.setup() should catch FileNotFoundException thrown from org.apache.hadoop.fs.FileSystem.listStatus()
[ https://issues.apache.org/jira/browse/GIRAPH-162?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13234164#comment-13234164 ] Avery Ching commented on GIRAPH-162: Looks good. +1. I'm committing. BspCase.setup() should catch FileNotFoundException thrown from org.apache.hadoop.fs.FileSystem.listStatus() --- Key: GIRAPH-162 URL: https://issues.apache.org/jira/browse/GIRAPH-162 Project: Giraph Issue Type: Bug Components: test Affects Versions: 0.2.0 Reporter: Eugene Koontz Fix For: 0.2.0 Attachments: GIRAPH-162.patch In hadoop trunk, org.apache.hadoop.fs.FileSystem.listStatus() is declared to throws both FileNotFoundException and IOException. The former (FileNotFoundException) is currently not caught when BspCase.setup() looks for the GiraphJob.ZOOKEEPER_MANAGER_DIR_DEFAULT directory in order to delete it. The listStatus() call throws FileNotException if this directory does not exist and causes several tests to fail when using Hadoop trunk. This exception should be caught and ignored during setup(), since it's not an error for this directory not to exist. -- 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
Re: hadoop version profiles
I agree with this approach, although munge is kinda hacky. It is easy though. =) Avery On 3/20/12 5:52 PM, Eugene Koontz wrote: Hi Giraphers, I think it might be good to look at how we can add support for new hadoop versions. Currently we have hadoop_facebook (https://issues.apache.org/jira/browse/GIRAPH-14). I am considering adding new ones such as hadoop_0.24. Looking at the code, it seems that the main hadoop variation between the stock hadoop used (0.203.0) versus facebook has to do with the new security-related APIs in the latter that is, fortunately, also available in hadoop 0.23 and 0.24. So, hopefully we can make use of the existing work that Avery has done for hadoop_facebook and apply it to other hadoop versions. Therefore I would propose that: 1. a new munge flag HADOOP_SECURE to be used in RPCCommunication.java and a few other places, where we are currently checking for HADOOP_FACEBOOK and HADOOP. 2. we make a new profile called hadoop_secure, which, as with hadoop_facebook, will use the above munge flag. 3. we make a new profile hadoop_0.20.203 for the existing default hadoop and make it the default profile (activeByDefault=true). This will makes it easier to handle the differences in the hadoop library dependency set that have happened between 0.20.203 and hadoop trunk. Please see https://github.com/ekoontz/giraph/tree/security-profile for my branch that implements the above. Thanks, -Eugene
[jira] [Commented] (GIRAPH-159) Case insensitive file/directory name matching will produce errors on M/R jar unpack.
[ https://issues.apache.org/jira/browse/GIRAPH-159?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13234177#comment-13234177 ] Avery Ching commented on GIRAPH-159: Brian, can you show me how to recreate this issue on OSX? Case insensitive file/directory name matching will produce errors on M/R jar unpack. - Key: GIRAPH-159 URL: https://issues.apache.org/jira/browse/GIRAPH-159 Project: Giraph Issue Type: Bug Components: build Affects Versions: 0.2.0 Environment: OSX 10.6.8 Reporter: Brian Femiano Priority: Minor Attachments: GIRAPH-159.patch, compile.xml This only seems to affect platforms where there can be a file/directory naming conflicts from case insensitive matches. I was able to reproduce running the pseudo-distributed unit tests within OSX. This has affected other projects: https://issues.apache.org/jira/browse/MAHOUT-780 I've been able to reproduce this on my local OSX install with the following error: https://groups.google.com/a/cloudera.org/group/cdh-user/browse_thread/thread/a201218000e956d3/cc6eca3ef9f80ff8 Since LICENSE.txt contains the same content as the file LICENSE, I propose we exclude any LICENSE matches found in the unpacked dependency jars when the maven assembly phase hits 'jar-with-dependencies'. I have a patch which moves the 'jar-with-dependencies' descriptor to an external compile.xml file which has the proper excludes. This might also come in handy down the road should any additional tweaks be needed to the compile phase. -- 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-164) fix 5 Line is longer than 80 characters style errors in GiraphRunner
[ https://issues.apache.org/jira/browse/GIRAPH-164?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13234843#comment-13234843 ] Avery Ching commented on GIRAPH-164: +1, thanks guys. Committing. fix 5 Line is longer than 80 characters style errors in GiraphRunner -- Key: GIRAPH-164 URL: https://issues.apache.org/jira/browse/GIRAPH-164 Project: Giraph Issue Type: Bug Affects Versions: 0.2.0 Reporter: Eugene Koontz Priority: Trivial Fix For: 0.2.0 Attachments: GIRAPH-164.patch {code} file name=/Users/ekoontz/giraph/src/main/java/org/apache/giraph/GiraphRunner.java error line=155 severity=error message=Line is longer than 80 characters. source=com.puppycrawl.tools.checkstyle.checks.sizes.LineLengthCheck/ error line=156 severity=error message=Line is longer than 80 characters. source=com.puppycrawl.tools.checkstyle.checks.sizes.LineLengthCheck/ error line=158 severity=error message=Line is longer than 80 characters. source=com.puppycrawl.tools.checkstyle.checks.sizes.LineLengthCheck/ error line=161 severity=error message=Line is longer than 80 characters. source=com.puppycrawl.tools.checkstyle.checks.sizes.LineLengthCheck/ /file {code} -- 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-154) Worker ports are not synched properly with its peers
[ https://issues.apache.org/jira/browse/GIRAPH-154?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13232012#comment-13232012 ] Avery Ching commented on GIRAPH-154: Nice work Zhiwei (+1), I verified it as well and committed. Will close once Hudson verifies as well. Worker ports are not synched properly with its peers Key: GIRAPH-154 URL: https://issues.apache.org/jira/browse/GIRAPH-154 Project: Giraph Issue Type: Bug Components: bsp Affects Versions: 0.2.0 Reporter: Zhiwei Gu Assignee: Zhiwei Gu Attachments: GIRAPH-154.patch When worker trying multiple ports to setup the rpc server, the final port is not synched with it's peer workers properly, and resulted in peer workers send message to the default port. Here is some logs: Base port: 34900 log for worker 161: IPC Server handler 98 on 36061: starting BasicRPCCommunications: Started RPC communication server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:36061 with 100 handlers and 199 flush threads on bind attempt 1 IPC Server handler 99 on 36061: starting setup: Registering health of this worker... getJobState: Job state already exists (/_hadoopBsp/job_201203130609_14838/_masterJobState) getApplicationAttempt: Node /_hadoopBsp/job_201203130609_14838/_applicationAttemptsDir already exists! getApplicationAttempt: Node /_hadoopBsp/job_201203130609_14838/_applicationAttemptsDir already exists! registerHealth: Created my health node for attempt=0, superstep=-1 with /_hadoopBsp/job_201203130609_14838/_applicationAttemptsDir/0/_superstepDir/-1/_workerHealthyDir/gsta32085.tan.ygrid.yahoo.com_161 and workerInfo= Worker(hostname=gsta32085.tan.ygrid.yahoo.com, MRpartition=161, port=35061) process: partitionAssignmentsReadyChanged (partitions are assigned) startSuperstep: Ready for computation on superstep -1 since worker selection and vertex range assignments are done in /_hadoopBsp/job_201203130609_14838/_applicationAttemptsDir/0/_superstepDir/-1/_partitionAssignments Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 0 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 1 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 2 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 3 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 4 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 5 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 6 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 7 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 8 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 9 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 10 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 11 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 12 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 13 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 14 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 15 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 16 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 17 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 18 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 19 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 20 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 21 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47
[jira] [Commented] (GIRAPH-156) Users should be able to set simple 'custom arguments' via org.apache.giraph.GiraphRunner
[ https://issues.apache.org/jira/browse/GIRAPH-156?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13232014#comment-13232014 ] Avery Ching commented on GIRAPH-156: +1, looks good. It would be great if you could wrap LOG.info with if (LOG.isInfoEnabled()), before committing. There are some other places in this file as well without the LOG enabled wrap. You can either make that change here or someone else can do it in another patch. Users should be able to set simple 'custom arguments' via org.apache.giraph.GiraphRunner Key: GIRAPH-156 URL: https://issues.apache.org/jira/browse/GIRAPH-156 Project: Giraph Issue Type: Improvement Components: conf and scripts Affects Versions: 0.1.0 Reporter: Sebastian Schelter Assignee: Sebastian Schelter Attachments: GIRAPH-156-1.patch, GIRAPH-156.patch Some vertices need custom arguments to run. The SimpleShortestPathsVertex for example needs to know the source vertex for the computation which is saved in the job's Configuration as _SimpleShortestPathsVertex.sourceId_. Users should be able to apply such simple custom arguments via GiraphRunner. I propose to add a new option _--customArguments_ where users can supply arguments in the form _param1=value1,param2=value2_ for this. -- 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] [Resolved] (GIRAPH-154) Worker ports are not synched properly with its peers
[ https://issues.apache.org/jira/browse/GIRAPH-154?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Avery Ching resolved GIRAPH-154. Resolution: Fixed Passed hudson, thanks again Zhiwei. Worker ports are not synched properly with its peers Key: GIRAPH-154 URL: https://issues.apache.org/jira/browse/GIRAPH-154 Project: Giraph Issue Type: Bug Components: bsp Affects Versions: 0.2.0 Reporter: Zhiwei Gu Assignee: Zhiwei Gu Attachments: GIRAPH-154.patch When worker trying multiple ports to setup the rpc server, the final port is not synched with it's peer workers properly, and resulted in peer workers send message to the default port. Here is some logs: Base port: 34900 log for worker 161: IPC Server handler 98 on 36061: starting BasicRPCCommunications: Started RPC communication server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:36061 with 100 handlers and 199 flush threads on bind attempt 1 IPC Server handler 99 on 36061: starting setup: Registering health of this worker... getJobState: Job state already exists (/_hadoopBsp/job_201203130609_14838/_masterJobState) getApplicationAttempt: Node /_hadoopBsp/job_201203130609_14838/_applicationAttemptsDir already exists! getApplicationAttempt: Node /_hadoopBsp/job_201203130609_14838/_applicationAttemptsDir already exists! registerHealth: Created my health node for attempt=0, superstep=-1 with /_hadoopBsp/job_201203130609_14838/_applicationAttemptsDir/0/_superstepDir/-1/_workerHealthyDir/gsta32085.tan.ygrid.yahoo.com_161 and workerInfo= Worker(hostname=gsta32085.tan.ygrid.yahoo.com, MRpartition=161, port=35061) process: partitionAssignmentsReadyChanged (partitions are assigned) startSuperstep: Ready for computation on superstep -1 since worker selection and vertex range assignments are done in /_hadoopBsp/job_201203130609_14838/_applicationAttemptsDir/0/_superstepDir/-1/_partitionAssignments Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 0 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 1 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 2 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 3 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 4 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 5 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 6 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 7 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 8 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 9 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 10 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 11 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 12 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 13 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 14 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 15 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 16 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 17 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 18 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 19 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 20 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 21 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com/10.216.148.47:35061. Already tried 22 time(s). Retrying connect to server: gsta32085.tan.ygrid.yahoo.com
Re: How to contribute page
That would be great, Jakob. Avery On 3/15/12 11:48 AM, Jakob Homan wrote: That's fine. Can we update the site to point to the wiki (and harmonize the content), so we don't have duplicate, soon-to-diverage information? If so, I'll try to do this pretty soon. On Wed, Mar 14, 2012 at 11:37 PM, Avery Chingach...@apache.org wrote: Main differences are the 'mvn verify' and running singe node unittest tests. It's easier for us to manage on confluence compared to maintaining the site =). Avery On 3/14/12 11:59 AM, Jakob Homan wrote: This page looks very similar in content to the Generating Patches and Getting Invovled sections on the main site: https://incubator.apache.org/giraph/ Are there any significant differences? On Wed, Mar 14, 2012 at 10:25 AM, Sebastian Schelters...@apache.org wrote: I added the 'Be involved' part from Mahout's [1] 'How to contribute' page. Maybe we could even copy a little more from there :) Best, Sebastian [1] https://cwiki.apache.org/MAHOUT/how-to-contribute.html On 14.03.2012 17:39, Avery Ching wrote: Yes, that is thanks to Sebastian. We should probably make that another confluence page though based on his notes. Anyone want to do it? =) Avery On 3/14/12 7:43 AM, Benjamin Heitmann wrote: On 14 Mar 2012, at 07:08, Avery Ching wrote: I just added a How to contribute page. https://cwiki.apache.org/confluence/display/GIRAPH/How+to+Contribute Thanks for setting up this page! Also, the link about running giraph's unit test in pseudo distributed mode [1] is very interesting. [1] http://ssc.io/running-giraphs-unit-tests-in-pseudo-distributed-mode/
[jira] [Commented] (GIRAPH-156) Users should be able to set simple 'custom arguments' via org.apache.giraph.GiraphRunner
[ https://issues.apache.org/jira/browse/GIRAPH-156?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13230942#comment-13230942 ] Avery Ching commented on GIRAPH-156: I think this makes sense. Go for it. =) Users should be able to set simple 'custom arguments' via org.apache.giraph.GiraphRunner Key: GIRAPH-156 URL: https://issues.apache.org/jira/browse/GIRAPH-156 Project: Giraph Issue Type: Improvement Components: conf and scripts Affects Versions: 0.1.0 Reporter: Sebastian Schelter Assignee: Sebastian Schelter Some vertices need custom arguments to run. The SimpleShortestPathsVertex for example needs to know the source vertex for the computation which is saved in the job's Configuration as _SimpleShortestPathsVertex.sourceId_. Users should be able to apply such simple custom arguments via GiraphRunner. I propose to add a new option _--customArguments_ where users can supply arguments in the form _param1=value1,param2=value2_ for this. -- 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
Re: Giraph and MRv2?
This would be great Eugene. So far, I don't know of any other JIRA to do this, so go for it! Thanks for doing this. Avery On 3/15/12 3:39 PM, Eugene Koontz wrote: Greetings fellow Giraphers, I'm working on compiling Giraph with support for MRv2 (available with Hadoop's 0.23 branch and on trunk (0.24.0-SNAPHOT)). Has anyone tried this yet? I think we'd want a maven profile switch (perhaps -Phadoop_trunk), similar to GIRAPH-14's -Phadoop_facebook (https://issues.apache.org/jira/browse/GIRAPH-14). Shall I file a JIRA with this intention, unless I've missed one already open? -Eugene
Please vote for our Giraph proposal for the upcoming Hadoop Summit
Hi Giraphers, We have a submission for the 2012 Hadoop summit and part of deciding whether it gets accepted is based on community voting. It would be great to get more folks interested and involved in what is going on with Giraph so please vote! Here's the link: https://hadoopsummit2012.uservoice.com/forums/151413-track-1-future-of-apache-hadoop/suggestions/2663542-processing-over-a-billion-edges-on-apache-giraph We had some great exposure at last year's Hadoop Summit and hope to be a part of this year's program as well. Thanks! Avery
[jira] [Commented] (GIRAPH-153) HBase/Accumulo Input and Output formats
[ https://issues.apache.org/jira/browse/GIRAPH-153?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13229042#comment-13229042 ] Avery Ching commented on GIRAPH-153: Brian, this is an awesome contribution and a lot of code. I'm really sorry that it took me so long to look at this. Is there any change that you could add some simple unittests for your formats? TestJsonBase64Format.java is an example that might be easy to adapt for your formats. Also, I just created a page for how to contribute. https://cwiki.apache.org/confluence/display/GIRAPH/How+to+Contribute Have you run 'mvn verify'? Thanks! HBase/Accumulo Input and Output formats --- Key: GIRAPH-153 URL: https://issues.apache.org/jira/browse/GIRAPH-153 Project: Giraph Issue Type: New Feature Components: bsp Affects Versions: 0.1.0 Environment: Single host OSX 10.6.8 2.2Ghz Intel i7, 8GB Reporter: Brian Femiano Attachments: AccumuloRootMarker.java, AccumuloRootMarkerInputFormat.java, AccumuloRootMarkerOutputFormat.java, AccumuloVertexInputFormat.java, AccumuloVertexOutputFormat.java, ComputeIsRoot.java, DistributedCacheHelper.java, HBaseVertexInputFormat.java, HBaseVertexOutputFormat.java, IdentifyAndMarkRoots.java, SetLongWritable.java, SetTextWritable.java, TableRootMarker.java, TableRootMarkerInputFormat.java, TableRootMarkerOutputFormat.java Four abstract classes that wrap their respective delegate input/output formats for easy hooks into vertex input format subclasses. I've included some sample programs that show two very simple graph algorithms. I have a graph generator that builds out a very simple directed structure, starting with a few 'root' nodes. Root nodes are defined as nodes which are not listed as a child anywhere in the graph. Algorithm 1) AccumuloRootMarker.java -- Accumulo as read/write source. Every vertex starts thinking it's a root. At superstep 0, send a message down to each child as a non-root notification. After superstep 1, only root nodes will have never been messaged. Algorithm 2) TableRootMarker -- HBase as read/write source. Expands on A1 by bundling the notification logic followed by root node propagation. Once we've marked the appropriate nodes as roots, tell every child which roots it can be traced back to via one or more spanning trees. This will take N + 2 supersteps where N is the maximum number of hops from any root to any leaf, plus 2 supersteps for the initial root flagging. I've included all relevant code plus DistributedCacheHelper.java for recursive cache file and archive searches. It is more hadoop centric than giraph, but these jobs use it so I figured why not commit here. These have been tested through local JobRunner, pseudo-distributed on the aforementioned hardware, and full distributed on EC2. More details in the comments. -- 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
How to contribute page
Hi guys, I just added a How to contribute page. https://cwiki.apache.org/confluence/display/GIRAPH/How+to+Contribute Please fix it if you have anything to add or I forgot something. Thanks! Avery
Re: How to contribute page
Yes, that is thanks to Sebastian. We should probably make that another confluence page though based on his notes. Anyone want to do it? =) Avery On 3/14/12 7:43 AM, Benjamin Heitmann wrote: On 14 Mar 2012, at 07:08, Avery Ching wrote: I just added a How to contribute page. https://cwiki.apache.org/confluence/display/GIRAPH/How+to+Contribute Thanks for setting up this page! Also, the link about running giraph's unit test in pseudo distributed mode [1] is very interesting. [1] http://ssc.io/running-giraphs-unit-tests-in-pseudo-distributed-mode/
[jira] [Commented] (GIRAPH-144) GiraphJob should not extend Job (users should not be able to call Job methods like waitForCompletion or setMapper..etc)
[ https://issues.apache.org/jira/browse/GIRAPH-144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13224096#comment-13224096 ] Avery Ching commented on GIRAPH-144: @Jakob, any more thoughts? GiraphJob should not extend Job (users should not be able to call Job methods like waitForCompletion or setMapper..etc) Key: GIRAPH-144 URL: https://issues.apache.org/jira/browse/GIRAPH-144 Project: Giraph Issue Type: Bug Reporter: Dave Assignee: Avery Ching Attachments: GIRAPH-144.patch Original Estimate: 24h Remaining Estimate: 24h -- 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
Re: Graph clustering via LinLog force directed layout
Hi Timmy, I don't know much about force directed layout, but it certainly sounds like a very interesting application for Giraph. Keep us posted on your progress and let us know how we can help. Avery On 3/6/12 8:34 AM, Claudio Martella wrote: Hi, I'm not definitely familiar with the algorithm or implementation of LinLog, I've been just a user. It should be doable with Giraph if you can express it in terms of message-passing between vertices and without a dependency on a global view of the graph (except for the convergence criteria, such as total energy). Please consider that Giraph's data model is based on a directed graph, this should be a quite interesting constraint for you, if your implementation is going to modify energy associated with edges (you'd have two views over the undirected edge, one in each endpoint). In general, a good way of doing community analysis would be to look at algorithms that belong to the family of label-propagation clustering algorithms. Hope this helps, Claudio On Tue, Mar 6, 2012 at 3:28 PM, Timmy Wilsontim...@smarttypes.org wrote: Hi giraph community, I'm interested in using giraph for distributed n-body simulation. Initially, i'm interested in force directed layouts -- ie, graph drawing: http://en.wikipedia.org/wiki/Force-based_algorithms_(graph_drawing) I'm interested specifically in Dr. Andreas Noack's LinLog energy model -- which performs well w/ community detection: http://www.informatik.tu-cottbus.de/~an/GD/linlog.html I have a few examples of a serial implementation here: http://www.smarttypes.org/ The model maximizes the distance between all nodes while minimizing the distance between connected nodes. Without getting into too much detail, i'm curious if anyone has considered using giraph for force directed graph embedding (yet another name for it)? I'm also considering something like http://www.mcs.anl.gov/petsc/ or http://www.cs.cmu.edu/~scandal/alg/nbody.html -- which have fast n-body simulation implementations (Barnes-Hut + Fast Multipole). That said, i think giraph may be a good fit -- curious what the community thinks? Thanks, Timmy Wilson Cleveland, OH
[jira] [Commented] (GIRAPH-85) Simplify return expression in RPCCommunications::getRPCProxy
[ https://issues.apache.org/jira/browse/GIRAPH-85?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13217096#comment-13217096 ] Avery Ching commented on GIRAPH-85: --- I just looked at your patch in Eclipse and see that there is now a warning due to Type safety: Unchecked cast from VersionedProtocol to CommunicationsInterfaceI,V,E,M. We can either keep this the way it was before, or add @SuppressWarnings(unchecked) to the method. I don't have a strong opinion here. Anyone else care to comment? Simplify return expression in RPCCommunications::getRPCProxy Key: GIRAPH-85 URL: https://issues.apache.org/jira/browse/GIRAPH-85 Project: Giraph Issue Type: Improvement Affects Versions: 0.2.0 Reporter: Jakob Homan Labels: newbie Fix For: 0.2.0 Attachments: GIRAPH-85.patch, GIRAPH-85.patch Twice in RPCCommunications::getRPCProxy a local variable, proxy, is created and immediately returned. We can simplify this to just return the value. -- 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-87) Simplify boolean expression in BspService::checkpointFrequencyMet
[ https://issues.apache.org/jira/browse/GIRAPH-87?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13216365#comment-13216365 ] Avery Ching commented on GIRAPH-87: --- +1 Thanks Eli, I committed on your behalf. Simplify boolean expression in BspService::checkpointFrequencyMet - Key: GIRAPH-87 URL: https://issues.apache.org/jira/browse/GIRAPH-87 Project: Giraph Issue Type: Improvement Affects Versions: 0.2.0 Reporter: Jakob Homan Assignee: Eli Reisman Labels: newbie Attachments: GIRAPH-87.patch, GIRAPH-87.patch {noformat}if (superstep firstCheckpoint) { return false; } else if (((superstep - firstCheckpoint) % checkpointFrequency) == 0) { return true; } else { return false; }{noformat} can be simplified to just return the result of the else if evaluation. -- 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] [Resolved] (GIRAPH-132) Simplify boolean expression in GraphMapper::map()
[ https://issues.apache.org/jira/browse/GIRAPH-132?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Avery Ching resolved GIRAPH-132. Resolution: Not A Problem Thanks Eli, this was resolved in the massive checkstyle fix (GIRAPH-40). Simplify boolean expression in GraphMapper::map() - Key: GIRAPH-132 URL: https://issues.apache.org/jira/browse/GIRAPH-132 Project: Giraph Issue Type: Improvement Reporter: Jakob Homan Priority: Trivial Labels: newbie The boolean expression in: {noformat}@Override public void map(Object key, Object value, Context context) throws IOException, InterruptedException { // map() only does computation // 1) Run checkpoint per frequency policy. // 2) For every vertex on this mapper, run the compute() function // 3) Wait until all messaging is done. // 4) Check if all vertices are done. If not goto 2). // 5) Dump output. if (done == true) { return; }{noformat} can be simplified. -- 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-85) Simplify return expression in RPCCommunications::getRPCProxy
[ https://issues.apache.org/jira/browse/GIRAPH-85?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13216047#comment-13216047 ] Avery Ching commented on GIRAPH-85: --- please make sure it passes 'mvn verify' as well. That will check rat and checkstyle. Simplify return expression in RPCCommunications::getRPCProxy Key: GIRAPH-85 URL: https://issues.apache.org/jira/browse/GIRAPH-85 Project: Giraph Issue Type: Improvement Affects Versions: 0.2.0 Reporter: Jakob Homan Labels: newbie Fix For: 0.2.0 Attachments: GIRAPH-85.patch Twice in RPCCommunications::getRPCProxy a local variable, proxy, is created and immediately returned. We can simplify this to just return the value. -- 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] [Resolved] (GIRAPH-150) PageRankBenchmark accesses wrong conf after GiraphJob is created
[ https://issues.apache.org/jira/browse/GIRAPH-150?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Avery Ching resolved GIRAPH-150. Resolution: Fixed Thanks Sebastian! Closing after hudson's approval. PageRankBenchmark accesses wrong conf after GiraphJob is created Key: GIRAPH-150 URL: https://issues.apache.org/jira/browse/GIRAPH-150 Project: Giraph Issue Type: Bug Reporter: Avery Ching Assignee: Avery Ching Attachments: GIRAPH-150.patch -- 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-40) Adding checkstyle enforcement of Giraph code conventions
[ https://issues.apache.org/jira/browse/GIRAPH-40?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13209727#comment-13209727 ] Avery Ching commented on GIRAPH-40: --- Can another committer please look at this as per Jakob's request? Adding checkstyle enforcement of Giraph code conventions Key: GIRAPH-40 URL: https://issues.apache.org/jira/browse/GIRAPH-40 Project: Giraph Issue Type: New Feature Reporter: Avery Ching Assignee: Avery Ching Priority: Minor Attachments: GIRAPH-40.2.patch, GIRAPH-40.3.patch, GIRAPH-40.patch, GIRAPH-40.patch Now that we have some code conventions (see GIRAPH-21), we should enforce them with a maven checkstyle plugin. -- 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-40) Adding checkstyle enforcement of Giraph code conventions
[ https://issues.apache.org/jira/browse/GIRAPH-40?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13209801#comment-13209801 ] Avery Ching commented on GIRAPH-40: --- Thanks so much for the reviews Jakob and Sebastian. It's committed. @Sebastian, 'mvn compile' and 'mvn package' will succeed with violations. Anything using 'verify', i.e. 'mvn verify' or 'mvn install' will hit problems with checkstyle and rat. Adding checkstyle enforcement of Giraph code conventions Key: GIRAPH-40 URL: https://issues.apache.org/jira/browse/GIRAPH-40 Project: Giraph Issue Type: New Feature Reporter: Avery Ching Assignee: Avery Ching Priority: Minor Attachments: GIRAPH-40.2.patch, GIRAPH-40.3.patch, GIRAPH-40.patch, GIRAPH-40.patch Now that we have some code conventions (see GIRAPH-21), we should enforce them with a maven checkstyle plugin. -- 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-150) PageRankBenchmark accesses wrong conf after GiraphJob is created
[ https://issues.apache.org/jira/browse/GIRAPH-150?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Avery Ching updated GIRAPH-150: --- Attachment: GIRAPH-150.patch Use the job.getConfiguration() instead of getConf() or else the vertex class doesn't get set properly. Also got rid of the other getConf() usage. Tested with 'mvn package' and 'mvn verify'. PageRankBenchmark accesses wrong conf after GiraphJob is created Key: GIRAPH-150 URL: https://issues.apache.org/jira/browse/GIRAPH-150 Project: Giraph Issue Type: Bug Reporter: Avery Ching Assignee: Avery Ching Attachments: GIRAPH-150.patch -- 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-150) PageRankBenchmark accesses wrong conf after GiraphJob is created
[ https://issues.apache.org/jira/browse/GIRAPH-150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13210025#comment-13210025 ] Avery Ching commented on GIRAPH-150: By the way, here was the full stack trace: hadoop jar target/giraph-0.2-SNAPSHOT-jar-with-dependencies.jar org.apache.giraph.benchmark.PageRankBenchmark -e 1 -s 3 -v -V 50 -w 3 -c 1 Exception in thread main java.lang.NullPointerException at org.apache.giraph.benchmark.PageRankBenchmark.run(PageRankBenchmark.java:127) at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:65) at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:79) at org.apache.giraph.benchmark.PageRankBenchmark.main(PageRankBenchmark.java:162) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) at java.lang.reflect.Method.invoke(Method.java:597) at org.apache.hadoop.util.RunJar.main(RunJar.java:156) After this fix, it works. PageRankBenchmark accesses wrong conf after GiraphJob is created Key: GIRAPH-150 URL: https://issues.apache.org/jira/browse/GIRAPH-150 Project: Giraph Issue Type: Bug Reporter: Avery Ching Assignee: Avery Ching Attachments: GIRAPH-150.patch -- 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-40) Adding checkstyle enforcement of Giraph code conventions
[ https://issues.apache.org/jira/browse/GIRAPH-40?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Avery Ching updated GIRAPH-40: -- Attachment: GIRAPH-40.3.patch Good suggestion, Jakob. I have addressed Jakob's comments by changing phases from 'compile'-'verify' for checkstyle. This is the only change I made. {noquote} execution -phasecompile/phase +phaseverify/phase goals {noquote} We need to require our users to have tested with 'mvn verify' (or 'mvn install') before submitting diffs. This matches the rat approach. Adding checkstyle enforcement of Giraph code conventions Key: GIRAPH-40 URL: https://issues.apache.org/jira/browse/GIRAPH-40 Project: Giraph Issue Type: New Feature Reporter: Avery Ching Assignee: Avery Ching Priority: Minor Attachments: GIRAPH-40.2.patch, GIRAPH-40.3.patch, GIRAPH-40.patch, GIRAPH-40.patch Now that we have some code conventions (see GIRAPH-21), we should enforce them with a maven checkstyle plugin. -- 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-40) Adding checkstyle enforcement of Giraph code conventions
[ https://issues.apache.org/jira/browse/GIRAPH-40?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Avery Ching updated GIRAPH-40: -- Attachment: GIRAPH-40.patch As promised, here is the full patch. Due to its massive size, I am not posting this to reviewboard. Here are the details of what I did: * Created a checkstyle.xml file that follows our CODE_CONVENTIONS as best as possible. * Compiles will now fail if checkstyle guidelines are not met. * While checkstyle isn't comprehensive, it should reduce our reviewer overhead for formatting issues and common code style violations * Current source code (not test code) now meets checkstyle checks It passes both the local and MR unittests and also passes rat installation. Take a look at a few files. I don't recommend looking at everything since $ git diff HEAD^ | grep -P ^(\+|\-) | wc -l 32848 Let's get this in soon to help us iterate faster and get rid of this technical debt! Adding checkstyle enforcement of Giraph code conventions Key: GIRAPH-40 URL: https://issues.apache.org/jira/browse/GIRAPH-40 Project: Giraph Issue Type: New Feature Reporter: Avery Ching Assignee: Avery Ching Priority: Minor Attachments: GIRAPH-40.patch, GIRAPH-40.patch Now that we have some code conventions (see GIRAPH-21), we should enforce them with a maven checkstyle plugin. -- 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-40) Adding checkstyle enforcement of Giraph code conventions
[ https://issues.apache.org/jira/browse/GIRAPH-40?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13207188#comment-13207188 ] Avery Ching commented on GIRAPH-40: --- So for the first example, we need to follow that format, or else checkstyle will mark it an error. For the second examples, checkstyle doesn't seem to enforce the line wrap indent. So we need to still keep an eye out for those issues. Adding checkstyle enforcement of Giraph code conventions Key: GIRAPH-40 URL: https://issues.apache.org/jira/browse/GIRAPH-40 Project: Giraph Issue Type: New Feature Reporter: Avery Ching Assignee: Avery Ching Priority: Minor Attachments: GIRAPH-40.patch Now that we have some code conventions (see GIRAPH-21), we should enforce them with a maven checkstyle plugin. -- 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-40) Adding checkstyle enforcement of Giraph code conventions
[ https://issues.apache.org/jira/browse/GIRAPH-40?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13207203#comment-13207203 ] Avery Ching commented on GIRAPH-40: --- I'm not a checkstyle expert, but I don't think so. I can play around with trying to fix that. Or we can fix in another issue. I should be done with this patch today. Adding checkstyle enforcement of Giraph code conventions Key: GIRAPH-40 URL: https://issues.apache.org/jira/browse/GIRAPH-40 Project: Giraph Issue Type: New Feature Reporter: Avery Ching Assignee: Avery Ching Priority: Minor Attachments: GIRAPH-40.patch Now that we have some code conventions (see GIRAPH-21), we should enforce them with a maven checkstyle plugin. -- 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-148) giraph-site.xml needs Apache header
[ https://issues.apache.org/jira/browse/GIRAPH-148?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13207211#comment-13207211 ] Avery Ching commented on GIRAPH-148: +1. giraph-site.xml needs Apache header --- Key: GIRAPH-148 URL: https://issues.apache.org/jira/browse/GIRAPH-148 Project: Giraph Issue Type: Bug Components: conf and scripts Affects Versions: 0.2.0 Reporter: Jakob Homan Assignee: Jakob Homan Fix For: 0.2.0 Attachments: GIRAPH-148-b.patch, GIRAPH-148.patch I forgot to add the license to the conf file and now rat is failing... -- 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-40) Adding checkstyle enforcement of Giraph code conventions
[ https://issues.apache.org/jira/browse/GIRAPH-40?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13206291#comment-13206291 ] Avery Ching commented on GIRAPH-40: --- Thank you for the feedback Claudio. I'll continue to transition the other files and submit a final patch unless anyone has any objections. Adding checkstyle enforcement of Giraph code conventions Key: GIRAPH-40 URL: https://issues.apache.org/jira/browse/GIRAPH-40 Project: Giraph Issue Type: New Feature Reporter: Avery Ching Assignee: Avery Ching Priority: Minor Attachments: GIRAPH-40.patch Now that we have some code conventions (see GIRAPH-21), we should enforce them with a maven checkstyle plugin. -- 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-139) Change PageRankBenchmark to be accessible via bin/giraph
[ https://issues.apache.org/jira/browse/GIRAPH-139?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13205598#comment-13205598 ] Avery Ching commented on GIRAPH-139: +1 Looks good to me. Change PageRankBenchmark to be accessible via bin/giraph Key: GIRAPH-139 URL: https://issues.apache.org/jira/browse/GIRAPH-139 Project: Giraph Issue Type: Improvement Affects Versions: 0.2.0 Reporter: Jakob Homan Assignee: Jakob Homan Fix For: 0.2.0 Attachments: GIRAPH-139-b.patch, GIRAPH-139.patch Currently the PageRankBenchmark has its own main and tool implementation and is difficult to access from the bin/giraph script. It would be better if everything were accessible via bin/giraph. The benchmark is particularly problematic because it uses inner classes for its two actual Vertex implementations, which have to be specified on the command line as their .class name(ie org.apache.giraph.benchmark.PageRankBenchmark$PageRankHashMapVertex) rather than just with dots, as one would expect. -- 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-147) Add Blueprints Tinkerpop support
Add Blueprints Tinkerpop support Key: GIRAPH-147 URL: https://issues.apache.org/jira/browse/GIRAPH-147 Project: Giraph Issue Type: New Feature Reporter: Avery Ching Priority: Minor Got this issue on the old Giraph GitHub (deprecated). Moving it here. jeffg2k opened this issue 2 hours ago Hoping that Giraph might add TinkerPop Blueprint support. :) -- 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-148) giraph-site.xml needs Apache header
[ https://issues.apache.org/jira/browse/GIRAPH-148?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13205947#comment-13205947 ] Avery Ching commented on GIRAPH-148: Jakob, this header is formatted slightly differently from the one in pom.xml and the .java files we have. giraph-site.xml needs Apache header --- Key: GIRAPH-148 URL: https://issues.apache.org/jira/browse/GIRAPH-148 Project: Giraph Issue Type: Bug Components: conf and scripts Affects Versions: 0.2.0 Reporter: Jakob Homan Assignee: Jakob Homan Fix For: 0.2.0 Attachments: GIRAPH-148.patch I forgot to add the license to the conf file and now rat is failing... -- 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-40) Adding checkstyle enforcement of Giraph code conventions
[ https://issues.apache.org/jira/browse/GIRAPH-40?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13205976#comment-13205976 ] Avery Ching commented on GIRAPH-40: --- Here are some examples of one problem: Checkstyle doesn't seem to be able to handle single indent versus double indent of 2 spaces when appropriate. The below examples are what Checkstyle wants to have us do. {noformat} @Override public BasicVertexLongWritable, DoubleWritable, DoubleWritable, M getCurrentVertex() throws IOException, InterruptedException { @Override public VertexReaderLongWritable, DoubleWritable, DoubleWritable, M createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException { {noformat} Also, checkstyle won't enforce indenting after a line wrap. So both of these examples are passing checkstyle. {noformat} aggregateVertices = configuration.getLong( PseudoRandomVertexInputFormat.AGGREGATE_VERTICES, 0); aggregateVertices = configuration.getLong( PseudoRandomVertexInputFormat.AGGREGATE_VERTICES, 0); {noformat} That being said, I think this is the right thing to do and we can make some sacrifices to have better, more uniform code. Please let me know your thoughts. Adding checkstyle enforcement of Giraph code conventions Key: GIRAPH-40 URL: https://issues.apache.org/jira/browse/GIRAPH-40 Project: Giraph Issue Type: New Feature Reporter: Avery Ching Assignee: Avery Ching Priority: Minor Attachments: GIRAPH-40.patch Now that we have some code conventions (see GIRAPH-21), we should enforce them with a maven checkstyle plugin. -- 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-148) giraph-site.xml needs Apache header
[ https://issues.apache.org/jira/browse/GIRAPH-148?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13205992#comment-13205992 ] Avery Ching commented on GIRAPH-148: It will with checkstyle (see GIRAPH-40). We will need to pick one or the other. I don't have a strong preference. giraph-site.xml needs Apache header --- Key: GIRAPH-148 URL: https://issues.apache.org/jira/browse/GIRAPH-148 Project: Giraph Issue Type: Bug Components: conf and scripts Affects Versions: 0.2.0 Reporter: Jakob Homan Assignee: Jakob Homan Fix For: 0.2.0 Attachments: GIRAPH-148.patch I forgot to add the license to the conf file and now rat is failing... -- 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: GIRAPH-144 GiraphJob should not extend Job (users should not be able to call Job methods like waitForCompletion or setMapper..etc)
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/3823/ --- Review request for giraph. Summary --- GiraphJob uses composition with Job instead of extending Job (exposing all Job methods to users is bad). This addresses bug GIRAPH-144. https://issues.apache.org/jira/browse/GIRAPH-144 Diffs - http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java 1241611 http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/test/java/org/apache/giraph/BspCase.java 1241611 http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/test/java/org/apache/giraph/TestJsonBase64Format.java 1241611 http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleVertexWithWorkerContext.java 1241611 http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/GiraphJob.java 1241611 http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/GiraphRunner.java 1241611 http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleCheckpointVertex.java 1241611 http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleShortestPathsVertex.java 1241611 Diff: https://reviews.apache.org/r/3823/diff Testing --- local and MR unittests. Thanks, Avery
[jira] [Commented] (GIRAPH-142) _hadoopBsp should be prefixable via configuration
[ https://issues.apache.org/jira/browse/GIRAPH-142?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13205047#comment-13205047 ] Avery Ching commented on GIRAPH-142: Looks fine, could we just add a check somewhere that the path must start with / and throw an exception explaining to the user the problem? _hadoopBsp should be prefixable via configuration - Key: GIRAPH-142 URL: https://issues.apache.org/jira/browse/GIRAPH-142 Project: Giraph Issue Type: Improvement Affects Versions: 0.1.0 Reporter: Jakob Homan Assignee: Jakob Homan Fix For: 0.2.0 Attachments: GIRAPH-142.patch In multitennant zookeeper clusters, it would be good to be able to specify the base directory that's created for the _hadoopBsp znodes. This would also fix the issue we have with creating that directory in the source root during tests. -- 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-139) Change PageRankBenchmark to be accessible via bin/giraph
[ https://issues.apache.org/jira/browse/GIRAPH-139?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13203900#comment-13203900 ] Avery Ching commented on GIRAPH-139: I agree the main() and run() code should be deprecated, but preferably after giraph-examples.jar is ready =). Change PageRankBenchmark to be accessible via bin/giraph Key: GIRAPH-139 URL: https://issues.apache.org/jira/browse/GIRAPH-139 Project: Giraph Issue Type: Improvement Affects Versions: 0.2.0 Reporter: Jakob Homan Assignee: Jakob Homan Fix For: 0.2.0 Attachments: GIRAPH-139.patch Currently the PageRankBenchmark has its own main and tool implementation and is difficult to access from the bin/giraph script. It would be better if everything were accessible via bin/giraph. The benchmark is particularly problematic because it uses inner classes for its two actual Vertex implementations, which have to be specified on the command line as their .class name(ie org.apache.giraph.benchmark.PageRankBenchmark$PageRankHashMapVertex) rather than just with dots, as one would expect. -- 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-139) Change PageRankBenchmark to be accessible via bin/giraph
[ https://issues.apache.org/jira/browse/GIRAPH-139?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13203967#comment-13203967 ] Avery Ching commented on GIRAPH-139: sounds good to me. Change PageRankBenchmark to be accessible via bin/giraph Key: GIRAPH-139 URL: https://issues.apache.org/jira/browse/GIRAPH-139 Project: Giraph Issue Type: Improvement Affects Versions: 0.2.0 Reporter: Jakob Homan Assignee: Jakob Homan Fix For: 0.2.0 Attachments: GIRAPH-139.patch Currently the PageRankBenchmark has its own main and tool implementation and is difficult to access from the bin/giraph script. It would be better if everything were accessible via bin/giraph. The benchmark is particularly problematic because it uses inner classes for its two actual Vertex implementations, which have to be specified on the command line as their .class name(ie org.apache.giraph.benchmark.PageRankBenchmark$PageRankHashMapVertex) rather than just with dots, as one would expect. -- 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-144) GiraphJob should not extend Job (users should not be able to call Job methods like waitForCompletion or setMapper..etc)
[ https://issues.apache.org/jira/browse/GIRAPH-144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13204213#comment-13204213 ] Avery Ching commented on GIRAPH-144: I'm working on this, should have a fix by tonight. GiraphJob should not extend Job (users should not be able to call Job methods like waitForCompletion or setMapper..etc) Key: GIRAPH-144 URL: https://issues.apache.org/jira/browse/GIRAPH-144 Project: Giraph Issue Type: Bug Reporter: Dave Assignee: Avery Ching Original Estimate: 24h Remaining Estimate: 24h -- 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] [Assigned] (GIRAPH-144) GiraphJob should not extend Job (users should not be able to call Job methods like waitForCompletion or setMapper..etc)
[ https://issues.apache.org/jira/browse/GIRAPH-144?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Avery Ching reassigned GIRAPH-144: -- Assignee: Avery Ching GiraphJob should not extend Job (users should not be able to call Job methods like waitForCompletion or setMapper..etc) Key: GIRAPH-144 URL: https://issues.apache.org/jira/browse/GIRAPH-144 Project: Giraph Issue Type: Bug Reporter: Dave Assignee: Avery Ching Original Estimate: 24h Remaining Estimate: 24h -- 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