You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hama.apache.org by ed...@apache.org on 2012/06/23 03:01:28 UTC

svn commit: r1353058 - in /hama/trunk: ./ examples/src/main/java/org/apache/hama/examples/ examples/src/test/java/org/apache/hama/examples/

Author: edwardyoon
Date: Sat Jun 23 01:01:27 2012
New Revision: 1353058

URL: http://svn.apache.org/viewvc?rev=1353058&view=rev
Log:
Change Pagerank IO format to human-readable text for easy debug

Modified:
    hama/trunk/CHANGES.txt
    hama/trunk/examples/src/main/java/org/apache/hama/examples/MindistSearch.java
    hama/trunk/examples/src/main/java/org/apache/hama/examples/PageRank.java
    hama/trunk/examples/src/main/java/org/apache/hama/examples/SSSP.java
    hama/trunk/examples/src/test/java/org/apache/hama/examples/MindistSearchTest.java
    hama/trunk/examples/src/test/java/org/apache/hama/examples/PageRankTest.java
    hama/trunk/examples/src/test/java/org/apache/hama/examples/SSSPTest.java

Modified: hama/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hama/trunk/CHANGES.txt?rev=1353058&r1=1353057&r2=1353058&view=diff
==============================================================================
--- hama/trunk/CHANGES.txt (original)
+++ hama/trunk/CHANGES.txt Sat Jun 23 01:01:27 2012
@@ -24,6 +24,7 @@ Release 0.5 - April 10, 2012 
 
   IMPROVEMENTS
 
+    HAMA-584: Change Pagerank IO format to human-readable text for easy debug (tjungblut vai edwardyoon)
     HAMA-590: Fix TestSubmitGraphJob tests (tjungblut)
     HAMA-582: Task's error logs should be displayed on client-end when job is failed (edwardyoon)
     HAMA-580: Improve input of graph module (tjungblut)

Modified: hama/trunk/examples/src/main/java/org/apache/hama/examples/MindistSearch.java
URL: http://svn.apache.org/viewvc/hama/trunk/examples/src/main/java/org/apache/hama/examples/MindistSearch.java?rev=1353058&r1=1353057&r2=1353058&view=diff
==============================================================================
--- hama/trunk/examples/src/main/java/org/apache/hama/examples/MindistSearch.java (original)
+++ hama/trunk/examples/src/main/java/org/apache/hama/examples/MindistSearch.java Sat Jun 23 01:01:27 2012
@@ -28,8 +28,8 @@ import org.apache.hadoop.io.Text;
 import org.apache.hama.HamaConfiguration;
 import org.apache.hama.bsp.Combiner;
 import org.apache.hama.bsp.HashPartitioner;
-import org.apache.hama.bsp.SequenceFileOutputFormat;
 import org.apache.hama.bsp.TextInputFormat;
+import org.apache.hama.bsp.TextOutputFormat;
 import org.apache.hama.graph.Edge;
 import org.apache.hama.graph.GraphJob;
 import org.apache.hama.graph.Vertex;
@@ -162,7 +162,7 @@ public class MindistSearch {
     job.setInputFormat(TextInputFormat.class);
     job.setVertexInputReaderClass(MindistSearchCountReader.class);
     job.setPartitioner(HashPartitioner.class);
-    job.setOutputFormat(SequenceFileOutputFormat.class);
+    job.setOutputFormat(TextOutputFormat.class);
     job.setOutputKeyClass(Text.class);
     job.setOutputValueClass(Text.class);
 

Modified: hama/trunk/examples/src/main/java/org/apache/hama/examples/PageRank.java
URL: http://svn.apache.org/viewvc/hama/trunk/examples/src/main/java/org/apache/hama/examples/PageRank.java?rev=1353058&r1=1353057&r2=1353058&view=diff
==============================================================================
--- hama/trunk/examples/src/main/java/org/apache/hama/examples/PageRank.java (original)
+++ hama/trunk/examples/src/main/java/org/apache/hama/examples/PageRank.java Sat Jun 23 01:01:27 2012
@@ -28,8 +28,8 @@ import org.apache.hadoop.io.NullWritable
 import org.apache.hadoop.io.Text;
 import org.apache.hama.HamaConfiguration;
 import org.apache.hama.bsp.HashPartitioner;
-import org.apache.hama.bsp.SequenceFileOutputFormat;
 import org.apache.hama.bsp.TextInputFormat;
+import org.apache.hama.bsp.TextOutputFormat;
 import org.apache.hama.graph.AbstractAggregator;
 import org.apache.hama.graph.AverageAggregator;
 import org.apache.hama.graph.Edge;
@@ -184,7 +184,7 @@ public class PageRank {
     pageJob.setInputFormat(TextInputFormat.class);
     pageJob.setVertexInputReaderClass(PagerankTextReader.class);
     pageJob.setPartitioner(HashPartitioner.class);
-    pageJob.setOutputFormat(SequenceFileOutputFormat.class);
+    pageJob.setOutputFormat(TextOutputFormat.class);
     pageJob.setOutputKeyClass(Text.class);
     pageJob.setOutputValueClass(DoubleWritable.class);
     return pageJob;

Modified: hama/trunk/examples/src/main/java/org/apache/hama/examples/SSSP.java
URL: http://svn.apache.org/viewvc/hama/trunk/examples/src/main/java/org/apache/hama/examples/SSSP.java?rev=1353058&r1=1353057&r2=1353058&view=diff
==============================================================================
--- hama/trunk/examples/src/main/java/org/apache/hama/examples/SSSP.java (original)
+++ hama/trunk/examples/src/main/java/org/apache/hama/examples/SSSP.java Sat Jun 23 01:01:27 2012
@@ -27,8 +27,8 @@ import org.apache.hadoop.io.Text;
 import org.apache.hama.HamaConfiguration;
 import org.apache.hama.bsp.Combiner;
 import org.apache.hama.bsp.HashPartitioner;
-import org.apache.hama.bsp.SequenceFileOutputFormat;
 import org.apache.hama.bsp.TextInputFormat;
+import org.apache.hama.bsp.TextOutputFormat;
 import org.apache.hama.graph.Edge;
 import org.apache.hama.graph.GraphJob;
 import org.apache.hama.graph.Vertex;
@@ -149,7 +149,7 @@ public class SSSP {
     ssspJob.setInputValueClass(Text.class);
 
     ssspJob.setPartitioner(HashPartitioner.class);
-    ssspJob.setOutputFormat(SequenceFileOutputFormat.class);
+    ssspJob.setOutputFormat(TextOutputFormat.class);
     ssspJob.setVertexInputReaderClass(SSSPTextReader.class);
     ssspJob.setOutputKeyClass(Text.class);
     ssspJob.setOutputValueClass(IntWritable.class);

Modified: hama/trunk/examples/src/test/java/org/apache/hama/examples/MindistSearchTest.java
URL: http://svn.apache.org/viewvc/hama/trunk/examples/src/test/java/org/apache/hama/examples/MindistSearchTest.java?rev=1353058&r1=1353057&r2=1353058&view=diff
==============================================================================
--- hama/trunk/examples/src/test/java/org/apache/hama/examples/MindistSearchTest.java (original)
+++ hama/trunk/examples/src/test/java/org/apache/hama/examples/MindistSearchTest.java Sat Jun 23 01:01:27 2012
@@ -17,9 +17,11 @@
  */
 package org.apache.hama.examples;
 
+import java.io.BufferedReader;
 import java.io.BufferedWriter;
 import java.io.FileWriter;
 import java.io.IOException;
+import java.io.InputStreamReader;
 import java.util.Arrays;
 import java.util.List;
 
@@ -29,9 +31,7 @@ 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.examples.MindistSearch.MinTextCombiner;
 
@@ -81,14 +81,13 @@ public class MindistSearchTest extends T
     FileStatus[] globStatus = fs.globStatus(new Path(OUTPUT + "/part-*"));
     int itemsRead = 0;
     for (FileStatus fts : globStatus) {
-      SequenceFile.Reader reader = new SequenceFile.Reader(fs, fts.getPath(),
-          conf);
-      Text key = new Text();
-      Writable value = new Text();
-      while (reader.next(key, value)) {
-        System.out.println(key + " | " + value);
-        assertEquals(resultList[Integer.parseInt(key.toString())],
-            value.toString());
+      BufferedReader reader = new BufferedReader(new InputStreamReader(
+          fs.open(fts.getPath())));
+      String line = null;
+      while ((line = reader.readLine()) != null) {
+        String[] split = line.split("\t");
+        System.out.println(split[0] + " | " + split[1]);
+        assertEquals(resultList[Integer.parseInt(split[0])], split[1]);
         itemsRead++;
       }
     }

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=1353058&r1=1353057&r2=1353058&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 Sat Jun 23 01:01:27 2012
@@ -17,9 +17,11 @@
  */
 package org.apache.hama.examples;
 
+import java.io.BufferedReader;
 import java.io.BufferedWriter;
 import java.io.FileWriter;
 import java.io.IOException;
+import java.io.InputStreamReader;
 
 import junit.framework.TestCase;
 
@@ -27,9 +29,6 @@ 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.DoubleWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Text;
 import org.apache.hama.HamaConfiguration;
 import org.apache.hama.graph.GraphJob;
 import org.apache.hama.graph.GraphJobRunner;
@@ -71,14 +70,13 @@ public class PageRankTest extends TestCa
     double sum = 0.0;
     FileStatus[] globStatus = fs.globStatus(new Path(OUTPUT + "/part-*"));
     for (FileStatus fts : globStatus) {
-      SequenceFile.Reader reader = new SequenceFile.Reader(fs, fts.getPath(),
-          conf);
-      Text key = new Text();
-      DoubleWritable value = new DoubleWritable();
-
-      while (reader.next(key, value)) {
-        System.out.println(key + " / " + value);
-        sum += value.get();
+      BufferedReader reader = new BufferedReader(new InputStreamReader(
+          fs.open(fts.getPath())));
+      String line = null;
+      while ((line = reader.readLine()) != null) {
+        String[] split = line.split("\t");
+        System.out.println(split[0] + " / " + split[1]);
+        sum += Double.parseDouble(split[1]);
       }
     }
     System.out.println("Sum is: " + sum);
@@ -89,7 +87,7 @@ public class PageRankTest extends TestCa
     generateTestData();
     try {
       HamaConfiguration conf = new HamaConfiguration(new Configuration());
-      conf.set("bsp.local.tasks.maximum", "1"); 
+      conf.set("bsp.local.tasks.maximum", "1");
       conf.setBoolean(GraphJobRunner.GRAPH_REPAIR, true);
       GraphJob pageJob = PageRank.createJob(new String[] { INPUT, OUTPUT },
           conf);

Modified: hama/trunk/examples/src/test/java/org/apache/hama/examples/SSSPTest.java
URL: http://svn.apache.org/viewvc/hama/trunk/examples/src/test/java/org/apache/hama/examples/SSSPTest.java?rev=1353058&r1=1353057&r2=1353058&view=diff
==============================================================================
--- hama/trunk/examples/src/test/java/org/apache/hama/examples/SSSPTest.java (original)
+++ hama/trunk/examples/src/test/java/org/apache/hama/examples/SSSPTest.java Sat Jun 23 01:01:27 2012
@@ -17,9 +17,11 @@
  */
 package org.apache.hama.examples;
 
+import java.io.BufferedReader;
 import java.io.BufferedWriter;
 import java.io.FileWriter;
 import java.io.IOException;
+import java.io.InputStreamReader;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -29,9 +31,6 @@ 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.IntWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Text;
 import org.apache.hama.HamaConfiguration;
 
 /**
@@ -82,12 +81,12 @@ public class SSSPTest extends TestCase {
 
     FileStatus[] globStatus = fs.globStatus(new Path(OUTPUT + "/part-*"));
     for (FileStatus fts : globStatus) {
-      SequenceFile.Reader reader = new SequenceFile.Reader(fs, fts.getPath(),
-          conf);
-      Text key = new Text();
-      IntWritable value = new IntWritable();
-      while (reader.next(key, value)) {
-        assertEquals(value.get(), (int) rs.get(key.toString()));
+      BufferedReader reader = new BufferedReader(new InputStreamReader(
+          fs.open(fts.getPath())));
+      String line = null;
+      while ((line = reader.readLine()) != null) {
+        String[] split = line.split("\t");
+        assertEquals(Integer.parseInt(split[1]), (int) rs.get(split[0]));
       }
     }
   }