[jira] [Created] (GIRAPH-80) Don't expose the list holding the messages in BasicVertex
Don't expose the list holding the messages in BasicVertex - Key: GIRAPH-80 URL: https://issues.apache.org/jira/browse/GIRAPH-80 Project: Giraph Issue Type: Improvement Affects Versions: 0.70.0 Reporter: Sebastian Schelter I'm currently trying to implement my own memory efficient vertex (similar to LongDoubleFloatDoubleVertex) and ran into problems with getMsgList() This method returns a list pointing to the messages of the vertex and it is modified externally (BasicRPCCommunications calls clear() and addAll() e.g.). This makes it very hard to use something else than a java.util.List internally (LongDoubleFloatDoubleVertex hacked around this) and it is generally dangerous to have the internal state of an object be modified externally. It also makes the code harder to read and understand. I'd suggest to change the API to let a vertex handle the modifications itself internally (e.g. add something like pushMessages(...)) -- 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: better way to update site?
could you remove /www/incubator.apache.org/giraph_ directory? Done. thanks for the update.
[jira] [Commented] (GIRAPH-51) Provide unit testing tool for Giraph algorithms
[ https://issues.apache.org/jira/browse/GIRAPH-51?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13149805#comment-13149805 ] Jakob Homan commented on GIRAPH-51: --- Looks great. A few comments: * It may make sense to move InternalVertexRunner to the src tree rather than test tree, since it's a user-facing class rather than something for Giraph's internal testing. I can imagine us generating a separate test jar soon and we'd want this class in the regular jar we ship to end users. * Is it necessary to specify the input and output formats and to write data out to the file system? In general a vertex implementation should be able to work with reasonable vertices from any input source (part of the GIRAPH-64 work). If the internal vertex runner just fed the values into the compute method we'd save file io and coupling of specific formats. * Can you add javadoc for the public methods? * It looks like the ZooKeeper exceptions are probably race conditions. I see similar ones during regular test execution. It would be nice to remove the need for ZooKeeper on these types of tests: if one is spinning up ZK, it's not really a unit test any more, and it should be possible to test vertex implementations without it. One should be able to just feed input state (vertices, edges, superstep #, etc.) and verify the output state without every actually spinning up any of the distributed infrastructure. But that's probably best done in another JIRA. I don't think the ZK exceptions are something to be concerned about. Provide unit testing tool for Giraph algorithms --- Key: GIRAPH-51 URL: https://issues.apache.org/jira/browse/GIRAPH-51 Project: Giraph Issue Type: Improvement Reporter: Jakob Homan Attachments: GIRAPH-51.patch It would be nice to have a little tool, similar to MRUnit, that would allow Giraph application writers to quickly unit test their algorithms. The tool could take a Vertex implementation, a set of input and expected output and verify that after the specified number of supersteps, we've gotten what we 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] [Updated] (GIRAPH-51) Provide unit testing tool for Giraph algorithms
[ https://issues.apache.org/jira/browse/GIRAPH-51?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jakob Homan updated GIRAPH-51: -- Assignee: Sebastian Schelter Provide unit testing tool for Giraph algorithms --- Key: GIRAPH-51 URL: https://issues.apache.org/jira/browse/GIRAPH-51 Project: Giraph Issue Type: Improvement Reporter: Jakob Homan Assignee: Sebastian Schelter Attachments: GIRAPH-51.patch It would be nice to have a little tool, similar to MRUnit, that would allow Giraph application writers to quickly unit test their algorithms. The tool could take a Vertex implementation, a set of input and expected output and verify that after the specified number of supersteps, we've gotten what we 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
Re: FOSDEM
It would be excellent to be at FOSDEM, please do. I'm hoping to submit something for next year's Berlin Buzzwords, so, uhm, dibs? (as much as is reasonably possible and polite, of course). On Mon, Nov 7, 2011 at 12:38 PM, Avery Ching ach...@apache.org wrote: I think it would be great if someone could talk about Giraph at FOSDEM. Thanks for volunteering, Claudio. I'm not planning to be there unfortunately. Avery On 11/7/11 12:33 PM, Claudio Martella wrote: Hello list, I was thinking about submitting for a talk at fosdem with topic Pregel Giraph. Am I overlapping with somebody else? Best, Claudio
[jira] [Commented] (GIRAPH-80) Don't expose the list holding the messages in BasicVertex
[ https://issues.apache.org/jira/browse/GIRAPH-80?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13149876#comment-13149876 ] Avery Ching commented on GIRAPH-80: --- Since public abstract void compute(IteratorM msgIterator) throws IOException; already gives the msgIterator to the user, getMsgList() should be used to internally transfer the messages to the vertex and get the message iterator. So instead of in BasicVertex public abstract ListM getMsgList(); do something like void abstract addMessages(CollectionM messages); IteratorM abstract getMessageIterator(); Thoughts? Don't expose the list holding the messages in BasicVertex - Key: GIRAPH-80 URL: https://issues.apache.org/jira/browse/GIRAPH-80 Project: Giraph Issue Type: Improvement Affects Versions: 0.70.0 Reporter: Sebastian Schelter I'm currently trying to implement my own memory efficient vertex (similar to LongDoubleFloatDoubleVertex) and ran into problems with getMsgList() This method returns a list pointing to the messages of the vertex and it is modified externally (BasicRPCCommunications calls clear() and addAll() e.g.). This makes it very hard to use something else than a java.util.List internally (LongDoubleFloatDoubleVertex hacked around this) and it is generally dangerous to have the internal state of an object be modified externally. It also makes the code harder to read and understand. I'd suggest to change the API to let a vertex handle the modifications itself internally (e.g. add something like pushMessages(...)) -- 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-80) Don't expose the list holding the messages in BasicVertex
[ https://issues.apache.org/jira/browse/GIRAPH-80?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13149887#comment-13149887 ] Jake Mannix commented on GIRAPH-80: --- Can we change both addMessages() and getMessages() to take/return IterableM? That way it can be scanned multiple times without multiple calls to the vertex. void abstract addMessages(IterableM messages); IterableM abstract getMessages(); Don't expose the list holding the messages in BasicVertex - Key: GIRAPH-80 URL: https://issues.apache.org/jira/browse/GIRAPH-80 Project: Giraph Issue Type: Improvement Affects Versions: 0.70.0 Reporter: Sebastian Schelter I'm currently trying to implement my own memory efficient vertex (similar to LongDoubleFloatDoubleVertex) and ran into problems with getMsgList() This method returns a list pointing to the messages of the vertex and it is modified externally (BasicRPCCommunications calls clear() and addAll() e.g.). This makes it very hard to use something else than a java.util.List internally (LongDoubleFloatDoubleVertex hacked around this) and it is generally dangerous to have the internal state of an object be modified externally. It also makes the code harder to read and understand. I'd suggest to change the API to let a vertex handle the modifications itself internally (e.g. add something like pushMessages(...)) -- 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-80) Don't expose the list holding the messages in BasicVertex
[ https://issues.apache.org/jira/browse/GIRAPH-80?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13149891#comment-13149891 ] Avery Ching commented on GIRAPH-80: --- Seems reasonable to me. Having the option of foreach and iterator is nice. Don't expose the list holding the messages in BasicVertex - Key: GIRAPH-80 URL: https://issues.apache.org/jira/browse/GIRAPH-80 Project: Giraph Issue Type: Improvement Affects Versions: 0.70.0 Reporter: Sebastian Schelter I'm currently trying to implement my own memory efficient vertex (similar to LongDoubleFloatDoubleVertex) and ran into problems with getMsgList() This method returns a list pointing to the messages of the vertex and it is modified externally (BasicRPCCommunications calls clear() and addAll() e.g.). This makes it very hard to use something else than a java.util.List internally (LongDoubleFloatDoubleVertex hacked around this) and it is generally dangerous to have the internal state of an object be modified externally. It also makes the code harder to read and understand. I'd suggest to change the API to let a vertex handle the modifications itself internally (e.g. add something like pushMessages(...)) -- 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-11 : Improve the graph distribution of Giraph
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/2788/#review3211 --- Overall I like it. Please avoid non-essential format changes in large patches; when reviewing it's like trying to run a marathon with a pebble in your shoe. There needs to be quite a bit of unit test coverage on the new classes. Most of them should be amenable to straight-up unit tests rather than ZK-involved integration tests. http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/bsp/CentralizedServiceWorker.java https://reviews.apache.org/r/2788/#comment7131 return type has changed. javadoc needs updated. http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleMutateGraphVertex.java https://reviews.apache.org/r/2788/#comment7152 switch statement? http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SuperstepHashPartitioner.java https://reviews.apache.org/r/2788/#comment7156 This seems like a dangerous thing to leave lying around, even for example purposes. Is there another example that we can generate which might be more useful? http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/VerifyMessage.java https://reviews.apache.org/r/2788/#comment7158 indent +4 http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/VerifyMessage.java https://reviews.apache.org/r/2788/#comment7159 need debugging guards here and +2 lines http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/VerifyMessage.java https://reviews.apache.org/r/2788/#comment7161 log guard http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/VerifyMessage.java https://reviews.apache.org/r/2788/#comment7162 ditto http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/BspServiceWorker.java https://reviews.apache.org/r/2788/#comment7168 typo. send - sent http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/BspServiceWorker.java https://reviews.apache.org/r/2788/#comment7173 typo. sent - send http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/partition/GraphPartitioner.java https://reviews.apache.org/r/2788/#comment7178 Better to call it a factory? http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/partition/HashWorkerPartitioner.java https://reviews.apache.org/r/2788/#comment7180 typo: dependant - dependent http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/partition/PartitionBalancer.java https://reviews.apache.org/r/2788/#comment7182 rename: value - totalValue, to be consistent with usage. http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/partition/RangeWorkerPartitioner.java https://reviews.apache.org/r/2788/#comment7185 I'm unclear on this. - Jakob On 2011-11-14 06:56:19, Avery Ching wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/2788/ --- (Updated 2011-11-14 06:56:19) Review request for giraph. Summary --- Warning: This is a very large change! Vertex ranges no longer exist. A generic partitioner handles the division of vertex ids to partitions. As a default, there is a HashPartitioner and a HashRangePartitioner that will use the hashCode of a Java object to decide which partition to place the vertex. Developers can write their own algorithm to determine how to change the partitioning as well as implement the assignment of partitions to workers. All vertices loaded from the input split are sent to the owner of the partition rather than loaded locally. This eliminates the constraint that the vertices must be ordered in the input split. The checkpoint format has been changed to suit the new partition style. Checkpoints are now a lot simpler. The master will assign partitions and the workers will only load their own partitions from the checkpoint. Unfortunately, the vertex range implementation was baked into almost every aspect of the code (hence the ridiculous size of this diff). But now it should be flexible to support several different graph partitioning schemes (i.e. hash-based, hash-ranged-based, and for special cases, fully ranged-based). Sorry for the long delay, but this way pretty involved. This addresses bug GIRAPH-11.
[jira] [Created] (GIRAPH-81) Create annotations on provided algorithms for cli
Create annotations on provided algorithms for cli - Key: GIRAPH-81 URL: https://issues.apache.org/jira/browse/GIRAPH-81 Project: Giraph Issue Type: New Feature Reporter: Jakob Homan As discussed in GIRAPH-64, it would be nice if the built-in algorithms that ship with Giraph were displayed from cli (ala Mahout) so that users can see what's available. Also, it was suggested that it would be good to use annotations for these to ease in documentation. This sounds like a good approach. -- 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-82) Remove default of using ImmutableOutputCommitter
Remove default of using ImmutableOutputCommitter Key: GIRAPH-82 URL: https://issues.apache.org/jira/browse/GIRAPH-82 Project: Giraph Issue Type: Improvement Reporter: Jakob Homan Currently if no outputformat is specified we default to ImmutableOutputCommitter, essentially piping the results of the work out to /dev/null. This is a useful output option in some cases, but the default probably shouldn't be to throw away work. Making outputformat a required option and providing DevNullOutputFormat, or whatever, would likely save some surprise from users. -- 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-64) Create VertexRunner to make it easier to run users' computations
[ https://issues.apache.org/jira/browse/GIRAPH-64?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13149932#comment-13149932 ] Hudson commented on GIRAPH-64: -- Integrated in Giraph-trunk-Commit #30 (See [https://builds.apache.org/job/Giraph-trunk-Commit/30/]) GIRAPH-64. Create VertexRunner to make it easier to run users' computations. jghoman : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVNview=revrev=1201899 Files : * /incubator/giraph/trunk/CHANGELOG * /incubator/giraph/trunk/bin * /incubator/giraph/trunk/bin/giraph * /incubator/giraph/trunk/pom.xml * /incubator/giraph/trunk/src/main/assembly * /incubator/giraph/trunk/src/main/assembly/assembly.xml * /incubator/giraph/trunk/src/main/java/org/apache/giraph/GiraphRunner.java * /incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/GraphMapper.java Create VertexRunner to make it easier to run users' computations Key: GIRAPH-64 URL: https://issues.apache.org/jira/browse/GIRAPH-64 Project: Giraph Issue Type: New Feature Reporter: Jakob Homan Assignee: Jakob Homan Fix For: 0.70.0 Attachments: GIRAPH-64-2.patch, GIRAPH-64.patch Currently, if a user wants to implement a Giraph algorithm by extending {{Vertex}} they must also write all the boilerplate around the {{Tool}} interface and bundle it with the Giraph jar (or get Giraph on the classpath and playing nice with the implementation). For example, what is included in the PageRankBenchmark and what Kohei has done: https://github.com/smly/java-Giraph-LabelPropagation It would be better if we had perhaps a Vertex implementation to be subclassed that already had all the standard Tooling included such that all one had to run would be (assuming the Giraph jar was already on the classpath): {noformat}hadoop jar my-awesome-vertex.jar my.awesome.vertex -i jazz_input -o jazz_output -if org.apache.giraph.lib.in.text.adjacency-list.LongDoubleDouble -of org.apache.giraph.lib.out.text.adjacency-list.LongDoubleDouble{noformat} This wouldn't work with every algorithm, but would be useful in a large number of cases. -- 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-47) Export Worker's Context/State to vertices through pre/post/Application/Superstep
[ https://issues.apache.org/jira/browse/GIRAPH-47?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jakob Homan resolved GIRAPH-47. --- Resolution: Fixed Export Worker's Context/State to vertices through pre/post/Application/Superstep Key: GIRAPH-47 URL: https://issues.apache.org/jira/browse/GIRAPH-47 Project: Giraph Issue Type: Improvement Components: graph Affects Versions: 0.70.0 Reporter: Claudio Martella Assignee: Claudio Martella Attachments: GIRAPH-47.diff, GIRAPH-47.diff, GIRAPH-47.diff, GIRAPH-47.final It would be quite useful for vertices to reach some worker-related information stored i.e. in the GraphState class. This information could be exported as a parameter to pre/post/Application/Superstep like this: public void preApplication(Configurable workerObject); public void postApplication(Configurable workerObject); public void preSuperstep(Configurable workerObject); public void postSuperstep(Configurable workerObject); public Configurable getWorkerObject(); Another possibility is to add a Context inner class to BasicVertex to store this information. -- 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-11 : Improve the graph distribution of Giraph
On 2011-11-14 20:54:28, Jakob Homan wrote: Overall I like it. Please avoid non-essential format changes in large patches; when reviewing it's like trying to run a marathon with a pebble in your shoe. There needs to be quite a bit of unit test coverage on the new classes. Most of them should be amenable to straight-up unit tests rather than ZK-involved integration tests. Since it's a straight switch from VertexRange objects to Partition objects, everything is tested by the same existing integration tests. I have also added a few more integration tests that ensure reverse ordering, different algorithms, etc. I agree more unittestting should be done rather than integration testing. If you don't mind, I'd like to add those in a later issue as this one is already too big and needs to be committed soon. On 2011-11-14 20:54:28, Jakob Homan wrote: http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/bsp/CentralizedServiceWorker.java, line 111 https://reviews.apache.org/r/2788/diff/2/?file=57771#file57771line111 return type has changed. javadoc needs updated. Fixed. On 2011-11-14 20:54:28, Jakob Homan wrote: http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleMutateGraphVertex.java, line 62 https://reviews.apache.org/r/2788/diff/2/?file=57781#file57781line62 switch statement? In this case, I prefer the if/else if/else logic due since I have to scope each block, making the switch a bit long with the extra cscoping i.e. case 3: { long... On 2011-11-14 20:54:28, Jakob Homan wrote: http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SuperstepHashPartitioner.java, line 49 https://reviews.apache.org/r/2788/diff/2/?file=57784#file57784line49 This seems like a dangerous thing to leave lying around, even for example purposes. Is there another example that we can generate which might be more useful? I have moved it to the test (TestGraphPartitioner), so that people don't just use it. On 2011-11-14 20:54:28, Jakob Homan wrote: http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/VerifyMessage.java, line 123 https://reviews.apache.org/r/2788/diff/2/?file=57785#file57785line123 indent +4 Fixed. On 2011-11-14 20:54:28, Jakob Homan wrote: http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/VerifyMessage.java, line 128 https://reviews.apache.org/r/2788/diff/2/?file=57785#file57785line128 need debugging guards here and +2 lines Fixed. On 2011-11-14 20:54:28, Jakob Homan wrote: http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/VerifyMessage.java, line 157 https://reviews.apache.org/r/2788/diff/2/?file=57785#file57785line157 log guard Fixed. On 2011-11-14 20:54:28, Jakob Homan wrote: http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/VerifyMessage.java, line 162 https://reviews.apache.org/r/2788/diff/2/?file=57785#file57785line162 ditto Fixed. On 2011-11-14 20:54:28, Jakob Homan wrote: http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/BspServiceWorker.java, line 745 https://reviews.apache.org/r/2788/diff/2/?file=57791#file57791line745 typo. send - sent Fixed. On 2011-11-14 20:54:28, Jakob Homan wrote: http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/BspServiceWorker.java, line 1579 https://reviews.apache.org/r/2788/diff/2/?file=57791#file57791line1579 typo. sent - send Fixed. This is sad, especially since English is my only language =). On 2011-11-14 20:54:28, Jakob Homan wrote: http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/partition/GraphPartitioner.java, line 33 https://reviews.apache.org/r/2788/diff/2/?file=57806#file57806line33 Better to call it a factory? Agreed, changed. On 2011-11-14 20:54:28, Jakob Homan wrote: http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/partition/HashWorkerPartitioner.java, line 76 https://reviews.apache.org/r/2788/diff/2/?file=57811#file57811line76 typo: dependant - dependent Changed. On 2011-11-14 20:54:28, Jakob Homan wrote: http://svn.apache.org/repos/asf/incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/partition/PartitionBalancer.java, line 123 https://reviews.apache.org/r/2788/diff/2/?file=57814#file57814line123 rename: value - totalValue, to be consistent with usage. Changed. On 2011-11-14 20:54:28, Jakob Homan wrote:
[jira] [Commented] (GIRAPH-11) Improve the graph distribution of Giraph
[ https://issues.apache.org/jira/browse/GIRAPH-11?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13150095#comment-13150095 ] Jakob Homan commented on GIRAPH-11: --- I think this is ready to go. Avery, just out of curiosity, beyond the MR unittests, have you run any test vertices on this? Improve the graph distribution of Giraph Key: GIRAPH-11 URL: https://issues.apache.org/jira/browse/GIRAPH-11 Project: Giraph Issue Type: Improvement Affects Versions: 0.70.0 Reporter: Avery Ching Assignee: Avery Ching Attachments: GIRAPH-11.2.diff, GIRAPH-11.3.diff, GIRAPH-11.4.diff, GIRAPH-11.diff Currently, Giraph assumes that the data from the VertexInputFormat is sorted. If the user data is not sorted by the vertex id, they must first run a MapReduce or Pig job to generate a sorted dataset. This is often a bit inconvenient. Giraph graph partitioning is currently range based and there are some advantages and disadvantages of this approach. The proposal of this JIRA would be to allow for both range and hash based partitioning and provide more flexibility to the user. Design goals for the graph distribution: * Allow vertices to be unordered or unordered * Ability to repartition * Select the partitioning scheme based on user needs (i.e. hash or range based) * Ability to provide user-specific hints about partitions Hash-based partitioning * Good vertex balancing across ranges for random data * Bad at vertex id locality Range-based partitioning * Good at vertex id locality * Ability to split ranges easily * Can cause hotspots for hot ranges -- 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-11) Improve the graph distribution of Giraph
[ https://issues.apache.org/jira/browse/GIRAPH-11?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13150099#comment-13150099 ] Jakob Homan commented on GIRAPH-11: --- +1 Improve the graph distribution of Giraph Key: GIRAPH-11 URL: https://issues.apache.org/jira/browse/GIRAPH-11 Project: Giraph Issue Type: Improvement Affects Versions: 0.70.0 Reporter: Avery Ching Assignee: Avery Ching Attachments: GIRAPH-11.2.diff, GIRAPH-11.3.diff, GIRAPH-11.4.diff, GIRAPH-11.diff Currently, Giraph assumes that the data from the VertexInputFormat is sorted. If the user data is not sorted by the vertex id, they must first run a MapReduce or Pig job to generate a sorted dataset. This is often a bit inconvenient. Giraph graph partitioning is currently range based and there are some advantages and disadvantages of this approach. The proposal of this JIRA would be to allow for both range and hash based partitioning and provide more flexibility to the user. Design goals for the graph distribution: * Allow vertices to be unordered or unordered * Ability to repartition * Select the partitioning scheme based on user needs (i.e. hash or range based) * Ability to provide user-specific hints about partitions Hash-based partitioning * Good vertex balancing across ranges for random data * Bad at vertex id locality Range-based partitioning * Good at vertex id locality * Ability to split ranges easily * Can cause hotspots for hot ranges -- 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