hama-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From edwardy...@apache.org
Subject svn commit: r1456332 - in /hama/trunk: examples/src/test/java/org/apache/hama/examples/PageRankTest.java graph/src/main/java/org/apache/hama/graph/GraphJobRunner.java graph/src/main/java/org/apache/hama/graph/ListVerticesInfo.java
Date Thu, 14 Mar 2013 05:36:48 GMT
Author: edwardyoon
Date: Thu Mar 14 05:36:47 2013
New Revision: 1456332

URL: http://svn.apache.org/r1456332
Log:
HAMA-745: Fix bugs of GraphJobRunner

Modified:
    hama/trunk/examples/src/test/java/org/apache/hama/examples/PageRankTest.java
    hama/trunk/graph/src/main/java/org/apache/hama/graph/GraphJobRunner.java
    hama/trunk/graph/src/main/java/org/apache/hama/graph/ListVerticesInfo.java

Modified: hama/trunk/examples/src/test/java/org/apache/hama/examples/PageRankTest.java
URL: http://svn.apache.org/viewvc/hama/trunk/examples/src/test/java/org/apache/hama/examples/PageRankTest.java?rev=1456332&r1=1456331&r2=1456332&view=diff
==============================================================================
--- hama/trunk/examples/src/test/java/org/apache/hama/examples/PageRankTest.java (original)
+++ hama/trunk/examples/src/test/java/org/apache/hama/examples/PageRankTest.java Thu Mar 14
05:36:47 2013
@@ -27,20 +27,14 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
 import org.apache.hama.HamaConfiguration;
-import org.apache.hama.bsp.TextArrayWritable;
+import org.apache.hama.examples.util.FastGraphGen;
 import org.junit.Test;
 
 /**
  * Testcase for {@link PageRank}
  */
 public class PageRankTest extends TestCase {
-  String[] input = new String[] { "1\t2\t3", "2", "3\t1\t2\t5", "4\t5\t6",
-      "5\t4\t6", "6\t4", "7\t2\t4" };
-
   private static String INPUT = "/tmp/page-tmp.seq";
   private static String TEXT_INPUT = "/tmp/page.txt";
   private static String TEXT_OUTPUT = INPUT + "page.txt.seq";
@@ -75,7 +69,6 @@ public class PageRankTest extends TestCa
           fs.open(fts.getPath())));
       String line = null;
       while ((line = reader.readLine()) != null) {
-        System.out.println(line);
         String[] split = line.split("\t");
         sum += Double.parseDouble(split[1]);
       }
@@ -86,25 +79,8 @@ public class PageRankTest extends TestCa
 
   private void generateTestData() {
     try {
-      SequenceFile.Writer writer1 = SequenceFile.createWriter(fs, conf,
-          new Path(INPUT + "/part0"), Text.class, TextArrayWritable.class);
-
-      for (int i = 0; i < input.length; i++) {
-        String[] x = input[i].split("\t");
-
-        Text vertex = new Text(x[0]);
-        TextArrayWritable arr = new TextArrayWritable();
-        Writable[] values = new Writable[x.length - 1];
-        for (int j = 1; j < x.length; j++) {
-          values[j - 1] = new Text(x[j]);
-        }
-        arr.set(values);
-        writer1.append(vertex, arr);
-      }
-
-      writer1.close();
-
-    } catch (IOException e) {
+      FastGraphGen.main(new String[] { "400", "10", INPUT, "2" });
+    } catch (Exception e) {
       e.printStackTrace();
     }
   }

Modified: hama/trunk/graph/src/main/java/org/apache/hama/graph/GraphJobRunner.java
URL: http://svn.apache.org/viewvc/hama/trunk/graph/src/main/java/org/apache/hama/graph/GraphJobRunner.java?rev=1456332&r1=1456331&r2=1456332&view=diff
==============================================================================
--- hama/trunk/graph/src/main/java/org/apache/hama/graph/GraphJobRunner.java (original)
+++ hama/trunk/graph/src/main/java/org/apache/hama/graph/GraphJobRunner.java Thu Mar 14 05:36:47
2013
@@ -345,7 +345,8 @@ public final class GraphJobRunner<V exte
     aggregationRunner = new AggregationRunner<V, E, M>();
     aggregationRunner.setupAggregators(peer);
 
-    vertices = new DiskVerticesInfo<V, E, M>();
+    // FIXME We should make this configurable.
+    vertices = new ListVerticesInfo();
     vertices.init(this, conf, peer.getTaskId());
   }
 
@@ -388,7 +389,13 @@ public final class GraphJobRunner<V exte
       if (selfReference) {
         vertex.addEdge(new Edge<V, E>(vertex.getVertexID(), null));
       }
+
       vertices.addVertex(vertex);
+
+      // Reinitializing vertex object for memory based implementations of
+      // VerticesInfo
+      vertex = GraphJobRunner.<V, E, M> newVertexInstance(VERTEX_CLASS);
+      vertex.runner = this;
     }
     vertices.finishAdditions();
     // finish the "superstep" because we have written a new file here

Modified: hama/trunk/graph/src/main/java/org/apache/hama/graph/ListVerticesInfo.java
URL: http://svn.apache.org/viewvc/hama/trunk/graph/src/main/java/org/apache/hama/graph/ListVerticesInfo.java?rev=1456332&r1=1456331&r2=1456332&view=diff
==============================================================================
--- hama/trunk/graph/src/main/java/org/apache/hama/graph/ListVerticesInfo.java (original)
+++ hama/trunk/graph/src/main/java/org/apache/hama/graph/ListVerticesInfo.java Thu Mar 14
05:36:47 2013
@@ -19,6 +19,7 @@ package org.apache.hama.graph;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
@@ -88,7 +89,7 @@ public final class ListVerticesInfo<V ex
 
   @Override
   public void finishAdditions() {
-
+    Collections.sort(vertices);
   }
 
   @Override



Mime
View raw message