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 2015/04/14 07:02:19 UTC
svn commit: r1673357 - in /hama/trunk: ./ core/
examples/src/main/java/org/apache/hama/examples/
examples/src/main/java/org/apache/hama/examples/util/
examples/src/test/java/org/apache/hama/examples/
Author: edwardyoon
Date: Tue Apr 14 05:02:19 2015
New Revision: 1673357
URL: http://svn.apache.org/r1673357
Log:
HAMA-944: Add JSON format option to fastgen command (Minho Kim via edwardyoon)
Modified:
hama/trunk/CHANGES.txt
hama/trunk/core/pom.xml
hama/trunk/examples/src/main/java/org/apache/hama/examples/PageRank.java
hama/trunk/examples/src/main/java/org/apache/hama/examples/util/FastGraphGen.java
hama/trunk/examples/src/test/java/org/apache/hama/examples/FastGraphGenTest.java
hama/trunk/examples/src/test/java/org/apache/hama/examples/PageRankTest.java
Modified: hama/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hama/trunk/CHANGES.txt?rev=1673357&r1=1673356&r2=1673357&view=diff
==============================================================================
--- hama/trunk/CHANGES.txt (original)
+++ hama/trunk/CHANGES.txt Tue Apr 14 05:02:19 2015
@@ -26,6 +26,7 @@ Release 0.7.0 (unreleased changes)
IMPROVEMENTS
+ HAMA-944: Add JSON format option to fastgen command (Minho Kim via edwardyoon)
HAMA-919: Manage messages per Vertex (edwardyoon)
HAMA-923: add a toString() method for FloatArrayWritable and TextArrayWritable classes (edwardyoon)
HAMA-921: Polish doSuperstep() function and VertexMessageIterable class (Anastasis Andronidis)
Modified: hama/trunk/core/pom.xml
URL: http://svn.apache.org/viewvc/hama/trunk/core/pom.xml?rev=1673357&r1=1673356&r2=1673357&view=diff
==============================================================================
--- hama/trunk/core/pom.xml (original)
+++ hama/trunk/core/pom.xml Tue Apr 14 05:02:19 2015
@@ -154,7 +154,12 @@
<groupId>io.netty</groupId>
<artifactId>netty-all</artifactId>
<version>4.0.21.Final</version>
- </dependency>
+ </dependency>
+ <dependency>
+ <groupId>com.googlecode.json-simple</groupId>
+ <artifactId>json-simple</artifactId>
+ <version>1.1</version>
+ </dependency>
</dependencies>
<build>
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=1673357&r1=1673356&r2=1673357&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 Tue Apr 14 05:02:19 2015
@@ -18,22 +18,22 @@
package org.apache.hama.examples;
import java.io.IOException;
+import java.util.Iterator;
+import org.apache.commons.cli.*;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.*;
import org.apache.hama.HamaConfiguration;
import org.apache.hama.bsp.HashPartitioner;
-import org.apache.hama.bsp.SequenceFileInputFormat;
+import org.apache.hama.bsp.TextInputFormat;
import org.apache.hama.bsp.TextOutputFormat;
-import org.apache.hama.commons.io.TextArrayWritable;
import org.apache.hama.graph.AverageAggregator;
import org.apache.hama.graph.Edge;
import org.apache.hama.graph.GraphJob;
import org.apache.hama.graph.Vertex;
import org.apache.hama.graph.VertexInputReader;
+import org.json.simple.JSONArray;
+import org.json.simple.parser.JSONParser;
/**
* Real pagerank with dangling node contribution.
@@ -75,7 +75,7 @@ public class PageRank {
// if we have not reached our global error yet, then proceed.
DoubleWritable globalError = getAggregatedValue(0);
-
+
if (globalError != null && this.getSuperstepCount() > 2
&& MAXIMUM_CONVERGENCE_ERROR > globalError.get()) {
voteToHalt();
@@ -87,30 +87,66 @@ public class PageRank {
}
}
- public static class PagerankSeqReader
- extends
- VertexInputReader<Text, TextArrayWritable, Text, NullWritable, DoubleWritable> {
+ public static class PagerankTextReader extends
+ VertexInputReader<LongWritable, Text, Text, NullWritable, DoubleWritable> {
+
+ @Override
+ public boolean parseVertex(LongWritable key, Text value,
+ Vertex<Text, NullWritable, DoubleWritable> vertex) throws Exception {
+
+ String[] tokenArray = value.toString().split("\t");
+ String vtx = tokenArray[0].trim();
+ String[] edges = tokenArray[1].trim().split(" ");
+
+ vertex.setVertexID(new Text(vtx));
+
+ for (String v : edges) {
+ vertex.addEdge(new Edge<Text, NullWritable>(new Text(v), null));
+ }
+
+ return true;
+ }
+ }
+
+ public static class PagerankJsonReader extends
+ VertexInputReader<LongWritable, Text, Text, NullWritable, DoubleWritable> {
+ JSONParser parser = new JSONParser();
+
@Override
- public boolean parseVertex(Text key, TextArrayWritable value,
+ public boolean parseVertex(LongWritable key, Text value,
Vertex<Text, NullWritable, DoubleWritable> vertex) throws Exception {
- vertex.setVertexID(key);
- for (Writable v : value.get()) {
- vertex.addEdge(new Edge<Text, NullWritable>((Text) v, null));
+ String strValue = value.toString();
+ JSONArray jsonArray = (JSONArray) parser.parse(strValue);
+
+ vertex.setVertexID(new Text(jsonArray.get(0).toString()));
+
+ Iterator<JSONArray> iter = ((JSONArray) jsonArray.get(2)).iterator();
+ while (iter.hasNext()) {
+ JSONArray edge = (JSONArray) iter.next();
+ vertex.addEdge(new Edge<Text, NullWritable>(new Text(edge.get(0)
+ .toString()), null));
}
return true;
}
}
- public static GraphJob createJob(String[] args, HamaConfiguration conf)
- throws IOException {
+ public static GraphJob createJob(String[] args, HamaConfiguration conf,
+ Options opts) throws IOException, ParseException {
+ CommandLine cliParser = new GnuParser().parse(opts, args);
+
+ if (!cliParser.hasOption("i") || !cliParser.hasOption("o")) {
+ System.out
+ .println("No input or output path specified for PageRank, exiting.");
+ }
+
GraphJob pageJob = new GraphJob(conf, PageRank.class);
pageJob.setJobName("Pagerank");
pageJob.setVertexClass(PageRankVertex.class);
- pageJob.setInputPath(new Path(args[0]));
- pageJob.setOutputPath(new Path(args[1]));
+ pageJob.setInputPath(new Path(cliParser.getOptionValue("i")));
+ pageJob.setOutputPath(new Path(cliParser.getOptionValue("o")));
// set the defaults
pageJob.setMaxIteration(30);
@@ -120,23 +156,37 @@ public class PageRank {
pageJob.set("hama.graph.self.ref", "true");
pageJob.set("hama.graph.max.convergence.error", "0.001");
- if (args.length == 3) {
- pageJob.setNumBspTask(Integer.parseInt(args[2]));
+ if (cliParser.hasOption("t")) {
+ pageJob.setNumBspTask(Integer.parseInt(cliParser.getOptionValue("t")));
}
// error
pageJob.setAggregatorClass(AverageAggregator.class);
// Vertex reader
- pageJob.setVertexInputReaderClass(PagerankSeqReader.class);
+ // According to file type, which is Text or Json,
+ // Vertex reader handle it differently.
+ if (cliParser.hasOption("f")) {
+ if (cliParser.getOptionValue("f").equals("text")) {
+ pageJob.setVertexInputReaderClass(PagerankTextReader.class);
+ } else if (cliParser.getOptionValue("f").equals("json")) {
+ pageJob.setVertexInputReaderClass(PagerankJsonReader.class);
+ } else {
+ System.out.println("File type is not available to run Pagerank... "
+ + "File type set default value, Text.");
+ pageJob.setVertexInputReaderClass(PagerankTextReader.class);
+ }
+ } else {
+ pageJob.setVertexInputReaderClass(PagerankTextReader.class);
+ }
pageJob.setVertexIDClass(Text.class);
pageJob.setVertexValueClass(DoubleWritable.class);
pageJob.setEdgeValueClass(NullWritable.class);
- pageJob.setInputFormat(SequenceFileInputFormat.class);
- pageJob.setInputKeyClass(Text.class);
- pageJob.setInputValueClass(TextArrayWritable.class);
+ pageJob.setInputFormat(TextInputFormat.class);
+ pageJob.setInputKeyClass(LongWritable.class);
+ pageJob.setInputValueClass(Text.class);
pageJob.setPartitioner(HashPartitioner.class);
pageJob.setOutputFormat(TextOutputFormat.class);
@@ -145,18 +195,25 @@ public class PageRank {
return pageJob;
}
- private static void printUsage() {
- System.out.println("Usage: <input> <output> [tasks]");
- System.exit(-1);
- }
-
public static void main(String[] args) throws IOException,
- InterruptedException, ClassNotFoundException {
- if (args.length < 2)
- printUsage();
+ InterruptedException, ClassNotFoundException, ParseException {
+ Options opts = new Options();
+ opts.addOption("i", "input_path", true, "The Location of output path.");
+ opts.addOption("o", "output_path", true, "The Location of input path.");
+ opts.addOption("h", "help", false, "Print usage");
+ opts.addOption("t", "task_num", true, "The number of tasks.");
+ opts.addOption("f", "file_type", true, "The file type of input data. Input"
+ + "file format which is \"text\" tab delimiter separated or \"json\"."
+ + "Default value - Text");
+
+ if (args.length < 2) {
+ new HelpFormatter().printHelp("PageRank -i INPUT_PATH -o OUTPUT_PATH "
+ + "[-t NUM_TASKS] [-f FILE_TYPE]", opts);
+ System.exit(-1);
+ }
HamaConfiguration conf = new HamaConfiguration();
- GraphJob pageJob = createJob(args, conf);
+ GraphJob pageJob = createJob(args, conf, opts);
long startTime = System.currentTimeMillis();
if (pageJob.waitForCompletion(true)) {
Modified: hama/trunk/examples/src/main/java/org/apache/hama/examples/util/FastGraphGen.java
URL: http://svn.apache.org/viewvc/hama/trunk/examples/src/main/java/org/apache/hama/examples/util/FastGraphGen.java?rev=1673357&r1=1673356&r2=1673357&view=diff
==============================================================================
--- hama/trunk/examples/src/main/java/org/apache/hama/examples/util/FastGraphGen.java (original)
+++ hama/trunk/examples/src/main/java/org/apache/hama/examples/util/FastGraphGen.java Tue Apr 14 05:02:19 2015
@@ -21,6 +21,7 @@ import java.io.IOException;
import java.util.HashSet;
import java.util.Random;
+import org.apache.commons.cli.*;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@@ -28,22 +29,21 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
import org.apache.hama.HamaConfiguration;
-import org.apache.hama.bsp.BSP;
-import org.apache.hama.bsp.BSPJob;
-import org.apache.hama.bsp.BSPPeer;
-import org.apache.hama.bsp.FileOutputFormat;
-import org.apache.hama.bsp.NullInputFormat;
-import org.apache.hama.bsp.SequenceFileOutputFormat;
+import org.apache.hama.bsp.*;
import org.apache.hama.bsp.sync.SyncException;
import org.apache.hama.commons.io.TextArrayWritable;
import com.google.common.collect.Sets;
+import org.json.simple.JSONArray;
+
public class FastGraphGen {
protected static Log LOG = LogFactory.getLog(FastGraphGen.class);
private static String SIZE_OF_MATRIX = "size.of.matrix";
private static String MAX_EDGES = "max.outlinks";
+ private static String OUTPUT_FORMAT = "graph.outputformat";
+ private static String WEIGHT = "graph.weight";
public static class FastGraphGenBSP extends
BSP<NullWritable, NullWritable, Text, TextArrayWritable, Text> {
@@ -51,15 +51,20 @@ public class FastGraphGen {
private Configuration conf;
private int sizeN;
private int maxOutEdges;
+ private boolean isJson;
+ private int weight;
@Override
public void setup(
BSPPeer<NullWritable, NullWritable, Text, TextArrayWritable, Text> peer) {
this.conf = peer.getConfiguration();
sizeN = conf.getInt(SIZE_OF_MATRIX, 10);
- maxOutEdges = conf.getInt(MAX_EDGES, 1);
+ maxOutEdges = conf.getInt(MAX_EDGES, 3);
+ isJson = conf.getBoolean(OUTPUT_FORMAT, false);
+ weight = conf.getInt(WEIGHT, 0);
}
+ @SuppressWarnings("unchecked")
@Override
public void bsp(
BSPPeer<NullWritable, NullWritable, Text, TextArrayWritable, Text> peer)
@@ -74,37 +79,99 @@ public class FastGraphGen {
}
Random r = new Random();
- for (int i = startID; i < endID; i++) {
- HashSet<Integer> set = Sets.newHashSet();
- for (int j = 0; j < maxOutEdges; j++) {
- set.add(r.nextInt(sizeN));
+ if (isJson) {
+ for (int i = startID; i < endID; i++) {
+
+ JSONArray vtxArray = new JSONArray();
+ vtxArray.add(i);
+ vtxArray.add(0);
+ JSONArray edgeArray = new JSONArray();
+ HashSet<Integer> set = Sets.newHashSet();
+ for (int j = 0; j < maxOutEdges; j++) {
+ set.add(r.nextInt(sizeN));
+ }
+ for (int x : set) {
+ JSONArray edge = new JSONArray();
+ edge.add(x);
+ if (weight == 0)
+ edge.add(0);
+ else if (weight > 0)
+ edge.add(r.nextInt(weight));
+ edgeArray.add(edge);
+ }
+ vtxArray.add(edgeArray);
+ peer.write(new Text(vtxArray.toString()), null);
}
- TextArrayWritable textArrayWritable = new TextArrayWritable();
- Text[] arr = new Text[set.size()];
- int index = 0;
- for (int x : set) {
- arr[index++] = new Text(String.valueOf(x));
+
+ } else {
+ for (int i = startID; i < endID; i++) {
+ HashSet<Integer> set = Sets.newHashSet();
+ for (int j = 0; j < maxOutEdges; j++) {
+ set.add(r.nextInt(sizeN));
+ }
+
+ TextArrayWritable textArrayWritable = new TextArrayWritable();
+ Text[] arr = new Text[set.size()];
+ int index = 0;
+ for (int x : set) {
+ arr[index++] = new Text(String.valueOf(x));
+ }
+ textArrayWritable.set(arr);
+
+ peer.write(new Text(String.valueOf(i)), textArrayWritable);
}
- textArrayWritable.set(arr);
- peer.write(new Text(String.valueOf(i)), textArrayWritable);
}
-
}
}
- public static void main(String[] args) throws InterruptedException,
- IOException, ClassNotFoundException {
- if (args.length < 4) {
- System.out
- .println("Usage: <size n> <max out-edges> <output path> <number of tasks>");
- System.exit(1);
+ public static void main(String[] args)
+ throws InterruptedException, IOException, ClassNotFoundException,
+ ParseException {
+ Options opts = new Options();
+ opts.addOption("v", "vertices", true, "The total number of vertices. Default value is 10.");
+ opts.addOption("e", "edges", true, "The maximum number of edges per vertex. Default value is 3.");
+ opts.addOption("o", "output_path", true, "The Location of output path.");
+ opts.addOption("t", "task_num", true, "The number of tasks. Default value is one.");
+ opts.addOption("h", "help", false, "Print usage");
+ opts.addOption("of", "output_format", true, "OutputFormat Type which is \"text\", "
+ + "tab delimiter separated or \"json\". Default value - text");
+ opts.addOption("w", "weight", true, "Enable to set weight of graph edges."
+ + "Default value - 0.");
+
+ CommandLine cliParser = new GnuParser().parse(opts, args);
+
+ // outputType, that has a value of "Text" unless true,
+ // when it has a value of "Json".
+ boolean outputType = false;
+
+ if (args.length == 0) {
+ System.out.println("No args specified for FastGraphGen to initialize");
+ System.exit(-1);
+ }
+
+ if (cliParser.hasOption("h")) {
+ new HelpFormatter().printHelp("FastGraphGen -p OUTPUT_PATH [options]", opts);
+ return;
+ }
+
+ if (!cliParser.hasOption("o")) {
+ System.out.println("No output path specified for FastGraphGen, exiting.");
+ System.exit(-1);
+ }
+
+ if (cliParser.hasOption("of")) {
+ if (cliParser.getOptionValue("of").equals("json"))
+ outputType = true;
}
// BSP job configuration
HamaConfiguration conf = new HamaConfiguration();
- conf.setInt(SIZE_OF_MATRIX, Integer.parseInt(args[0]));
- conf.setInt(MAX_EDGES, Integer.parseInt(args[1]));
+ conf.setInt(SIZE_OF_MATRIX, Integer.parseInt(cliParser.getOptionValue("vertices", "5")));
+ conf.setInt(MAX_EDGES,
+ Integer.parseInt(cliParser.getOptionValue("edges", "3")));
+ conf.setBoolean(OUTPUT_FORMAT, outputType);
+ conf.setInt(WEIGHT, Integer.parseInt(cliParser.getOptionValue("weight", "1")));
BSPJob bsp = new BSPJob(conf, FastGraphGenBSP.class);
// Set the job name
@@ -113,9 +180,9 @@ public class FastGraphGen {
bsp.setInputFormat(NullInputFormat.class);
bsp.setOutputKeyClass(Text.class);
bsp.setOutputValueClass(TextArrayWritable.class);
- bsp.setOutputFormat(SequenceFileOutputFormat.class);
- FileOutputFormat.setOutputPath(bsp, new Path(args[2]));
- bsp.setNumBspTask(Integer.parseInt(args[3]));
+ bsp.setOutputFormat(TextOutputFormat.class);
+ FileOutputFormat.setOutputPath(bsp, new Path(cliParser.getOptionValue("output_path")));
+ bsp.setNumBspTask(Integer.parseInt(cliParser.getOptionValue("task_num", "1")));
long startTime = System.currentTimeMillis();
if (bsp.waitForCompletion(true)) {
Modified: hama/trunk/examples/src/test/java/org/apache/hama/examples/FastGraphGenTest.java
URL: http://svn.apache.org/viewvc/hama/trunk/examples/src/test/java/org/apache/hama/examples/FastGraphGenTest.java?rev=1673357&r1=1673356&r2=1673357&view=diff
==============================================================================
--- hama/trunk/examples/src/test/java/org/apache/hama/examples/FastGraphGenTest.java (original)
+++ hama/trunk/examples/src/test/java/org/apache/hama/examples/FastGraphGenTest.java Tue Apr 14 05:02:19 2015
@@ -25,13 +25,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.commons.io.TextArrayWritable;
import org.apache.hama.examples.util.FastGraphGen;
+import org.json.simple.JSONArray;
+import org.json.simple.parser.JSONParser;
import org.junit.Test;
+import java.io.BufferedReader;
+import java.io.InputStreamReader;
+
public class FastGraphGenTest extends TestCase {
protected static Log LOG = LogFactory.getLog(FastGraphGenTest.class);
private static String TEST_OUTPUT = "/tmp/test";
@@ -40,26 +41,80 @@ public class FastGraphGenTest extends Te
public void testGraphGenerator() throws Exception {
Configuration conf = new Configuration();
- FastGraphGen.main(new String[] { "20", "10", TEST_OUTPUT, "3" });
+ // vertex size : 20
+ // maximum edges : 10
+ // output path : /tmp/test
+ // tasks num : 3
+ FastGraphGen.main(new String[] { "-v", "20", "-e", "10",
+ "-o", TEST_OUTPUT, "-t", "3" });
+ FileSystem fs = FileSystem.get(conf);
+
+ FileStatus[] globStatus = fs.globStatus(new Path(TEST_OUTPUT + "/part-*"));
+ for (FileStatus fts : globStatus) {
+ BufferedReader br = new BufferedReader(
+ new InputStreamReader(fs.open(fts.getPath())));
+ try {
+ String line;
+ line = br.readLine();
+ while (line != null) {
+ String[] keyValue = line.split("\t");
+ String[] outlinkId = keyValue[1].split(" ");
+ assertTrue(outlinkId.length <= 10);
+ for (String edge : outlinkId) {
+ assertTrue(Integer.parseInt(edge) < 20);
+ assertTrue(Integer.parseInt(edge) >= 0);
+ }
+ line = br.readLine();
+ }
+ } finally {
+ br.close();
+ }
+ }
+
+ fs.delete(new Path(TEST_OUTPUT), true);
+ }
+
+ @Test
+ public void testJsonGraphGenerator() throws Exception {
+ Configuration conf = new Configuration();
+
+ // vertex size : 20
+ // maximum edges : 10
+ // output path : /tmp/test
+ // tasks num : 3
+ // output type : json
+ // weight : 0
+ FastGraphGen.main(new String[] { "-v", "20", "-e", "10",
+ "-o", TEST_OUTPUT, "-t", "1", "-of", "json", "-w", "0" });
FileSystem fs = FileSystem.get(conf);
FileStatus[] globStatus = fs.globStatus(new Path(TEST_OUTPUT + "/part-*"));
+ JSONParser parser = new JSONParser();
for (FileStatus fts : globStatus) {
- SequenceFile.Reader reader = new SequenceFile.Reader(fs, fts.getPath(),
- conf);
- Text key = new Text();
- TextArrayWritable value = new TextArrayWritable();
-
- while (reader.next(key, value)) {
- Writable[] writables = value.get();
- assertTrue(writables.length <= 10);
- for (Writable t : writables) {
- int outlinkId = Integer.parseInt(t.toString());
- assertTrue(outlinkId <= 20);
- assertTrue(outlinkId >= 0);
+ BufferedReader br = new BufferedReader(
+ new InputStreamReader(fs.open(fts.getPath())));
+ try {
+ String line;
+ line = br.readLine();
+
+ while (line != null) {
+ JSONArray jsonArray = (JSONArray)parser.parse(line);
+
+ // the edge data begins at the third element.
+ JSONArray edgeArray = (JSONArray)jsonArray.get(2);
+ assertTrue(edgeArray.size() <= 10);
+
+ for (Object obj : edgeArray) {
+ JSONArray edge = (JSONArray)obj;
+ assertTrue(Integer.parseInt(edge.get(0).toString()) < 20);
+ assertTrue(Integer.parseInt(edge.get(0).toString()) >= 0);
+ assertTrue(Integer.parseInt(edge.get(1).toString()) == 0);
+ }
+ line = br.readLine();
}
+ } finally {
+ br.close();
}
- reader.close();
}
fs.delete(new Path(TEST_OUTPUT), true);
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=1673357&r1=1673356&r2=1673357&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 Tue Apr 14 05:02:19 2015
@@ -23,6 +23,7 @@ import java.io.InputStreamReader;
import junit.framework.TestCase;
+import org.apache.commons.cli.ParseException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@@ -54,8 +55,11 @@ public class PageRankTest extends TestCa
generateTestData();
try {
- PageRank.main(new String[] { INPUT, OUTPUT, "3" });
+ PageRank.main(new String[] { "-input_path", INPUT, "-output_path",
+ OUTPUT, "-task_num", "3", "-f", "json" });
verifyResult();
+ } catch (ParseException e) {
+ e.printStackTrace();
} finally {
deleteTempDirs();
}
@@ -67,7 +71,7 @@ public class PageRankTest extends TestCa
for (FileStatus fts : globStatus) {
BufferedReader reader = new BufferedReader(new InputStreamReader(
fs.open(fts.getPath())));
- String line = null;
+ String line;
while ((line = reader.readLine()) != null) {
String[] split = line.split("\t");
sum += Double.parseDouble(split[1]);
@@ -79,7 +83,8 @@ public class PageRankTest extends TestCa
private void generateTestData() {
try {
- FastGraphGen.main(new String[] { "60", "3", INPUT, "3" });
+ FastGraphGen.main(new String[] { "-v", "60", "-e", "3", "-output_path",
+ INPUT, "-task_num", "3", "-of", "json"});
} catch (Exception e) {
e.printStackTrace();
}