Details

    • Type: New Feature New Feature
    • Status: Resolved
    • Priority: Major Major
    • Resolution: Fixed
    • Affects Version/s: 0.4.0
    • Fix Version/s: 0.5.0
    • Component/s: bsp core, examples
    • Labels:

      Description

      According to what we've discussed on the mailing list, we should add a Pregel-like API.

      1. test.txt
        5 kB
        Edward J. Yoon
      2. pregel_v03.patch
        81 kB
        Edward J. Yoon
      3. pregel_v02.patch
        77 kB
        Edward J. Yoon
      4. pregel_v01.patch
        64 kB
        Edward J. Yoon
      5. InlinkCount.java
        3 kB
        Thomas Jungblut
      6. hama-graph.zip
        674 kB
        Miklos Erdelyi
      7. HAMA-409_v05.patch
        91 kB
        Thomas Jungblut
      8. HAMA-409_v04.patch
        91 kB
        Edward J. Yoon
      9. HAMA-409_v03.patch
        75 kB
        Thomas Jungblut
      10. HAMA-409_v02.patch
        67 kB
        Thomas Jungblut
      11. HAMA-409_v01.patch
        66 kB
        Edward J. Yoon

        Issue Links

        There are no Sub-Tasks for this issue.

          Activity

          Hide
          Edward J. Yoon added a comment -

          Graph examples' unit tests are also fixed.

          I just closed this issue.

          Show
          Edward J. Yoon added a comment - Graph examples' unit tests are also fixed. I just closed this issue.
          Hide
          Edward J. Yoon added a comment -
          
          12/02/27 13:54:08 DEBUG bsp.Counters: Creating group org.apache.hama.bsp.JobInProgress$JobCounter with nothing
          12/02/27 13:54:11 DEBUG bsp.Counters: Creating group org.apache.hama.bsp.JobInProgress$JobCounter with nothing
          12/02/27 13:54:11 DEBUG bsp.Counters: Creating group org.apache.hama.bsp.BSPPeerImpl$PeerCounter with nothing
          12/02/27 13:54:11 INFO bsp.BSPJobClient: Current supersteps number: 5
          12/02/27 13:54:11 DEBUG bsp.Counters: Creating group org.apache.hama.bsp.JobInProgress$JobCounter with nothing
          12/02/27 13:54:11 DEBUG bsp.Counters: Creating group org.apache.hama.bsp.BSPPeerImpl$PeerCounter with nothing
          12/02/27 13:54:11 DEBUG bsp.Counters: Creating group org.apache.hama.bsp.JobInProgress$JobCounter with nothing
          12/02/27 13:54:11 DEBUG bsp.Counters: Creating group org.apache.hama.bsp.BSPPeerImpl$PeerCounter with nothing
          12/02/27 13:54:11 INFO bsp.BSPJobClient: The total number of supersteps: 5
          12/02/27 13:54:11 DEBUG bsp.Counters: Adding SUPERSTEPS
          12/02/27 13:54:11 INFO bsp.BSPJobClient: Counters: 3
          12/02/27 13:54:11 INFO bsp.BSPJobClient:   org.apache.hama.bsp.JobInProgress$JobCounter
          12/02/27 13:54:11 INFO bsp.BSPJobClient:     LAUNCHED_TASKS=2
          12/02/27 13:54:11 INFO bsp.BSPJobClient:   org.apache.hama.bsp.BSPPeerImpl$PeerCounter
          12/02/27 13:54:11 INFO bsp.BSPJobClient:     SUPERSTEPS=5
          12/02/27 13:54:11 INFO bsp.BSPJobClient:     SUPERSTEP_SUM=5
          12/02/27 13:54:11 DEBUG bsp.Counters: Creating group org.apache.hama.bsp.JobInProgress$JobCounter with nothing
          12/02/27 13:54:11 DEBUG bsp.Counters: Creating group org.apache.hama.bsp.BSPPeerImpl$PeerCounter with nothing
          12/02/27 13:54:11 DEBUG bsp.Counters: Creating group org.apache.hama.bsp.JobInProgress$JobCounter with nothing
          12/02/27 13:54:11 DEBUG bsp.Counters: Creating group org.apache.hama.bsp.BSPPeerImpl$PeerCounter with nothing
          Job Finished in 3.638 seconds
          

          Tested on my laptop.
          I'm commit to TRUNK directly and will open new ticket for other issues.

          Show
          Edward J. Yoon added a comment - 12/02/27 13:54:08 DEBUG bsp.Counters: Creating group org.apache.hama.bsp.JobInProgress$JobCounter with nothing 12/02/27 13:54:11 DEBUG bsp.Counters: Creating group org.apache.hama.bsp.JobInProgress$JobCounter with nothing 12/02/27 13:54:11 DEBUG bsp.Counters: Creating group org.apache.hama.bsp.BSPPeerImpl$PeerCounter with nothing 12/02/27 13:54:11 INFO bsp.BSPJobClient: Current supersteps number: 5 12/02/27 13:54:11 DEBUG bsp.Counters: Creating group org.apache.hama.bsp.JobInProgress$JobCounter with nothing 12/02/27 13:54:11 DEBUG bsp.Counters: Creating group org.apache.hama.bsp.BSPPeerImpl$PeerCounter with nothing 12/02/27 13:54:11 DEBUG bsp.Counters: Creating group org.apache.hama.bsp.JobInProgress$JobCounter with nothing 12/02/27 13:54:11 DEBUG bsp.Counters: Creating group org.apache.hama.bsp.BSPPeerImpl$PeerCounter with nothing 12/02/27 13:54:11 INFO bsp.BSPJobClient: The total number of supersteps: 5 12/02/27 13:54:11 DEBUG bsp.Counters: Adding SUPERSTEPS 12/02/27 13:54:11 INFO bsp.BSPJobClient: Counters: 3 12/02/27 13:54:11 INFO bsp.BSPJobClient: org.apache.hama.bsp.JobInProgress$JobCounter 12/02/27 13:54:11 INFO bsp.BSPJobClient: LAUNCHED_TASKS=2 12/02/27 13:54:11 INFO bsp.BSPJobClient: org.apache.hama.bsp.BSPPeerImpl$PeerCounter 12/02/27 13:54:11 INFO bsp.BSPJobClient: SUPERSTEPS=5 12/02/27 13:54:11 INFO bsp.BSPJobClient: SUPERSTEP_SUM=5 12/02/27 13:54:11 DEBUG bsp.Counters: Creating group org.apache.hama.bsp.JobInProgress$JobCounter with nothing 12/02/27 13:54:11 DEBUG bsp.Counters: Creating group org.apache.hama.bsp.BSPPeerImpl$PeerCounter with nothing 12/02/27 13:54:11 DEBUG bsp.Counters: Creating group org.apache.hama.bsp.JobInProgress$JobCounter with nothing 12/02/27 13:54:11 DEBUG bsp.Counters: Creating group org.apache.hama.bsp.BSPPeerImpl$PeerCounter with nothing Job Finished in 3.638 seconds Tested on my laptop. I'm commit to TRUNK directly and will open new ticket for other issues.
          Hide
          Edward J. Yoon added a comment -

          And also, Vertex class reflection seems problematic. I'll fix tomorrow.

          Thanks and Enjoy your weekend Thomas!

          Show
          Edward J. Yoon added a comment - And also, Vertex class reflection seems problematic. I'll fix tomorrow. Thanks and Enjoy your weekend Thomas!
          Hide
          Thomas Jungblut added a comment -

          I believe it is because of the updated condition.

          In each task you are going through a part of the graph and check if they send some messages, if no message is sent, you leave the loop.
          I guess this causes a lock in the barrier sync, because not all tasks are reaching it then.

          We had similar problems with SSSP and Pagerank in the beginning
          We could use counters to check if vertices have been updated globally (HAMA-515). Or we have to manage this be messaging like in SSSP we had before.

          Show
          Thomas Jungblut added a comment - I believe it is because of the updated condition. In each task you are going through a part of the graph and check if they send some messages, if no message is sent, you leave the loop. I guess this causes a lock in the barrier sync, because not all tasks are reaching it then. We had similar problems with SSSP and Pagerank in the beginning We could use counters to check if vertices have been updated globally ( HAMA-515 ). Or we have to manage this be messaging like in SSSP we had before.
          Hide
          Edward J. Yoon added a comment -

          maybe it's related with multiple graph, examples jar files when job submission is done.

          Show
          Edward J. Yoon added a comment - maybe it's related with multiple graph, examples jar files when job submission is done.
          Hide
          Edward J. Yoon added a comment -

          Reopen. It doesn't work on distributed mode.

          Show
          Edward J. Yoon added a comment - Reopen. It doesn't work on distributed mode.
          Hide
          Edward J. Yoon added a comment -

          Just committed this.

          Vertex Combiners/Aggregators, output interfaces, .., etc. can be added later.

          Show
          Edward J. Yoon added a comment - Just committed this. Vertex Combiners/Aggregators, output interfaces, .., etc. can be added later.
          Hide
          Thomas Jungblut added a comment -

          Yeah, let's add this to trunk.
          We can always open new issues regarding improvements.

          Show
          Thomas Jungblut added a comment - Yeah, let's add this to trunk. We can always open new issues regarding improvements.
          Hide
          Edward J. Yoon added a comment -

          args.length checker added and replaced compute parameter like:

          public void compute(Iterator<MSGTYPE> messages) throws IOException;
          

          Others are little related with HAMA-502 and new APIs.

          I'm going to add this to trunk. What do you think?

          Show
          Edward J. Yoon added a comment - args.length checker added and replaced compute parameter like: public void compute(Iterator<MSGTYPE> messages) throws IOException; Others are little related with HAMA-502 and new APIs. I'm going to add this to trunk. What do you think?
          Hide
          Edward J. Yoon added a comment -

          Maybe I can improve more tomorrow.

          Show
          Edward J. Yoon added a comment - Maybe I can improve more tomorrow.
          Hide
          Thomas Jungblut added a comment -

          Here's my inlink count.
          Now I know why there is this getValue().

          Show
          Thomas Jungblut added a comment - Here's my inlink count. Now I know why there is this getValue().
          Hide
          Thomas Jungblut added a comment -

          I'm just trying to use it for inlinkcount.

          Things I notice:

          • Vertex class constructor is swallowing the exception (does the value be need to be in the basic vertex class?)
          • Why are the setters avaiable? We should declare them package visible.
          • SSSP and pagerank needs to check the args (minor thing)
          • MessageIterator should have a hasNext() method.

          We have to add the ability to write...

          But otherwise good work. I like it.

          Show
          Thomas Jungblut added a comment - I'm just trying to use it for inlinkcount. Things I notice: Vertex class constructor is swallowing the exception (does the value be need to be in the basic vertex class?) Why are the setters avaiable? We should declare them package visible. SSSP and pagerank needs to check the args (minor thing) MessageIterator should have a hasNext() method. We have to add the ability to write... But otherwise good work. I like it.
          Hide
          Edward J. Yoon added a comment -

          This v02 patch replaces PageRank example, too.

          Please review about Interfaces.

          Show
          Edward J. Yoon added a comment - This v02 patch replaces PageRank example, too. Please review about Interfaces.
          Hide
          Thomas Jungblut added a comment -

          This looks cool! I have a closer look later when I'm in my lecture.
          Let's translate pagerank as well and add inlink count from Miklos.

          Show
          Thomas Jungblut added a comment - This looks cool! I have a closer look later when I'm in my lecture. Let's translate pagerank as well and add inlink count from Miklos.
          Hide
          Edward J. Yoon added a comment -

          This patch replaces SSSP example to GraphJobRunner based.

          Show
          Edward J. Yoon added a comment - This patch replaces SSSP example to GraphJobRunner based.
          Hide
          Edward J. Yoon added a comment -

          This is my test patch. Unit test is OK.

          In ShortestPathVertex class, SSSP can be wrote like this:

            public void compute(List<ShortestPathVertexMessage> value) throws IOException {
              int minDist = this.getCost();
          
              for (ShortestPathVertexMessage msg : value) {
                if (msg.getData() < minDist)
                  minDist = msg.getData();
              }
          
              if (minDist < this.getCost()) {
                this.setCost(minDist);
                for (ShortestPathVertex e : this.getEdges()) {
                  sendMessage(e, minDist + e.getWeight());
                }
              }
            }
          

          In this issue, the key pointi is interface designing. Optimization issues can be handled later.

          Show
          Edward J. Yoon added a comment - This is my test patch. Unit test is OK. In ShortestPathVertex class, SSSP can be wrote like this: public void compute(List<ShortestPathVertexMessage> value) throws IOException { int minDist = this .getCost(); for (ShortestPathVertexMessage msg : value) { if (msg.getData() < minDist) minDist = msg.getData(); } if (minDist < this .getCost()) { this .setCost(minDist); for (ShortestPathVertex e : this .getEdges()) { sendMessage(e, minDist + e.getWeight()); } } } In this issue, the key pointi is interface designing. Optimization issues can be handled later.
          Hide
          Edward J. Yoon added a comment -

          I'd like to add this feature to 0.4 even if it's not perfect or simple.

          I noticed that many people who are unfamiliar with internals recognizes this as one of the main difference between others and the weakness of our project. :/

          Show
          Edward J. Yoon added a comment - I'd like to add this feature to 0.4 even if it's not perfect or simple. I noticed that many people who are unfamiliar with internals recognizes this as one of the main difference between others and the weakness of our project. :/
          Hide
          Thomas Jungblut added a comment -

          Rescheduled this to 0.5.0 and add a blocked by i/o system.

          Show
          Thomas Jungblut added a comment - Rescheduled this to 0.5.0 and add a blocked by i/o system.
          Hide
          Edward J. Yoon added a comment -

          Let's add basic interfaces first. I'll do.

          Show
          Edward J. Yoon added a comment - Let's add basic interfaces first. I'll do.
          Hide
          Thomas Jungblut added a comment -

          Hey guys, what is going on here? Should we split the task? Commit it?
          What's the current state?

          Show
          Thomas Jungblut added a comment - Hey guys, what is going on here? Should we split the task? Commit it? What's the current state?
          Hide
          Thomas Jungblut added a comment - - edited

          How about we separate this big issue into smaller tasks?

          How about splitting it into:

          > Create graph module
          > Integrate HAMA-423 (Partitioning and vertices)
          > Integrate the vertex class to our BSPJob

          So we can commit this now...

          Show
          Thomas Jungblut added a comment - - edited How about we separate this big issue into smaller tasks? How about splitting it into: > Create graph module > Integrate HAMA-423 (Partitioning and vertices) > Integrate the vertex class to our BSPJob So we can commit this now...
          Hide
          Thomas Jungblut added a comment - - edited

          Updated to current revision 1159872.

          Several things to do:

          -Integrate the vertices and partitioners of HAMA-423.
          -Integrate the vertex class to our BSPJob
          Get rid of the ZooKeeper stuff in the PartitionManager and ComputeRunner < This is used for dynamic partition assignment to peers.
          ->Extract a graph module for maven

          Show
          Thomas Jungblut added a comment - - edited Updated to current revision 1159872. Several things to do: -Integrate the vertices and partitioners of HAMA-423 . -Integrate the vertex class to our BSPJob Get rid of the ZooKeeper stuff in the PartitionManager and ComputeRunner < This is used for dynamic partition assignment to peers. ->Extract a graph module for maven
          Hide
          Edward J. Yoon added a comment -

          Miklos seems busy. How about we separate this big issue into smaller tasks?

          Show
          Edward J. Yoon added a comment - Miklos seems busy. How about we separate this big issue into smaller tasks?
          Hide
          Edward J. Yoon added a comment -

          I renamed methods

          • voteForHalt -> voteToHalt
          • outEdgeIterator -> getOutEdgeIterator
          Show
          Edward J. Yoon added a comment - I renamed methods voteForHalt -> voteToHalt outEdgeIterator -> getOutEdgeIterator
          Hide
          Thomas Jungblut added a comment -

          Don't have the paper anymore
          Would you mind to correct this if it is really voteToHalt()?^^

          Show
          Thomas Jungblut added a comment - Don't have the paper anymore Would you mind to correct this if it is really voteToHalt()?^^
          Hide
          Edward J. Yoon added a comment -

          Isn't it called voteToHalt()?

          Show
          Edward J. Yoon added a comment - Isn't it called voteToHalt()?
          Hide
          Thomas Jungblut added a comment -

          actually this should be version 3

          Show
          Thomas Jungblut added a comment - actually this should be version 3
          Hide
          Thomas Jungblut added a comment -

          Okay next try, this time under ubuntu.

          • voteForHalt() is added to the vertex
          • BSP class is now a concrete implementation, the interface isn't implemented anymore.
          • added setup / cleanup instead of configurable.
          • refactored somewhat to get the things running again.

          Next step is the integration to the BSPJob. I leave this to you

          Thanks Miklos, great work.

          Show
          Thomas Jungblut added a comment - Okay next try, this time under ubuntu. voteForHalt() is added to the vertex BSP class is now a concrete implementation, the interface isn't implemented anymore. added setup / cleanup instead of configurable. refactored somewhat to get the things running again. Next step is the integration to the BSPJob. I leave this to you Thanks Miklos, great work.
          Hide
          Thomas Jungblut added a comment - - edited

          Yeah that is what I meant, although I dislike the design.

          EDIT:
          Sorry I have to delay the voteforhalt method and refactoring to tomorrow.
          Please discard the last patch, this wasn't build on linux. I'll have my ubuntu machine back tomorrow.

          Show
          Thomas Jungblut added a comment - - edited Yeah that is what I meant, although I dislike the design. EDIT: Sorry I have to delay the voteforhalt method and refactoring to tomorrow. Please discard the last patch, this wasn't build on linux. I'll have my ubuntu machine back tomorrow.
          Hide
          Edward J. Yoon added a comment -

          >> But we can actually add job.setVertexClass(Blahblah.class), if this is set we can automatically set the ComputeGenericBSP/Pregel BSP.

          Like this?

          public class MyVertex extends Vertex {
            public void compute(MessageValueIterator<LongWritable> messages)
                  throws IOException {
               ...
            }
          }
          
          public void main(String[] args) {
            HamaConfiguration conf = new HamaConfiguration();
            BSPJob job = new BSPJob(conf);
          
            job.setVertexClass(MyVertex.class);
            job.setMessageValueClass(LongWritable.class);
            ...
          
          Show
          Edward J. Yoon added a comment - >> But we can actually add job.setVertexClass(Blahblah.class), if this is set we can automatically set the ComputeGenericBSP/Pregel BSP. Like this? public class MyVertex extends Vertex { public void compute(MessageValueIterator<LongWritable> messages) throws IOException { ... } } public void main( String [] args) { HamaConfiguration conf = new HamaConfiguration(); BSPJob job = new BSPJob(conf); job.setVertexClass(MyVertex.class); job.setMessageValueClass(LongWritable.class); ...
          Hide
          Edward J. Yoon added a comment -

          If Hama job can be submitted to Hadoop nextGen, all BSP-based Hama jobs e.g., pure BSP job, GraphJob, .., etc should be submitted as a Hama job.

          Otherwise, I think it's too incoherent ..

          Show
          Edward J. Yoon added a comment - If Hama job can be submitted to Hadoop nextGen, all BSP-based Hama jobs e.g., pure BSP job, GraphJob, .., etc should be submitted as a Hama job. Otherwise, I think it's too incoherent ..
          Hide
          Thomas Jungblut added a comment -

          I think, we should consider about the integrated management and consistent job configuration interface

          -1, I don't think this is good encapsulation.

          But we can actually add job.setVertexClass(Blahblah.class), if this is set we can automatically set the ComputeGenericBSP/Pregel BSP.
          If the user decides to override this with his own ComputationStuff, he should feel free to do so. But I don't think this is needed by Matrixcomputation.

          Show
          Thomas Jungblut added a comment - I think, we should consider about the integrated management and consistent job configuration interface -1, I don't think this is good encapsulation. But we can actually add job.setVertexClass(Blahblah.class), if this is set we can automatically set the ComputeGenericBSP/Pregel BSP. If the user decides to override this with his own ComputationStuff, he should feel free to do so. But I don't think this is needed by Matrixcomputation.
          Hide
          Edward J. Yoon added a comment -

          Yeah, it's a hard to decision.

          I think, we should consider about the integrated management and consistent job configuration interface. Maybe another computing framework on top of Hama BSP can be added.

          Show
          Edward J. Yoon added a comment - Yeah, it's a hard to decision. I think, we should consider about the integrated management and consistent job configuration interface. Maybe another computing framework on top of Hama BSP can be added.
          Hide
          Thomas Jungblut added a comment -

          This is a hard decision.

          I think we should keep the "BSPJob", and just extend a "HamaJob" which has the Pregel API.
          So we keep the BSP engine under the hood and available for matrix computation (which we can replace with something like "thinking like a vector" haha).

          I would not rename it to "Job" solely since we have Hadoop in our classpath, this would just let everybody think that we're something similar like hive or pig once again. With "HamaJob" we can leech the fame of the PregelAPI and establish our name within

          Since we give an iterator to the compute function we should consider using just one instance of the message and refill it. I did too extensive use of reflections then and it is causing so much GC. This should be possible when setting the messagevalue class and vertex class.

          Show
          Thomas Jungblut added a comment - This is a hard decision. I think we should keep the "BSPJob", and just extend a "HamaJob" which has the Pregel API. So we keep the BSP engine under the hood and available for matrix computation (which we can replace with something like "thinking like a vector" haha). I would not rename it to "Job" solely since we have Hadoop in our classpath, this would just let everybody think that we're something similar like hive or pig once again. With "HamaJob" we can leech the fame of the PregelAPI and establish our name within Since we give an iterator to the compute function we should consider using just one instance of the message and refill it. I did too extensive use of reflections then and it is causing so much GC. This should be possible when setting the messagevalue class and vertex class.
          Hide
          Edward J. Yoon added a comment -

          I think, we should re-name the BSPJob to just Job or HamaJob.

          Show
          Edward J. Yoon added a comment - I think, we should re-name the BSPJob to just Job or HamaJob.
          Hide
          Edward J. Yoon added a comment -

          Looks good.

          >> Should we extend a GraphComputationJob for this? Don't want to mess up the BSPJob for that.

          Like this?

          BSPJob job = new PregelJob(conf);
          
          Show
          Edward J. Yoon added a comment - Looks good. >> Should we extend a GraphComputationJob for this? Don't want to mess up the BSPJob for that. Like this? BSPJob job = new PregelJob(conf);
          Hide
          Thomas Jungblut added a comment - - edited
          • Added a generic message with classcaching
          • Refactored to generic messaging

          I add the voteForHalt() method tomorrow.

          EDIT:
          added a testcase for the generic messaging. As Edward mentioned, we should add a job.setMessageValueClass(X.class). Should we extend a GraphComputationJob for this? Don't want to mess up the BSPJob for that.

          Show
          Thomas Jungblut added a comment - - edited Added a generic message with classcaching Refactored to generic messaging I add the voteForHalt() method tomorrow. EDIT: added a testcase for the generic messaging. As Edward mentioned, we should add a job.setMessageValueClass(X.class). Should we extend a GraphComputationJob for this? Don't want to mess up the BSPJob for that.
          Hide
          Thomas Jungblut added a comment -

          Very nice work Miklos, thank you very much.
          I'm going to extend this with voteForHalt(), as an alternative to the iteration maximum.

          Show
          Thomas Jungblut added a comment - Very nice work Miklos, thank you very much. I'm going to extend this with voteForHalt(), as an alternative to the iteration maximum.
          Hide
          Edward J. Yoon added a comment -

          I just created a wiki page to design more elegant programming model.

          http://wiki.apache.org/hama/GraphModel

          Show
          Edward J. Yoon added a comment - I just created a wiki page to design more elegant programming model. http://wiki.apache.org/hama/GraphModel
          Hide
          Edward J. Yoon added a comment -

          This patch only contains an interfaces and few core codes. Generally looks good.

          Here's my few comments.

          1. vertexClass and messageValueClass can be defined when configure a Hama job like Map/Reduce e.g., job.setMessageValueClass(LongWritable.class); instead of using constructor. Then, we can program a graph algorithm and configure a job at one class.

          2. Let's re-code the In/Output parts.

          Show
          Edward J. Yoon added a comment - This patch only contains an interfaces and few core codes. Generally looks good. Here's my few comments. 1. vertexClass and messageValueClass can be defined when configure a Hama job like Map/Reduce e.g., job.setMessageValueClass(LongWritable.class); instead of using constructor. Then, we can program a graph algorithm and configure a job at one class. 2. Let's re-code the In/Output parts.
          Hide
          Edward J. Yoon added a comment -
           * Graph transitions.
           * 
           * @author ela, ekr@cs.vu.nl
           * 
           * @param <TNode>
          

          Where this code came from? We should remove all GNU/GPL codes.

          Show
          Edward J. Yoon added a comment - * Graph transitions. * * @author ela, ekr@cs.vu.nl * * @param <TNode> Where this code came from? We should remove all GNU/GPL codes.
          Hide
          Edward J. Yoon added a comment -

          zip file shouldn't submitted to hudson. I'll upload new patch for this.

          Show
          Edward J. Yoon added a comment - zip file shouldn't submitted to hudson. I'll upload new patch for this.
          Hide
          Hudson added a comment -

          -1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12486149/hama-graph.zip
          against trunk revision 1143241.

          @author +1. The patch does not contain any @author tags.

          tests included +1. The patch appears to include 8 new or modified tests.

          patch -1. The patch command could not apply the patch.

          Console output: http://builds.apache.org/hudson/job/Hama-Patch/345/console

          This message is automatically generated.

          Show
          Hudson added a comment - -1 overall. Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12486149/hama-graph.zip against trunk revision 1143241. @author +1. The patch does not contain any @author tags. tests included +1. The patch appears to include 8 new or modified tests. patch -1. The patch command could not apply the patch. Console output: http://builds.apache.org/hudson/job/Hama-Patch/345/console This message is automatically generated.
          Hide
          Edward J. Yoon added a comment -

          Minor comment, we should remove all "a) Graph input" related code and GNU licensed libraries.

          Show
          Edward J. Yoon added a comment - Minor comment, we should remove all "a) Graph input" related code and GNU licensed libraries.
          Hide
          Edward J. Yoon added a comment -

          Excellent.

          Show
          Edward J. Yoon added a comment - Excellent.
          Hide
          Miklos Erdelyi added a comment -

          To get HAMA graph running from the attached archive please follow the below steps:
          1) Unzip the archive in the root of a freshly checked out HAMA trunk.

          2) Get fastutil (http://fastutil.dsi.unimi.it/) and put its jar under $

          {basedir}

          /lib. I needed to put two other dependencies there because I could not find them in a public maven repository (dsiutils and webgraph).

          3) To run the example inlink counter hama-graph job, invoke:
          java org.apache.hama.graph.GraphComputeRunner <path-to-partitioned-graph>/ <number-of-partitions> org.apache.hama.graph.examples.InlinkCountComputer <max-iterations>

          All the dependencies need to be on the classpath including the used HAMA distribution's config directory. The partitioned graph needs to be accessible through NFS on all the cluster nodes.

          A few thoughts on current implementation:

          a) Graph input
          Currently HipG's (www.cs.vu.nl/~ekr/hipg/) segmented graph loader is used for loading partitioned graphs. A conversion util is provided (org.apache.hama.graph.format.ConvertFromWebGraphASCII.java) which can convert from WebGraph's (http://webgraph.dsi.unimi.it/) ASCII format into the segmented graph format.

          This mechanism should be replaced with a builder-like approach employed by e.g. GoldenOrb: based on some input specification a factory class dependent on the input format of the graph should load vertexes into memory.
          A good approach would be letting workers load data-local segments of the graph and do a partition-worker assignment based on data-locality, after which loaded vertexes not belonging to one worker's partition(s) could be sent to the proper workers via network.

          b) Message buffering
          In the current implementation messages are grouped by vertexes after a new superstep starts, i.e., VertexComputerGeneric iterates through all messages recevied by the BSPPeer and puts them into the queue of respective vertexes.
          This memory copying could be avoided by letting the BSP framework do the grouping while receiving the messages, i.e., by grouping messages by tag such that messages intended for the same vertex ID will have the same tag.

          c) Output
          Currently nothing is output from the vertexes. However, it would be relatively easy to add, e.g., by requiring that the Vertex class implement Writable and serializing all vertexes at the end of certain supersteps. This would be a step towards fault-tolerance too.

          Show
          Miklos Erdelyi added a comment - To get HAMA graph running from the attached archive please follow the below steps: 1) Unzip the archive in the root of a freshly checked out HAMA trunk. 2) Get fastutil ( http://fastutil.dsi.unimi.it/ ) and put its jar under $ {basedir} /lib. I needed to put two other dependencies there because I could not find them in a public maven repository (dsiutils and webgraph). 3) To run the example inlink counter hama-graph job, invoke: java org.apache.hama.graph.GraphComputeRunner <path-to-partitioned-graph>/ <number-of-partitions> org.apache.hama.graph.examples.InlinkCountComputer <max-iterations> All the dependencies need to be on the classpath including the used HAMA distribution's config directory. The partitioned graph needs to be accessible through NFS on all the cluster nodes. A few thoughts on current implementation: a) Graph input Currently HipG's (www.cs.vu.nl/~ekr/hipg/) segmented graph loader is used for loading partitioned graphs. A conversion util is provided (org.apache.hama.graph.format.ConvertFromWebGraphASCII.java) which can convert from WebGraph's ( http://webgraph.dsi.unimi.it/ ) ASCII format into the segmented graph format. This mechanism should be replaced with a builder-like approach employed by e.g. GoldenOrb: based on some input specification a factory class dependent on the input format of the graph should load vertexes into memory. A good approach would be letting workers load data-local segments of the graph and do a partition-worker assignment based on data-locality, after which loaded vertexes not belonging to one worker's partition(s) could be sent to the proper workers via network. b) Message buffering In the current implementation messages are grouped by vertexes after a new superstep starts, i.e., VertexComputerGeneric iterates through all messages recevied by the BSPPeer and puts them into the queue of respective vertexes. This memory copying could be avoided by letting the BSP framework do the grouping while receiving the messages, i.e., by grouping messages by tag such that messages intended for the same vertex ID will have the same tag. c) Output Currently nothing is output from the vertexes. However, it would be relatively easy to add, e.g., by requiring that the Vertex class implement Writable and serializing all vertexes at the end of certain supersteps. This would be a step towards fault-tolerance too.
          Hide
          Thomas Jungblut added a comment -

          In addition we should update the Website to be "Pregel-Like"

          Show
          Thomas Jungblut added a comment - In addition we should update the Website to be "Pregel-Like"

            People

            • Assignee:
              Edward J. Yoon
              Reporter:
              Thomas Jungblut
            • Votes:
              3 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development