Author: claudio
Date: Wed Dec 21 17:50:24 2011
New Revision: 1221823
URL: http://svn.apache.org/viewvc?rev=1221823&view=rev
Log:
GIRAPH-109: GiraphRunner should provide support for combiners.
Modified:
incubator/giraph/trunk/CHANGELOG
incubator/giraph/trunk/src/main/java/org/apache/giraph/GiraphRunner.java
Modified: incubator/giraph/trunk/CHANGELOG
URL:
http://svn.apache.org/viewvc/incubator/giraph/trunk/CHANGELOG?rev=1221823&r1=1221822&r2=1221823&view=diff
==============================================================================
--- incubator/giraph/trunk/CHANGELOG (original)
+++ incubator/giraph/trunk/CHANGELOG Wed Dec 21 17:50:24 2011
@@ -2,6 +2,9 @@ Giraph Change Log
Release 0.70.0 - unreleased
+ GIRAPH-109: GiraphRunner should provide support for combiners.
+ (ssc via claudio)
+
GIRAPH-113: Change cast to Vertex used in prepareSuperstep() to
BasicVertex. (humming80 via aching)
Modified:
incubator/giraph/trunk/src/main/java/org/apache/giraph/GiraphRunner.java
URL:
http://svn.apache.org/viewvc/incubator/giraph/trunk/src/main/java/org/apache/giraph/GiraphRunner.java?rev=1221823&r1=1221822&r2=1221823&view=diff
==============================================================================
--- incubator/giraph/trunk/src/main/java/org/apache/giraph/GiraphRunner.java
(original)
+++ incubator/giraph/trunk/src/main/java/org/apache/giraph/GiraphRunner.java
Wed Dec 21 17:50:24 2011
@@ -48,6 +48,9 @@ public class GiraphRunner implements Too
options.addOption("of", "outputFormat", true, "Graph outputformat");
options.addOption("ip", "inputPath", true, "Graph input path");
options.addOption("op", "outputPath", true, "Graph output path");
+ options.addOption("c", "combiner", true, "VertexCombiner class");
+ options.addOption("wc", "workerContext", true, "WorkerContext class");
+ options.addOption("aw", "aggregatorWriter", true, "AggregatorWriter
class");
return options;
}
@@ -95,13 +98,27 @@ public class GiraphRunner implements Too
if(cmd.hasOption("ip")) {
FileInputFormat.addInputPath(job, new Path(cmd.getOptionValue("ip")));
} else {
- LOG.info("No input path specified. Ensure your InputFormat does not
require one.");
+ LOG.info("No input path specified. Ensure your InputFormat does not " +
+ "require one.");
}
if(cmd.hasOption("op")) {
FileOutputFormat.setOutputPath(job, new Path(cmd.getOptionValue("op")));
} else {
- LOG.info("No output path specified. Ensure your OutputFormat does not
require one.");
+ LOG.info("No output path specified. Ensure your OutputFormat does not " +
+ "require one.");
+ }
+
+ if (cmd.hasOption("c")) {
+ job.setVertexCombinerClass(Class.forName(cmd.getOptionValue("c")));
+ }
+
+ if (cmd.hasOption("wc")) {
+ job.setWorkerContextClass(Class.forName(cmd.getOptionValue("wc")));
+ }
+
+ if (cmd.hasOption("aw")) {
+ job.setAggregatorWriterClass(Class.forName(cmd.getOptionValue("aw")));
}
job.setWorkerConfiguration(workers, workers, 100.0f);