From 33572a5e2325c8b15b2ed133dca337abb7c2e94c Mon Sep 17 00:00:00 2001 From: Avery Ching Date: Thu, 16 Feb 2012 22:12:31 +0000 Subject: [PATCH] GIRAPH-40: Added checkstyle for enforcement of code conventions. All Giraph source files now pass checkstyle. (aching). git-svn-id: https://svn.apache.org/repos/asf/incubator/giraph/trunk@1245205 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGELOG | 3 + checkstyle.xml | 296 ++ license-header.txt | 17 + pom.xml | 57 +- .../java/org/apache/giraph/GiraphRunner.java | 50 +- .../HashMapVertexPageRankBenchmark.java | 7 +- .../giraph/benchmark/PageRankBenchmark.java | 222 +- .../giraph/benchmark/PageRankComputation.java | 19 +- .../PseudoRandomVertexInputFormat.java | 257 +- .../benchmark/RandomMessageBenchmark.java | 687 ++-- .../apache/giraph/benchmark/package-info.java | 21 + .../apache/giraph/bsp/ApplicationState.java | 12 +- .../org/apache/giraph/bsp/BspInputFormat.java | 89 +- .../org/apache/giraph/bsp/BspInputSplit.java | 95 +- .../apache/giraph/bsp/BspOutputFormat.java | 58 +- .../apache/giraph/bsp/BspRecordReader.java | 61 +- .../apache/giraph/bsp/BspRecordWriter.java | 24 +- .../apache/giraph/bsp/CentralizedService.java | 71 +- .../giraph/bsp/CentralizedServiceMaster.java | 109 +- .../giraph/bsp/CentralizedServiceWorker.java | 260 +- .../giraph/bsp/ImmutableOutputCommitter.java | 44 +- .../org/apache/giraph/bsp/SuperstepState.java | 12 +- .../org/apache/giraph/bsp/package-info.java | 21 + .../apache/giraph/comm/ArrayListWritable.java | 144 +- .../giraph/comm/BasicRPCCommunications.java | 2159 +++++------ .../giraph/comm/CommunicationsInterface.java | 160 +- .../java/org/apache/giraph/comm/MsgList.java | 39 +- .../apache/giraph/comm/RPCCommunications.java | 225 +- .../apache/giraph/comm/ServerInterface.java | 70 +- .../apache/giraph/comm/VertexIdMessages.java | 112 +- .../giraph/comm/VertexIdMessagesList.java | 42 +- .../org/apache/giraph/comm/VertexList.java | 34 +- .../giraph/comm/WorkerCommunications.java | 130 +- .../org/apache/giraph/comm/package-info.java | 21 + .../examples/ConnectedComponentsVertex.java | 144 +- .../examples/GeneratedVertexInputFormat.java | 31 +- .../examples/GeneratedVertexReader.java | 89 +- .../IntIntNullIntTextInputFormat.java | 127 +- .../giraph/examples/LongSumAggregator.java | 57 +- .../apache/giraph/examples/MaxAggregator.java | 24 +- .../apache/giraph/examples/MinAggregator.java | 24 +- .../giraph/examples/MinimumIntCombiner.java | 61 +- .../examples/SimpleAggregatorWriter.java | 53 +- .../examples/SimpleCheckpointVertex.java | 408 ++- .../giraph/examples/SimpleCombinerVertex.java | 63 +- .../giraph/examples/SimpleFailVertex.java | 69 +- .../giraph/examples/SimpleMsgVertex.java | 58 +- .../examples/SimpleMutateGraphVertex.java | 314 +- .../giraph/examples/SimplePageRankVertex.java | 387 +- .../examples/SimpleShortestPathsVertex.java | 379 +- .../giraph/examples/SimpleSumCombiner.java | 24 +- .../examples/SimpleSuperstepVertex.java | 194 +- .../SimpleTextVertexOutputFormat.java | 57 +- .../SimpleVertexWithWorkerContext.java | 227 +- .../apache/giraph/examples/SumAggregator.java | 22 +- .../apache/giraph/examples/VerifyMessage.java | 306 +- .../VertexWithComponentTextOutputFormat.java | 97 +- .../apache/giraph/examples/package-info.java | 21 + .../org/apache/giraph/graph/Aggregator.java | 60 +- .../apache/giraph/graph/AggregatorUsage.java | 57 +- .../apache/giraph/graph/AggregatorWriter.java | 76 +- .../org/apache/giraph/graph/BasicVertex.java | 488 +-- .../giraph/graph/BasicVertexResolver.java | 60 +- .../org/apache/giraph/graph/BspService.java | 1925 +++++----- .../apache/giraph/graph/BspServiceMaster.java | 3212 +++++++++-------- .../apache/giraph/graph/BspServiceWorker.java | 2707 +++++++------- .../org/apache/giraph/graph/BspUtils.java | 837 +++-- .../giraph/graph/DefaultWorkerContext.java | 22 +- .../java/org/apache/giraph/graph/Edge.java | 243 +- .../apache/giraph/graph/EdgeListVertex.java | 455 +-- .../org/apache/giraph/graph/GiraphJob.java | 1109 +++--- .../org/apache/giraph/graph/GlobalStats.java | 102 +- .../org/apache/giraph/graph/GraphMapper.java | 1124 +++--- .../org/apache/giraph/graph/GraphState.java | 160 +- .../apache/giraph/graph/HashMapVertex.java | 382 +- .../giraph/graph/IntIntNullIntVertex.java | 241 +- .../graph/LongDoubleFloatDoubleVertex.java | 525 +-- .../org/apache/giraph/graph/MasterThread.java | 283 +- .../apache/giraph/graph/MutableVertex.java | 177 +- .../giraph/graph/TextAggregatorWriter.java | 164 +- .../apache/giraph/graph/VertexChanges.java | 62 +- .../apache/giraph/graph/VertexCombiner.java | 31 +- .../apache/giraph/graph/VertexEdgeCount.java | 90 +- .../giraph/graph/VertexInputFormat.java | 74 +- .../apache/giraph/graph/VertexMutations.java | 144 +- .../giraph/graph/VertexOutputFormat.java | 79 +- .../org/apache/giraph/graph/VertexReader.java | 99 +- .../apache/giraph/graph/VertexResolver.java | 187 +- .../org/apache/giraph/graph/VertexWriter.java | 58 +- .../apache/giraph/graph/WorkerContext.java | 177 +- .../org/apache/giraph/graph/WorkerInfo.java | 143 +- .../org/apache/giraph/graph/package-info.java | 22 + .../graph/partition/BasicPartitionOwner.java | 232 +- .../partition/GraphPartitionerFactory.java | 30 +- .../partition/HashMasterPartitioner.java | 218 +- .../partition/HashPartitionerFactory.java | 40 +- .../HashRangePartitionerFactory.java | 39 +- .../partition/HashRangeWorkerPartitioner.java | 16 +- .../partition/HashWorkerPartitioner.java | 130 +- .../partition/MasterGraphPartitioner.java | 83 +- .../giraph/graph/partition/Partition.java | 195 +- .../graph/partition/PartitionBalancer.java | 428 +-- .../graph/partition/PartitionExchange.java | 82 +- .../graph/partition/PartitionOwner.java | 94 +- .../graph/partition/PartitionStats.java | 191 +- .../graph/partition/PartitionUtils.java | 189 +- .../partition/RangeMasterPartitioner.java | 13 +- .../graph/partition/RangePartitionOwner.java | 66 +- .../graph/partition/RangePartitionStats.java | 56 +- .../graph/partition/RangeSplitHint.java | 60 +- .../partition/RangeWorkerPartitioner.java | 56 +- .../partition/WorkerGraphPartitioner.java | 107 +- .../giraph/graph/partition/package-info.java | 21 + .../giraph/hadoop/BspPolicyProvider.java | 23 +- .../giraph/hadoop/BspTokenSelector.java | 35 +- .../apache/giraph/hadoop/package-info.java | 21 + .../SuperstepHashPartitionerFactory.java | 144 +- .../giraph/integration/package-info.java | 21 + .../AdjacencyListTextVertexOutputFormat.java | 32 +- .../giraph/lib/AdjacencyListVertexReader.java | 50 +- .../lib/IdWithValueTextOutputFormat.java | 36 +- .../giraph/lib/JsonBase64VertexFormat.java | 19 +- .../lib/JsonBase64VertexInputFormat.java | 190 +- .../lib/JsonBase64VertexOutputFormat.java | 136 +- ...eDoubleAdjacencyListVertexInputFormat.java | 39 +- .../lib/SequenceFileVertexInputFormat.java | 55 +- ...eDoubleAdjacencyListVertexInputFormat.java | 35 +- .../giraph/lib/TextVertexInputFormat.java | 135 +- .../giraph/lib/TextVertexOutputFormat.java | 125 +- .../org/apache/giraph/lib/package-info.java | 21 + .../java/org/apache/giraph/package-info.java | 21 + .../apache/giraph/utils/ComparisonUtils.java | 54 +- .../apache/giraph/utils/EmptyIterable.java | 44 +- .../giraph/utils/InternalVertexRunner.java | 407 ++- .../org/apache/giraph/utils/MemoryUtils.java | 30 +- .../apache/giraph/utils/ReflectionUtils.java | 209 +- .../utils/UnmodifiableIntArrayIterator.java | 42 +- .../apache/giraph/utils/WritableUtils.java | 324 +- .../org/apache/giraph/utils/package-info.java | 21 + .../java/org/apache/giraph/zk/BspEvent.java | 40 +- .../org/apache/giraph/zk/ContextLock.java | 44 +- .../org/apache/giraph/zk/PredicateLock.java | 149 +- .../org/apache/giraph/zk/ZooKeeperExt.java | 472 ++- .../apache/giraph/zk/ZooKeeperManager.java | 1435 ++++---- .../org/apache/giraph/zk/package-info.java | 21 + .../java/org/apache/giraph/TestBspBasic.java | 630 ++-- .../apache/giraph/TestJsonBase64Format.java | 139 +- .../apache/giraph/TestManualCheckpoint.java | 156 +- .../SimpleShortestPathVertexTest.java | 211 +- .../giraph/graph/TestEdgeListVertex.java | 236 +- .../org/apache/giraph/utils/MockUtils.java | 2 +- src/test/java/zk/TestZooKeeperManager.java | 3 +- 152 files changed, 17201 insertions(+), 15391 deletions(-) create mode 100644 checkstyle.xml create mode 100644 license-header.txt create mode 100644 src/main/java/org/apache/giraph/benchmark/package-info.java create mode 100644 src/main/java/org/apache/giraph/bsp/package-info.java create mode 100644 src/main/java/org/apache/giraph/comm/package-info.java create mode 100644 src/main/java/org/apache/giraph/examples/package-info.java create mode 100644 src/main/java/org/apache/giraph/graph/package-info.java create mode 100644 src/main/java/org/apache/giraph/graph/partition/package-info.java create mode 100644 src/main/java/org/apache/giraph/hadoop/package-info.java create mode 100644 src/main/java/org/apache/giraph/integration/package-info.java create mode 100644 src/main/java/org/apache/giraph/lib/package-info.java create mode 100644 src/main/java/org/apache/giraph/package-info.java create mode 100644 src/main/java/org/apache/giraph/utils/package-info.java create mode 100644 src/main/java/org/apache/giraph/zk/package-info.java diff --git a/CHANGELOG b/CHANGELOG index adbbd583a..561d6500a 100644 --- a/CHANGELOG +++ b/CHANGELOG @@ -1,6 +1,9 @@ Giraph Change Log Release 0.2.0 - unreleased + + GIRAPH-40: Added checkstyle for enforcement of code conventions. + All Giraph source files now pass checkstyle. (aching) GIRAPH-148: giraph-site.xml needs Apache header. (jghoman) diff --git a/checkstyle.xml b/checkstyle.xml new file mode 100644 index 000000000..6d0c9b7af --- /dev/null +++ b/checkstyle.xml @@ -0,0 +1,296 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/license-header.txt b/license-header.txt new file mode 100644 index 000000000..2379ddac1 --- /dev/null +++ b/license-header.txt @@ -0,0 +1,17 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ diff --git a/pom.xml b/pom.xml index 756149f11..e00c4a018 100644 --- a/pom.xml +++ b/pom.xml @@ -183,7 +183,26 @@ under the License. - + + org.apache.maven.plugins + maven-checkstyle-plugin + 2.9 + + checkstyle.xml + false + license-header.txt + true + false + + + + verify + + check + + + + org.apache.maven.plugins maven-enforcer-plugin @@ -346,7 +365,7 @@ under the License. org.apache.maven.plugins maven-checkstyle-plugin - 2.6 + 2.9 org.codehaus.mojo @@ -391,6 +410,7 @@ under the License. **/target/** _bsp/** + .checkstyle .git/** .idea/** @@ -398,6 +418,39 @@ under the License. + + + + + org.eclipse.m2e + lifecycle-mapping + 1.0.0 + + + + + + + org.apache.maven.plugins + + + maven-assembly-plugin + + [2.2,) + + single + + + + + + + + + + + + diff --git a/src/main/java/org/apache/giraph/GiraphRunner.java b/src/main/java/org/apache/giraph/GiraphRunner.java index 3eb948eff..346f2d398 100644 --- a/src/main/java/org/apache/giraph/GiraphRunner.java +++ b/src/main/java/org/apache/giraph/GiraphRunner.java @@ -31,19 +31,33 @@ import org.apache.hadoop.util.ToolRunner; import org.apache.log4j.Logger; +/** + * Helper class to run Giraph applications by specifying the actual class name + * to use (i.e. vertex, vertex input/output format, combiner, etc.). + */ public class GiraphRunner implements Tool { static { Configuration.addDefaultResource("giraph-site.xml"); } + /** Class logger */ private static final Logger LOG = Logger.getLogger(GiraphRunner.class); + /** Writable conf */ private Configuration conf; - final String [][] requiredOptions = - {{"w", "Need to choose the number of workers (-w)"}, - {"if", "Need to set inputformat (-if)"}}; - - private Options getOptions() { + /** + * Required options. + */ + private final String [][] requiredOptions = + {{"w", "Need to choose the number of workers (-w)"}, + {"if", "Need to set inputformat (-if)"}}; + + /** + * Get the options available. + * + * @return Options available. + */ + private static Options getOptions() { Options options = new Options(); options.addOption("h", "help", false, "Help"); options.addOption("q", "quiet", false, "Quiet output"); @@ -78,7 +92,7 @@ public int run(String[] args) throws Exception { } String vertexClassName = args[0]; - if(LOG.isDebugEnabled()) { + if (LOG.isDebugEnabled()) { LOG.debug("Attempting to run Vertex: " + vertexClassName); } @@ -87,8 +101,8 @@ public int run(String[] args) throws Exception { // Verify all the options have been provided for (String[] requiredOption : requiredOptions) { - if(!cmd.hasOption(requiredOption[0])) { - System.out.println(requiredOption[1]); + if (!cmd.hasOption(requiredOption[0])) { + LOG.info(requiredOption[1]); return -1; } } @@ -99,30 +113,30 @@ public int run(String[] args) throws Exception { job.setVertexInputFormatClass(Class.forName(cmd.getOptionValue("if"))); job.setVertexOutputFormatClass(Class.forName(cmd.getOptionValue("of"))); - if(cmd.hasOption("ip")) { + 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."); + "require one."); } - if(cmd.hasOption("op")) { + 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."); + "require one."); } if (cmd.hasOption("c")) { - job.setVertexCombinerClass(Class.forName(cmd.getOptionValue("c"))); + job.setVertexCombinerClass(Class.forName(cmd.getOptionValue("c"))); } if (cmd.hasOption("wc")) { - job.setWorkerContextClass(Class.forName(cmd.getOptionValue("wc"))); + job.setWorkerContextClass(Class.forName(cmd.getOptionValue("wc"))); } if (cmd.hasOption("aw")) { - job.setAggregatorWriterClass(Class.forName(cmd.getOptionValue("aw"))); + job.setAggregatorWriterClass(Class.forName(cmd.getOptionValue("aw"))); } job.setWorkerConfiguration(workers, workers, 100.0f); @@ -132,6 +146,12 @@ public int run(String[] args) throws Exception { return job.run(isQuiet) ? 0 : -1; } + /** + * Execute GiraphRunner. + * + * @param args Typically command line arguments. + * @throws Exception Any exceptions thrown. + */ public static void main(String[] args) throws Exception { System.exit(ToolRunner.run(new GiraphRunner(), args)); } diff --git a/src/main/java/org/apache/giraph/benchmark/HashMapVertexPageRankBenchmark.java b/src/main/java/org/apache/giraph/benchmark/HashMapVertexPageRankBenchmark.java index aaf9015ba..2c5449674 100644 --- a/src/main/java/org/apache/giraph/benchmark/HashMapVertexPageRankBenchmark.java +++ b/src/main/java/org/apache/giraph/benchmark/HashMapVertexPageRankBenchmark.java @@ -25,13 +25,12 @@ import java.util.Iterator; /** - * Same benchmark code as {@link PageRankBenchmark}, but uses Hashmap-backed Vertex - * implementation rather than {@link org.apache.giraph.graph.EdgeListVertex} + * Same benchmark code as {@link PageRankBenchmark}, but uses + * Hashmap-backed Vertex implementation rather than + * {@link org.apache.giraph.graph.EdgeListVertex} */ public class HashMapVertexPageRankBenchmark extends HashMapVertex< LongWritable, DoubleWritable, DoubleWritable, DoubleWritable> { - public final static String SUPERSTEP_COUNT = PageRankComputation.SUPERSTEP_COUNT; - @Override public void compute(Iterator msgIterator) throws IOException { PageRankComputation.computePageRank(this, msgIterator); diff --git a/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java b/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java index 067362e69..0e7612278 100644 --- a/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java +++ b/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java @@ -23,6 +23,7 @@ import org.apache.commons.cli.HelpFormatter; import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; +import org.apache.giraph.graph.BspUtils; import org.apache.giraph.graph.EdgeListVertex; import org.apache.giraph.graph.GiraphJob; import org.apache.hadoop.conf.Configuration; @@ -30,6 +31,7 @@ import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; +import org.apache.log4j.Logger; import java.io.IOException; import java.util.Iterator; @@ -38,121 +40,125 @@ * Default Pregel-style PageRank computation using a {@link EdgeListVertex}. */ public class PageRankBenchmark extends EdgeListVertex< - LongWritable, DoubleWritable, DoubleWritable, DoubleWritable> implements Tool { - public final static String SUPERSTEP_COUNT = PageRankComputation.SUPERSTEP_COUNT; + LongWritable, DoubleWritable, DoubleWritable, DoubleWritable> + implements Tool { + /** Class logger */ + private static final Logger LOG = Logger.getLogger(PageRankBenchmark.class); + /** Configuration from Configurable */ + private Configuration conf; - /** Configuration from Configurable */ - private Configuration conf; + @Override + public void compute(Iterator msgIterator) throws IOException { + PageRankComputation.computePageRank(this, msgIterator); + } - @Override - public void compute(Iterator msgIterator) throws IOException { - PageRankComputation.computePageRank(this, msgIterator); - } + @Override + public Configuration getConf() { + return conf; + } - @Override - public Configuration getConf() { - return conf; - } + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } - @Override - public void setConf(Configuration conf) { - this.conf = conf; + @Override + public final int run(final String[] args) throws Exception { + Options options = new Options(); + options.addOption("h", "help", false, "Help"); + options.addOption("v", "verbose", false, "Verbose"); + options.addOption("w", + "workers", + true, + "Number of workers"); + options.addOption("s", + "supersteps", + true, + "Supersteps to execute before finishing"); + options.addOption("V", + "aggregateVertices", + true, + "Aggregate vertices"); + options.addOption("e", + "edgesPerVertex", + true, + "Edges per vertex"); + options.addOption("c", + "vertexClass", + true, + "Vertex class (0 for Vertex, 1 for EdgeListVertex)"); + HelpFormatter formatter = new HelpFormatter(); + if (args.length == 0) { + formatter.printHelp(getClass().getName(), options, true); + return 0; + } + CommandLineParser parser = new PosixParser(); + CommandLine cmd = parser.parse(options, args); + if (cmd.hasOption('h')) { + formatter.printHelp(getClass().getName(), options, true); + return 0; + } + if (!cmd.hasOption('w')) { + LOG.info("Need to choose the number of workers (-w)"); + return -1; + } + if (!cmd.hasOption('s')) { + LOG.info("Need to set the number of supersteps (-s)"); + return -1; + } + if (!cmd.hasOption('V')) { + LOG.info("Need to set the aggregate vertices (-V)"); + return -1; + } + if (!cmd.hasOption('e')) { + LOG.info("Need to set the number of edges " + + "per vertex (-e)"); + return -1; } - @Override - public int run(String[] args) throws Exception { - Options options = new Options(); - options.addOption("h", "help", false, "Help"); - options.addOption("v", "verbose", false, "Verbose"); - options.addOption("w", - "workers", - true, - "Number of workers"); - options.addOption("s", - "supersteps", - true, - "Supersteps to execute before finishing"); - options.addOption("V", - "aggregateVertices", - true, - "Aggregate vertices"); - options.addOption("e", - "edgesPerVertex", - true, - "Edges per vertex"); - options.addOption("c", - "vertexClass", - true, - "Vertex class (0 for HashMapVertex, 1 for EdgeListVertex)"); - HelpFormatter formatter = new HelpFormatter(); - if (args.length == 0) { - formatter.printHelp(getClass().getName(), options, true); - return 0; - } - CommandLineParser parser = new PosixParser(); - CommandLine cmd = parser.parse(options, args); - if (cmd.hasOption('h')) { - formatter.printHelp(getClass().getName(), options, true); - return 0; - } - if (!cmd.hasOption('w')) { - System.out.println("Need to choose the number of workers (-w)"); - return -1; - } - if (!cmd.hasOption('s')) { - System.out.println("Need to set the number of supersteps (-s)"); - return -1; - } - if (!cmd.hasOption('V')) { - System.out.println("Need to set the aggregate vertices (-V)"); - return -1; - } - if (!cmd.hasOption('e')) { - System.out.println("Need to set the number of edges " + - "per vertex (-e)"); - return -1; - } - - int workers = Integer.parseInt(cmd.getOptionValue('w')); - GiraphJob job = new GiraphJob(getConf(), getClass().getName()); - if (!cmd.hasOption('c') || - (Integer.parseInt(cmd.getOptionValue('c')) == 0)) { - System.out.println("Using " + - HashMapVertexPageRankBenchmark.class.getName()); - job.setVertexClass(HashMapVertexPageRankBenchmark.class); - } else { - System.out.println("Using " + - PageRankBenchmark.class.getName()); - job.setVertexClass(PageRankBenchmark.class); - } - job.setVertexInputFormatClass(PseudoRandomVertexInputFormat.class); - job.setWorkerConfiguration(workers, workers, 100.0f); - job.getConfiguration().setLong( - PseudoRandomVertexInputFormat.AGGREGATE_VERTICES, - Long.parseLong(cmd.getOptionValue('V'))); - job.getConfiguration().setLong( - PseudoRandomVertexInputFormat.EDGES_PER_VERTEX, - Long.parseLong(cmd.getOptionValue('e'))); - job.getConfiguration().setInt( - SUPERSTEP_COUNT, - Integer.parseInt(cmd.getOptionValue('s'))); - - boolean isVerbose = false; - if (cmd.hasOption('v')) { - isVerbose = true; - } - if (cmd.hasOption('s')) { - getConf().setInt(SUPERSTEP_COUNT, - Integer.parseInt(cmd.getOptionValue('s'))); - } - if (job.run(isVerbose) == true) { - return 0; - } else { - return -1; - } + int workers = Integer.parseInt(cmd.getOptionValue('w')); + GiraphJob job = new GiraphJob(getConf(), getClass().getName()); + if (!cmd.hasOption('c') || + (Integer.parseInt(cmd.getOptionValue('c')) == 0)) { + job.setVertexClass(HashMapVertexPageRankBenchmark.class); + } else { + job.setVertexClass(PageRankBenchmark.class); } + LOG.info("Using class " + BspUtils.getVertexClass(getConf()).getName()); + job.setVertexInputFormatClass(PseudoRandomVertexInputFormat.class); + job.setWorkerConfiguration(workers, workers, 100.0f); + job.getConfiguration().setLong( + PseudoRandomVertexInputFormat.AGGREGATE_VERTICES, + Long.parseLong(cmd.getOptionValue('V'))); + job.getConfiguration().setLong( + PseudoRandomVertexInputFormat.EDGES_PER_VERTEX, + Long.parseLong(cmd.getOptionValue('e'))); + job.getConfiguration().setInt( + PageRankComputation.SUPERSTEP_COUNT, + Integer.parseInt(cmd.getOptionValue('s'))); - public static void main(String[] args) throws Exception { - System.exit(ToolRunner.run(new PageRankBenchmark(), args)); + boolean isVerbose = false; + if (cmd.hasOption('v')) { + isVerbose = true; } + if (cmd.hasOption('s')) { + getConf().setInt(PageRankComputation.SUPERSTEP_COUNT, + Integer.parseInt(cmd.getOptionValue('s'))); + } + if (job.run(isVerbose)) { + return 0; + } else { + return -1; + } + } + + /** + * Execute the benchmark. + * + * @param args Typically the command line arguments. + * @throws Exception Any exception from the computation. + */ + public static void main(final String[] args) throws Exception { + System.exit(ToolRunner.run(new PageRankBenchmark(), args)); + } } diff --git a/src/main/java/org/apache/giraph/benchmark/PageRankComputation.java b/src/main/java/org/apache/giraph/benchmark/PageRankComputation.java index ba98d6c16..6fb47a637 100644 --- a/src/main/java/org/apache/giraph/benchmark/PageRankComputation.java +++ b/src/main/java/org/apache/giraph/benchmark/PageRankComputation.java @@ -28,11 +28,24 @@ * classes. */ public class PageRankComputation { - public final static String SUPERSTEP_COUNT = "PageRankBenchmark.superstepCount"; + /** Number of supersteps */ + public static final String SUPERSTEP_COUNT = + "PageRankBenchmark.superstepCount"; + /** + * Do not construct. + */ + private PageRankComputation() { } + + /** + * Generic page rank algorithm. + * + * @param vertex Vertex to compute on. + * @param msgIterator Iterator of messages from previous superstep. + */ public static void computePageRank( - MutableVertex - vertex, Iterator msgIterator) { + MutableVertex vertex, Iterator msgIterator) { if (vertex.getSuperstep() >= 1) { double sum = 0; while (msgIterator.hasNext()) { diff --git a/src/main/java/org/apache/giraph/benchmark/PseudoRandomVertexInputFormat.java b/src/main/java/org/apache/giraph/benchmark/PseudoRandomVertexInputFormat.java index 970232a6c..ab80198c5 100644 --- a/src/main/java/org/apache/giraph/benchmark/PseudoRandomVertexInputFormat.java +++ b/src/main/java/org/apache/giraph/benchmark/PseudoRandomVertexInputFormat.java @@ -44,147 +44,148 @@ * to create an input data source that a variable number of aggregate vertices * and edges per vertex that is repeatable for the exact same parameter * (pseudo-random). + * + * @param Message data */ public class PseudoRandomVertexInputFormat extends - VertexInputFormat { - /** Set the number of aggregate vertices */ - public static final String AGGREGATE_VERTICES = - "pseudoRandomVertexReader.aggregateVertices"; - /** Set the number of edges per vertex (pseudo-random destination) */ - public static final String EDGES_PER_VERTEX = - "pseudoRandomVertexReader.edgesPerVertex"; + VertexInputFormat { + /** Set the number of aggregate vertices. */ + public static final String AGGREGATE_VERTICES = + "pseudoRandomVertexReader.aggregateVertices"; + /** Set the number of edges per vertex (pseudo-random destination). */ + public static final String EDGES_PER_VERTEX = + "pseudoRandomVertexReader.edgesPerVertex"; - @Override - public List getSplits(JobContext context, int numWorkers) - throws IOException, InterruptedException { - // This is meaningless, the PseudoRandomVertexReader will generate - // all the test data - List inputSplitList = new ArrayList(); - for (int i = 0; i < numWorkers; ++i) { - inputSplitList.add(new BspInputSplit(i, numWorkers)); - } - return inputSplitList; + @Override + public final List getSplits(final JobContext context, + final int numWorkers) throws IOException, InterruptedException { + // This is meaningless, the PseudoRandomVertexReader will generate + // all the test data + List inputSplitList = new ArrayList(); + for (int i = 0; i < numWorkers; ++i) { + inputSplitList.add(new BspInputSplit(i, numWorkers)); } + return inputSplitList; + } - @Override - public VertexReader - createVertexReader(InputSplit split, TaskAttemptContext context) - throws IOException { - return new PseudoRandomVertexReader(); - } + @Override + public VertexReader + createVertexReader(InputSplit split, TaskAttemptContext context) + throws IOException { + return new PseudoRandomVertexReader(); + } + + /** + * Used by {@link PseudoRandomVertexInputFormat} to read + * pseudo-randomly generated data. + */ + private static class PseudoRandomVertexReader implements + VertexReader { + /** Logger. */ + private static final Logger LOG = + Logger.getLogger(PseudoRandomVertexReader.class); + /** Starting vertex id. */ + private long startingVertexId = -1; + /** Vertices read so far. */ + private long verticesRead = 0; + /** Total vertices to read (on this split alone). */ + private long totalSplitVertices = -1; + /** Aggregate vertices (all input splits). */ + private long aggregateVertices = -1; + /** Edges per vertex. */ + private long edgesPerVertex = -1; + /** BspInputSplit (used only for index). */ + private BspInputSplit bspInputSplit; + /** Saved configuration */ + private Configuration configuration; /** - * Used by {@link PseudoRandomVertexInputFormat} to read - * pseudo-randomly generated data + * Default constructor for reflection. */ - private static class PseudoRandomVertexReader implements - VertexReader { - /** Logger */ - private static final Logger LOG = - Logger.getLogger(PseudoRandomVertexReader.class); - /** Starting vertex id */ - private long startingVertexId = -1; - /** Vertices read so far */ - private long verticesRead = 0; - /** Total vertices to read (on this split alone) */ - private long totalSplitVertices = -1; - /** Aggregate vertices (all input splits) */ - private long aggregateVertices = -1; - /** Edges per vertex */ - private long edgesPerVertex = -1; - /** BspInputSplit (used only for index) */ - private BspInputSplit bspInputSplit; - - private Configuration configuration; - - public PseudoRandomVertexReader() { - } + public PseudoRandomVertexReader() { + } - @Override - public void initialize(InputSplit inputSplit, - TaskAttemptContext context) throws IOException { - configuration = context.getConfiguration(); - aggregateVertices = - configuration.getLong( - PseudoRandomVertexInputFormat.AGGREGATE_VERTICES, 0); - if (aggregateVertices <= 0) { - throw new IllegalArgumentException( - "initialize: " + - PseudoRandomVertexInputFormat.AGGREGATE_VERTICES + " <= 0"); - } - if (inputSplit instanceof BspInputSplit) { - bspInputSplit = (BspInputSplit) inputSplit; - long extraVertices = - aggregateVertices % bspInputSplit.getNumSplits(); - totalSplitVertices = - aggregateVertices / bspInputSplit.getNumSplits(); - if (bspInputSplit.getSplitIndex() < extraVertices) { - ++totalSplitVertices; - } - startingVertexId = (bspInputSplit.getSplitIndex() * - (aggregateVertices / bspInputSplit.getNumSplits())) + - Math.min(bspInputSplit.getSplitIndex(), - extraVertices); - } else { - throw new IllegalArgumentException( - "initialize: Got " + inputSplit.getClass() + - " instead of " + BspInputSplit.class); - } - edgesPerVertex = - configuration.getLong( - PseudoRandomVertexInputFormat.EDGES_PER_VERTEX, 0); - if (edgesPerVertex <= 0) { - throw new IllegalArgumentException( - "initialize: " + - PseudoRandomVertexInputFormat.EDGES_PER_VERTEX + " <= 0"); - } + @Override + public void initialize(InputSplit inputSplit, + TaskAttemptContext context) throws IOException { + configuration = context.getConfiguration(); + aggregateVertices = + configuration.getLong( + PseudoRandomVertexInputFormat.AGGREGATE_VERTICES, 0); + if (aggregateVertices <= 0) { + throw new IllegalArgumentException( + PseudoRandomVertexInputFormat.AGGREGATE_VERTICES + " <= 0"); + } + if (inputSplit instanceof BspInputSplit) { + bspInputSplit = (BspInputSplit) inputSplit; + long extraVertices = + aggregateVertices % bspInputSplit.getNumSplits(); + totalSplitVertices = + aggregateVertices / bspInputSplit.getNumSplits(); + if (bspInputSplit.getSplitIndex() < extraVertices) { + ++totalSplitVertices; } + startingVertexId = (bspInputSplit.getSplitIndex() * + (aggregateVertices / bspInputSplit.getNumSplits())) + + Math.min(bspInputSplit.getSplitIndex(), + extraVertices); + } else { + throw new IllegalArgumentException( + "initialize: Got " + inputSplit.getClass() + + " instead of " + BspInputSplit.class); + } + edgesPerVertex = configuration.getLong( + PseudoRandomVertexInputFormat.EDGES_PER_VERTEX, 0); + if (edgesPerVertex <= 0) { + throw new IllegalArgumentException( + PseudoRandomVertexInputFormat.EDGES_PER_VERTEX + " <= 0"); + } + } - @Override - public boolean nextVertex() throws IOException, InterruptedException { - return totalSplitVertices > verticesRead; - } + @Override + public boolean nextVertex() throws IOException, InterruptedException { + return totalSplitVertices > verticesRead; + } - @Override - public BasicVertex getCurrentVertex() - throws IOException, InterruptedException { - BasicVertex - vertex = BspUtils.createVertex(configuration); - long vertexId = startingVertexId + verticesRead; - // Seed on the vertex id to keep the vertex data the same when - // on different number of workers, but other parameters are the - // same. - Random rand = new Random(vertexId); - DoubleWritable vertexValue = new DoubleWritable(rand.nextDouble()); - Map edges = Maps.newHashMap(); - for (long i = 0; i < edgesPerVertex; ++i) { - LongWritable destVertexId = null; - do { - destVertexId = - new LongWritable(Math.abs(rand.nextLong()) % - aggregateVertices); - } while (edges.containsKey(destVertexId)); - edges.put(destVertexId, new DoubleWritable(rand.nextDouble())); - } - vertex.initialize( - new LongWritable(vertexId), vertexValue, edges, null); - ++verticesRead; - if (LOG.isDebugEnabled()) { - LOG.debug("next: Return vertexId=" + - vertex.getVertexId().get() + - ", vertexValue=" + vertex.getVertexValue() + - ", edgeMap=" + vertex.iterator()); - } - return vertex; - } + @Override + public BasicVertex + getCurrentVertex() throws IOException, InterruptedException { + BasicVertex + vertex = BspUtils.createVertex(configuration); + long vertexId = startingVertexId + verticesRead; + // Seed on the vertex id to keep the vertex data the same when + // on different number of workers, but other parameters are the + // same. + Random rand = new Random(vertexId); + DoubleWritable vertexValue = new DoubleWritable(rand.nextDouble()); + Map edges = Maps.newHashMap(); + for (long i = 0; i < edgesPerVertex; ++i) { + LongWritable destVertexId = null; + do { + destVertexId = + new LongWritable(Math.abs(rand.nextLong()) % + aggregateVertices); + } while (edges.containsKey(destVertexId)); + edges.put(destVertexId, new DoubleWritable(rand.nextDouble())); + } + vertex.initialize(new LongWritable(vertexId), vertexValue, edges, null); + ++verticesRead; + if (LOG.isDebugEnabled()) { + LOG.debug("next: Return vertexId=" + + vertex.getVertexId().get() + + ", vertexValue=" + vertex.getVertexValue() + + ", edgeMap=" + vertex.iterator()); + } + return vertex; + } - @Override - public void close() throws IOException { - } + @Override + public void close() throws IOException { + } - @Override - public float getProgress() throws IOException { - return verticesRead * 100.0f / totalSplitVertices; - } + @Override + public float getProgress() throws IOException { + return verticesRead * 100.0f / totalSplitVertices; } + } } diff --git a/src/main/java/org/apache/giraph/benchmark/RandomMessageBenchmark.java b/src/main/java/org/apache/giraph/benchmark/RandomMessageBenchmark.java index ff6c986e9..39f2b512a 100644 --- a/src/main/java/org/apache/giraph/benchmark/RandomMessageBenchmark.java +++ b/src/main/java/org/apache/giraph/benchmark/RandomMessageBenchmark.java @@ -41,361 +41,386 @@ * Random Message Benchmark for evaluating the messaging performance. */ public class RandomMessageBenchmark implements Tool { - /** Configuration from Configurable */ - private Configuration conf; + /** How many supersteps to run */ + public static final String SUPERSTEP_COUNT = + "RandomMessageBenchmark.superstepCount"; + /** How many bytes per message */ + public static final String NUM_BYTES_PER_MESSAGE = + "RandomMessageBenchmark.numBytesPerMessage"; + /** Default bytes per message */ + public static final int DEFAULT_NUM_BYTES_PER_MESSAGE = 16; + /** How many messages per edge */ + public static final String NUM_MESSAGES_PER_EDGE = + "RandomMessageBenchmark.numMessagesPerEdge"; + /** Default messages per edge */ + public static final int DEFAULT_NUM_MESSAGES_PER_EDGE = 1; + /** All bytes sent during this superstep */ + public static final String AGG_SUPERSTEP_TOTAL_BYTES = + "superstep total bytes sent"; + /** All bytes sent during this application */ + public static final String AGG_TOTAL_BYTES = "total bytes sent"; + /** All messages during this superstep */ + public static final String AGG_SUPERSTEP_TOTAL_MESSAGES = + "superstep total messages"; + /** All messages during this application */ + public static final String AGG_TOTAL_MESSAGES = "total messages"; + /** All millis during this superstep */ + public static final String AGG_SUPERSTEP_TOTAL_MILLIS = + "superstep total millis"; + /** All millis during this application */ + public static final String AGG_TOTAL_MILLIS = "total millis"; + /** Workers for that superstep */ + public static final String WORKERS = "workers"; + /** Class logger */ + private static final Logger LOG = + Logger.getLogger(RandomMessageBenchmarkWorkerContext.class); + /** Configuration from Configurable */ + private Configuration conf; - /** How many supersteps to run */ - public static final String SUPERSTEP_COUNT = - "RandomMessageBenchmark.superstepCount"; - /** How many bytes per message */ - public static final String NUM_BYTES_PER_MESSAGE = - "RandomMessageBenchmark.numBytesPerMessage"; - /** Default bytes per message */ - public static final int DEFAULT_NUM_BYTES_PER_MESSAGE = 16; - /** How many messages per edge */ - public static final String NUM_MESSAGES_PER_EDGE= - "RandomMessageBenchmark.numMessagesPerEdge"; - /** Default messages per edge */ - public static final int DEFAULT_NUM_MESSAGES_PER_EDGE = 1; - /** All bytes sent during this superstep */ - public static final String AGG_SUPERSTEP_TOTAL_BYTES = - "superstep total bytes sent"; - /** All bytes sent during this application */ - public static final String AGG_TOTAL_BYTES = "total bytes sent"; - /** All messages during this superstep */ - public static final String AGG_SUPERSTEP_TOTAL_MESSAGES = - "superstep total messages"; - /** All messages during this application */ - public static final String AGG_TOTAL_MESSAGES = "total messages"; - /** All millis during this superstep */ - public static final String AGG_SUPERSTEP_TOTAL_MILLIS = - "superstep total millis"; - /** All millis during this application */ - public static final String AGG_TOTAL_MILLIS = "total millis"; - /** Workers for that superstep */ - public static final String WORKERS = "workers"; + /** + * {@link WorkerContext} forRandomMessageBenchmark. + */ + private static class RandomMessageBenchmarkWorkerContext extends + WorkerContext { + /** Class logger */ + private static final Logger LOG = + Logger.getLogger(RandomMessageBenchmarkWorkerContext.class); + /** Bytes to be sent */ + private byte[] messageBytes; + /** Number of messages sent per edge */ + private int numMessagesPerEdge = -1; + /** Number of supersteps */ + private int numSupersteps = -1; + /** Random generator for random bytes message */ + private final Random random = new Random(System.currentTimeMillis()); + /** Start superstep millis */ + private long startSuperstepMillis = 0; + /** Total bytes */ + private long totalBytes = 0; + /** Total messages */ + private long totalMessages = 0; + /** Total millis */ + private long totalMillis = 0; + /** Class logger */ - /** - * {@link WorkerContext} forRandomMessageBenchmark. - */ - private static class RandomMessageBenchmarkWorkerContext extends - WorkerContext { - /** Bytes to be sent */ - private byte[] messageBytes; - /** Number of messages sent per edge */ - private int numMessagesPerEdge = -1; - /** Number of supersteps */ - private int numSupersteps = -1; - /** Random generator for random bytes message */ - private final Random random = new Random(System.currentTimeMillis()); - /** Start superstep millis */ - private long startSuperstepMillis = 0; - /** Total bytes */ - private long totalBytes = 0; - /** Total messages */ - private long totalMessages = 0; - /** Total millis */ - private long totalMillis = 0; - /** Class logger */ - private static final Logger LOG = - Logger.getLogger(RandomMessageBenchmarkWorkerContext.class); - - @Override - public void preApplication() - throws InstantiationException, IllegalAccessException { - messageBytes = - new byte[getContext().getConfiguration(). - getInt(NUM_BYTES_PER_MESSAGE, - DEFAULT_NUM_BYTES_PER_MESSAGE)]; - numMessagesPerEdge = - getContext().getConfiguration(). - getInt(NUM_MESSAGES_PER_EDGE, - DEFAULT_NUM_MESSAGES_PER_EDGE); - numSupersteps = getContext().getConfiguration(). - getInt(SUPERSTEP_COUNT, -1); - registerAggregator(AGG_SUPERSTEP_TOTAL_BYTES, - LongSumAggregator.class); - registerAggregator(AGG_SUPERSTEP_TOTAL_MESSAGES, - LongSumAggregator.class); - registerAggregator(AGG_SUPERSTEP_TOTAL_MILLIS, - LongSumAggregator.class); - registerAggregator(WORKERS, - LongSumAggregator.class); - } - - @Override - public void preSuperstep() { - LongSumAggregator superstepBytesAggregator = - (LongSumAggregator) getAggregator(AGG_SUPERSTEP_TOTAL_BYTES); - LongSumAggregator superstepMessagesAggregator = - (LongSumAggregator) getAggregator(AGG_SUPERSTEP_TOTAL_MESSAGES); - LongSumAggregator superstepMillisAggregator = - (LongSumAggregator) getAggregator(AGG_SUPERSTEP_TOTAL_MILLIS); - LongSumAggregator workersAggregator = - (LongSumAggregator) getAggregator(WORKERS); - - // For timing and tracking the supersteps - // - superstep 0 starts the time, but cannot display any stats - // since nothing has been aggregated yet - // - supersteps > 0 can display the stats - if (getSuperstep() == 0) { - startSuperstepMillis = System.currentTimeMillis(); - } else { - totalBytes += - superstepBytesAggregator.getAggregatedValue().get(); - totalMessages += - superstepMessagesAggregator.getAggregatedValue().get(); - totalMillis += - superstepMillisAggregator.getAggregatedValue().get(); - double superstepMegabytesPerSecond = - superstepBytesAggregator.getAggregatedValue().get() * - workersAggregator.getAggregatedValue().get() * - 1000d / 1024d / 1024d / - superstepMillisAggregator.getAggregatedValue().get(); - double megabytesPerSecond = totalBytes * - workersAggregator.getAggregatedValue().get() * - 1000d / 1024d / 1024d / totalMillis; - double superstepMessagesPerSecond = - superstepMessagesAggregator.getAggregatedValue().get() * - workersAggregator.getAggregatedValue().get() * 1000d / - superstepMillisAggregator.getAggregatedValue().get(); - double messagesPerSecond = totalMessages * - workersAggregator.getAggregatedValue().get() * 1000d / - totalMillis; - if (LOG.isInfoEnabled()) { - LOG.info("Outputing statistics for superstep " + - getSuperstep()); - LOG.info(AGG_SUPERSTEP_TOTAL_BYTES + " : " + - superstepBytesAggregator.getAggregatedValue()); - LOG.info(AGG_TOTAL_BYTES + " : " + totalBytes); - LOG.info(AGG_SUPERSTEP_TOTAL_MESSAGES + " : " + - superstepMessagesAggregator.getAggregatedValue()); - LOG.info(AGG_TOTAL_MESSAGES + " : " + totalMessages); - LOG.info(AGG_SUPERSTEP_TOTAL_MILLIS + " : " + - superstepMillisAggregator.getAggregatedValue()); - LOG.info(AGG_TOTAL_MILLIS + " : " + totalMillis); - LOG.info(WORKERS + " : " + - workersAggregator.getAggregatedValue()); - LOG.info("Superstep megabytes / second = " + - superstepMegabytesPerSecond); - LOG.info("Total megabytes / second = " + - megabytesPerSecond); - LOG.info("Superstep messages / second = " + - superstepMessagesPerSecond); - LOG.info("Total messages / second = " + - messagesPerSecond); - LOG.info("Superstep megabytes / second / worker = " + - superstepMegabytesPerSecond / - workersAggregator.getAggregatedValue().get()); - LOG.info("Total megabytes / second / worker = " + - megabytesPerSecond / - workersAggregator.getAggregatedValue().get()); - LOG.info("Superstep messages / second / worker = " + - superstepMessagesPerSecond / - workersAggregator.getAggregatedValue().get()); - LOG.info("Total messages / second / worker = " + - messagesPerSecond / - workersAggregator.getAggregatedValue().get()); - } - } + @Override + public void preApplication() + throws InstantiationException, IllegalAccessException { + messageBytes = + new byte[getContext().getConfiguration(). + getInt(NUM_BYTES_PER_MESSAGE, + DEFAULT_NUM_BYTES_PER_MESSAGE)]; + numMessagesPerEdge = + getContext().getConfiguration(). + getInt(NUM_MESSAGES_PER_EDGE, + DEFAULT_NUM_MESSAGES_PER_EDGE); + numSupersteps = getContext().getConfiguration(). + getInt(SUPERSTEP_COUNT, -1); + registerAggregator(AGG_SUPERSTEP_TOTAL_BYTES, + LongSumAggregator.class); + registerAggregator(AGG_SUPERSTEP_TOTAL_MESSAGES, + LongSumAggregator.class); + registerAggregator(AGG_SUPERSTEP_TOTAL_MILLIS, + LongSumAggregator.class); + registerAggregator(WORKERS, + LongSumAggregator.class); + } - superstepBytesAggregator.setAggregatedValue( - new LongWritable(0L)); - superstepMessagesAggregator.setAggregatedValue( - new LongWritable(0L)); - workersAggregator.setAggregatedValue( - new LongWritable(1L)); - useAggregator(AGG_SUPERSTEP_TOTAL_BYTES); - useAggregator(AGG_SUPERSTEP_TOTAL_MILLIS); - useAggregator(AGG_SUPERSTEP_TOTAL_MESSAGES); - useAggregator(WORKERS); - } + @Override + public void preSuperstep() { + LongSumAggregator superstepBytesAggregator = + (LongSumAggregator) getAggregator(AGG_SUPERSTEP_TOTAL_BYTES); + LongSumAggregator superstepMessagesAggregator = + (LongSumAggregator) getAggregator(AGG_SUPERSTEP_TOTAL_MESSAGES); + LongSumAggregator superstepMillisAggregator = + (LongSumAggregator) getAggregator(AGG_SUPERSTEP_TOTAL_MILLIS); + LongSumAggregator workersAggregator = + (LongSumAggregator) getAggregator(WORKERS); - @Override - public void postSuperstep() { - LongSumAggregator superstepMillisAggregator = - (LongSumAggregator) getAggregator(AGG_SUPERSTEP_TOTAL_MILLIS); - long endSuperstepMillis = System.currentTimeMillis(); - long superstepMillis = endSuperstepMillis - startSuperstepMillis; - startSuperstepMillis = endSuperstepMillis; - superstepMillisAggregator.setAggregatedValue( - new LongWritable(superstepMillis)); + // For timing and tracking the supersteps + // - superstep 0 starts the time, but cannot display any stats + // since nothing has been aggregated yet + // - supersteps > 0 can display the stats + if (getSuperstep() == 0) { + startSuperstepMillis = System.currentTimeMillis(); + } else { + totalBytes += + superstepBytesAggregator.getAggregatedValue().get(); + totalMessages += + superstepMessagesAggregator.getAggregatedValue().get(); + totalMillis += + superstepMillisAggregator.getAggregatedValue().get(); + double superstepMegabytesPerSecond = + superstepBytesAggregator.getAggregatedValue().get() * + workersAggregator.getAggregatedValue().get() * + 1000d / 1024d / 1024d / + superstepMillisAggregator.getAggregatedValue().get(); + double megabytesPerSecond = totalBytes * + workersAggregator.getAggregatedValue().get() * + 1000d / 1024d / 1024d / totalMillis; + double superstepMessagesPerSecond = + superstepMessagesAggregator.getAggregatedValue().get() * + workersAggregator.getAggregatedValue().get() * 1000d / + superstepMillisAggregator.getAggregatedValue().get(); + double messagesPerSecond = totalMessages * + workersAggregator.getAggregatedValue().get() * 1000d / + totalMillis; + if (LOG.isInfoEnabled()) { + LOG.info("Outputing statistics for superstep " + + getSuperstep()); + LOG.info(AGG_SUPERSTEP_TOTAL_BYTES + " : " + + superstepBytesAggregator.getAggregatedValue()); + LOG.info(AGG_TOTAL_BYTES + " : " + totalBytes); + LOG.info(AGG_SUPERSTEP_TOTAL_MESSAGES + " : " + + superstepMessagesAggregator.getAggregatedValue()); + LOG.info(AGG_TOTAL_MESSAGES + " : " + totalMessages); + LOG.info(AGG_SUPERSTEP_TOTAL_MILLIS + " : " + + superstepMillisAggregator.getAggregatedValue()); + LOG.info(AGG_TOTAL_MILLIS + " : " + totalMillis); + LOG.info(WORKERS + " : " + + workersAggregator.getAggregatedValue()); + LOG.info("Superstep megabytes / second = " + + superstepMegabytesPerSecond); + LOG.info("Total megabytes / second = " + + megabytesPerSecond); + LOG.info("Superstep messages / second = " + + superstepMessagesPerSecond); + LOG.info("Total messages / second = " + + messagesPerSecond); + LOG.info("Superstep megabytes / second / worker = " + + superstepMegabytesPerSecond / + workersAggregator.getAggregatedValue().get()); + LOG.info("Total megabytes / second / worker = " + + megabytesPerSecond / + workersAggregator.getAggregatedValue().get()); + LOG.info("Superstep messages / second / worker = " + + superstepMessagesPerSecond / + workersAggregator.getAggregatedValue().get()); + LOG.info("Total messages / second / worker = " + + messagesPerSecond / + workersAggregator.getAggregatedValue().get()); } + } - @Override - public void postApplication() {} - - public byte[] getMessageBytes() { - return messageBytes; - } + superstepBytesAggregator.setAggregatedValue( + new LongWritable(0L)); + superstepMessagesAggregator.setAggregatedValue( + new LongWritable(0L)); + workersAggregator.setAggregatedValue( + new LongWritable(1L)); + useAggregator(AGG_SUPERSTEP_TOTAL_BYTES); + useAggregator(AGG_SUPERSTEP_TOTAL_MILLIS); + useAggregator(AGG_SUPERSTEP_TOTAL_MESSAGES); + useAggregator(WORKERS); + } - public int getNumMessagePerEdge() { - return numMessagesPerEdge; - } + @Override + public void postSuperstep() { + LongSumAggregator superstepMillisAggregator = + (LongSumAggregator) getAggregator(AGG_SUPERSTEP_TOTAL_MILLIS); + long endSuperstepMillis = System.currentTimeMillis(); + long superstepMillis = endSuperstepMillis - startSuperstepMillis; + startSuperstepMillis = endSuperstepMillis; + superstepMillisAggregator.setAggregatedValue( + new LongWritable(superstepMillis)); + } - public int getNumSupersteps() { - return numSupersteps; - } + @Override + public void postApplication() { } - public void randomizeMessageBytes() { - random.nextBytes(messageBytes); - } + /** + * Get the message bytes to be used for sending. + * + * @return Byte array used for messages. + */ + public byte[] getMessageBytes() { + return messageBytes; } /** - * Actual message computation (messaging in this case) + * Get the number of edges per message. + * + * @return Messages per edge. */ - public static class RandomMessageVertex extends EdgeListVertex< - LongWritable, DoubleWritable, DoubleWritable, BytesWritable> { + public int getNumMessagePerEdge() { + return numMessagesPerEdge; + } - @Override - public void compute(Iterator msgIterator) { - RandomMessageBenchmarkWorkerContext workerContext = - (RandomMessageBenchmarkWorkerContext) getWorkerContext(); - LongSumAggregator superstepBytesAggregator = - (LongSumAggregator) getAggregator(AGG_SUPERSTEP_TOTAL_BYTES); - LongSumAggregator superstepMessagesAggregator = - (LongSumAggregator) getAggregator(AGG_SUPERSTEP_TOTAL_MESSAGES); - if (getSuperstep() < workerContext.getNumSupersteps()) { - for (int i = 0; i < workerContext.getNumMessagePerEdge(); - i++) { - workerContext.randomizeMessageBytes(); - sendMsgToAllEdges( - new BytesWritable(workerContext.getMessageBytes())); - long bytesSent = workerContext.getMessageBytes().length * - getNumOutEdges(); - superstepBytesAggregator.aggregate(bytesSent); - superstepMessagesAggregator.aggregate(getNumOutEdges()); - } - } else { - voteToHalt(); - } - } + /** + * Get the number of supersteps. + * + * @return Number of supersteps. + */ + public int getNumSupersteps() { + return numSupersteps; } - @Override - public Configuration getConf() { - return conf; + /** + * Randomize the message bytes. + */ + public void randomizeMessageBytes() { + random.nextBytes(messageBytes); } + } + /** + * Actual message computation (messaging in this case) + */ + public static class RandomMessageVertex extends EdgeListVertex< + LongWritable, DoubleWritable, DoubleWritable, BytesWritable> { @Override - public void setConf(Configuration conf) { - this.conf = conf; + public void compute(Iterator msgIterator) { + RandomMessageBenchmarkWorkerContext workerContext = + (RandomMessageBenchmarkWorkerContext) getWorkerContext(); + LongSumAggregator superstepBytesAggregator = + (LongSumAggregator) getAggregator(AGG_SUPERSTEP_TOTAL_BYTES); + LongSumAggregator superstepMessagesAggregator = + (LongSumAggregator) getAggregator(AGG_SUPERSTEP_TOTAL_MESSAGES); + if (getSuperstep() < workerContext.getNumSupersteps()) { + for (int i = 0; i < workerContext.getNumMessagePerEdge(); i++) { + workerContext.randomizeMessageBytes(); + sendMsgToAllEdges( + new BytesWritable(workerContext.getMessageBytes())); + long bytesSent = workerContext.getMessageBytes().length * + getNumOutEdges(); + superstepBytesAggregator.aggregate(bytesSent); + superstepMessagesAggregator.aggregate(getNumOutEdges()); + } + } else { + voteToHalt(); + } } + } - @Override - public int run(String[] args) throws Exception { - Options options = new Options(); - options.addOption("h", "help", false, "Help"); - options.addOption("v", "verbose", false, "Verbose"); - options.addOption("w", - "workers", - true, - "Number of workers"); - options.addOption("b", - "bytes", - true, - "Message bytes per memssage"); - options.addOption("n", - "number", - true, - "Number of messages per edge"); - options.addOption("s", - "supersteps", - true, - "Supersteps to execute before finishing"); - options.addOption("V", - "aggregateVertices", - true, - "Aggregate vertices"); - options.addOption("e", - "edgesPerVertex", - true, - "Edges per vertex"); - options.addOption("f", - "flusher", - true, - "Number of flush threads"); + @Override + public Configuration getConf() { + return conf; + } - HelpFormatter formatter = new HelpFormatter(); - if (args.length == 0) { - formatter.printHelp(getClass().getName(), options, true); - return 0; - } - CommandLineParser parser = new PosixParser(); - CommandLine cmd = parser.parse(options, args); - if (cmd.hasOption('h')) { - formatter.printHelp(getClass().getName(), options, true); - return 0; - } - if (!cmd.hasOption('w')) { - System.out.println("Need to choose the number of workers (-w)"); - return -1; - } - if (!cmd.hasOption('s')) { - System.out.println("Need to set the number of supersteps (-s)"); - return -1; - } - if (!cmd.hasOption('V')) { - System.out.println("Need to set the aggregate vertices (-V)"); - return -1; - } - if (!cmd.hasOption('e')) { - System.out.println("Need to set the number of edges " + - "per vertex (-e)"); - return -1; - } - if (!cmd.hasOption('b')) { - System.out.println("Need to set the number of message bytes (-b)"); - return -1; - } - if (!cmd.hasOption('n')) { - System.out.println("Need to set the number of messages per edge (-n)"); - return -1; - } - int workers = Integer.parseInt(cmd.getOptionValue('w')); - GiraphJob job = new GiraphJob(getConf(), getClass().getName()); - job.getConfiguration().setInt(GiraphJob.CHECKPOINT_FREQUENCY, 0); - job.setVertexClass(RandomMessageVertex.class); - job.setVertexInputFormatClass(PseudoRandomVertexInputFormat.class); - job.setWorkerContextClass(RandomMessageBenchmarkWorkerContext.class); - job.setWorkerConfiguration(workers, workers, 100.0f); - job.getConfiguration().setLong( - PseudoRandomVertexInputFormat.AGGREGATE_VERTICES, - Long.parseLong(cmd.getOptionValue('V'))); - job.getConfiguration().setLong( - PseudoRandomVertexInputFormat.EDGES_PER_VERTEX, - Long.parseLong(cmd.getOptionValue('e'))); - job.getConfiguration().setInt( - SUPERSTEP_COUNT, - Integer.parseInt(cmd.getOptionValue('s'))); - job.getConfiguration().setInt( - RandomMessageBenchmark.NUM_BYTES_PER_MESSAGE, - Integer.parseInt(cmd.getOptionValue('b'))); - job.getConfiguration().setInt( - RandomMessageBenchmark.NUM_MESSAGES_PER_EDGE, - Integer.parseInt(cmd.getOptionValue('n'))); + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } - boolean isVerbose = false; - if (cmd.hasOption('v')) { - isVerbose = true; - } - if (cmd.hasOption('s')) { - getConf().setInt(SUPERSTEP_COUNT, - Integer.parseInt(cmd.getOptionValue('s'))); - } - if (cmd.hasOption('f')) { - job.getConfiguration().setInt(GiraphJob.MSG_NUM_FLUSH_THREADS, - Integer.parseInt(cmd.getOptionValue('f'))); - } - if (job.run(isVerbose) == true) { - return 0; - } else { - return -1; - } + @Override + public int run(String[] args) throws Exception { + Options options = new Options(); + options.addOption("h", "help", false, "Help"); + options.addOption("v", "verbose", false, "Verbose"); + options.addOption("w", + "workers", + true, + "Number of workers"); + options.addOption("b", + "bytes", + true, + "Message bytes per memssage"); + options.addOption("n", + "number", + true, + "Number of messages per edge"); + options.addOption("s", + "supersteps", + true, + "Supersteps to execute before finishing"); + options.addOption("V", + "aggregateVertices", + true, + "Aggregate vertices"); + options.addOption("e", + "edgesPerVertex", + true, + "Edges per vertex"); + options.addOption("f", + "flusher", + true, + "Number of flush threads"); + + HelpFormatter formatter = new HelpFormatter(); + if (args.length == 0) { + formatter.printHelp(getClass().getName(), options, true); + return 0; + } + CommandLineParser parser = new PosixParser(); + CommandLine cmd = parser.parse(options, args); + if (cmd.hasOption('h')) { + formatter.printHelp(getClass().getName(), options, true); + return 0; + } + if (!cmd.hasOption('w')) { + LOG.info("Need to choose the number of workers (-w)"); + return -1; + } + if (!cmd.hasOption('s')) { + LOG.info("Need to set the number of supersteps (-s)"); + return -1; + } + if (!cmd.hasOption('V')) { + LOG.info("Need to set the aggregate vertices (-V)"); + return -1; + } + if (!cmd.hasOption('e')) { + LOG.info("Need to set the number of edges " + + "per vertex (-e)"); + return -1; } + if (!cmd.hasOption('b')) { + LOG.info("Need to set the number of message bytes (-b)"); + return -1; + } + if (!cmd.hasOption('n')) { + LOG.info("Need to set the number of messages per edge (-n)"); + return -1; + } + int workers = Integer.parseInt(cmd.getOptionValue('w')); + GiraphJob job = new GiraphJob(getConf(), getClass().getName()); + job.getConfiguration().setInt(GiraphJob.CHECKPOINT_FREQUENCY, 0); + job.setVertexClass(RandomMessageVertex.class); + job.setVertexInputFormatClass(PseudoRandomVertexInputFormat.class); + job.setWorkerContextClass(RandomMessageBenchmarkWorkerContext.class); + job.setWorkerConfiguration(workers, workers, 100.0f); + job.getConfiguration().setLong( + PseudoRandomVertexInputFormat.AGGREGATE_VERTICES, + Long.parseLong(cmd.getOptionValue('V'))); + job.getConfiguration().setLong( + PseudoRandomVertexInputFormat.EDGES_PER_VERTEX, + Long.parseLong(cmd.getOptionValue('e'))); + job.getConfiguration().setInt( + SUPERSTEP_COUNT, + Integer.parseInt(cmd.getOptionValue('s'))); + job.getConfiguration().setInt( + RandomMessageBenchmark.NUM_BYTES_PER_MESSAGE, + Integer.parseInt(cmd.getOptionValue('b'))); + job.getConfiguration().setInt( + RandomMessageBenchmark.NUM_MESSAGES_PER_EDGE, + Integer.parseInt(cmd.getOptionValue('n'))); - public static void main(String[] args) throws Exception { - System.exit(ToolRunner.run(new RandomMessageBenchmark(), args)); + boolean isVerbose = false; + if (cmd.hasOption('v')) { + isVerbose = true; + } + if (cmd.hasOption('s')) { + getConf().setInt(SUPERSTEP_COUNT, + Integer.parseInt(cmd.getOptionValue('s'))); } + if (cmd.hasOption('f')) { + job.getConfiguration().setInt(GiraphJob.MSG_NUM_FLUSH_THREADS, + Integer.parseInt(cmd.getOptionValue('f'))); + } + if (job.run(isVerbose)) { + return 0; + } else { + return -1; + } + } + + /** + * Execute the benchmark. + * + * @param args Typically, this is the command line arguments. + * @throws Exception Any exception thrown during computation. + */ + public static void main(String[] args) throws Exception { + System.exit(ToolRunner.run(new RandomMessageBenchmark(), args)); + } } diff --git a/src/main/java/org/apache/giraph/benchmark/package-info.java b/src/main/java/org/apache/giraph/benchmark/package-info.java new file mode 100644 index 000000000..66743fca3 --- /dev/null +++ b/src/main/java/org/apache/giraph/benchmark/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/** + * Package of benchmarks for performance testing and optimization + */ +package org.apache.giraph.benchmark; diff --git a/src/main/java/org/apache/giraph/bsp/ApplicationState.java b/src/main/java/org/apache/giraph/bsp/ApplicationState.java index b9ec28714..303ed066c 100644 --- a/src/main/java/org/apache/giraph/bsp/ApplicationState.java +++ b/src/main/java/org/apache/giraph/bsp/ApplicationState.java @@ -22,8 +22,12 @@ * State of the BSP application */ public enum ApplicationState { - UNKNOWN, ///< Shouldn't be seen, just an initial state - START_SUPERSTEP, ///< Start from a desired superstep - FAILED, ///< Unrecoverable - FINISHED ///< Successful completion + /** Shouldn't be seen, just an initial state */ + UNKNOWN, + /** Start from a desired superstep */ + START_SUPERSTEP, + /** Unrecoverable */ + FAILED, + /** Successful completion */ + FINISHED, } diff --git a/src/main/java/org/apache/giraph/bsp/BspInputFormat.java b/src/main/java/org/apache/giraph/bsp/BspInputFormat.java index 6e80ad9eb..2a93519e3 100644 --- a/src/main/java/org/apache/giraph/bsp/BspInputFormat.java +++ b/src/main/java/org/apache/giraph/bsp/BspInputFormat.java @@ -39,52 +39,55 @@ * separate. It is not meant to do any meaningful split of user-data. */ public class BspInputFormat extends InputFormat { - /** Logger */ - private static final Logger LOG = Logger.getLogger(BspInputFormat.class); + /** Class Logger */ + private static final Logger LOG = Logger.getLogger(BspInputFormat.class); - /** - * Get the correct number of mappers based on the configuration - * - * @param conf Configuration to determine the number of mappers - */ - public static int getMaxTasks(Configuration conf) { - int maxWorkers = conf.getInt(GiraphJob.MAX_WORKERS, 0); - boolean splitMasterWorker = - conf.getBoolean(GiraphJob.SPLIT_MASTER_WORKER, - GiraphJob.SPLIT_MASTER_WORKER_DEFAULT); - int maxTasks = maxWorkers; - if (splitMasterWorker) { - int zkServers = - conf.getInt(GiraphJob.ZOOKEEPER_SERVER_COUNT, - GiraphJob.ZOOKEEPER_SERVER_COUNT_DEFAULT); - maxTasks += zkServers; - } - if (LOG.isDebugEnabled()) { - LOG.debug("getMaxTasks: Max workers = " + maxWorkers + - ", split master/worker = " + splitMasterWorker + - ", total max tasks = " + maxTasks); - } - return maxTasks; + /** + * Get the correct number of mappers based on the configuration + * + * @param conf Configuration to determine the number of mappers + * @return Maximum number of tasks + */ + public static int getMaxTasks(Configuration conf) { + int maxWorkers = conf.getInt(GiraphJob.MAX_WORKERS, 0); + boolean splitMasterWorker = + conf.getBoolean(GiraphJob.SPLIT_MASTER_WORKER, + GiraphJob.SPLIT_MASTER_WORKER_DEFAULT); + int maxTasks = maxWorkers; + if (splitMasterWorker) { + int zkServers = + conf.getInt(GiraphJob.ZOOKEEPER_SERVER_COUNT, + GiraphJob.ZOOKEEPER_SERVER_COUNT_DEFAULT); + maxTasks += zkServers; } - - public List getSplits(JobContext context) - throws IOException, InterruptedException { - Configuration conf = context.getConfiguration(); - int maxTasks = getMaxTasks(conf); - if (maxTasks <= 0) { - throw new InterruptedException( - "getSplits: Cannot have maxTasks <= 0 - " + maxTasks); - } - List inputSplitList = new ArrayList(); - for (int i = 0; i < maxTasks; ++i) { - inputSplitList.add(new BspInputSplit()); - } - return inputSplitList; + if (LOG.isDebugEnabled()) { + LOG.debug("getMaxTasks: Max workers = " + maxWorkers + + ", split master/worker = " + splitMasterWorker + + ", total max tasks = " + maxTasks); } + return maxTasks; + } - public RecordReader - createRecordReader(InputSplit split, TaskAttemptContext context) - throws IOException, InterruptedException { - return new BspRecordReader(); + @Override + public List getSplits(JobContext context) + throws IOException, InterruptedException { + Configuration conf = context.getConfiguration(); + int maxTasks = getMaxTasks(conf); + if (maxTasks <= 0) { + throw new InterruptedException( + "getSplits: Cannot have maxTasks <= 0 - " + maxTasks); + } + List inputSplitList = new ArrayList(); + for (int i = 0; i < maxTasks; ++i) { + inputSplitList.add(new BspInputSplit()); } + return inputSplitList; + } + + @Override + public RecordReader + createRecordReader(InputSplit split, TaskAttemptContext context) + throws IOException, InterruptedException { + return new BspRecordReader(); + } } diff --git a/src/main/java/org/apache/giraph/bsp/BspInputSplit.java b/src/main/java/org/apache/giraph/bsp/BspInputSplit.java index 916090e5e..2258917d2 100644 --- a/src/main/java/org/apache/giraph/bsp/BspInputSplit.java +++ b/src/main/java/org/apache/giraph/bsp/BspInputSplit.java @@ -32,51 +32,70 @@ * directly. */ public class BspInputSplit extends InputSplit implements Writable { - /** Number of splits */ - private int numSplits = -1; - /** Split index */ - private int splitIndex = -1; + /** Number of splits */ + private int numSplits = -1; + /** Split index */ + private int splitIndex = -1; - public BspInputSplit() {} + /** + * Reflection constructor. + */ + public BspInputSplit() { } - public BspInputSplit(int splitIndex, int numSplits) { - this.splitIndex = splitIndex; - this.numSplits = numSplits; - } + /** + * Constructor used by {@link BspInputFormat}. + * + * @param splitIndex Index of this split. + * @param numSplits Total number of splits. + */ + public BspInputSplit(int splitIndex, int numSplits) { + this.splitIndex = splitIndex; + this.numSplits = numSplits; + } - @Override - public long getLength() throws IOException, InterruptedException { - return 0; - } + @Override + public long getLength() throws IOException, InterruptedException { + return 0; + } - @Override - public String[] getLocations() throws IOException, InterruptedException { - return new String[]{}; - } + @Override + public String[] getLocations() throws IOException, InterruptedException { + return new String[]{}; + } - @Override - public void readFields(DataInput in) throws IOException { - splitIndex = in.readInt(); - numSplits = in.readInt(); - } + @Override + public void readFields(DataInput in) throws IOException { + splitIndex = in.readInt(); + numSplits = in.readInt(); + } - @Override - public void write(DataOutput out) throws IOException { - out.writeInt(splitIndex); - out.writeInt(numSplits); - } + @Override + public void write(DataOutput out) throws IOException { + out.writeInt(splitIndex); + out.writeInt(numSplits); + } - public int getSplitIndex() { - return splitIndex; - } + /** + * Get the index of this split. + * + * @return Index of this split. + */ + public int getSplitIndex() { + return splitIndex; + } - public int getNumSplits() { - return numSplits; - } + /** + * Get the number of splits for this application. + * + * @return Total number of splits. + */ + public int getNumSplits() { + return numSplits; + } - @Override - public String toString() { - return "'" + getClass().getCanonicalName() + - ", index=" + getSplitIndex() + ", num=" + getNumSplits(); - } + @Override + public String toString() { + return "'" + getClass().getCanonicalName() + + ", index=" + getSplitIndex() + ", num=" + getNumSplits(); + } } diff --git a/src/main/java/org/apache/giraph/bsp/BspOutputFormat.java b/src/main/java/org/apache/giraph/bsp/BspOutputFormat.java index df0737346..9e43ca6be 100644 --- a/src/main/java/org/apache/giraph/bsp/BspOutputFormat.java +++ b/src/main/java/org/apache/giraph/bsp/BspOutputFormat.java @@ -34,38 +34,38 @@ * to be called as if a normal Hadoop job. */ public class BspOutputFormat extends OutputFormat { - /** Class logger */ - private static Logger LOG = Logger.getLogger(BspOutputFormat.class); + /** Class logger */ + private static Logger LOG = Logger.getLogger(BspOutputFormat.class); - @Override - public void checkOutputSpecs(JobContext context) - throws IOException, InterruptedException { - if (BspUtils.getVertexOutputFormatClass(context.getConfiguration()) - == null) { - LOG.warn("checkOutputSpecs: ImmutableOutputCommiter" + - " will not check anything"); - return; - } - BspUtils.createVertexOutputFormat(context.getConfiguration()). - checkOutputSpecs(context); + @Override + public void checkOutputSpecs(JobContext context) + throws IOException, InterruptedException { + if (BspUtils.getVertexOutputFormatClass(context.getConfiguration()) == + null) { + LOG.warn("checkOutputSpecs: ImmutableOutputCommiter" + + " will not check anything"); + return; } + BspUtils.createVertexOutputFormat(context.getConfiguration()). + checkOutputSpecs(context); + } - @Override - public OutputCommitter getOutputCommitter(TaskAttemptContext context) - throws IOException, InterruptedException { - if (BspUtils.getVertexOutputFormatClass(context.getConfiguration()) - == null) { - LOG.warn("getOutputCommitter: Returning " + - "ImmutableOutputCommiter (does nothing)."); - return new ImmutableOutputCommitter(); - } - return BspUtils.createVertexOutputFormat(context.getConfiguration()). - getOutputCommitter(context); + @Override + public OutputCommitter getOutputCommitter(TaskAttemptContext context) + throws IOException, InterruptedException { + if (BspUtils.getVertexOutputFormatClass(context.getConfiguration()) == + null) { + LOG.warn("getOutputCommitter: Returning " + + "ImmutableOutputCommiter (does nothing)."); + return new ImmutableOutputCommitter(); } + return BspUtils.createVertexOutputFormat(context.getConfiguration()). + getOutputCommitter(context); + } - @Override - public RecordWriter getRecordWriter(TaskAttemptContext context) - throws IOException, InterruptedException { - return new BspRecordWriter(); - } + @Override + public RecordWriter getRecordWriter(TaskAttemptContext context) + throws IOException, InterruptedException { + return new BspRecordWriter(); + } } diff --git a/src/main/java/org/apache/giraph/bsp/BspRecordReader.java b/src/main/java/org/apache/giraph/bsp/BspRecordReader.java index 7f8811c5d..5646018c2 100644 --- a/src/main/java/org/apache/giraph/bsp/BspRecordReader.java +++ b/src/main/java/org/apache/giraph/bsp/BspRecordReader.java @@ -29,40 +29,45 @@ * Only returns a single key-value pair so that the map() can run. */ class BspRecordReader extends RecordReader { + /** Singular key object */ + private static final Text ONLY_KEY = new Text("only key"); + /** Single value object */ + private static final Text ONLY_VALUE = new Text("only value"); - private static final Text ONLY_KEY = new Text("only key"); - private static final Text ONLY_VALUE = new Text("only value"); + /** Has the one record been seen? */ + private boolean seenRecord = false; - /** Has the one record been seen? */ - private boolean seenRecord = false; + @Override + public void close() throws IOException { + return; + } - @Override - public void close() throws IOException { - return; - } - - @Override - public float getProgress() throws IOException { - return (seenRecord ? 1f : 0f); - } + @Override + public float getProgress() throws IOException { + return seenRecord ? 1f : 0f; + } - @Override - public Text getCurrentKey() throws IOException, InterruptedException { - return ONLY_KEY; - } + @Override + public Text getCurrentKey() throws IOException, InterruptedException { + return ONLY_KEY; + } - @Override - public Text getCurrentValue() throws IOException, InterruptedException { - return ONLY_VALUE; - } + @Override + public Text getCurrentValue() throws IOException, InterruptedException { + return ONLY_VALUE; + } - @Override - public void initialize(InputSplit inputSplit, TaskAttemptContext context) - throws IOException, InterruptedException { - } + @Override + public void initialize(InputSplit inputSplit, TaskAttemptContext context) + throws IOException, InterruptedException { + } - @Override - public boolean nextKeyValue() throws IOException, InterruptedException { - return (seenRecord ? false : (seenRecord = true)); + @Override + public boolean nextKeyValue() throws IOException, InterruptedException { + if (!seenRecord) { + seenRecord = true; + return true; } + return false; + } } diff --git a/src/main/java/org/apache/giraph/bsp/BspRecordWriter.java b/src/main/java/org/apache/giraph/bsp/BspRecordWriter.java index 3838c1ae8..c6bde1e46 100644 --- a/src/main/java/org/apache/giraph/bsp/BspRecordWriter.java +++ b/src/main/java/org/apache/giraph/bsp/BspRecordWriter.java @@ -31,17 +31,17 @@ */ public class BspRecordWriter extends RecordWriter { - @Override - public void close(TaskAttemptContext context) - throws IOException, InterruptedException { - // Do nothing - } + @Override + public void close(TaskAttemptContext context) + throws IOException, InterruptedException { + // Do nothing + } - @Override - public void write(Text key, Text value) - throws IOException, InterruptedException { - throw new IOException("write: Cannot write with " + - getClass().getName() + - ". Should never be called"); - } + @Override + public void write(Text key, Text value) + throws IOException, InterruptedException { + throw new IOException("write: Cannot write with " + + getClass().getName() + + ". Should never be called"); + } } diff --git a/src/main/java/org/apache/giraph/bsp/CentralizedService.java b/src/main/java/org/apache/giraph/bsp/CentralizedService.java index a72f1425b..049bb2fe8 100644 --- a/src/main/java/org/apache/giraph/bsp/CentralizedService.java +++ b/src/main/java/org/apache/giraph/bsp/CentralizedService.java @@ -26,45 +26,48 @@ /** * Basic service interface shared by both {@link CentralizedServiceMaster} and * {@link CentralizedServiceWorker}. + * + * @param Vertex id + * @param Vertex value + * @param Edge value + * @param Message data */ @SuppressWarnings("rawtypes") public interface CentralizedService { - /** - * Setup (must be called prior to any other function) - */ - void setup(); + V extends Writable, E extends Writable, M extends Writable> { + /** + * Setup (must be called prior to any other function) + */ + void setup(); - /** - * Get the current global superstep of the application to work on. - * - * @return global superstep (begins at INPUT_SUPERSTEP) - */ - long getSuperstep(); + /** + * Get the current global superstep of the application to work on. + * + * @return global superstep (begins at INPUT_SUPERSTEP) + */ + long getSuperstep(); - /** - * Get the restarted superstep - * - * @return -1 if not manually restarted, otherwise the superstep id - */ - long getRestartedSuperstep(); + /** + * Get the restarted superstep + * + * @return -1 if not manually restarted, otherwise the superstep id + */ + long getRestartedSuperstep(); - /** - * Given a superstep, should it be checkpointed based on the - * checkpoint frequency? - * - * @param superstep superstep to check against frequency - * @return true if checkpoint frequency met or superstep is 1. - */ - boolean checkpointFrequencyMet(long superstep); + /** + * Given a superstep, should it be checkpointed based on the + * checkpoint frequency? + * + * @param superstep superstep to check against frequency + * @return true if checkpoint frequency met or superstep is 1. + */ + boolean checkpointFrequencyMet(long superstep); - /** - * Clean up the service (no calls may be issued after this) - * - * @throws IOException - * @throws InterruptedException - */ - void cleanup() throws IOException, InterruptedException; + /** + * Clean up the service (no calls may be issued after this) + * + * @throws IOException + * @throws InterruptedException + */ + void cleanup() throws IOException, InterruptedException; } diff --git a/src/main/java/org/apache/giraph/bsp/CentralizedServiceMaster.java b/src/main/java/org/apache/giraph/bsp/CentralizedServiceMaster.java index 9e44c1baa..3e034c1b5 100644 --- a/src/main/java/org/apache/giraph/bsp/CentralizedServiceMaster.java +++ b/src/main/java/org/apache/giraph/bsp/CentralizedServiceMaster.java @@ -22,70 +22,73 @@ import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableComparable; -import org.apache.hadoop.mapreduce.InputSplit; import org.apache.zookeeper.KeeperException; /** * At most, there will be one active master at a time, but many threads can * be trying to be the active master. + * + * @param Vertex id + * @param Vertex value + * @param Edge value + * @param Message data */ @SuppressWarnings("rawtypes") public interface CentralizedServiceMaster< - I extends WritableComparable, - V extends Writable, - E extends Writable, - M extends Writable> - extends CentralizedService { - /** - * Become the master. - * @return true if became the master, false if the application is done. - */ - boolean becomeMaster(); + I extends WritableComparable, V extends Writable, E extends Writable, + M extends Writable> extends CentralizedService { + /** + * Become the master. + * @return true if became the master, false if the application is done. + */ + boolean becomeMaster(); - /** - * Create the {@link InputSplit} objects from the index range based on the - * user-defined VertexInputFormat. The {@link InputSplit} objects will - * processed by the workers later on during the INPUT_SUPERSTEP. - * - * @return Number of partitions. Returns -1 on failure to create - * valid input splits. - */ - int createInputSplits(); + /** + * Create the {@link InputSplit} objects from the index range based on the + * user-defined VertexInputFormat. The {@link InputSplit} objects will + * processed by the workers later on during the INPUT_SUPERSTEP. + * + * @return Number of partitions. Returns -1 on failure to create + * valid input splits. + */ + int createInputSplits(); - /** - * Master coordinates the superstep - * - * @return State of the application as a result of this superstep - * @throws InterruptedException - * @throws KeeperException - */ - SuperstepState coordinateSuperstep() - throws KeeperException, InterruptedException; + /** + * Master coordinates the superstep + * + * @return State of the application as a result of this superstep + * @throws InterruptedException + * @throws KeeperException + */ + SuperstepState coordinateSuperstep() + throws KeeperException, InterruptedException; - /** - * Master can decide to restart from the last good checkpoint if a - * worker fails during a superstep. - * - * @param checkpoint Checkpoint to restart from - */ - void restartFromCheckpoint(long checkpoint); + /** + * Master can decide to restart from the last good checkpoint if a + * worker fails during a superstep. + * + * @param checkpoint Checkpoint to restart from + */ + void restartFromCheckpoint(long checkpoint); - /** - * Get the last known good checkpoint - * @throws IOException - */ - long getLastGoodCheckpoint() throws IOException; + /** + * Get the last known good checkpoint + * + * @return Last good superstep number + * @throws IOException + */ + long getLastGoodCheckpoint() throws IOException; - /** - * If the master decides that this job doesn't have the resources to - * continue, it can fail the job. It can also designate what to do next. - * Typically this is mainly informative. - * - * @param state - * @param applicationAttempt attempt to start on - * @param desiredSuperstep Superstep to restart from (if applicable) - */ - void setJobState(ApplicationState state, - long applicationAttempt, - long desiredSuperstep); + /** + * If the master decides that this job doesn't have the resources to + * continue, it can fail the job. It can also designate what to do next. + * Typically this is mainly informative. + * + * @param state State of the application. + * @param applicationAttempt Attempt to start on + * @param desiredSuperstep Superstep to restart from (if applicable) + */ + void setJobState(ApplicationState state, + long applicationAttempt, + long desiredSuperstep); } diff --git a/src/main/java/org/apache/giraph/bsp/CentralizedServiceWorker.java b/src/main/java/org/apache/giraph/bsp/CentralizedServiceWorker.java index 6ca866362..fd4bcf6ed 100644 --- a/src/main/java/org/apache/giraph/bsp/CentralizedServiceWorker.java +++ b/src/main/java/org/apache/giraph/bsp/CentralizedServiceWorker.java @@ -31,141 +31,143 @@ import org.apache.giraph.graph.partition.Partition; import org.apache.giraph.graph.partition.PartitionOwner; import org.apache.giraph.graph.partition.PartitionStats; -import org.apache.giraph.graph.partition.WorkerGraphPartitioner; import org.apache.giraph.graph.WorkerInfo; import org.apache.giraph.graph.WorkerContext; /** * All workers should have access to this centralized service to * execute the following methods. + * + * @param Vertex id + * @param Vertex value + * @param Edge value + * @param Message data */ @SuppressWarnings("rawtypes") -public interface CentralizedServiceWorker< - I extends WritableComparable, - V extends Writable, - E extends Writable, - M extends Writable> - extends CentralizedService, AggregatorUsage { - /** - * Get the worker information - * - * @return Worker information - */ - WorkerInfo getWorkerInfo(); - - /** - * - * @return worker's WorkerContext - */ - WorkerContext getWorkerContext(); - - /** - * Get a map of the partition id to the partition for this worker. - * The partitions contain the vertices for - * this worker and can be used to run compute() for the vertices or do - * checkpointing. - * - * @return List of partitions that this worker owns. - */ - Map> getPartitionMap(); - - /** - * Get a collection of all the partition owners. - * - * @return Collection of all the partition owners. - */ - Collection getPartitionOwners(); - - /** - * Both the vertices and the messages need to be checkpointed in order - * for them to be used. This is done after all messages have been - * delivered, but prior to a superstep starting. - */ - void storeCheckpoint() throws IOException; - - /** - * Load the vertices, edges, messages from the beginning of a superstep. - * Will load the vertex partitions as designated by the master and set the - * appropriate superstep. - * - * @param superstep which checkpoint to use - * @throws IOException - */ - void loadCheckpoint(long superstep) throws IOException; - - /** - * Take all steps prior to actually beginning the computation of a - * superstep. - * - * @return Collection of all the partition owners from the master for this - * superstep. - */ - Collection startSuperstep(); - - /** - * Worker is done with its portion of the superstep. Report the - * worker level statistics after the computation. - * - * @param partitionStatsList All the partition stats for this worker - * @return true if this is the last superstep, false otherwise - */ - boolean finishSuperstep(List partitionStatsList); - /** - * Get the partition that a vertex index would belong to - * - * @param vertexIndex Index of the vertex that is used to find the correct - * partition. - * @return Correct partition if exists on this worker, null otherwise. - */ - public Partition getPartition(I vertexIndex); - - /** - * Every client will need to get a partition owner from a vertex id so that - * they know which worker to sent the request to. - * - * @param vertexIndex Vertex index to look for - * @return PartitionOnwer that should contain this vertex if it exists - */ - PartitionOwner getVertexPartitionOwner(I vertexIndex); - - /** - * Look up a vertex on a worker given its vertex index. - * - * @param vertexIndex Vertex index to look for - * @return Vertex if it exists on this worker. - */ - BasicVertex getVertex(I vertexIndex); - - /** - * If desired by the user, vertex partitions are redistributed among - * workers according to the chosen {@link WorkerGraphPartitioner}. - * - * @param masterSetPartitionOwners Partition owner info passed from the - * master. - */ - void exchangeVertexPartitions( - Collection masterSetPartitionOwners); - - /** - * Assign messages to a vertex (bypasses package-private access to - * setMessages() for internal classes). - * - * @param vertex Vertex (owned by worker) - * @param messageIterator Messages to assign to the vertex - */ - void assignMessagesToVertex(BasicVertex vertex, - Iterable messageIterator); - - /** - * Get the GraphMapper that this service is using. Vertices need to know - * this. - * - * @return BspMapper - */ - GraphMapper getGraphMapper(); - - /** - * Operations that will be called if there is a failure by a worker. - */ - void failureCleanup(); +public interface CentralizedServiceWorker + extends CentralizedService, AggregatorUsage { + /** + * Get the worker information + * + * @return Worker information + */ + WorkerInfo getWorkerInfo(); + + /** + * + * @return worker's WorkerContext + */ + WorkerContext getWorkerContext(); + + /** + * Get a map of the partition id to the partition for this worker. + * The partitions contain the vertices for + * this worker and can be used to run compute() for the vertices or do + * checkpointing. + * + * @return List of partitions that this worker owns. + */ + Map> getPartitionMap(); + + /** + * Get a collection of all the partition owners. + * + * @return Collection of all the partition owners. + */ + Collection getPartitionOwners(); + + /** + * Both the vertices and the messages need to be checkpointed in order + * for them to be used. This is done after all messages have been + * delivered, but prior to a superstep starting. + */ + void storeCheckpoint() throws IOException; + + /** + * Load the vertices, edges, messages from the beginning of a superstep. + * Will load the vertex partitions as designated by the master and set the + * appropriate superstep. + * + * @param superstep which checkpoint to use + * @throws IOException + */ + void loadCheckpoint(long superstep) throws IOException; + + /** + * Take all steps prior to actually beginning the computation of a + * superstep. + * + * @return Collection of all the partition owners from the master for this + * superstep. + */ + Collection startSuperstep(); + + /** + * Worker is done with its portion of the superstep. Report the + * worker level statistics after the computation. + * + * @param partitionStatsList All the partition stats for this worker + * @return true if this is the last superstep, false otherwise + */ + boolean finishSuperstep(List partitionStatsList); + + /** + * Get the partition that a vertex index would belong to + * + * @param vertexIndex Index of the vertex that is used to find the correct + * partition. + * @return Correct partition if exists on this worker, null otherwise. + */ + Partition getPartition(I vertexIndex); + + /** + * Every client will need to get a partition owner from a vertex id so that + * they know which worker to sent the request to. + * + * @param vertexIndex Vertex index to look for + * @return PartitionOnwer that should contain this vertex if it exists + */ + PartitionOwner getVertexPartitionOwner(I vertexIndex); + + /** + * Look up a vertex on a worker given its vertex index. + * + * @param vertexIndex Vertex index to look for + * @return Vertex if it exists on this worker. + */ + BasicVertex getVertex(I vertexIndex); + + /** + * If desired by the user, vertex partitions are redistributed among + * workers according to the chosen {@link WorkerGraphPartitioner}. + * + * @param masterSetPartitionOwners Partition owner info passed from the + * master. + */ + void exchangeVertexPartitions( + Collection masterSetPartitionOwners); + + /** + * Assign messages to a vertex (bypasses package-private access to + * setMessages() for internal classes). + * + * @param vertex Vertex (owned by worker) + * @param messageIterator Messages to assign to the vertex + */ + void assignMessagesToVertex(BasicVertex vertex, + Iterable messageIterator); + + /** + * Get the GraphMapper that this service is using. Vertices need to know + * this. + * + * @return BspMapper + */ + GraphMapper getGraphMapper(); + + /** + * Operations that will be called if there is a failure by a worker. + */ + void failureCleanup(); } diff --git a/src/main/java/org/apache/giraph/bsp/ImmutableOutputCommitter.java b/src/main/java/org/apache/giraph/bsp/ImmutableOutputCommitter.java index 5a85bbd8b..5c8a2ccab 100644 --- a/src/main/java/org/apache/giraph/bsp/ImmutableOutputCommitter.java +++ b/src/main/java/org/apache/giraph/bsp/ImmutableOutputCommitter.java @@ -30,35 +30,35 @@ * FileOutputCommitter. */ public class ImmutableOutputCommitter extends OutputCommitter { - @Override - public void abortTask(TaskAttemptContext context) throws IOException { - } + @Override + public void abortTask(TaskAttemptContext context) throws IOException { + } - @Override - public void commitTask(TaskAttemptContext context) throws IOException { - } + @Override + public void commitTask(TaskAttemptContext context) throws IOException { + } - @Override - public boolean needsTaskCommit(TaskAttemptContext context) - throws IOException { - return false; - } + @Override + public boolean needsTaskCommit(TaskAttemptContext context) + throws IOException { + return false; + } - @Override - public void setupJob(JobContext context) throws IOException { - } + @Override + public void setupJob(JobContext context) throws IOException { + } - @Override - public void setupTask(TaskAttemptContext context) throws IOException { - } + @Override + public void setupTask(TaskAttemptContext context) throws IOException { + } - /*if[HADOOP_NON_SECURE] + /*if[HADOOP_NON_SECURE] @Override public void cleanupJob(JobContext jobContext) throws IOException { } else[HADOOP_NON_SECURE]*/ - @Override - /*end[HADOOP_NON_SECURE]*/ - public void commitJob(JobContext jobContext) throws IOException { - } + @Override + /*end[HADOOP_NON_SECURE]*/ + public void commitJob(JobContext jobContext) throws IOException { + } } diff --git a/src/main/java/org/apache/giraph/bsp/SuperstepState.java b/src/main/java/org/apache/giraph/bsp/SuperstepState.java index d61f1af6d..c384fbfe8 100644 --- a/src/main/java/org/apache/giraph/bsp/SuperstepState.java +++ b/src/main/java/org/apache/giraph/bsp/SuperstepState.java @@ -22,8 +22,12 @@ * State of a coordinated superstep */ public enum SuperstepState { - INITIAL, ///< Nothing happened yet - WORKER_FAILURE, ///< A worker died during this superstep - THIS_SUPERSTEP_DONE, ///< This superstep completed correctly - ALL_SUPERSTEPS_DONE, ///< All supersteps are complete + /** Nothing happened yet */ + INITIAL, + /** A worker died during this superstep */ + WORKER_FAILURE, + /** This superstep completed correctly */ + THIS_SUPERSTEP_DONE, + /** All supersteps are complete */ + ALL_SUPERSTEPS_DONE, } diff --git a/src/main/java/org/apache/giraph/bsp/package-info.java b/src/main/java/org/apache/giraph/bsp/package-info.java new file mode 100644 index 000000000..b5e7dc323 --- /dev/null +++ b/src/main/java/org/apache/giraph/bsp/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/** + * Package of generic bulk synchronous processing objects. + */ +package org.apache.giraph.bsp; diff --git a/src/main/java/org/apache/giraph/comm/ArrayListWritable.java b/src/main/java/org/apache/giraph/comm/ArrayListWritable.java index 034f39fe3..4dd36e933 100644 --- a/src/main/java/org/apache/giraph/comm/ArrayListWritable.java +++ b/src/main/java/org/apache/giraph/comm/ArrayListWritable.java @@ -30,84 +30,96 @@ /** * A Writable for ListArray containing instances of a class. + * + * @param Message data */ public abstract class ArrayListWritable extends ArrayList - implements Writable, Configurable { - /** Used for instantiation */ - private Class refClass = null; - /** Defining a layout version for a serializable class. */ - private static final long serialVersionUID = 1L; - /** Configuration */ - private Configuration conf; + implements Writable, Configurable { + /** Defining a layout version for a serializable class. */ + private static final long serialVersionUID = 1L; + /** Used for instantiation */ + private Class refClass = null; - /** - * Using the default constructor requires that the user implement - * setClass(), guaranteed to be invoked prior to instantiation in - * readFields() - */ - public ArrayListWritable() { - } + /** Configuration */ + private Configuration conf; - public ArrayListWritable(ArrayListWritable arrayListWritable) { - super(arrayListWritable); - } + /** + * Using the default constructor requires that the user implement + * setClass(), guaranteed to be invoked prior to instantiation in + * readFields() + */ + public ArrayListWritable() { + } - /** - * This constructor allows setting the refClass during construction. - * - * @param refClass internal type class - */ - public ArrayListWritable(Class refClass) { - super(); - this.refClass = refClass; - } + /** + * Constructor with another {@link ArrayListWritable}. + * + * @param arrayListWritable Array list to be used internally. + */ + public ArrayListWritable(ArrayListWritable arrayListWritable) { + super(arrayListWritable); + } - /** - * This is a one-time operation to set the class type - * - * @param refClass internal type class - */ - public void setClass(Class refClass) { - if (this.refClass != null) { - throw new RuntimeException( - "setClass: refClass is already set to " + - this.refClass.getName()); - } - this.refClass = refClass; + /** + * This constructor allows setting the refClass during construction. + * + * @param refClass internal type class + */ + public ArrayListWritable(Class refClass) { + super(); + this.refClass = refClass; + } + + /** + * This is a one-time operation to set the class type + * + * @param refClass internal type class + */ + public void setClass(Class refClass) { + if (this.refClass != null) { + throw new RuntimeException( + "setClass: refClass is already set to " + + this.refClass.getName()); } + this.refClass = refClass; + } - /** - * Subclasses must set the class type appropriately and can use - * setClass(Class refClass) to do it. - */ - public abstract void setClass(); + /** + * Subclasses must set the class type appropriately and can use + * setClass(Class refClass) to do it. + */ + public abstract void setClass(); - public void readFields(DataInput in) throws IOException { - if (this.refClass == null) { - setClass(); - } - int numValues = in.readInt(); // read number of values - ensureCapacity(numValues); - for (int i = 0; i < numValues; i++) { - M value = ReflectionUtils.newInstance(refClass, conf); - value.readFields(in); // read a value - add(value); // store it in values - } + @Override + public void readFields(DataInput in) throws IOException { + if (this.refClass == null) { + setClass(); } - - public void write(DataOutput out) throws IOException { - int numValues = size(); - out.writeInt(numValues); // write number of values - for (int i = 0; i < numValues; i++) { - get(i).write(out); - } + int numValues = in.readInt(); // read number of values + ensureCapacity(numValues); + for (int i = 0; i < numValues; i++) { + M value = ReflectionUtils.newInstance(refClass, conf); + value.readFields(in); // read a value + add(value); // store it in values } + } - public final Configuration getConf() { - return conf; + @Override + public void write(DataOutput out) throws IOException { + int numValues = size(); + out.writeInt(numValues); // write number of values + for (int i = 0; i < numValues; i++) { + get(i).write(out); } + } - public final void setConf(Configuration conf) { - this.conf = conf; - } + @Override + public final Configuration getConf() { + return conf; + } + + @Override + public final void setConf(Configuration conf) { + this.conf = conf; + } } diff --git a/src/main/java/org/apache/giraph/comm/BasicRPCCommunications.java b/src/main/java/org/apache/giraph/comm/BasicRPCCommunications.java index 2f46d9d8b..ad1c695e6 100644 --- a/src/main/java/org/apache/giraph/comm/BasicRPCCommunications.java +++ b/src/main/java/org/apache/giraph/comm/BasicRPCCommunications.java @@ -39,7 +39,6 @@ import java.net.BindException; import java.net.InetAddress; import java.net.InetSocketAddress; -import java.net.UnknownHostException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -66,555 +65,623 @@ import org.apache.hadoop.ipc.ProtocolSignature; end[HADOOP_FACEBOOK]*/ +/** + * Basic RPC communications object that implements the lower level operations + * for RPC communication. + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data + * @param Job token + */ @SuppressWarnings("rawtypes") -public abstract class BasicRPCCommunications< - I extends WritableComparable, - V extends Writable, - E extends Writable, - M extends Writable, J> - implements CommunicationsInterface, - ServerInterface { - /** Class logger */ - private static final Logger LOG = - Logger.getLogger(BasicRPCCommunications.class); - /** Indicates whether in superstep preparation */ - private boolean inPrepareSuperstep = false; - /** Local hostname */ - private final String localHostname; - /** Name of RPC server, == myAddress.toString() */ - private final String myName; - /** RPC server */ - private Server server; - /** Centralized service, needed to get vertex ranges */ - private final CentralizedServiceWorker service; - /** Hadoop configuration */ - protected final Configuration conf; - /** Combiner instance, can be null */ - private final VertexCombiner combiner; - /** Address of RPC server */ - private InetSocketAddress myAddress; - /** Messages sent during the last superstep */ - private long totalMsgsSentInSuperstep = 0; - /** Maximum messages sent per putVertexIdMessagesList RPC */ - private final int maxMessagesPerFlushPut; +public abstract class BasicRPCCommunications + implements CommunicationsInterface, + ServerInterface { + /** Class logger */ + private static final Logger LOG = + Logger.getLogger(BasicRPCCommunications.class); + /** Maximum number of vertices sent in a single RPC */ + private static final int MAX_VERTICES_PER_RPC = 1024; + /** Hadoop configuration */ + protected final Configuration conf; + /** Indicates whether in superstep preparation */ + private boolean inPrepareSuperstep = false; + /** Local hostname */ + private final String localHostname; + /** Name of RPC server, == myAddress.toString() */ + private final String myName; + /** RPC server */ + private Server server; + /** Centralized service, needed to get vertex ranges */ + private final CentralizedServiceWorker service; + /** Combiner instance, can be null */ + private final VertexCombiner combiner; + /** Address of RPC server */ + private InetSocketAddress myAddress; + /** Messages sent during the last superstep */ + private long totalMsgsSentInSuperstep = 0; + /** Maximum messages sent per putVertexIdMessagesList RPC */ + private final int maxMessagesPerFlushPut; + /** + * Map of the peer connections, mapping from remote socket address to client + * meta data + */ + private final Map peerConnections = + new HashMap(); + /** + * Cached map of partition ids to remote socket address. Needs to be + * synchronized. + */ + private final Map partitionIndexAddressMap = + new HashMap(); + /** + * Thread pool for message flush threads + */ + private final ExecutorService executor; + /** + * Map of outbound messages, mapping from remote server to + * destination vertex index to list of messages + * (Synchronized between peer threads and main thread for each internal + * map) + */ + private final Map>> outMessages = + new HashMap>>(); + /** + * Map of incoming messages, mapping from vertex index to list of messages. + * Only accessed by the main thread (no need to synchronize). + */ + private final Map> inMessages = new HashMap>(); + /** + * Map of inbound messages, mapping from vertex index to list of messages. + * Transferred to inMessages at beginning of a superstep. This + * intermediary step exists so that the combiner will run not only at the + * client, but also at the server. Also, allows the sending of large + * message lists during the superstep computation. (Synchronized) + */ + private final Map> transientInMessages = + new HashMap>(); + /** + * Map of partition ids to incoming vertices from other workers. + * (Synchronized) + */ + private final Map>> + inPartitionVertexMap = new HashMap>>(); + + /** + * Map from vertex index to all vertex mutations + */ + private final Map> inVertexMutationsMap = + new HashMap>(); + + /** Maximum size of cached message list, before sending it out */ + private final int maxSize; + /** Cached job id */ + private final String jobId; + /** Cached job token */ + private final J jobToken; + + + /** + * PeerConnection contains RPC client and accumulated messages + * for a specific peer. + */ + private class PeerConnection { /** - * Map of the peer connections, mapping from remote socket address to client - * meta data + * Map of outbound messages going to a particular remote server, + * mapping from the destination vertex to a list of messages. + * (Synchronized with itself). */ - private final Map peerConnections = - new HashMap(); + private final Map> outMessagesPerPeer; /** - * Cached map of partition ids to remote socket address. Needs to be - * synchronized. + * Client interface: RPC proxy for remote server, this class for local */ - private final Map partitionIndexAddressMap = - new HashMap(); - /** - * Thread pool for message flush threads - */ - private final ExecutorService executor; - /** - * Map of outbound messages, mapping from remote server to - * destination vertex index to list of messages - * (Synchronized between peer threads and main thread for each internal - * map) - */ - private final Map>> outMessages = - new HashMap>>(); - /** - * Map of incoming messages, mapping from vertex index to list of messages. - * Only accessed by the main thread (no need to synchronize). - */ - private final Map> inMessages = new HashMap>(); - /** - * Map of inbound messages, mapping from vertex index to list of messages. - * Transferred to inMessages at beginning of a superstep. This - * intermediary step exists so that the combiner will run not only at the - * client, but also at the server. Also, allows the sending of large - * message lists during the superstep computation. (Synchronized) - */ - private final Map> transientInMessages = - new HashMap>(); - /** - * Map of partition ids to incoming vertices from other workers. - * (Synchronized) - */ - private final Map>> - inPartitionVertexMap = - new HashMap>>(); - - /** - * Map from vertex index to all vertex mutations - */ - private final Map> - inVertexMutationsMap = - new HashMap>(); - - /** Maximum size of cached message list, before sending it out */ - private final int maxSize; - /** Cached job id */ - private final String jobId; - /** Cached job token */ - private final J jobToken; - /** Maximum number of vertices sent in a single RPC */ - private static final int MAX_VERTICES_PER_RPC = 1024; + private final CommunicationsInterface peer; + /** Boolean, set to false when local client (self), true otherwise */ + private final boolean isProxy; /** - * PeerConnection contains RPC client and accumulated messages - * for a specific peer. + * Constructor + * @param idMessageMap Map of vertex id to message list + * @param peerConnection Peer connection + * @param isProxy Is this a proxy (true) or local (false)? */ - private class PeerConnection { - /** - * Map of outbound messages going to a particular remote server, - * mapping from the destination vertex to a list of messages. - * (Synchronized with itself). - */ - private final Map> outMessagesPerPeer; - /** - * Client interface: RPC proxy for remote server, this class for local - */ - private final CommunicationsInterface peer; - /** Boolean, set to false when local client (self), true otherwise */ - private final boolean isProxy; - - public PeerConnection(Map> m, - CommunicationsInterface i, - boolean isProxy) { - - this.outMessagesPerPeer = m; - this.peer = i; - this.isProxy = isProxy; - } + public PeerConnection(Map> idMessageMap, + CommunicationsInterface peerConnection, + boolean isProxy) { - public void close() { - if (LOG.isDebugEnabled()) { - LOG.debug("close: Done"); - } - } - - public CommunicationsInterface getRPCProxy() { - return peer; - } - - @Override - public String toString() { - return peer.getName() + ", proxy=" + isProxy; - } + this.outMessagesPerPeer = idMessageMap; + this.peer = peerConnection; + this.isProxy = isProxy; } - private class PeerFlushExecutor implements Runnable { - private final PeerConnection peerConnection; - private final Mapper.Context context; - // Report on the status of this flusher if this interval was exceeded - private static final int REPORTING_INTERVAL_MIN_MILLIS = 60000; - - PeerFlushExecutor(PeerConnection peerConnection, - Mapper.Context context) { - this.peerConnection = peerConnection; - this.context = context; - } - - @Override - public void run() { - CommunicationsInterface proxy - = peerConnection.getRPCProxy(); - long startMillis = System.currentTimeMillis(); - long lastReportedMillis = startMillis; - try { - int verticesDone = 0; - synchronized(peerConnection.outMessagesPerPeer) { - final int vertices = - peerConnection.outMessagesPerPeer.size(); - // 1. Check for null messages and combine if possible - // 2. Send vertex ids and messages in bulk to the - // destination servers. - for (Entry> entry : - peerConnection.outMessagesPerPeer.entrySet()) { - for (M msg : entry.getValue()) { - if (msg == null) { - throw new IllegalArgumentException( - "run: Cannot put null message on " + - "vertex id " + entry.getKey()); - } - } - if (combiner != null && entry.getValue().size() > 1) { - Iterable messages = combiner.combine( - entry.getKey(), entry.getValue()); - if (messages == null) { - throw new IllegalStateException( - "run: Combiner cannot return null"); - } - if (Iterables.size(entry.getValue()) < - Iterables.size(messages)) { - throw new IllegalStateException( - "run: The number of combined " + - "messages is required to be <= to " + - "number of messages to be combined"); - } - entry.getValue().clear(); - for (M msg: messages) { - entry.getValue().add(msg); - } - } - if (entry.getValue().isEmpty()) { - throw new IllegalStateException( - "run: Impossible for no messages in " + - entry.getKey()); - } - } - while (!peerConnection.outMessagesPerPeer.isEmpty()) { - int bulkedMessages = 0; - Iterator>> vertexIdMessagesListIt = - peerConnection.outMessagesPerPeer.entrySet(). - iterator(); - VertexIdMessagesList vertexIdMessagesList = - new VertexIdMessagesList(); - while (vertexIdMessagesListIt.hasNext()) { - Entry> entry = - vertexIdMessagesListIt.next(); - // Add this entry if the list is empty or we - // haven't reached the maximum number of messages - if (vertexIdMessagesList.isEmpty() || - ((bulkedMessages + entry.getValue().size()) - < maxMessagesPerFlushPut)) { - vertexIdMessagesList.add( - new VertexIdMessages( - entry.getKey(), entry.getValue())); - bulkedMessages += entry.getValue().size(); - } - } - - // Clean up references to the vertex id and messages - for (VertexIdMessagesvertexIdMessages : - vertexIdMessagesList) { - peerConnection.outMessagesPerPeer.remove( - vertexIdMessages.getVertexId()); - } - - proxy.putVertexIdMessagesList(vertexIdMessagesList); - context.progress(); - - verticesDone += vertexIdMessagesList.size(); - long curMillis = System.currentTimeMillis(); - if ((lastReportedMillis + - REPORTING_INTERVAL_MIN_MILLIS) < curMillis) { - lastReportedMillis = curMillis; - if (LOG.isInfoEnabled()) { - float percentDone = - (100f * verticesDone) / - vertices; - float minutesUsed = - (curMillis - startMillis) / 1000f / 60f; - float minutesRemaining = - (minutesUsed * 100f / percentDone) - - minutesUsed; - LOG.info("run: " + peerConnection + ", " + - verticesDone + " out of " + - vertices + - " done in " + minutesUsed + - " minutes, " + - percentDone + "% done, ETA " + - minutesRemaining + - " minutes remaining, " + - MemoryUtils.getRuntimeMemoryStats()); - } - } - } - } - - if (LOG.isDebugEnabled()) { - LOG.debug("run: " + proxy.getName() + - ": all messages flushed"); - } - } catch (IOException e) { - LOG.error(e); - if (peerConnection.isProxy) { - RPC.stopProxy(peerConnection.peer); - } - throw new RuntimeException(e); - } - } + /** + * Nothing to do here to cleanup, just notify. + */ + public void close() { + if (LOG.isDebugEnabled()) { + LOG.debug("close: Done"); + } } /** - * LargeMessageFlushExecutor flushes all outgoing messages destined to some vertices. - * This is executed when the number of messages destined to certain vertex - * exceeds maxSize. + * Get the RPC proxy of this connection. + * + * @return RPC proxy of this connection. */ - private class LargeMessageFlushExecutor implements Runnable { - private final I destVertex; - private final MsgList outMessageList; - private PeerConnection peerConnection; - - LargeMessageFlushExecutor(PeerConnection peerConnection, I destVertex) { - this.peerConnection = peerConnection; - synchronized(peerConnection.outMessagesPerPeer) { - this.destVertex = destVertex; - outMessageList = - peerConnection.outMessagesPerPeer.get(destVertex); - peerConnection.outMessagesPerPeer.remove(destVertex); - } - } - - @Override - public void run() { - try { - CommunicationsInterface proxy = - peerConnection.getRPCProxy(); - - if (combiner != null) { - Iterable messages = combiner.combine(destVertex, - outMessageList); - if (messages == null) { - throw new IllegalStateException( - "run: Combiner cannot return null"); - } - if (Iterables.size(outMessageList) < - Iterables.size(messages)) { - throw new IllegalStateException( - "run: The number of combined messages is " + - "required to be <= to the number of " + - "messages to be combined"); - } - for (M msg: messages) { - proxy.putMsg(destVertex, msg); - } - } else { - proxy.putMsgList(destVertex, outMessageList); - } - } catch (IOException e) { - LOG.error(e); - if (peerConnection.isProxy) { - RPC.stopProxy(peerConnection.peer); - } - throw new RuntimeException("run: Got IOException", e); - } finally { - outMessageList.clear(); - } - } + public CommunicationsInterface getRPCProxy() { + return peer; } - private void submitLargeMessageSend(InetSocketAddress addr, I destVertex) { - PeerConnection pc = peerConnections.get(addr); - executor.execute(new LargeMessageFlushExecutor(pc, destVertex)); + @Override + public String toString() { + return peer.getName() + ", proxy=" + isProxy; } - - protected abstract J createJobToken() throws IOException; - - protected abstract Server getRPCServer( - InetSocketAddress addr, - int numHandlers, String jobId, J jobToken) throws IOException; + } + + /** + * Runnable to flush messages to a given connection. + */ + private class PeerFlushExecutor implements Runnable { + /** Report on the status of this flusher if this interval was exceeded */ + private static final int REPORTING_INTERVAL_MIN_MILLIS = 60000; + /** Connection to send the messages to. */ + private final PeerConnection peerConnection; + /** Saved context. */ + private final Mapper.Context context; /** - * Only constructor. + * Constructor. * - * @param context Context for getting configuration - * @param service Service worker to get the vertex ranges - * @throws IOException - * @throws UnknownHostException - * @throws InterruptedException + * @param peerConnection Connection to send the messsages to. + * @param context Context of the mapper. */ - public BasicRPCCommunications(Mapper.Context context, - CentralizedServiceWorker service) - throws IOException, UnknownHostException, InterruptedException { - this.service = service; - this.conf = context.getConfiguration(); - this.maxSize = conf.getInt(GiraphJob.MSG_SIZE, - GiraphJob.MSG_SIZE_DEFAULT); - this.maxMessagesPerFlushPut = - conf.getInt(GiraphJob.MAX_MESSAGES_PER_FLUSH_PUT, - GiraphJob.DEFAULT_MAX_MESSAGES_PER_FLUSH_PUT); - if (BspUtils.getVertexCombinerClass(conf) == null) { - this.combiner = null; - } else { - this.combiner = BspUtils.createVertexCombiner(conf); - } - - this.localHostname = InetAddress.getLocalHost().getHostName(); - int taskId = conf.getInt("mapred.task.partition", -1); - int numTasks = conf.getInt("mapred.map.tasks", 1); + PeerFlushExecutor(PeerConnection peerConnection, + Mapper.Context context) { + this.peerConnection = peerConnection; + this.context = context; + } + @Override + public void run() { + CommunicationsInterface proxy = peerConnection.getRPCProxy(); + long startMillis = System.currentTimeMillis(); + long lastReportedMillis = startMillis; + try { + int verticesDone = 0; + synchronized (peerConnection.outMessagesPerPeer) { + final int vertices = + peerConnection.outMessagesPerPeer.size(); + // 1. Check for null messages and combine if possible + // 2. Send vertex ids and messages in bulk to the + // destination servers. + for (Entry> entry : + peerConnection.outMessagesPerPeer.entrySet()) { + for (M msg : entry.getValue()) { + if (msg == null) { + throw new IllegalArgumentException( + "run: Cannot put null message on " + + "vertex id " + entry.getKey()); + } + } + if (combiner != null && entry.getValue().size() > 1) { + Iterable messages = combiner.combine( + entry.getKey(), entry.getValue()); + if (messages == null) { + throw new IllegalStateException( + "run: Combiner cannot return null"); + } + if (Iterables.size(entry.getValue()) < + Iterables.size(messages)) { + throw new IllegalStateException( + "run: The number of combined " + + "messages is required to be <= to " + + "number of messages to be combined"); + } + entry.getValue().clear(); + for (M msg: messages) { + entry.getValue().add(msg); + } + } + if (entry.getValue().isEmpty()) { + throw new IllegalStateException( + "run: Impossible for no messages in " + + entry.getKey()); + } + } + while (!peerConnection.outMessagesPerPeer.isEmpty()) { + int bulkedMessages = 0; + Iterator>> vertexIdMessagesListIt = + peerConnection.outMessagesPerPeer.entrySet(). + iterator(); + VertexIdMessagesList vertexIdMessagesList = + new VertexIdMessagesList(); + while (vertexIdMessagesListIt.hasNext()) { + Entry> entry = + vertexIdMessagesListIt.next(); + // Add this entry if the list is empty or we + // haven't reached the maximum number of messages + if (vertexIdMessagesList.isEmpty() || + ((bulkedMessages + entry.getValue().size()) < + maxMessagesPerFlushPut)) { + vertexIdMessagesList.add( + new VertexIdMessages( + entry.getKey(), entry.getValue())); + bulkedMessages += entry.getValue().size(); + } + } + // Clean up references to the vertex id and messages + for (VertexIdMessages vertexIdMessages : + vertexIdMessagesList) { + peerConnection.outMessagesPerPeer.remove( + vertexIdMessages.getVertexId()); + } - int numHandlers = conf.getInt(GiraphJob.RPC_NUM_HANDLERS, - GiraphJob.RPC_NUM_HANDLERS_DEFAULT); - if (numTasks < numHandlers) { - numHandlers = numTasks; - } - this.jobToken = createJobToken(); - this.jobId = context.getJobID().toString(); - - int numWorkers = conf.getInt(GiraphJob.MAX_WORKERS, numTasks); - // If the number of flush threads is unset, it is set to - // the number of max workers - 1 or a minimum of 1. - int numFlushThreads = - Math.max(conf.getInt(GiraphJob.MSG_NUM_FLUSH_THREADS, - numWorkers - 1), - 1); - this.executor = Executors.newFixedThreadPool(numFlushThreads); - - // Simple handling of port collisions on the same machine while - // preserving debugability from the port number alone. - // Round up the max number of workers to the next power of 10 and use - // it as a constant to increase the port number with. - int portIncrementConstant = - (int) Math.pow(10, Math.ceil(Math.log10(numWorkers))); - String bindAddress = localHostname; - int bindPort = conf.getInt(GiraphJob.RPC_INITIAL_PORT, - GiraphJob.RPC_INITIAL_PORT_DEFAULT) + - taskId; - int bindAttempts = 0; - final int maxRpcPortBindAttempts = - conf.getInt(GiraphJob.MAX_RPC_PORT_BIND_ATTEMPTS, - GiraphJob.MAX_RPC_PORT_BIND_ATTEMPTS_DEFAULT); - while (bindAttempts < maxRpcPortBindAttempts) { - this.myAddress = new InetSocketAddress(bindAddress, bindPort); - try { - this.server = - getRPCServer( - myAddress, numHandlers, this.jobId, this.jobToken); - break; - } catch (BindException e) { - LOG.info("BasicRPCCommunications: Failed to bind with port " + - bindPort + " on bind attempt " + bindAttempts); - ++bindAttempts; - bindPort += portIncrementConstant; + proxy.putVertexIdMessagesList(vertexIdMessagesList); + context.progress(); + + verticesDone += vertexIdMessagesList.size(); + long curMillis = System.currentTimeMillis(); + if ((lastReportedMillis + + REPORTING_INTERVAL_MIN_MILLIS) < curMillis) { + lastReportedMillis = curMillis; + if (LOG.isInfoEnabled()) { + float percentDone = + (100f * verticesDone) / + vertices; + float minutesUsed = + (curMillis - startMillis) / 1000f / 60f; + float minutesRemaining = + (minutesUsed * 100f / percentDone) - + minutesUsed; + LOG.info("run: " + peerConnection + ", " + + verticesDone + " out of " + + vertices + + " done in " + minutesUsed + + " minutes, " + + percentDone + "% done, ETA " + + minutesRemaining + + " minutes remaining, " + + MemoryUtils.getRuntimeMemoryStats()); + } } + } } - if (bindAttempts == maxRpcPortBindAttempts) { - throw new IllegalStateException( - "BasicRPCCommunications: Failed to start RPCServer with " + - maxRpcPortBindAttempts + " attempts"); - } - - this.server.start(); - this.myName = myAddress.toString(); - if (LOG.isInfoEnabled()) { - LOG.info("BasicRPCCommunications: Started RPC " + - "communication server: " + myName + " with " + - numHandlers + " handlers and " + numFlushThreads + - " flush threads on bind attempt " + bindAttempts); + if (LOG.isDebugEnabled()) { + LOG.debug("run: " + proxy.getName() + + ": all messages flushed"); + } + } catch (IOException e) { + LOG.error(e); + if (peerConnection.isProxy) { + RPC.stopProxy(peerConnection.peer); } + throw new RuntimeException(e); + } } + } + + /** + * LargeMessageFlushExecutor flushes all outgoing messages destined to + * some vertices. This is executed when the number of messages destined to + * certain vertex exceeds maxSize. + */ + private class LargeMessageFlushExecutor implements Runnable { + /** Destination vertex of message. */ + private final I destVertex; + /** List of messages to the destination vertex */ + private final MsgList outMessageList; + /** Connection to send the message to. */ + private PeerConnection peerConnection; /** - * Get the final port of the RPC server that it bound to. + * Constructor of the executor for flushing large messages. * - * @return Port that RPC server was bound to. + * @param peerConnection Connection to send the message to. + * @param destVertex Destination vertex of message. */ - public int getPort() { - return myAddress.getPort(); + LargeMessageFlushExecutor(PeerConnection peerConnection, I destVertex) { + this.peerConnection = peerConnection; + synchronized (peerConnection.outMessagesPerPeer) { + this.destVertex = destVertex; + outMessageList = + peerConnection.outMessagesPerPeer.get(destVertex); + peerConnection.outMessagesPerPeer.remove(destVertex); + } } @Override - public void setup() { - try { - connectAllRPCProxys(this.jobId, this.jobToken); - } catch (IOException e) { - throw new IllegalStateException("setup: Got IOException", e); - } catch (InterruptedException e) { - throw new IllegalStateException("setup: Got InterruptedException", - e); + public void run() { + try { + CommunicationsInterface proxy = + peerConnection.getRPCProxy(); + + if (combiner != null) { + Iterable messages = combiner.combine(destVertex, + outMessageList); + if (messages == null) { + throw new IllegalStateException( + "run: Combiner cannot return null"); + } + if (Iterables.size(outMessageList) < + Iterables.size(messages)) { + throw new IllegalStateException( + "run: The number of combined messages is " + + "required to be <= to the number of " + + "messages to be combined"); + } + for (M msg: messages) { + proxy.putMsg(destVertex, msg); + } + } else { + proxy.putMsgList(destVertex, outMessageList); } - } - - protected abstract CommunicationsInterface getRPCProxy( - final InetSocketAddress addr, String jobId, J jobToken) - throws IOException, InterruptedException; - - /** - * Establish connections to every RPC proxy server that will be used in - * the upcoming messaging. This method is idempotent. - * - * @param jobId Stringified job id - * @param jobToken used for - * @throws InterruptedException - * @throws IOException - */ - private void connectAllRPCProxys(String jobId, J jobToken) - throws IOException, InterruptedException { - final int maxTries = 5; - for (PartitionOwner partitionOwner : service.getPartitionOwners()) { - int tries = 0; - while (tries < maxTries) { - try { - startPeerConnectionThread( - partitionOwner.getWorkerInfo(), jobId, jobToken); - break; - } catch (IOException e) { - LOG.warn("connectAllRPCProxys: Failed on attempt " + - tries + " of " + maxTries + - " to connect to " + partitionOwner.toString(), e); - ++tries; - } - } + } catch (IOException e) { + LOG.error(e); + if (peerConnection.isProxy) { + RPC.stopProxy(peerConnection.peer); } + throw new RuntimeException("run: Got IOException", e); + } finally { + outMessageList.clear(); + } + } + } + + /** + * Only constructor. + * + * @param context Context for getting configuration + * @param service Service worker to get the vertex ranges + * @throws IOException + * @throws UnknownHostException + * @throws InterruptedException + */ + public BasicRPCCommunications(Mapper.Context context, + CentralizedServiceWorker service) + throws IOException, InterruptedException { + this.service = service; + this.conf = context.getConfiguration(); + this.maxSize = conf.getInt(GiraphJob.MSG_SIZE, + GiraphJob.MSG_SIZE_DEFAULT); + this.maxMessagesPerFlushPut = + conf.getInt(GiraphJob.MAX_MESSAGES_PER_FLUSH_PUT, + GiraphJob.DEFAULT_MAX_MESSAGES_PER_FLUSH_PUT); + if (BspUtils.getVertexCombinerClass(conf) == null) { + this.combiner = null; + } else { + this.combiner = BspUtils.createVertexCombiner(conf); } - /** - * Creates the connections to remote RPCs if any only if the inet socket - * address doesn't already exist. - * - * @param workerInfo My worker info - * @param jobId Id of the job - * @param jobToken Required for secure Hadoop - * @throws IOException - * @throws InterruptedException - */ - private void startPeerConnectionThread(WorkerInfo workerInfo, - String jobId, - J jobToken) - throws IOException, InterruptedException { - if (LOG.isDebugEnabled()) { - LOG.debug("startPeerConnectionThread: hostname " + - workerInfo.getHostname() + ", port " + - workerInfo.getPort()); - } - final InetSocketAddress addr = - new InetSocketAddress(workerInfo.getHostname(), - workerInfo.getPort()); - // Cheap way to hold both the hostname and port (rather than - // make a class) - InetSocketAddress addrUnresolved = - InetSocketAddress.createUnresolved(addr.getHostName(), - addr.getPort()); - Map> outMsgMap = null; - boolean isProxy = true; - CommunicationsInterface peer = this; - synchronized(outMessages) { - outMsgMap = outMessages.get(addrUnresolved); - if (LOG.isDebugEnabled()) { - LOG.debug("startPeerConnectionThread: Connecting to " + - workerInfo.toString() + ", addr = " + addr + - " if outMsgMap (" + outMsgMap + ") == null "); - } - if (outMsgMap != null) { // this host has already been added - return; - } + this.localHostname = InetAddress.getLocalHost().getHostName(); + int taskId = conf.getInt("mapred.task.partition", -1); + int numTasks = conf.getInt("mapred.map.tasks", 1); - if (myName.equals(addr.toString())) { - isProxy = false; - } else { - peer = getRPCProxy(addr, jobId, jobToken); - } - outMsgMap = new HashMap>(); - outMessages.put(addrUnresolved, outMsgMap); - } - PeerConnection peerConnection = - new PeerConnection(outMsgMap, peer, isProxy); - peerConnections.put(addrUnresolved, peerConnection); + int numHandlers = conf.getInt(GiraphJob.RPC_NUM_HANDLERS, + GiraphJob.RPC_NUM_HANDLERS_DEFAULT); + if (numTasks < numHandlers) { + numHandlers = numTasks; + } + this.jobToken = createJobToken(); + this.jobId = context.getJobID().toString(); + + int numWorkers = conf.getInt(GiraphJob.MAX_WORKERS, numTasks); + // If the number of flush threads is unset, it is set to + // the number of max workers - 1 or a minimum of 1. + int numFlushThreads = + Math.max(conf.getInt(GiraphJob.MSG_NUM_FLUSH_THREADS, + numWorkers - 1), + 1); + this.executor = Executors.newFixedThreadPool(numFlushThreads); + + // Simple handling of port collisions on the same machine while + // preserving debugability from the port number alone. + // Round up the max number of workers to the next power of 10 and use + // it as a constant to increase the port number with. + int portIncrementConstant = + (int) Math.pow(10, Math.ceil(Math.log10(numWorkers))); + String bindAddress = localHostname; + int bindPort = conf.getInt(GiraphJob.RPC_INITIAL_PORT, + GiraphJob.RPC_INITIAL_PORT_DEFAULT) + + taskId; + int bindAttempts = 0; + final int maxRpcPortBindAttempts = + conf.getInt(GiraphJob.MAX_RPC_PORT_BIND_ATTEMPTS, + GiraphJob.MAX_RPC_PORT_BIND_ATTEMPTS_DEFAULT); + while (bindAttempts < maxRpcPortBindAttempts) { + this.myAddress = new InetSocketAddress(bindAddress, bindPort); + try { + this.server = + getRPCServer( + myAddress, numHandlers, this.jobId, this.jobToken); + break; + } catch (BindException e) { + LOG.info("BasicRPCCommunications: Failed to bind with port " + + bindPort + " on bind attempt " + bindAttempts); + ++bindAttempts; + bindPort += portIncrementConstant; + } + } + if (bindAttempts == maxRpcPortBindAttempts) { + throw new IllegalStateException( + "BasicRPCCommunications: Failed to start RPCServer with " + + maxRpcPortBindAttempts + " attempts"); } - @Override - public final long getProtocolVersion(String protocol, long clientVersion) - throws IOException { - return versionID; + this.server.start(); + this.myName = myAddress.toString(); + + if (LOG.isInfoEnabled()) { + LOG.info("BasicRPCCommunications: Started RPC " + + "communication server: " + myName + " with " + + numHandlers + " handlers and " + numFlushThreads + + " flush threads on bind attempt " + bindAttempts); + } + } + + /** + * Submit a large message to be sent. + * + * @param addr Message destination. + * @param destVertex Index of the destination vertex. + */ + private void submitLargeMessageSend(InetSocketAddress addr, I destVertex) { + PeerConnection pc = peerConnections.get(addr); + executor.execute(new LargeMessageFlushExecutor(pc, destVertex)); + } + + /** + * Create the job token. + * + * @return Job token. + * @throws IOException + */ + protected abstract J createJobToken() throws IOException; + + /** + * Get the RPC server. + * @param addr Server address. + * @param numHandlers Number of handlers. + * @param jobId Job id. + * @param jobToken Job token. + * @return RPC server. + * @throws IOException + */ + protected abstract Server getRPCServer(InetSocketAddress addr, + int numHandlers, String jobId, J jobToken) throws IOException; + + /** + * Get the final port of the RPC server that it bound to. + * + * @return Port that RPC server was bound to. + */ + public int getPort() { + return myAddress.getPort(); + } + + @Override + public void setup() { + try { + connectAllRPCProxys(this.jobId, this.jobToken); + } catch (IOException e) { + throw new IllegalStateException("setup: Got IOException", e); + } catch (InterruptedException e) { + throw new IllegalStateException("setup: Got InterruptedException", + e); + } + } + + /** + * Get the RPC proxy (handled by subclasses) + * + * @param addr Socket address. + * @param jobId Job id. + * @param jobToken Jobtoken (if any) + * @return The RPC proxy. + * @throws IOException + * @throws InterruptedException + */ + protected abstract CommunicationsInterface getRPCProxy( + final InetSocketAddress addr, String jobId, J jobToken) + throws IOException, InterruptedException; + + /** + * Establish connections to every RPC proxy server that will be used in + * the upcoming messaging. This method is idempotent. + * + * @param jobId Stringified job id + * @param jobToken used for + * @throws InterruptedException + * @throws IOException + */ + private void connectAllRPCProxys(String jobId, J jobToken) + throws IOException, InterruptedException { + final int maxTries = 5; + for (PartitionOwner partitionOwner : service.getPartitionOwners()) { + int tries = 0; + while (tries < maxTries) { + try { + startPeerConnectionThread( + partitionOwner.getWorkerInfo(), jobId, jobToken); + break; + } catch (IOException e) { + LOG.warn("connectAllRPCProxys: Failed on attempt " + + tries + " of " + maxTries + + " to connect to " + partitionOwner.toString(), e); + ++tries; + } + } + } + } + + /** + * Creates the connections to remote RPCs if any only if the inet socket + * address doesn't already exist. + * + * @param workerInfo My worker info + * @param jobId Id of the job + * @param jobToken Required for secure Hadoop + * @throws IOException + * @throws InterruptedException + */ + private void startPeerConnectionThread(WorkerInfo workerInfo, + String jobId, + J jobToken) throws IOException, InterruptedException { + if (LOG.isDebugEnabled()) { + LOG.debug("startPeerConnectionThread: hostname " + + workerInfo.getHostname() + ", port " + + workerInfo.getPort()); + } + final InetSocketAddress addr = + new InetSocketAddress(workerInfo.getHostname(), + workerInfo.getPort()); + // Cheap way to hold both the hostname and port (rather than + // make a class) + InetSocketAddress addrUnresolved = + InetSocketAddress.createUnresolved(addr.getHostName(), + addr.getPort()); + Map> outMsgMap = null; + boolean isProxy = true; + CommunicationsInterface peer = this; + synchronized (outMessages) { + outMsgMap = outMessages.get(addrUnresolved); + if (LOG.isDebugEnabled()) { + LOG.debug("startPeerConnectionThread: Connecting to " + + workerInfo.toString() + ", addr = " + addr + + " if outMsgMap (" + outMsgMap + ") == null "); + } + if (outMsgMap != null) { // this host has already been added + return; + } + + if (myName.equals(addr.toString())) { + isProxy = false; + } else { + peer = getRPCProxy(addr, jobId, jobToken); + } + + outMsgMap = new HashMap>(); + outMessages.put(addrUnresolved, outMsgMap); } -/*if[HADOOP_FACEBOOK] + PeerConnection peerConnection = + new PeerConnection(outMsgMap, peer, isProxy); + peerConnections.put(addrUnresolved, peerConnection); + } + + @Override + public final long getProtocolVersion(String protocol, long clientVersion) + throws IOException { + return VERSION_ID; + } + + /*if[HADOOP_FACEBOOK] public ProtocolSignature getProtocolSignature( String protocol, long clientVersion, @@ -623,602 +690,600 @@ public ProtocolSignature getProtocolSignature( } end[HADOOP_FACEBOOK]*/ - @Override - public void closeConnections() throws IOException { - for(PeerConnection pc : peerConnections.values()) { - pc.close(); - } + @Override + public void closeConnections() throws IOException { + for (PeerConnection pc : peerConnections.values()) { + pc.close(); } + } - @Override - public final void close() { - LOG.info("close: shutting down RPC server"); - server.stop(); - } + @Override + public final void close() { + LOG.info("close: shutting down RPC server"); + server.stop(); + } - @Override - public final void putMsg(I vertex, M msg) throws IOException { - List msgs = null; - if (LOG.isDebugEnabled()) { - LOG.debug("putMsg: Adding msg " + msg + " on vertex " + vertex); - } - if (inPrepareSuperstep) { - // Called by combiner (main thread) during superstep preparation - msgs = inMessages.get(vertex); - if (msgs == null) { - msgs = new ArrayList(); - inMessages.put(vertex, msgs); - } - msgs.add(msg); - } - else { - synchronized(transientInMessages) { - msgs = transientInMessages.get(vertex); - if (msgs == null) { - msgs = new ArrayList(); - transientInMessages.put(vertex, msgs); - } - } - synchronized(msgs) { - msgs.add(msg); - } - } + @Override + public final void putMsg(I vertex, M msg) throws IOException { + List msgs = null; + if (LOG.isDebugEnabled()) { + LOG.debug("putMsg: Adding msg " + msg + " on vertex " + vertex); } - - @Override - public final void putMsgList(I vertex, - MsgList msgList) throws IOException { - List msgs = null; - if (LOG.isDebugEnabled()) { - LOG.debug("putMsgList: Adding msgList " + msgList + - " on vertex " + vertex); - } - synchronized(transientInMessages) { - msgs = transientInMessages.get(vertex); - if (msgs == null) { - msgs = new ArrayList(msgList.size()); - transientInMessages.put(vertex, msgs); - } - } - synchronized(msgs) { - msgs.addAll(msgList); + if (inPrepareSuperstep) { + // Called by combiner (main thread) during superstep preparation + msgs = inMessages.get(vertex); + if (msgs == null) { + msgs = new ArrayList(); + inMessages.put(vertex, msgs); + } + msgs.add(msg); + } else { + synchronized (transientInMessages) { + msgs = transientInMessages.get(vertex); + if (msgs == null) { + msgs = new ArrayList(); + transientInMessages.put(vertex, msgs); } + } + synchronized (msgs) { + msgs.add(msg); + } } - - @Override - public final void putVertexIdMessagesList( - VertexIdMessagesList vertexIdMessagesList) - throws IOException { - if (LOG.isDebugEnabled()) { - LOG.debug("putVertexIdMessagesList: Adding msgList " + - vertexIdMessagesList); - } - - List messageList = null; - for (VertexIdMessages vertexIdMessages : vertexIdMessagesList) { - synchronized(transientInMessages) { - messageList = - transientInMessages.get(vertexIdMessages.getVertexId()); - if (messageList == null) { - messageList = new ArrayList( - vertexIdMessages.getMessageList().size()); - transientInMessages.put( - vertexIdMessages.getVertexId(), messageList); - } - } - synchronized(messageList) { - messageList.addAll(vertexIdMessages.getMessageList()); - } - } + } + + @Override + public final void putMsgList(I vertex, + MsgList msgList) throws IOException { + List msgs = null; + if (LOG.isDebugEnabled()) { + LOG.debug("putMsgList: Adding msgList " + msgList + + " on vertex " + vertex); } - - @Override - public final void putVertexList(int partitionId, - VertexList vertexList) - throws IOException { - if (LOG.isDebugEnabled()) { - LOG.debug("putVertexList: On partition id " + partitionId + - " adding vertex list of size " + vertexList.size()); - } - synchronized(inPartitionVertexMap) { - if (vertexList.size() == 0) { - return; - } - if (!inPartitionVertexMap.containsKey(partitionId)) { - inPartitionVertexMap.put(partitionId, - new ArrayList>(vertexList)); - } else { - List> tmpVertexList = - inPartitionVertexMap.get(partitionId); - tmpVertexList.addAll(vertexList); - } - } + synchronized (transientInMessages) { + msgs = transientInMessages.get(vertex); + if (msgs == null) { + msgs = new ArrayList(msgList.size()); + transientInMessages.put(vertex, msgs); + } } - - @Override - public final void addEdge(I vertexIndex, Edge edge) { - if (LOG.isDebugEnabled()) { - LOG.debug("addEdge: Adding edge " + edge); - } - synchronized(inVertexMutationsMap) { - VertexMutations vertexMutations = null; - if (!inVertexMutationsMap.containsKey(vertexIndex)) { - vertexMutations = new VertexMutations(); - inVertexMutationsMap.put(vertexIndex, vertexMutations); - } else { - vertexMutations = inVertexMutationsMap.get(vertexIndex); - } - vertexMutations.addEdge(edge); - } + synchronized (msgs) { + msgs.addAll(msgList); } - - @Override - public void removeEdge(I vertexIndex, I destinationVertexIndex) { - if (LOG.isDebugEnabled()) { - LOG.debug("removeEdge: Removing edge on destination " + - destinationVertexIndex); - } - synchronized(inVertexMutationsMap) { - VertexMutations vertexMutations = null; - if (!inVertexMutationsMap.containsKey(vertexIndex)) { - vertexMutations = new VertexMutations(); - inVertexMutationsMap.put(vertexIndex, vertexMutations); - } else { - vertexMutations = inVertexMutationsMap.get(vertexIndex); - } - vertexMutations.removeEdge(destinationVertexIndex); - } + } + + @Override + public final void putVertexIdMessagesList( + VertexIdMessagesList vertexIdMessagesList) + throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("putVertexIdMessagesList: Adding msgList " + + vertexIdMessagesList); } - @Override - public final void addVertex(BasicVertex vertex) { - if (LOG.isDebugEnabled()) { - LOG.debug("addVertex: Adding vertex " + vertex); - } - synchronized(inVertexMutationsMap) { - VertexMutations vertexMutations = null; - if (!inVertexMutationsMap.containsKey(vertex.getVertexId())) { - vertexMutations = new VertexMutations(); - inVertexMutationsMap.put(vertex.getVertexId(), vertexMutations); - } else { - vertexMutations = inVertexMutationsMap.get(vertex.getVertexId()); - } - vertexMutations.addVertex(vertex); + List messageList = null; + for (VertexIdMessages vertexIdMessages : vertexIdMessagesList) { + synchronized (transientInMessages) { + messageList = + transientInMessages.get(vertexIdMessages.getVertexId()); + if (messageList == null) { + messageList = new ArrayList( + vertexIdMessages.getMessageList().size()); + transientInMessages.put( + vertexIdMessages.getVertexId(), messageList); } + } + synchronized (messageList) { + messageList.addAll(vertexIdMessages.getMessageList()); + } } - - @Override - public void removeVertex(I vertexIndex) { - if (LOG.isDebugEnabled()) { - LOG.debug("removeVertex: Removing vertex " + vertexIndex); - } - synchronized(inVertexMutationsMap) { - VertexMutations vertexMutations = null; - if (!inVertexMutationsMap.containsKey(vertexIndex)) { - vertexMutations = new VertexMutations(); - inVertexMutationsMap.put(vertexIndex, vertexMutations); - } else { - vertexMutations = inVertexMutationsMap.get(vertexIndex); - } - vertexMutations.removeVertex(); - } + } + + @Override + public final void putVertexList(int partitionId, + VertexList vertexList) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("putVertexList: On partition id " + partitionId + + " adding vertex list of size " + vertexList.size()); } - - @Override - public final void sendPartitionReq(WorkerInfo workerInfo, - Partition partition) { - // Internally, break up the sending so that the list doesn't get too - // big. - VertexList hadoopVertexList = - new VertexList(); - InetSocketAddress addr = - getInetSocketAddress(workerInfo, partition.getPartitionId()); - CommunicationsInterface rpcProxy = - peerConnections.get(addr).getRPCProxy(); - - if (LOG.isDebugEnabled()) { - LOG.debug("sendPartitionReq: Sending to " + rpcProxy.getName() + - " " + addr + " from " + workerInfo + - ", with partition " + partition); - } - for (BasicVertex vertex : partition.getVertices()) { - hadoopVertexList.add(vertex); - if (hadoopVertexList.size() >= MAX_VERTICES_PER_RPC) { - try { - rpcProxy.putVertexList(partition.getPartitionId(), - hadoopVertexList); - } catch (IOException e) { - throw new RuntimeException(e); - } - hadoopVertexList.clear(); - } - } - if (hadoopVertexList.size() > 0) { - try { - rpcProxy.putVertexList(partition.getPartitionId(), - hadoopVertexList); - } catch (IOException e) { - throw new RuntimeException(e); - } - } + synchronized (inPartitionVertexMap) { + if (vertexList.size() == 0) { + return; + } + if (!inPartitionVertexMap.containsKey(partitionId)) { + inPartitionVertexMap.put(partitionId, + new ArrayList>(vertexList)); + } else { + List> tmpVertexList = + inPartitionVertexMap.get(partitionId); + tmpVertexList.addAll(vertexList); + } } + } - /** - * Fill the socket address cache for the worker info and its partition. - * - * @param workerInfo Worker information to get the socket address - * @param partitionId - * @return address of the vertex range server containing this vertex - */ - private InetSocketAddress getInetSocketAddress(WorkerInfo workerInfo, - int partitionId) { - synchronized(partitionIndexAddressMap) { - InetSocketAddress address = - partitionIndexAddressMap.get(partitionId); - if (address == null) { - address = InetSocketAddress.createUnresolved( - workerInfo.getHostname(), - workerInfo.getPort()); - partitionIndexAddressMap.put(partitionId, address); - } - - if (address.getPort() != workerInfo.getPort() || - !address.getHostName().equals(workerInfo.getHostname())) { - throw new IllegalStateException( - "getInetSocketAddress: Impossible that address " + - address + " does not match " + workerInfo); - } - - return address; - } + @Override + public final void addEdge(I vertexIndex, Edge edge) { + if (LOG.isDebugEnabled()) { + LOG.debug("addEdge: Adding edge " + edge); } - - /** - * Fill the socket address cache for the partition owner. - * - * @param destVertex vertex to be sent - * @return address of the vertex range server containing this vertex - */ - private InetSocketAddress getInetSocketAddress(I destVertex) { - PartitionOwner partitionOwner = - service.getVertexPartitionOwner(destVertex); - return getInetSocketAddress(partitionOwner.getWorkerInfo(), - partitionOwner.getPartitionId()); + synchronized (inVertexMutationsMap) { + VertexMutations vertexMutations = null; + if (!inVertexMutationsMap.containsKey(vertexIndex)) { + vertexMutations = new VertexMutations(); + inVertexMutationsMap.put(vertexIndex, vertexMutations); + } else { + vertexMutations = inVertexMutationsMap.get(vertexIndex); + } + vertexMutations.addEdge(edge); } + } - @Override - public final void sendMessageReq(I destVertex, M msg) { - InetSocketAddress addr = getInetSocketAddress(destVertex); - if (LOG.isDebugEnabled()) { - LOG.debug("sendMessage: Send bytes (" + msg.toString() + - ") to " + destVertex + " with address " + addr); - } - ++totalMsgsSentInSuperstep; - Map> msgMap = null; - synchronized(outMessages) { - msgMap = outMessages.get(addr); - } - if (msgMap == null) { // should never happen after constructor - throw new RuntimeException( - "sendMessage: msgMap did not exist for " + addr + - " for vertex " + destVertex); - } - - synchronized(msgMap) { - MsgList msgList = msgMap.get(destVertex); - if (msgList == null) { // should only happen once - msgList = new MsgList(); - msgMap.put(destVertex, msgList); - } - msgList.add(msg); - if (LOG.isDebugEnabled()) { - LOG.debug("sendMessage: added msg=" + msg + ", size=" + - msgList.size()); - } - if (msgList.size() > maxSize) { - submitLargeMessageSend(addr, destVertex); - } - } + @Override + public void removeEdge(I vertexIndex, I destinationVertexIndex) { + if (LOG.isDebugEnabled()) { + LOG.debug("removeEdge: Removing edge on destination " + + destinationVertexIndex); } - - @Override - public final void addEdgeReq(I destVertex, Edge edge) - throws IOException { - InetSocketAddress addr = getInetSocketAddress(destVertex); - if (LOG.isDebugEnabled()) { - LOG.debug("addEdgeReq: Add edge (" + edge.toString() + ") to " + - destVertex + " with address " + addr); - } - CommunicationsInterface rpcProxy = - peerConnections.get(addr).getRPCProxy(); - rpcProxy.addEdge(destVertex, edge); + synchronized (inVertexMutationsMap) { + VertexMutations vertexMutations = null; + if (!inVertexMutationsMap.containsKey(vertexIndex)) { + vertexMutations = new VertexMutations(); + inVertexMutationsMap.put(vertexIndex, vertexMutations); + } else { + vertexMutations = inVertexMutationsMap.get(vertexIndex); + } + vertexMutations.removeEdge(destinationVertexIndex); } + } - @Override - public final void removeEdgeReq(I vertexIndex, I destVertexIndex) - throws IOException { - InetSocketAddress addr = getInetSocketAddress(vertexIndex); - if (LOG.isDebugEnabled()) { - LOG.debug("removeEdgeReq: remove edge (" + destVertexIndex + - ") from" + vertexIndex + " with address " + addr); - } - CommunicationsInterface rpcProxy = - peerConnections.get(addr).getRPCProxy(); - rpcProxy.removeEdge(vertexIndex, destVertexIndex); + @Override + public final void addVertex(BasicVertex vertex) { + if (LOG.isDebugEnabled()) { + LOG.debug("addVertex: Adding vertex " + vertex); } - - @Override - public final void addVertexReq(BasicVertex vertex) - throws IOException { - InetSocketAddress addr = getInetSocketAddress(vertex.getVertexId()); - if (LOG.isDebugEnabled()) { - LOG.debug("addVertexReq: Add vertex (" + vertex + ") " + - " with address " + addr); - } - CommunicationsInterface rpcProxy = - peerConnections.get(addr).getRPCProxy(); - rpcProxy.addVertex(vertex); + synchronized (inVertexMutationsMap) { + VertexMutations vertexMutations = null; + if (!inVertexMutationsMap.containsKey(vertex.getVertexId())) { + vertexMutations = new VertexMutations(); + inVertexMutationsMap.put(vertex.getVertexId(), vertexMutations); + } else { + vertexMutations = inVertexMutationsMap.get(vertex.getVertexId()); + } + vertexMutations.addVertex(vertex); } + } - @Override - public void removeVertexReq(I vertexIndex) throws IOException { - InetSocketAddress addr = - getInetSocketAddress(vertexIndex); - if (LOG.isDebugEnabled()) { - LOG.debug("removeVertexReq: Remove vertex index (" - + vertexIndex + ") with address " + addr); - } - CommunicationsInterface rpcProxy = - peerConnections.get(addr).getRPCProxy(); - rpcProxy.removeVertex(vertexIndex); + @Override + public void removeVertex(I vertexIndex) { + if (LOG.isDebugEnabled()) { + LOG.debug("removeVertex: Removing vertex " + vertexIndex); } - - @Override - public long flush(Mapper.Context context) throws IOException { - if (LOG.isInfoEnabled()) { - LOG.info("flush: starting for superstep " + - service.getSuperstep() + " " + - MemoryUtils.getRuntimeMemoryStats()); - } - for (List msgList : inMessages.values()) { - msgList.clear(); + synchronized (inVertexMutationsMap) { + VertexMutations vertexMutations = null; + if (!inVertexMutationsMap.containsKey(vertexIndex)) { + vertexMutations = new VertexMutations(); + inVertexMutationsMap.put(vertexIndex, vertexMutations); + } else { + vertexMutations = inVertexMutationsMap.get(vertexIndex); + } + vertexMutations.removeVertex(); + } + } + + @Override + public final void sendPartitionReq(WorkerInfo workerInfo, + Partition partition) { + // Internally, break up the sending so that the list doesn't get too + // big. + VertexList hadoopVertexList = + new VertexList(); + InetSocketAddress addr = + getInetSocketAddress(workerInfo, partition.getPartitionId()); + CommunicationsInterface rpcProxy = + peerConnections.get(addr).getRPCProxy(); + + if (LOG.isDebugEnabled()) { + LOG.debug("sendPartitionReq: Sending to " + rpcProxy.getName() + + " " + addr + " from " + workerInfo + + ", with partition " + partition); + } + for (BasicVertex vertex : partition.getVertices()) { + hadoopVertexList.add(vertex); + if (hadoopVertexList.size() >= MAX_VERTICES_PER_RPC) { + try { + rpcProxy.putVertexList(partition.getPartitionId(), + hadoopVertexList); + } catch (IOException e) { + throw new RuntimeException(e); } - inMessages.clear(); - - Collection> futures = new ArrayList>(); + hadoopVertexList.clear(); + } + } + if (hadoopVertexList.size() > 0) { + try { + rpcProxy.putVertexList(partition.getPartitionId(), + hadoopVertexList); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } + + /** + * Fill the socket address cache for the worker info and its partition. + * + * @param workerInfo Worker information to get the socket address + * @param partitionId Partition id to look up. + * @return address of the vertex range server containing this vertex + */ + private InetSocketAddress getInetSocketAddress(WorkerInfo workerInfo, + int partitionId) { + synchronized (partitionIndexAddressMap) { + InetSocketAddress address = + partitionIndexAddressMap.get(partitionId); + if (address == null) { + address = InetSocketAddress.createUnresolved( + workerInfo.getHostname(), + workerInfo.getPort()); + partitionIndexAddressMap.put(partitionId, address); + } + + if (address.getPort() != workerInfo.getPort() || + !address.getHostName().equals(workerInfo.getHostname())) { + throw new IllegalStateException( + "getInetSocketAddress: Impossible that address " + + address + " does not match " + workerInfo); + } + + return address; + } + } + + /** + * Fill the socket address cache for the partition owner. + * + * @param destVertex vertex to be sent + * @return address of the vertex range server containing this vertex + */ + private InetSocketAddress getInetSocketAddress(I destVertex) { + PartitionOwner partitionOwner = + service.getVertexPartitionOwner(destVertex); + return getInetSocketAddress(partitionOwner.getWorkerInfo(), + partitionOwner.getPartitionId()); + } + + @Override + public final void sendMessageReq(I destVertex, M msg) { + InetSocketAddress addr = getInetSocketAddress(destVertex); + if (LOG.isDebugEnabled()) { + LOG.debug("sendMessage: Send bytes (" + msg.toString() + + ") to " + destVertex + " with address " + addr); + } + ++totalMsgsSentInSuperstep; + Map> msgMap = null; + synchronized (outMessages) { + msgMap = outMessages.get(addr); + } + if (msgMap == null) { // should never happen after constructor + throw new RuntimeException( + "sendMessage: msgMap did not exist for " + addr + + " for vertex " + destVertex); + } - // randomize peers in order to avoid hotspot on racks - List peerList = - new ArrayList(peerConnections.values()); - Collections.shuffle(peerList); + synchronized (msgMap) { + MsgList msgList = msgMap.get(destVertex); + if (msgList == null) { // should only happen once + msgList = new MsgList(); + msgMap.put(destVertex, msgList); + } + msgList.add(msg); + if (LOG.isDebugEnabled()) { + LOG.debug("sendMessage: added msg=" + msg + ", size=" + + msgList.size()); + } + if (msgList.size() > maxSize) { + submitLargeMessageSend(addr, destVertex); + } + } + } + + @Override + public final void addEdgeReq(I destVertex, Edge edge) + throws IOException { + InetSocketAddress addr = getInetSocketAddress(destVertex); + if (LOG.isDebugEnabled()) { + LOG.debug("addEdgeReq: Add edge (" + edge.toString() + ") to " + + destVertex + " with address " + addr); + } + CommunicationsInterface rpcProxy = + peerConnections.get(addr).getRPCProxy(); + rpcProxy.addEdge(destVertex, edge); + } + + @Override + public final void removeEdgeReq(I vertexIndex, I destVertexIndex) + throws IOException { + InetSocketAddress addr = getInetSocketAddress(vertexIndex); + if (LOG.isDebugEnabled()) { + LOG.debug("removeEdgeReq: remove edge (" + destVertexIndex + + ") from" + vertexIndex + " with address " + addr); + } + CommunicationsInterface rpcProxy = + peerConnections.get(addr).getRPCProxy(); + rpcProxy.removeEdge(vertexIndex, destVertexIndex); + } + + @Override + public final void addVertexReq(BasicVertex vertex) + throws IOException { + InetSocketAddress addr = getInetSocketAddress(vertex.getVertexId()); + if (LOG.isDebugEnabled()) { + LOG.debug("addVertexReq: Add vertex (" + vertex + ") " + + " with address " + addr); + } + CommunicationsInterface rpcProxy = + peerConnections.get(addr).getRPCProxy(); + rpcProxy.addVertex(vertex); + } + + @Override + public void removeVertexReq(I vertexIndex) throws IOException { + InetSocketAddress addr = + getInetSocketAddress(vertexIndex); + if (LOG.isDebugEnabled()) { + LOG.debug("removeVertexReq: Remove vertex index (" + + vertexIndex + ") with address " + addr); + } + CommunicationsInterface rpcProxy = + peerConnections.get(addr).getRPCProxy(); + rpcProxy.removeVertex(vertexIndex); + } + + @Override + public long flush(Mapper.Context context) throws IOException { + if (LOG.isInfoEnabled()) { + LOG.info("flush: starting for superstep " + + service.getSuperstep() + " " + + MemoryUtils.getRuntimeMemoryStats()); + } + for (List msgList : inMessages.values()) { + msgList.clear(); + } + inMessages.clear(); - for (PeerConnection pc : peerList) { - futures.add(executor.submit(new PeerFlushExecutor(pc, context))); - } + Collection> futures = new ArrayList>(); - // wait for all flushes - for (Future future : futures) { - try { - future.get(); - context.progress(); - } catch (InterruptedException e) { - throw new IllegalStateException("flush: Got IOException", e); - } catch (ExecutionException e) { - throw new IllegalStateException( - "flush: Got ExecutionException", e); - } - } + // randomize peers in order to avoid hotspot on racks + List peerList = + new ArrayList(peerConnections.values()); + Collections.shuffle(peerList); - if (LOG.isInfoEnabled()) { - LOG.info("flush: ended for superstep " + - service.getSuperstep() + " " + - MemoryUtils.getRuntimeMemoryStats()); - } + for (PeerConnection pc : peerList) { + futures.add(executor.submit(new PeerFlushExecutor(pc, context))); + } - long msgs = totalMsgsSentInSuperstep; - totalMsgsSentInSuperstep = 0; - return msgs; + // wait for all flushes + for (Future future : futures) { + try { + future.get(); + context.progress(); + } catch (InterruptedException e) { + throw new IllegalStateException("flush: Got IOException", e); + } catch (ExecutionException e) { + throw new IllegalStateException( + "flush: Got ExecutionException", e); + } } - @Override - public void prepareSuperstep() { - if (LOG.isInfoEnabled()) { - LOG.info("prepareSuperstep: Superstep " + - service.getSuperstep() + " " + - MemoryUtils.getRuntimeMemoryStats()); - } - inPrepareSuperstep = true; - - // Combine and put the transient messages into the inMessages. - synchronized(transientInMessages) { - for (Entry> entry : transientInMessages.entrySet()) { - if (combiner != null) { - try { - Iterable messages = - combiner.combine(entry.getKey(), - entry.getValue()); - if (messages == null) { - throw new IllegalStateException( - "prepareSuperstep: Combiner cannot " + - "return null"); - } - if (Iterables.size(entry.getValue()) < - Iterables.size(messages)) { - throw new IllegalStateException( - "prepareSuperstep: The number of " + - "combined messages is " + - "required to be <= to the number of " + - "messages to be combined"); - } - for (M msg: messages) { - putMsg(entry.getKey(), msg); - } - } catch (IOException e) { - // no actual IO -- should never happen - throw new RuntimeException(e); - } - } else { - List msgs = inMessages.get(entry.getKey()); - if (msgs == null) { - msgs = new ArrayList(); - inMessages.put(entry.getKey(), msgs); - } - msgs.addAll(entry.getValue()); - } - entry.getValue().clear(); - } - transientInMessages.clear(); - } + if (LOG.isInfoEnabled()) { + LOG.info("flush: ended for superstep " + + service.getSuperstep() + " " + + MemoryUtils.getRuntimeMemoryStats()); + } - if (inMessages.size() > 0) { - // Assign the messages to each destination vertex (getting rid of - // the old ones) - for (Partition partition : - service.getPartitionMap().values()) { - for (BasicVertex vertex : partition.getVertices()) { - List msgList = inMessages.get(vertex.getVertexId()); - if (msgList != null) { - if (LOG.isDebugEnabled()) { - LOG.debug("prepareSuperstep: Assigning " + - msgList.size() + - " mgs to vertex index " + vertex); - } - for (M msg : msgList) { - if (msg == null) { - LOG.warn("prepareSuperstep: Null message " + - "in inMessages"); - } - } - service.assignMessagesToVertex(vertex, msgList); - msgList.clear(); - if (inMessages.remove(vertex.getVertexId()) == null) { - throw new IllegalStateException( - "prepareSuperstep: Impossible to not remove " + - vertex); - } - } - } + long msgs = totalMsgsSentInSuperstep; + totalMsgsSentInSuperstep = 0; + return msgs; + } + + @Override + public void prepareSuperstep() { + if (LOG.isInfoEnabled()) { + LOG.info("prepareSuperstep: Superstep " + + service.getSuperstep() + " " + + MemoryUtils.getRuntimeMemoryStats()); + } + inPrepareSuperstep = true; + + // Combine and put the transient messages into the inMessages. + synchronized (transientInMessages) { + for (Entry> entry : transientInMessages.entrySet()) { + if (combiner != null) { + try { + Iterable messages = + combiner.combine(entry.getKey(), + entry.getValue()); + if (messages == null) { + throw new IllegalStateException( + "prepareSuperstep: Combiner cannot " + + "return null"); } - } - - inPrepareSuperstep = false; - - // Resolve what happens when messages are sent to non-existent vertices - // and vertices that have mutations. Also make sure that the messages - // are being sent to the correct destination - Set resolveVertexIndexSet = new TreeSet(); - if (inMessages.size() > 0) { - for (Entry> entry : inMessages.entrySet()) { - if (service.getPartition(entry.getKey()) == null) { - throw new IllegalStateException( - "prepareSuperstep: Impossible that this worker " + - service.getWorkerInfo() + " was sent " + - entry.getValue().size() + " message(s) with " + - "vertex id " + entry.getKey() + - " when it does not own this partition. It should " + - "have gone to partition owner " + - service.getVertexPartitionOwner(entry.getKey()) + - ". The partition owners are " + - service.getPartitionOwners()); - } - resolveVertexIndexSet.add(entry.getKey()); + if (Iterables.size(entry.getValue()) < + Iterables.size(messages)) { + throw new IllegalStateException( + "prepareSuperstep: The number of " + + "combined messages is " + + "required to be <= to the number of " + + "messages to be combined"); } - } - synchronized(inVertexMutationsMap) { - for (I vertexIndex : inVertexMutationsMap.keySet()) { - resolveVertexIndexSet.add(vertexIndex); + for (M msg: messages) { + putMsg(entry.getKey(), msg); } + } catch (IOException e) { + // no actual IO -- should never happen + throw new RuntimeException(e); + } + } else { + List msgs = inMessages.get(entry.getKey()); + if (msgs == null) { + msgs = new ArrayList(); + inMessages.put(entry.getKey(), msgs); + } + msgs.addAll(entry.getValue()); } + entry.getValue().clear(); + } + transientInMessages.clear(); + } - // Resolve all graph mutations - for (I vertexIndex : resolveVertexIndexSet) { - VertexResolver vertexResolver = - BspUtils.createVertexResolver( - conf, service.getGraphMapper().getGraphState()); - BasicVertex originalVertex = - service.getVertex(vertexIndex); - Iterable messages = inMessages.get(vertexIndex); - if (originalVertex != null) { - messages = originalVertex.getMessages(); - } - VertexMutations vertexMutations = - inVertexMutationsMap.get(vertexIndex); - BasicVertex vertex = - vertexResolver.resolve(vertexIndex, - originalVertex, - vertexMutations, - messages); + if (inMessages.size() > 0) { + // Assign the messages to each destination vertex (getting rid of + // the old ones) + for (Partition partition : + service.getPartitionMap().values()) { + for (BasicVertex vertex : partition.getVertices()) { + List msgList = inMessages.get(vertex.getVertexId()); + if (msgList != null) { if (LOG.isDebugEnabled()) { - LOG.debug("prepareSuperstep: Resolved vertex index " + - vertexIndex + " with original vertex " + - originalVertex + ", returned vertex " + vertex + - " on superstep " + service.getSuperstep() + - " with mutations " + - vertexMutations); + LOG.debug("prepareSuperstep: Assigning " + + msgList.size() + + " mgs to vertex index " + vertex); } - - Partition partition = - service.getPartition(vertexIndex); - if (partition == null) { - throw new IllegalStateException( - "prepareSuperstep: No partition for index " + vertexIndex + - " in " + service.getPartitionMap() + " should have been " + - service.getVertexPartitionOwner(vertexIndex)); + for (M msg : msgList) { + if (msg == null) { + LOG.warn("prepareSuperstep: Null message " + + "in inMessages"); + } } - if (vertex != null) { - ((MutableVertex) vertex).setVertexId(vertexIndex); - partition.putVertex((BasicVertex) vertex); - } else if (originalVertex != null) { - partition.removeVertex(originalVertex.getVertexId()); + service.assignMessagesToVertex(vertex, msgList); + msgList.clear(); + if (inMessages.remove(vertex.getVertexId()) == null) { + throw new IllegalStateException( + "prepareSuperstep: Impossible to not remove " + + vertex); } + } } - synchronized(inVertexMutationsMap) { - inVertexMutationsMap.clear(); - } + } } - @Override - public void fixPartitionIdToSocketAddrMap() { - // 1. Fix all the cached inet addresses (remove all changed entries) - // 2. Connect to any new RPC servers - synchronized(partitionIndexAddressMap) { - for (PartitionOwner partitionOwner : service.getPartitionOwners()) { - InetSocketAddress address = - partitionIndexAddressMap.get( - partitionOwner.getPartitionId()); - if (address != null && - (!address.getHostName().equals( - partitionOwner.getWorkerInfo().getHostname()) || - address.getPort() != - partitionOwner.getWorkerInfo().getPort())) { - if (LOG.isInfoEnabled()) { - LOG.info("fixPartitionIdToSocketAddrMap: " + - "Partition owner " + - partitionOwner + " changed from " + - address); - } - partitionIndexAddressMap.remove( - partitionOwner.getPartitionId()); - } - } - } - try { - connectAllRPCProxys(this.jobId, this.jobToken); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } catch (IOException e) { - throw new RuntimeException(e); + inPrepareSuperstep = false; + + // Resolve what happens when messages are sent to non-existent vertices + // and vertices that have mutations. Also make sure that the messages + // are being sent to the correct destination + Set resolveVertexIndexSet = new TreeSet(); + if (inMessages.size() > 0) { + for (Entry> entry : inMessages.entrySet()) { + if (service.getPartition(entry.getKey()) == null) { + throw new IllegalStateException( + "prepareSuperstep: Impossible that this worker " + + service.getWorkerInfo() + " was sent " + + entry.getValue().size() + " message(s) with " + + "vertex id " + entry.getKey() + + " when it does not own this partition. It should " + + "have gone to partition owner " + + service.getVertexPartitionOwner(entry.getKey()) + + ". The partition owners are " + + service.getPartitionOwners()); } + resolveVertexIndexSet.add(entry.getKey()); + } } - - @Override - public String getName() { - return myName; + synchronized (inVertexMutationsMap) { + for (I vertexIndex : inVertexMutationsMap.keySet()) { + resolveVertexIndexSet.add(vertexIndex); + } } - @Override - public Map>> getInPartitionVertexMap() { - return inPartitionVertexMap; + // Resolve all graph mutations + for (I vertexIndex : resolveVertexIndexSet) { + VertexResolver vertexResolver = + BspUtils.createVertexResolver( + conf, service.getGraphMapper().getGraphState()); + BasicVertex originalVertex = + service.getVertex(vertexIndex); + Iterable messages = inMessages.get(vertexIndex); + if (originalVertex != null) { + messages = originalVertex.getMessages(); + } + VertexMutations vertexMutations = + inVertexMutationsMap.get(vertexIndex); + BasicVertex vertex = + vertexResolver.resolve(vertexIndex, + originalVertex, + vertexMutations, + messages); + if (LOG.isDebugEnabled()) { + LOG.debug("prepareSuperstep: Resolved vertex index " + + vertexIndex + " with original vertex " + + originalVertex + ", returned vertex " + vertex + + " on superstep " + service.getSuperstep() + + " with mutations " + + vertexMutations); + } + + Partition partition = + service.getPartition(vertexIndex); + if (partition == null) { + throw new IllegalStateException( + "prepareSuperstep: No partition for index " + vertexIndex + + " in " + service.getPartitionMap() + " should have been " + + service.getVertexPartitionOwner(vertexIndex)); + } + if (vertex != null) { + ((MutableVertex) vertex).setVertexId(vertexIndex); + partition.putVertex((BasicVertex) vertex); + } else if (originalVertex != null) { + partition.removeVertex(originalVertex.getVertexId()); + } + } + synchronized (inVertexMutationsMap) { + inVertexMutationsMap.clear(); + } + } + + @Override + public void fixPartitionIdToSocketAddrMap() { + // 1. Fix all the cached inet addresses (remove all changed entries) + // 2. Connect to any new RPC servers + synchronized (partitionIndexAddressMap) { + for (PartitionOwner partitionOwner : service.getPartitionOwners()) { + InetSocketAddress address = + partitionIndexAddressMap.get( + partitionOwner.getPartitionId()); + if (address != null && + (!address.getHostName().equals( + partitionOwner.getWorkerInfo().getHostname()) || + address.getPort() != + partitionOwner.getWorkerInfo().getPort())) { + if (LOG.isInfoEnabled()) { + LOG.info("fixPartitionIdToSocketAddrMap: " + + "Partition owner " + + partitionOwner + " changed from " + + address); + } + partitionIndexAddressMap.remove( + partitionOwner.getPartitionId()); + } + } + } + try { + connectAllRPCProxys(this.jobId, this.jobToken); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } catch (IOException e) { + throw new RuntimeException(e); } + } + + @Override + public String getName() { + return myName; + } + + @Override + public Map>> getInPartitionVertexMap() { + return inPartitionVertexMap; + } } diff --git a/src/main/java/org/apache/giraph/comm/CommunicationsInterface.java b/src/main/java/org/apache/giraph/comm/CommunicationsInterface.java index e35dc803b..71c0b5548 100644 --- a/src/main/java/org/apache/giraph/comm/CommunicationsInterface.java +++ b/src/main/java/org/apache/giraph/comm/CommunicationsInterface.java @@ -34,100 +34,98 @@ /** * Basic interface for communication between workers. * - * @param vertex id - * @param message data + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data */ @SuppressWarnings("rawtypes") /*if_not[HADOOP] else[HADOOP]*/ @TokenInfo(BspTokenSelector.class) /*end[HADOOP]*/ -public interface CommunicationsInterface< - I extends WritableComparable, - V extends Writable, - E extends Writable, - M extends Writable> - extends VersionedProtocol { +public interface CommunicationsInterface + extends VersionedProtocol { + /** + * Interface Version History + * + * 0 - First Version + */ + long VERSION_ID = 0L; - /** - * Interface Version History - * - * 0 - First Version - */ - static final long versionID = 0L; + /** + * Adds incoming message. + * + * @param vertexIndex Destination vertex index. + * @param message Message to store. + * @throws IOException + */ + void putMsg(I vertexIndex, M message) throws IOException; - /** - * Adds incoming message. - * - * @param vertexIndex - * @param msg - * @throws IOException - */ - void putMsg(I vertexIndex, M msg) throws IOException; + /** + * Adds incoming message list. + * + * @param vertexIndex Vertex index where the message are added + * @param msgList messages added + * @throws IOException + */ + void putMsgList(I vertexIndex, MsgList msgList) throws IOException; - /** - * Adds incoming message list. - * - * @param vertexIndex Vertex index where the message are added - * @param msgList messages added - * @throws IOException - */ - void putMsgList(I vertexIndex, MsgList msgList) throws IOException; + /** + * Adds a list of vertex ids and their respective message lists. + * + * @param vertexIdMessagesList messages to be added + * @throws IOException + */ + void putVertexIdMessagesList( + VertexIdMessagesList vertexIdMessagesList) throws IOException; - /** - * Adds a list of vertex ids and their respective message lists. - * - * @param vertexIdMessagesList messages to be added - * @throws IOException - */ - void putVertexIdMessagesList( - VertexIdMessagesList vertexIdMessagesList) throws IOException; + /** + * Adds vertex list (index, value, edges, etc.) to the appropriate worker. + * + * @param partitionId Partition id of the vertices to be added. + * @param vertexList List of vertices to add + */ + void putVertexList(int partitionId, + VertexList vertexList) throws IOException; - /** - * Adds vertex list (index, value, edges, etc.) to the appropriate worker. - * - * @param partitionId Partition id of the vertices to be added. - * @param vertexList List of vertices to add - */ - void putVertexList(int partitionId, - VertexList vertexList) throws IOException; + /** + * Add an edge to a remote vertex + * + * @param vertexIndex Vertex index where the edge is added + * @param edge Edge to be added + * @throws IOException + */ + void addEdge(I vertexIndex, Edge edge) throws IOException; - /** - * Add an edge to a remote vertex - * - * @param vertexIndex Vertex index where the edge is added - * @param edge Edge to be added - * @throws IOException - */ - void addEdge(I vertexIndex, Edge edge) throws IOException; + /** + * Remove an edge on a remote vertex + * + * @param vertexIndex Vertex index where the edge is added + * @param destinationVertexIndex Edge vertex index to be removed + * @throws IOException + */ + void removeEdge(I vertexIndex, I destinationVertexIndex) throws IOException; - /** - * Remove an edge on a remote vertex - * - * @param vertexIndex Vertex index where the edge is added - * @param destinationVertexIndex Edge vertex index to be removed - * @throws IOException - */ - void removeEdge(I vertexIndex, I destinationVertexIndex) throws IOException; + /** + * Add a remote vertex + * + * @param vertex Vertex that will be added + * @throws IOException + */ + void addVertex(BasicVertex vertex) throws IOException; - /** - * Add a remote vertex - * - * @param vertex Vertex that will be added - * @throws IOException - */ - void addVertex(BasicVertex vertex) throws IOException; + /** + * Removed a remote vertex + * + * @param vertexIndex Vertex index representing vertex to be removed + * @throws IOException + */ + void removeVertex(I vertexIndex) throws IOException; - /** - * Removed a remote vertex - * - * @param vertexIndex Vertex index representing vertex to be removed - * @throws IOException - */ - void removeVertex(I vertexIndex) throws IOException; - - /** - * @return The name of this worker in the format "hostname:port". - */ - String getName(); + /** + * @return The name of this worker in the format "hostname:port". + */ + String getName(); } diff --git a/src/main/java/org/apache/giraph/comm/MsgList.java b/src/main/java/org/apache/giraph/comm/MsgList.java index 69579b0cd..bd10d01ee 100644 --- a/src/main/java/org/apache/giraph/comm/MsgList.java +++ b/src/main/java/org/apache/giraph/comm/MsgList.java @@ -27,22 +27,29 @@ * * @param message type */ -public class MsgList - extends ArrayListWritable { - /** Defining a layout version for a serializable class. */ - private static final long serialVersionUID = 100L; +public class MsgList extends ArrayListWritable { + /** Defining a layout version for a serializable class. */ + private static final long serialVersionUID = 100L; - public MsgList() { - super(); - } - - public MsgList(MsgList msgList) { - super(msgList); - } + /** + * Default constructor. + */ + public MsgList() { + super(); + } - @SuppressWarnings("unchecked") - @Override - public void setClass() { - setClass((Class) BspUtils.getMessageValueClass(getConf())); - } + /** + * Copy constructor. + * + * @param msgList List of messages for writing. + */ + public MsgList(MsgList msgList) { + super(msgList); + } + + @SuppressWarnings("unchecked") + @Override + public void setClass() { + setClass((Class) BspUtils.getMessageValueClass(getConf())); + } } diff --git a/src/main/java/org/apache/giraph/comm/RPCCommunications.java b/src/main/java/org/apache/giraph/comm/RPCCommunications.java index 152bbfa92..135418a72 100644 --- a/src/main/java/org/apache/giraph/comm/RPCCommunications.java +++ b/src/main/java/org/apache/giraph/comm/RPCCommunications.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.net.InetSocketAddress; -import java.net.UnknownHostException; /*if_not[HADOOP] else[HADOOP]*/ @@ -48,119 +47,155 @@ import org.apache.hadoop.ipc.RPC.Server; import org.apache.hadoop.mapreduce.Mapper; +/** + * Used to implement abstract {@link BasicRPCCommunications} methods. + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data + */ @SuppressWarnings("rawtypes") -public class RPCCommunications< - I extends WritableComparable, - V extends Writable, - E extends Writable, - M extends Writable> -/*if_not[HADOOP] -extends BasicRPCCommunications { -else[HADOOP]*/ - extends BasicRPCCommunications> { -/*end[HADOOP]*/ +public class RPCCommunications + /*if_not[HADOOP] + extends BasicRPCCommunications { + else[HADOOP]*/ + extends BasicRPCCommunications> { + /*end[HADOOP]*/ - /** Class logger */ - public static final Logger LOG = Logger.getLogger(RPCCommunications.class); + /** Class logger */ + public static final Logger LOG = Logger.getLogger(RPCCommunications.class); - public RPCCommunications(Mapper.Context context, - CentralizedServiceWorker service, - GraphState graphState) - throws IOException, UnknownHostException, InterruptedException { - super(context, service); - } + /** + * Constructor. + * + * @param context Context to be saved. + * @param service Server worker. + * @param graphState Graph state from infrastructure. + * @throws IOException + * @throws UnknownHostException + * @throws InterruptedException + */ + public RPCCommunications(Mapper.Context context, + CentralizedServiceWorker service, + GraphState graphState) throws + IOException, InterruptedException { + super(context, service); + } -/*if_not[HADOOP] + /*if_not[HADOOP] protected Object createJobToken() throws IOException { return null; } -else[HADOOP]*/ - protected Token createJobToken() throws IOException { - String localJobTokenFile = System.getenv().get( - UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION); - if (localJobTokenFile != null) { - Credentials credentials = - TokenCache.loadTokens(localJobTokenFile, conf); - return TokenCache.getJobToken(credentials); - } - return null; + else[HADOOP]*/ + /** + * Create the job token. + * + * @return Job token. + */ + protected Token createJobToken() throws IOException { + String localJobTokenFile = System.getenv().get( + UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION); + if (localJobTokenFile != null) { + Credentials credentials = + TokenCache.loadTokens(localJobTokenFile, conf); + return TokenCache.getJobToken(credentials); } -/*end[HADOOP]*/ + return null; + } + /*end[HADOOP]*/ - protected Server getRPCServer( - InetSocketAddress myAddress, int numHandlers, String jobId, -/*if_not[HADOOP] + /** + * Get the RPC server. + * + * @param myAddress My address. + * @param numHandlers Number of handler threads. + * @param jobId Job id. + * @param jt Jobtoken indentifier. + * @return RPC server. + */ + protected Server getRPCServer( + InetSocketAddress myAddress, int numHandlers, String jobId, + /*if_not[HADOOP] Object jt) throws IOException { return RPC.getServer(this, myAddress.getHostName(), myAddress.getPort(), numHandlers, false, conf); } -else[HADOOP]*/ - Token jt) throws IOException { - @SuppressWarnings("deprecation") - String hadoopSecurityAuthorization = - ServiceAuthorizationManager.SERVICE_AUTHORIZATION_CONFIG; - if (conf.getBoolean( - hadoopSecurityAuthorization, - false)) { - ServiceAuthorizationManager.refresh(conf, new BspPolicyProvider()); - } - JobTokenSecretManager jobTokenSecretManager = - new JobTokenSecretManager(); - if (jt != null) { //could be null in the case of some unit tests - jobTokenSecretManager.addTokenForJob(jobId, jt); - if (LOG.isInfoEnabled()) { - LOG.info("getRPCServer: Added jobToken " + jt); - } - } - return RPC.getServer(this, myAddress.getHostName(), myAddress.getPort(), - numHandlers, false, conf, jobTokenSecretManager); + else[HADOOP]*/ + Token jt) throws IOException { + @SuppressWarnings("deprecation") + String hadoopSecurityAuthorization = + ServiceAuthorizationManager.SERVICE_AUTHORIZATION_CONFIG; + if (conf.getBoolean( + hadoopSecurityAuthorization, + false)) { + ServiceAuthorizationManager.refresh(conf, new BspPolicyProvider()); } -/*end[HADOOP]*/ - - protected CommunicationsInterface getRPCProxy( - final InetSocketAddress addr, - String jobId, -/*if_not[HADOOP] - Object jt) -else[HADOOP]*/ - Token jt) -/*end[HADOOP]*/ - throws IOException, InterruptedException { - final Configuration config = new Configuration(conf); + JobTokenSecretManager jobTokenSecretManager = + new JobTokenSecretManager(); + if (jt != null) { //could be null in the case of some unit tests + jobTokenSecretManager.addTokenForJob(jobId, jt); + if (LOG.isInfoEnabled()) { + LOG.info("getRPCServer: Added jobToken " + jt); + } + } + return RPC.getServer(this, myAddress.getHostName(), myAddress.getPort(), + numHandlers, false, conf, jobTokenSecretManager); + } + /*end[HADOOP]*/ -/*if_not[HADOOP] + /** + * Get the RPC proxy. + * + * @param addr Address of the RPC server. + * @param jobId Job id. + * @param jt Job token. + * @return Proxy of the RPC server. + */ + protected CommunicationsInterface getRPCProxy( + final InetSocketAddress addr, + String jobId, + /*if_not[HADOOP] + Object jt) + else[HADOOP]*/ + Token jt) + /*end[HADOOP]*/ + throws IOException, InterruptedException { + final Configuration config = new Configuration(conf); + /*if_not[HADOOP] @SuppressWarnings("unchecked") CommunicationsInterface proxy = (CommunicationsInterface)RPC.getProxy( CommunicationsInterface.class, versionID, addr, config); return proxy; -else[HADOOP]*/ - if (jt == null) { - @SuppressWarnings("unchecked") - CommunicationsInterface proxy = - (CommunicationsInterface)RPC.getProxy( - CommunicationsInterface.class, versionID, addr, config); - return proxy; - } - jt.setService(new Text(addr.getAddress().getHostAddress() + ":" - + addr.getPort())); - UserGroupInformation current = UserGroupInformation.getCurrentUser(); - current.addToken(jt); - UserGroupInformation owner = - UserGroupInformation.createRemoteUser(jobId); - owner.addToken(jt); - @SuppressWarnings("unchecked") - CommunicationsInterface proxy = - owner.doAs(new PrivilegedExceptionAction< - CommunicationsInterface>() { - @Override - public CommunicationsInterface run() throws Exception { - // All methods in CommunicationsInterface will be used for RPC - return (CommunicationsInterface )RPC.getProxy( - CommunicationsInterface.class, versionID, addr, config); - } - }); - return proxy; -/*end[HADOOP]*/ + else[HADOOP]*/ + if (jt == null) { + @SuppressWarnings("unchecked") + CommunicationsInterface proxy = + (CommunicationsInterface) RPC.getProxy( + CommunicationsInterface.class, VERSION_ID, addr, config); + return proxy; } + jt.setService(new Text(addr.getAddress().getHostAddress() + ":" + + addr.getPort())); + UserGroupInformation current = UserGroupInformation.getCurrentUser(); + current.addToken(jt); + UserGroupInformation owner = + UserGroupInformation.createRemoteUser(jobId); + owner.addToken(jt); + @SuppressWarnings("unchecked") + CommunicationsInterface proxy = + owner.doAs(new PrivilegedExceptionAction< + CommunicationsInterface>() { + @Override + public CommunicationsInterface run() throws Exception { + // All methods in CommunicationsInterface will be used for RPC + return (CommunicationsInterface) RPC.getProxy( + CommunicationsInterface.class, VERSION_ID, addr, config); + } + }); + return proxy; + /*end[HADOOP]*/ + } } diff --git a/src/main/java/org/apache/giraph/comm/ServerInterface.java b/src/main/java/org/apache/giraph/comm/ServerInterface.java index 9ba95d173..cf732fc5a 100644 --- a/src/main/java/org/apache/giraph/comm/ServerInterface.java +++ b/src/main/java/org/apache/giraph/comm/ServerInterface.java @@ -26,45 +26,47 @@ import org.apache.hadoop.mapreduce.Mapper; /** - * Interface for message communication server + * Interface for message communication server. + * + * @param Vertex id + * @param Vertex value + * @param Edge value + * @param Message data */ @SuppressWarnings("rawtypes") public interface ServerInterface - extends Closeable, - WorkerCommunications { - /** - * Setup the server. - */ - void setup(); + V extends Writable, E extends Writable, M extends Writable> + extends Closeable, WorkerCommunications { + /** + * Setup the server. + */ + void setup(); - /** - * Move the in transition messages to the in messages for every vertex and - * add new connections to any newly appearing RPC proxies. - */ - void prepareSuperstep(); + /** + * Move the in transition messages to the in messages for every vertex and + * add new connections to any newly appearing RPC proxies. + */ + void prepareSuperstep(); - /** - * Flush all outgoing messages. This will synchronously ensure that all - * messages have been send and delivered prior to returning. - * - * @param context Context used to signal process - * @return Number of messages sent during the last superstep - * @throws IOException - */ - long flush(Mapper.Context context) throws IOException; + /** + * Flush all outgoing messages. This will synchronously ensure that all + * messages have been send and delivered prior to returning. + * + * @param context Context used to signal process + * @return Number of messages sent during the last superstep + * @throws IOException + */ + long flush(Mapper.Context context) throws IOException; - /** - * Closes all connections. - * - * @throws IOException - */ - void closeConnections() throws IOException; + /** + * Closes all connections. + * + * @throws IOException + */ + void closeConnections() throws IOException; - /** - * Shuts down. - */ - void close(); + /** + * Shuts down. + */ + void close(); } diff --git a/src/main/java/org/apache/giraph/comm/VertexIdMessages.java b/src/main/java/org/apache/giraph/comm/VertexIdMessages.java index 09380f6ed..06fb50ea0 100644 --- a/src/main/java/org/apache/giraph/comm/VertexIdMessages.java +++ b/src/main/java/org/apache/giraph/comm/VertexIdMessages.java @@ -32,65 +32,75 @@ * This object is only used for transporting list of vertices and their * respective messages to a destination RPC server. * - * @param vertex id - * @param message data + * @param Vertex id + * @param Message data */ @SuppressWarnings("rawtypes") public class VertexIdMessages - implements Writable, Configurable { - /** Vertex id */ - private I vertexId; - /** Message list corresponding to vertex id */ - private MsgList msgList; - /** Configuration from Configurable */ - private Configuration conf; + implements Writable, Configurable { + /** Vertex id */ + private I vertexId; + /** Message list corresponding to vertex id */ + private MsgList msgList; + /** Configuration from Configurable */ + private Configuration conf; - /** - * Reflective constructor. - */ - public VertexIdMessages() {} + /** + * Reflective constructor. + */ + public VertexIdMessages() { } - /** - * Constructor used with creating initial values. - * - * @param vertexId Vertex id to be sent - * @param msgList Mesage list for the vertex id to be sent - */ - public VertexIdMessages(I vertexId, MsgList msgList) { - this.vertexId = vertexId; - this.msgList = msgList; - } + /** + * Constructor used with creating initial values. + * + * @param vertexId Vertex id to be sent + * @param msgList Mesage list for the vertex id to be sent + */ + public VertexIdMessages(I vertexId, MsgList msgList) { + this.vertexId = vertexId; + this.msgList = msgList; + } - @Override - public void readFields(DataInput input) throws IOException { - vertexId = BspUtils.createVertexIndex(getConf()); - vertexId.readFields(input); - msgList = new MsgList(); - msgList.setConf(getConf()); - msgList.readFields(input); - } + @Override + public void readFields(DataInput input) throws IOException { + vertexId = BspUtils.createVertexIndex(getConf()); + vertexId.readFields(input); + msgList = new MsgList(); + msgList.setConf(getConf()); + msgList.readFields(input); + } - @Override - public void write(DataOutput output) throws IOException { - vertexId.write(output); - msgList.write(output); - } + @Override + public void write(DataOutput output) throws IOException { + vertexId.write(output); + msgList.write(output); + } - @Override - public Configuration getConf() { - return conf; - } + @Override + public Configuration getConf() { + return conf; + } - @Override - public void setConf(Configuration conf) { - this.conf = conf; - } + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } - public I getVertexId() { - return vertexId; - } + /** + * Get the vertex id. + * + * @return Vertex id. + */ + public I getVertexId() { + return vertexId; + } - public MsgList getMessageList() { - return msgList; - } - } + /** + * Get the message list. + * + * @return Message list. + */ + public MsgList getMessageList() { + return msgList; + } +} diff --git a/src/main/java/org/apache/giraph/comm/VertexIdMessagesList.java b/src/main/java/org/apache/giraph/comm/VertexIdMessagesList.java index 372d4328a..2449078ec 100644 --- a/src/main/java/org/apache/giraph/comm/VertexIdMessagesList.java +++ b/src/main/java/org/apache/giraph/comm/VertexIdMessagesList.java @@ -25,27 +25,35 @@ * Wrapper around {@link ArrayListWritable} that provides the list for * {@link VertexIdMessages}. * - * @param vertex id - * @param message data + * @param Vertex id + * @param Message data */ @SuppressWarnings("rawtypes") public class VertexIdMessagesList extends ArrayListWritable> { - /** Defining a layout version for a serializable class. */ - private static final long serialVersionUID = 100L; + M extends Writable> extends ArrayListWritable> { + /** Defining a layout version for a serializable class. */ + private static final long serialVersionUID = 100L; - public VertexIdMessagesList() { - super(); - } + /** + * Default constructor. + */ + public VertexIdMessagesList() { + super(); + } - public VertexIdMessagesList(VertexIdMessagesList vertexIdMessagesList) { - super(vertexIdMessagesList); - } + /** + * Copy constructor. + * + * @param vertexIdMessagesList List to be copied. + */ + public VertexIdMessagesList(VertexIdMessagesList vertexIdMessagesList) { + super(vertexIdMessagesList); + } - @SuppressWarnings("unchecked") - @Override - public void setClass() { - setClass((Class>) - (new VertexIdMessages()).getClass()); - } + @SuppressWarnings("unchecked") + @Override + public void setClass() { + setClass((Class>) + (new VertexIdMessages()).getClass()); + } } diff --git a/src/main/java/org/apache/giraph/comm/VertexList.java b/src/main/java/org/apache/giraph/comm/VertexList.java index 2e7e24960..5379f9f0b 100644 --- a/src/main/java/org/apache/giraph/comm/VertexList.java +++ b/src/main/java/org/apache/giraph/comm/VertexList.java @@ -33,24 +33,22 @@ * @param Message value */ @SuppressWarnings("rawtypes") -public class VertexList< - I extends WritableComparable, - V extends Writable, - E extends Writable, - M extends Writable> - extends ArrayListWritable> { - /** Defining a layout version for a serializable class. */ - private static final long serialVersionUID = 1000L; +public class VertexList + extends ArrayListWritable> { + /** Defining a layout version for a serializable class. */ + private static final long serialVersionUID = 1000L; - /** - * Default constructor for reflection - */ - public VertexList() {} + /** + * Default constructor for reflection + */ + public VertexList() { } - @SuppressWarnings("unchecked") - @Override - public void setClass() { - setClass((Class>) - BspUtils.getVertexClass(getConf())); - } + @SuppressWarnings("unchecked") + @Override + public void setClass() { + setClass((Class>) + BspUtils.getVertexClass(getConf())); + } } diff --git a/src/main/java/org/apache/giraph/comm/WorkerCommunications.java b/src/main/java/org/apache/giraph/comm/WorkerCommunications.java index 0abbc2f02..1d7e718ef 100644 --- a/src/main/java/org/apache/giraph/comm/WorkerCommunications.java +++ b/src/main/java/org/apache/giraph/comm/WorkerCommunications.java @@ -33,80 +33,78 @@ /** * Public interface for workers to do message communication * - * @param vertex id - * @param vertex value - * @param edge value - * @param message data + * @param Vertex id + * @param Vertex value + * @param Edge value + * @param Message data */ @SuppressWarnings("rawtypes") public interface WorkerCommunications { - /** - * Fix changes to the workers and the mapping between partitions and - * workers. - */ - void fixPartitionIdToSocketAddrMap(); + V extends Writable, E extends Writable, M extends Writable> { + /** + * Fix changes to the workers and the mapping between partitions and + * workers. + */ + void fixPartitionIdToSocketAddrMap(); - /** - * Sends a message to destination vertex. - * - * @param id - * @param msg - */ - void sendMessageReq(I id, M msg); + /** + * Sends a message to destination vertex. + * + * @param destVertexId Destination vertex id. + * @param message Message to send. + */ + void sendMessageReq(I destVertexId, M message); - /** - * Sends a partition to the appropriate partition owner - * - * @param workerInfo Owner the vertices belong to - * @param partition Partition to send - */ - void sendPartitionReq(WorkerInfo workerInfo, - Partition partition); + /** + * Sends a partition to the appropriate partition owner + * + * @param workerInfo Owner the vertices belong to + * @param partition Partition to send + */ + void sendPartitionReq(WorkerInfo workerInfo, + Partition partition); - /** - * Sends a request to the appropriate vertex range owner to add an edge - * - * @param vertexIndex Index of the vertex to get the request - * @param edge Edge to be added - * @throws IOException - */ - void addEdgeReq(I vertexIndex, Edge edge) throws IOException; + /** + * Sends a request to the appropriate vertex range owner to add an edge + * + * @param vertexIndex Index of the vertex to get the request + * @param edge Edge to be added + * @throws IOException + */ + void addEdgeReq(I vertexIndex, Edge edge) throws IOException; - /** - * Sends a request to the appropriate vertex range owner to remove an edge - * - * @param vertexIndex Index of the vertex to get the request - * @param destinationVertexIndex Index of the edge to be removed - * @throws IOException - */ - void removeEdgeReq(I vertexIndex, I destinationVertexIndex) - throws IOException; + /** + * Sends a request to the appropriate vertex range owner to remove an edge + * + * @param vertexIndex Index of the vertex to get the request + * @param destinationVertexIndex Index of the edge to be removed + * @throws IOException + */ + void removeEdgeReq(I vertexIndex, I destinationVertexIndex) + throws IOException; - /** - * Sends a request to the appropriate vertex range owner to add a vertex - * - * @param vertex Vertex to be added - * @throws IOException - */ - void addVertexReq(BasicVertex vertex) throws IOException; + /** + * Sends a request to the appropriate vertex range owner to add a vertex + * + * @param vertex Vertex to be added + * @throws IOException + */ + void addVertexReq(BasicVertex vertex) throws IOException; - /** - * Sends a request to the appropriate vertex range owner to remove a vertex - * - * @param vertexIndex Index of the vertex to be removed - * @throws IOException - */ - void removeVertexReq(I vertexIndex) throws IOException; + /** + * Sends a request to the appropriate vertex range owner to remove a vertex + * + * @param vertexIndex Index of the vertex to be removed + * @throws IOException + */ + void removeVertexReq(I vertexIndex) throws IOException; - /** - * Get the vertices that were sent in the last iteration. After getting - * the map, the user should synchronize with it to insure it - * is thread-safe. - * - * @return map of vertex ranges to vertices - */ - Map>> getInPartitionVertexMap(); + /** + * Get the vertices that were sent in the last iteration. After getting + * the map, the user should synchronize with it to insure it + * is thread-safe. + * + * @return map of vertex ranges to vertices + */ + Map>> getInPartitionVertexMap(); } diff --git a/src/main/java/org/apache/giraph/comm/package-info.java b/src/main/java/org/apache/giraph/comm/package-info.java new file mode 100644 index 000000000..00152a92b --- /dev/null +++ b/src/main/java/org/apache/giraph/comm/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/** + * Package of communication related objects, RPC service. + */ +package org.apache.giraph.comm; diff --git a/src/main/java/org/apache/giraph/examples/ConnectedComponentsVertex.java b/src/main/java/org/apache/giraph/examples/ConnectedComponentsVertex.java index 5f26c65ae..f88d43da1 100644 --- a/src/main/java/org/apache/giraph/examples/ConnectedComponentsVertex.java +++ b/src/main/java/org/apache/giraph/examples/ConnectedComponentsVertex.java @@ -1,20 +1,20 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one -* or more contributor license agreements. See the NOTICE file -* distributed with this work for additional information -* regarding copyright ownership. The ASF licenses this file -* to you under the Apache License, Version 2.0 (the -* "License"); you may not use this file except in compliance -* with the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.giraph.examples; @@ -25,72 +25,72 @@ import java.util.Iterator; /** - * Implementation of the HCC algorithm that identifies connected components and assigns each - * vertex its "component identifier" (the smallest vertex id in the component) + * Implementation of the HCC algorithm that identifies connected components and + * assigns each vertex its "component identifier" (the smallest vertex id + * in the component) * - * The idea behind the algorithm is very simple: propagate the smallest vertex id along the - * edges to all vertices of a connected component. The number of supersteps necessary is - * equal to the length of the maximum diameter of all components + 1 + * The idea behind the algorithm is very simple: propagate the smallest + * vertex id along the edges to all vertices of a connected component. The + * number of supersteps necessary is equal to the length of the maximum + * diameter of all components + 1 * - * The original Hadoop-based variant of this algorithm was proposed by Kang, Charalampos - * Tsourakakis and Faloutsos in "PEGASUS: Mining Peta-Scale Graphs", 2010 + * The original Hadoop-based variant of this algorithm was proposed by Kang, + * Charalampos, Tsourakakis and Faloutsos in + * "PEGASUS: Mining Peta-Scale Graphs", 2010 * * http://www.cs.cmu.edu/~ukang/papers/PegasusKAIS.pdf */ public class ConnectedComponentsVertex extends IntIntNullIntVertex { + /** + * Propagates the smallest vertex id to all neighbors. Will always choose to + * halt and only reactivate if a smaller id has been sent to it. + * + * @param messages Iterator of messages from the previous superstep. + * @throws IOException + */ + @Override + public void compute(Iterator messages) throws IOException { + int currentComponent = getVertexValue().get(); - /** - * Propagates the smallest vertex id to all neighbors. Will always choose to halt and only - * reactivate if a smaller id has been sent to it. - * - * @param messages - * @throws IOException - */ - @Override - public void compute(Iterator messages) throws IOException { - - int currentComponent = getVertexValue().get(); - - // first superstep is special, because we can simply look at the neighbors - if (getSuperstep() == 0) { - for (Iterator edges = iterator(); edges.hasNext();) { - int neighbor = edges.next().get(); - if (neighbor < currentComponent) { - currentComponent = neighbor; - } - } - // only need to send value if it is not the own id - if (currentComponent != getVertexValue().get()) { - setVertexValue(new IntWritable(currentComponent)); - for (Iterator edges = iterator(); - edges.hasNext();) { - int neighbor = edges.next().get(); - if (neighbor > currentComponent) { - sendMsg(new IntWritable(neighbor), getVertexValue()); - } - } - } - - voteToHalt(); - return; + // First superstep is special, because we can simply look at the neighbors + if (getSuperstep() == 0) { + for (Iterator edges = iterator(); edges.hasNext();) { + int neighbor = edges.next().get(); + if (neighbor < currentComponent) { + currentComponent = neighbor; } - - boolean changed = false; - // did we get a smaller id ? - while (messages.hasNext()) { - int candidateComponent = messages.next().get(); - if (candidateComponent < currentComponent) { - currentComponent = candidateComponent; - changed = true; - } + } + // Only need to send value if it is not the own id + if (currentComponent != getVertexValue().get()) { + setVertexValue(new IntWritable(currentComponent)); + for (Iterator edges = iterator(); + edges.hasNext();) { + int neighbor = edges.next().get(); + if (neighbor > currentComponent) { + sendMsg(new IntWritable(neighbor), getVertexValue()); + } } + } - // propagate new component id to the neighbors - if (changed) { - setVertexValue(new IntWritable(currentComponent)); - sendMsgToAllEdges(getVertexValue()); - } - voteToHalt(); + voteToHalt(); + return; } + boolean changed = false; + // did we get a smaller id ? + while (messages.hasNext()) { + int candidateComponent = messages.next().get(); + if (candidateComponent < currentComponent) { + currentComponent = candidateComponent; + changed = true; + } + } + + // propagate new component id to the neighbors + if (changed) { + setVertexValue(new IntWritable(currentComponent)); + sendMsgToAllEdges(getVertexValue()); + } + voteToHalt(); + } } diff --git a/src/main/java/org/apache/giraph/examples/GeneratedVertexInputFormat.java b/src/main/java/org/apache/giraph/examples/GeneratedVertexInputFormat.java index 983b5c3a7..6d46d8142 100644 --- a/src/main/java/org/apache/giraph/examples/GeneratedVertexInputFormat.java +++ b/src/main/java/org/apache/giraph/examples/GeneratedVertexInputFormat.java @@ -32,22 +32,25 @@ /** * This VertexInputFormat is meant for testing/debugging. It simply generates * some vertex data that can be consumed by test applications. + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data */ @SuppressWarnings("rawtypes") public abstract class GeneratedVertexInputFormat< - I extends WritableComparable, V extends Writable, E extends Writable, - M extends Writable> - extends VertexInputFormat { - - @Override - public List getSplits(JobContext context, int numWorkers) - throws IOException, InterruptedException { - // This is meaningless, the VertexReader will generate all the test - // data. - List inputSplitList = new ArrayList(); - for (int i = 0; i < numWorkers; ++i) { - inputSplitList.add(new BspInputSplit(i, numWorkers)); - } - return inputSplitList; + I extends WritableComparable, V extends Writable, E extends Writable, + M extends Writable> extends VertexInputFormat { + @Override + public List getSplits(JobContext context, int numWorkers) + throws IOException, InterruptedException { + // This is meaningless, the VertexReader will generate all the test + // data. + List inputSplitList = new ArrayList(); + for (int i = 0; i < numWorkers; ++i) { + inputSplitList.add(new BspInputSplit(i, numWorkers)); } + return inputSplitList; + } } diff --git a/src/main/java/org/apache/giraph/examples/GeneratedVertexReader.java b/src/main/java/org/apache/giraph/examples/GeneratedVertexReader.java index 83f98f15e..4fdde637e 100644 --- a/src/main/java/org/apache/giraph/examples/GeneratedVertexReader.java +++ b/src/main/java/org/apache/giraph/examples/GeneratedVertexReader.java @@ -34,53 +34,58 @@ * @param Vertex index value * @param Vertex value * @param Edge value + * @param Message data */ @SuppressWarnings("rawtypes") public abstract class GeneratedVertexReader< - I extends WritableComparable, V extends Writable, E extends Writable, - M extends Writable> - implements VertexReader { - /** Records read so far */ - protected long recordsRead = 0; - /** Total records to read (on this split alone) */ - protected long totalRecords = 0; - /** The input split from initialize(). */ - protected BspInputSplit inputSplit = null; - /** Reverse the id order? */ - protected boolean reverseIdOrder; + I extends WritableComparable, V extends Writable, E extends Writable, + M extends Writable> implements VertexReader { + /** Vertices produced by this reader */ + public static final String READER_VERTICES = + "GeneratedVertexReader.reader_vertices"; + /** Default vertices produced by this reader */ + public static final long DEFAULT_READER_VERTICES = 10; + /** Reverse the order of the vertices? */ + public static final String REVERSE_ID_ORDER = + "GeneratedVertexReader.reverseIdOrder"; + /** Default ordering is not reversed */ + public static final boolean DEAFULT_REVERSE_ID_ORDER = false; + /** Records read so far */ + protected long recordsRead = 0; + /** Total records to read (on this split alone) */ + protected long totalRecords = 0; + /** The input split from initialize(). */ + protected BspInputSplit inputSplit = null; + /** Reverse the id order? */ + protected boolean reverseIdOrder; + /** Saved configuration */ + protected Configuration configuration = null; - protected Configuration configuration = null; + /** + * Default constructor for reflection. + */ + public GeneratedVertexReader() { + } - public static final String READER_VERTICES = - "GeneratedVertexReader.reader_vertices"; - public static final long DEFAULT_READER_VERTICES = 10; - public static final String REVERSE_ID_ORDER = - "GeneratedVertexReader.reverseIdOrder"; - public static final boolean DEAFULT_REVERSE_ID_ORDER = false; + @Override + public final void initialize(InputSplit inputSplit, + TaskAttemptContext context) throws IOException { + configuration = context.getConfiguration(); + totalRecords = configuration.getLong( + GeneratedVertexReader.READER_VERTICES, + GeneratedVertexReader.DEFAULT_READER_VERTICES); + reverseIdOrder = configuration.getBoolean( + GeneratedVertexReader.REVERSE_ID_ORDER, + GeneratedVertexReader.DEAFULT_REVERSE_ID_ORDER); + this.inputSplit = (BspInputSplit) inputSplit; + } - public GeneratedVertexReader() { - } + @Override + public void close() throws IOException { + } - @Override - final public void initialize(InputSplit inputSplit, - TaskAttemptContext context) - throws IOException { - configuration = context.getConfiguration(); - totalRecords = configuration.getLong( - GeneratedVertexReader.READER_VERTICES, - GeneratedVertexReader.DEFAULT_READER_VERTICES); - reverseIdOrder = configuration.getBoolean( - GeneratedVertexReader.REVERSE_ID_ORDER, - GeneratedVertexReader.DEAFULT_REVERSE_ID_ORDER); - this.inputSplit = (BspInputSplit) inputSplit; - } - - @Override - public void close() throws IOException { - } - - @Override - final public float getProgress() throws IOException { - return recordsRead * 100.0f / totalRecords; - } + @Override + public final float getProgress() throws IOException { + return recordsRead * 100.0f / totalRecords; + } } diff --git a/src/main/java/org/apache/giraph/examples/IntIntNullIntTextInputFormat.java b/src/main/java/org/apache/giraph/examples/IntIntNullIntTextInputFormat.java index d6edfc0ce..eafcbef54 100644 --- a/src/main/java/org/apache/giraph/examples/IntIntNullIntTextInputFormat.java +++ b/src/main/java/org/apache/giraph/examples/IntIntNullIntTextInputFormat.java @@ -1,20 +1,20 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one -* or more contributor license agreements. See the NOTICE file -* distributed with this work for additional information -* regarding copyright ownership. The ASF licenses this file -* to you under the Apache License, Version 2.0 (the -* "License"); you may not use this file except in compliance -* with the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.giraph.examples; @@ -37,61 +37,68 @@ import java.util.regex.Pattern; /** - * Simple text-based {@link org.apache.giraph.graph.VertexInputFormat} for unweighted - * graphs with int ids. + * Simple text-based {@link org.apache.giraph.graph.VertexInputFormat} for + * unweighted graphs with int ids. * * Each line consists of: vertex neighbor1 neighbor2 ... */ public class IntIntNullIntTextInputFormat extends - TextVertexInputFormat { - - @Override - public VertexReader - createVertexReader(InputSplit split, TaskAttemptContext context) - throws IOException { - return new IntIntNullIntVertexReader( - textInputFormat.createRecordReader(split, context)); - } + TextVertexInputFormat { - public static class IntIntNullIntVertexReader extends - TextVertexInputFormat.TextVertexReader { + @Override + public VertexReader + createVertexReader(InputSplit split, TaskAttemptContext context) + throws IOException { + return new IntIntNullIntVertexReader( + textInputFormat.createRecordReader(split, context)); + } - private static final Pattern SEPARATOR = Pattern.compile("[\t ]"); + /** + * Vertex reader associated with {@link IntIntNullIntTextInputFormat}. + */ + public static class IntIntNullIntVertexReader extends + TextVertexInputFormat.TextVertexReader { + /** Separator of the vertex and neighbors */ + private static final Pattern SEPARATOR = Pattern.compile("[\t ]"); - public IntIntNullIntVertexReader(RecordReader - lineReader) { - super(lineReader); - } - - @Override - public BasicVertex - getCurrentVertex() throws IOException, InterruptedException { - BasicVertex - vertex = BspUtils.createVertex(getContext().getConfiguration()); + /** + * Constructor with the line reader. + * + * @param lineReader Internal line reader. + */ + public IntIntNullIntVertexReader(RecordReader + lineReader) { + super(lineReader); + } - String[] tokens = SEPARATOR.split(getRecordReader() - .getCurrentValue().toString()); - Map edges = - Maps.newHashMapWithExpectedSize(tokens.length - 1); - for (int n = 1; n < tokens.length; n++) { - edges.put(new IntWritable(Integer.parseInt(tokens[n])), - NullWritable.get()); - } + @Override + public BasicVertex + getCurrentVertex() throws IOException, InterruptedException { + BasicVertex + vertex = BspUtils.createVertex(getContext().getConfiguration()); - IntWritable vertexId = new IntWritable(Integer.parseInt(tokens[0])); - vertex.initialize(vertexId, vertexId, edges, - Lists.newArrayList()); + String[] tokens = SEPARATOR.split(getRecordReader() + .getCurrentValue().toString()); + Map edges = + Maps.newHashMapWithExpectedSize(tokens.length - 1); + for (int n = 1; n < tokens.length; n++) { + edges.put(new IntWritable(Integer.parseInt(tokens[n])), + NullWritable.get()); + } - return vertex; - } + IntWritable vertexId = new IntWritable(Integer.parseInt(tokens[0])); + vertex.initialize(vertexId, vertexId, edges, + Lists.newArrayList()); - @Override - public boolean nextVertex() throws IOException, InterruptedException { - return getRecordReader().nextKeyValue(); - } + return vertex; } + @Override + public boolean nextVertex() throws IOException, InterruptedException { + return getRecordReader().nextKeyValue(); + } + } } diff --git a/src/main/java/org/apache/giraph/examples/LongSumAggregator.java b/src/main/java/org/apache/giraph/examples/LongSumAggregator.java index c0811d206..7ee29e536 100644 --- a/src/main/java/org/apache/giraph/examples/LongSumAggregator.java +++ b/src/main/java/org/apache/giraph/examples/LongSumAggregator.java @@ -26,30 +26,35 @@ * Aggregator for summing up values. */ public class LongSumAggregator implements Aggregator { - /** Internal sum */ - private long sum = 0; - - public void aggregate(long value) { - sum += value; - } - - @Override - public void aggregate(LongWritable value) { - sum += value.get(); - } - - @Override - public void setAggregatedValue(LongWritable value) { - sum = value.get(); - } - - @Override - public LongWritable getAggregatedValue() { - return new LongWritable(sum); - } - - @Override - public LongWritable createAggregatedValue() { - return new LongWritable(); - } + /** Internal sum */ + private long sum = 0; + + /** + * Aggregate with a primitive long. + * + * @param value Long value to aggregate. + */ + public void aggregate(long value) { + sum += value; + } + + @Override + public void aggregate(LongWritable value) { + sum += value.get(); + } + + @Override + public void setAggregatedValue(LongWritable value) { + sum = value.get(); + } + + @Override + public LongWritable getAggregatedValue() { + return new LongWritable(sum); + } + + @Override + public LongWritable createAggregatedValue() { + return new LongWritable(); + } } diff --git a/src/main/java/org/apache/giraph/examples/MaxAggregator.java b/src/main/java/org/apache/giraph/examples/MaxAggregator.java index 4e7a9f3cb..aae7c07f2 100644 --- a/src/main/java/org/apache/giraph/examples/MaxAggregator.java +++ b/src/main/java/org/apache/giraph/examples/MaxAggregator.java @@ -25,29 +25,31 @@ /** * Aggregator for getting max value. * - **/ - + */ public class MaxAggregator implements Aggregator { - + /** Saved maximum value */ private double max = Double.MIN_VALUE; + @Override public void aggregate(DoubleWritable value) { - double val = value.get(); - if (val > max) { - max = val; - } + double val = value.get(); + if (val > max) { + max = val; + } } + @Override public void setAggregatedValue(DoubleWritable value) { - max = value.get(); + max = value.get(); } + @Override public DoubleWritable getAggregatedValue() { - return new DoubleWritable(max); + return new DoubleWritable(max); } + @Override public DoubleWritable createAggregatedValue() { - return new DoubleWritable(); + return new DoubleWritable(); } - } diff --git a/src/main/java/org/apache/giraph/examples/MinAggregator.java b/src/main/java/org/apache/giraph/examples/MinAggregator.java index 1714c942f..a0843cdb7 100644 --- a/src/main/java/org/apache/giraph/examples/MinAggregator.java +++ b/src/main/java/org/apache/giraph/examples/MinAggregator.java @@ -24,30 +24,32 @@ /** * Aggregator for getting min value. - * - **/ - + */ public class MinAggregator implements Aggregator { - + /** Internal aggregator */ private double min = Double.MAX_VALUE; + @Override public void aggregate(DoubleWritable value) { - double val = value.get(); - if (val < min) { - min = val; - } + double val = value.get(); + if (val < min) { + min = val; + } } + @Override public void setAggregatedValue(DoubleWritable value) { - min = value.get(); + min = value.get(); } + @Override public DoubleWritable getAggregatedValue() { - return new DoubleWritable(min); + return new DoubleWritable(min); } + @Override public DoubleWritable createAggregatedValue() { - return new DoubleWritable(); + return new DoubleWritable(); } } diff --git a/src/main/java/org/apache/giraph/examples/MinimumIntCombiner.java b/src/main/java/org/apache/giraph/examples/MinimumIntCombiner.java index 1758388d7..e6554a67e 100644 --- a/src/main/java/org/apache/giraph/examples/MinimumIntCombiner.java +++ b/src/main/java/org/apache/giraph/examples/MinimumIntCombiner.java @@ -1,20 +1,20 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one -* or more contributor license agreements. See the NOTICE file -* distributed with this work for additional information -* regarding copyright ownership. The ASF licenses this file -* to you under the Apache License, Version 2.0 (the -* "License"); you may not use this file except in compliance -* with the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.giraph.examples; @@ -29,20 +29,19 @@ * {@link VertexCombiner} that finds the minimum {@link IntWritable} */ public class MinimumIntCombiner - extends VertexCombiner { - - @Override - public Iterable combine(IntWritable target, - Iterable messages) throws IOException { - int minimum = Integer.MAX_VALUE; - for (IntWritable message : messages) { - if (message.get() < minimum) { - minimum = message.get(); - } - } - List value = new ArrayList(); - value.add(new IntWritable(minimum)); - - return value; + extends VertexCombiner { + @Override + public Iterable combine(IntWritable target, + Iterable messages) throws IOException { + int minimum = Integer.MAX_VALUE; + for (IntWritable message : messages) { + if (message.get() < minimum) { + minimum = message.get(); + } } + List value = new ArrayList(); + value.add(new IntWritable(minimum)); + + return value; + } } diff --git a/src/main/java/org/apache/giraph/examples/SimpleAggregatorWriter.java b/src/main/java/org/apache/giraph/examples/SimpleAggregatorWriter.java index afdcb2717..06c125063 100644 --- a/src/main/java/org/apache/giraph/examples/SimpleAggregatorWriter.java +++ b/src/main/java/org/apache/giraph/examples/SimpleAggregatorWriter.java @@ -37,32 +37,37 @@ * directory. */ public class SimpleAggregatorWriter implements AggregatorWriter { - /** the name of the file we wrote to */ - public static String filename; - private FSDataOutputStream output; - - @SuppressWarnings("rawtypes") - @Override - public void initialize(Context context, long applicationAttempt) - throws IOException { - filename = "aggregatedValues_"+applicationAttempt; - Path p = new Path(filename); - FileSystem fs = FileSystem.get(context.getConfiguration()); - output = fs.create(p, true); - } + /** Name of the file we wrote to */ + private static String FILENAME; + /** Saved output stream to write to */ + private FSDataOutputStream output; - @Override - public void writeAggregator(Map> map, - long superstep) throws IOException { + public static String getFilename() { + return FILENAME; + } - for (Entry> aggregator: map.entrySet()) { - aggregator.getValue().getAggregatedValue().write(output); - } - output.flush(); - } + @SuppressWarnings("rawtypes") + @Override + public void initialize(Context context, long applicationAttempt) + throws IOException { + FILENAME = "aggregatedValues_" + applicationAttempt; + Path p = new Path(FILENAME); + FileSystem fs = FileSystem.get(context.getConfiguration()); + output = fs.create(p, true); + } - @Override - public void close() throws IOException { - output.close(); + @Override + public void writeAggregator(Map> map, + long superstep) throws IOException { + + for (Entry> aggregator: map.entrySet()) { + aggregator.getValue().getAggregatedValue().write(output); } + output.flush(); + } + + @Override + public void close() throws IOException { + output.close(); + } } diff --git a/src/main/java/org/apache/giraph/examples/SimpleCheckpointVertex.java b/src/main/java/org/apache/giraph/examples/SimpleCheckpointVertex.java index 59d8bdc3a..89ed0d197 100644 --- a/src/main/java/org/apache/giraph/examples/SimpleCheckpointVertex.java +++ b/src/main/java/org/apache/giraph/examples/SimpleCheckpointVertex.java @@ -18,8 +18,14 @@ package org.apache.giraph.examples; -import org.apache.commons.cli.*; -import org.apache.giraph.graph.*; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.CommandLineParser; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.PosixParser; +import org.apache.giraph.graph.EdgeListVertex; +import org.apache.giraph.graph.GiraphJob; +import org.apache.giraph.graph.WorkerContext; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.FloatWritable; @@ -38,208 +44,222 @@ * can also test automated checkpoint restarts. */ public class SimpleCheckpointVertex extends - EdgeListVertex - implements Tool { - private static Logger LOG = - Logger.getLogger(SimpleCheckpointVertex.class); - /** Configuration */ - private Configuration conf; - /** Which superstep to cause the worker to fail */ - public final int faultingSuperstep = 4; - /** Vertex id to fault on */ - public final long faultingVertexId = 1; - /** Dynamically set number of supersteps */ - public static final String SUPERSTEP_COUNT = - "simpleCheckpointVertex.superstepCount"; - /** Should fault? */ - public static final String ENABLE_FAULT= - "simpleCheckpointVertex.enableFault"; + EdgeListVertex + implements Tool { + /** Which superstep to cause the worker to fail */ + public static final int FAULTING_SUPERSTEP = 4; + /** Vertex id to fault on */ + public static final long FAULTING_VERTEX_ID = 1; + /** Dynamically set number of supersteps */ + public static final String SUPERSTEP_COUNT = + "simpleCheckpointVertex.superstepCount"; + /** Should fault? */ + public static final String ENABLE_FAULT = + "simpleCheckpointVertex.enableFault"; + /** Class logger */ + private static final Logger LOG = + Logger.getLogger(SimpleCheckpointVertex.class); + /** Configuration */ + private Configuration conf; + + @Override + public void compute(Iterator msgIterator) { + SimpleCheckpointVertexWorkerContext workerContext = + (SimpleCheckpointVertexWorkerContext) getWorkerContext(); + + LongSumAggregator sumAggregator = (LongSumAggregator) + getAggregator(LongSumAggregator.class.getName()); + + boolean enableFault = workerContext.getEnableFault(); + int supersteps = workerContext.getSupersteps(); + + if (enableFault && (getSuperstep() == FAULTING_SUPERSTEP) && + (getContext().getTaskAttemptID().getId() == 0) && + (getVertexId().get() == FAULTING_VERTEX_ID)) { + LOG.info("compute: Forced a fault on the first " + + "attempt of superstep " + + FAULTING_SUPERSTEP + " and vertex id " + + FAULTING_VERTEX_ID); + System.exit(-1); + } + if (getSuperstep() > supersteps) { + voteToHalt(); + return; + } + LOG.info("compute: " + sumAggregator); + sumAggregator.aggregate(getVertexId().get()); + LOG.info("compute: sum = " + + sumAggregator.getAggregatedValue().get() + + " for vertex " + getVertexId()); + float msgValue = 0.0f; + while (msgIterator.hasNext()) { + float curMsgValue = msgIterator.next().get(); + msgValue += curMsgValue; + LOG.info("compute: got msgValue = " + curMsgValue + + " for vertex " + getVertexId() + + " on superstep " + getSuperstep()); + } + int vertexValue = getVertexValue().get(); + setVertexValue(new IntWritable(vertexValue + (int) msgValue)); + LOG.info("compute: vertex " + getVertexId() + + " has value " + getVertexValue() + + " on superstep " + getSuperstep()); + for (LongWritable targetVertexId : this) { + FloatWritable edgeValue = getEdgeValue(targetVertexId); + LOG.info("compute: vertex " + getVertexId() + + " sending edgeValue " + edgeValue + + " vertexValue " + vertexValue + + " total " + (edgeValue.get() + + (float) vertexValue) + + " to vertex " + targetVertexId + + " on superstep " + getSuperstep()); + edgeValue.set(edgeValue.get() + (float) vertexValue); + addEdge(targetVertexId, edgeValue); + sendMsg(targetVertexId, new FloatWritable(edgeValue.get())); + } + } + + /** + * Worker context associated with {@link SimpleCheckpointVertex}. + */ + public static class SimpleCheckpointVertexWorkerContext + extends WorkerContext { + /** Filename to indicate whether a fault was found */ + public static final String FAULT_FILE = "/tmp/faultFile"; + /** User can access this after the application finishes if local */ + private static long FINAL_SUM; + /** Number of supersteps to run (6 by default) */ + private int supersteps = 6; + /** Enable the fault at the particular vertex id and superstep? */ + private boolean enableFault = false; + + public static long getFinalSum() { + return FINAL_SUM; + } @Override - public void compute(Iterator msgIterator) { - SimpleCheckpointVertexWorkerContext workerContext = - (SimpleCheckpointVertexWorkerContext) getWorkerContext(); - - LongSumAggregator sumAggregator = (LongSumAggregator) - getAggregator(LongSumAggregator.class.getName()); - - boolean enableFault = workerContext.getEnableFault(); - int supersteps = workerContext.getSupersteps(); - - if (enableFault && (getSuperstep() == faultingSuperstep) && - (getContext().getTaskAttemptID().getId() == 0) && - (getVertexId().get() == faultingVertexId)) { - System.out.println("compute: Forced a fault on the first " + - "attempt of superstep " + - faultingSuperstep + " and vertex id " + - faultingVertexId); - System.exit(-1); - } - if (getSuperstep() > supersteps) { - voteToHalt(); - return; - } - System.out.println("compute: " + sumAggregator); - sumAggregator.aggregate(getVertexId().get()); - System.out.println("compute: sum = " + - sumAggregator.getAggregatedValue().get() + - " for vertex " + getVertexId()); - float msgValue = 0.0f; - while (msgIterator.hasNext()) { - float curMsgValue = msgIterator.next().get(); - msgValue += curMsgValue; - System.out.println("compute: got msgValue = " + curMsgValue + - " for vertex " + getVertexId() + - " on superstep " + getSuperstep()); - } - int vertexValue = getVertexValue().get(); - setVertexValue(new IntWritable(vertexValue + (int) msgValue)); - System.out.println("compute: vertex " + getVertexId() + - " has value " + getVertexValue() + - " on superstep " + getSuperstep()); - for (LongWritable targetVertexId : this) { - FloatWritable edgeValue = getEdgeValue(targetVertexId); - System.out.println("compute: vertex " + getVertexId() + - " sending edgeValue " + edgeValue + - " vertexValue " + vertexValue + - " total " + (edgeValue.get() + - (float) vertexValue) + - " to vertex " + targetVertexId + - " on superstep " + getSuperstep()); - edgeValue.set(edgeValue.get() + (float) vertexValue); - addEdge(targetVertexId, edgeValue); - sendMsg(targetVertexId, new FloatWritable(edgeValue.get())); - } - } - - public static class SimpleCheckpointVertexWorkerContext - extends WorkerContext { - /** User can access this after the application finishes if local */ - public static long finalSum; - /** Number of supersteps to run (6 by default) */ - private int supersteps = 6; - /** Filename to indicate whether a fault was found */ - public final String faultFile = "/tmp/faultFile"; - /** Enable the fault at the particular vertex id and superstep? */ - private boolean enableFault = false; - - @Override - public void preApplication() - throws InstantiationException, IllegalAccessException { - registerAggregator(LongSumAggregator.class.getName(), - LongSumAggregator.class); - LongSumAggregator sumAggregator = (LongSumAggregator) - getAggregator(LongSumAggregator.class.getName()); - sumAggregator.setAggregatedValue(new LongWritable(0)); - supersteps = getContext().getConfiguration() - .getInt(SUPERSTEP_COUNT, supersteps); - enableFault = getContext().getConfiguration() - .getBoolean(ENABLE_FAULT, false); - } - - @Override - public void postApplication() { - LongSumAggregator sumAggregator = (LongSumAggregator) - getAggregator(LongSumAggregator.class.getName()); - finalSum = sumAggregator.getAggregatedValue().get(); - LOG.info("finalSum="+ finalSum); - } - - @Override - public void preSuperstep() { - useAggregator(LongSumAggregator.class.getName()); - } - - @Override - public void postSuperstep() { } - - public int getSupersteps() { - return this.supersteps; - } - - public boolean getEnableFault() { - return this.enableFault; - } + public void preApplication() + throws InstantiationException, IllegalAccessException { + registerAggregator(LongSumAggregator.class.getName(), + LongSumAggregator.class); + LongSumAggregator sumAggregator = (LongSumAggregator) + getAggregator(LongSumAggregator.class.getName()); + sumAggregator.setAggregatedValue(new LongWritable(0)); + supersteps = getContext().getConfiguration() + .getInt(SUPERSTEP_COUNT, supersteps); + enableFault = getContext().getConfiguration() + .getBoolean(ENABLE_FAULT, false); } @Override - public int run(String[] args) throws Exception { - Options options = new Options(); - options.addOption("h", "help", false, "Help"); - options.addOption("v", "verbose", false, "Verbose"); - options.addOption("w", - "workers", - true, - "Number of workers"); - options.addOption("s", - "supersteps", - true, - "Supersteps to execute before finishing"); - options.addOption("w", - "workers", - true, - "Minimum number of workers"); - options.addOption("o", - "outputDirectory", - true, - "Output directory"); - HelpFormatter formatter = new HelpFormatter(); - if (args.length == 0) { - formatter.printHelp(getClass().getName(), options, true); - return 0; - } - CommandLineParser parser = new PosixParser(); - CommandLine cmd = parser.parse(options, args); - if (cmd.hasOption('h')) { - formatter.printHelp(getClass().getName(), options, true); - return 0; - } - if (!cmd.hasOption('w')) { - System.out.println("Need to choose the number of workers (-w)"); - return -1; - } - if (!cmd.hasOption('o')) { - System.out.println("Need to set the output directory (-o)"); - return -1; - } - - GiraphJob bspJob = new GiraphJob(getConf(), getClass().getName()); - bspJob.setVertexClass(getClass()); - bspJob.setVertexInputFormatClass(GeneratedVertexInputFormat.class); - bspJob.setVertexOutputFormatClass(SimpleTextVertexOutputFormat.class); - bspJob.setWorkerContextClass(SimpleCheckpointVertexWorkerContext.class); - int minWorkers = Integer.parseInt(cmd.getOptionValue('w')); - int maxWorkers = Integer.parseInt(cmd.getOptionValue('w')); - bspJob.setWorkerConfiguration(minWorkers, maxWorkers, 100.0f); - - FileOutputFormat.setOutputPath(bspJob, - new Path(cmd.getOptionValue('o'))); - boolean verbose = false; - if (cmd.hasOption('v')) { - verbose = true; - } - if (cmd.hasOption('s')) { - getConf().setInt(SUPERSTEP_COUNT, - Integer.parseInt(cmd.getOptionValue('s'))); - } - if (bspJob.run(verbose) == true) { - return 0; - } else { - return -1; - } - } - - public static void main(String[] args) throws Exception { - System.exit(ToolRunner.run(new SimpleCheckpointVertex(), args)); + public void postApplication() { + LongSumAggregator sumAggregator = (LongSumAggregator) + getAggregator(LongSumAggregator.class.getName()); + FINAL_SUM = sumAggregator.getAggregatedValue().get(); + LOG.info("FINAL_SUM=" + FINAL_SUM); } @Override - public Configuration getConf() { - return conf; + public void preSuperstep() { + useAggregator(LongSumAggregator.class.getName()); } @Override - public void setConf(Configuration conf) { - this.conf = conf; + public void postSuperstep() { } + + public int getSupersteps() { + return this.supersteps; + } + + public boolean getEnableFault() { + return this.enableFault; } + } + + @Override + public int run(String[] args) throws Exception { + Options options = new Options(); + options.addOption("h", "help", false, "Help"); + options.addOption("v", "verbose", false, "Verbose"); + options.addOption("w", + "workers", + true, + "Number of workers"); + options.addOption("s", + "supersteps", + true, + "Supersteps to execute before finishing"); + options.addOption("w", + "workers", + true, + "Minimum number of workers"); + options.addOption("o", + "outputDirectory", + true, + "Output directory"); + HelpFormatter formatter = new HelpFormatter(); + if (args.length == 0) { + formatter.printHelp(getClass().getName(), options, true); + return 0; + } + CommandLineParser parser = new PosixParser(); + CommandLine cmd = parser.parse(options, args); + if (cmd.hasOption('h')) { + formatter.printHelp(getClass().getName(), options, true); + return 0; + } + if (!cmd.hasOption('w')) { + LOG.info("Need to choose the number of workers (-w)"); + return -1; + } + if (!cmd.hasOption('o')) { + LOG.info("Need to set the output directory (-o)"); + return -1; + } + + GiraphJob bspJob = new GiraphJob(getConf(), getClass().getName()); + bspJob.setVertexClass(getClass()); + bspJob.setVertexInputFormatClass(GeneratedVertexInputFormat.class); + bspJob.setVertexOutputFormatClass(SimpleTextVertexOutputFormat.class); + bspJob.setWorkerContextClass(SimpleCheckpointVertexWorkerContext.class); + int minWorkers = Integer.parseInt(cmd.getOptionValue('w')); + int maxWorkers = Integer.parseInt(cmd.getOptionValue('w')); + bspJob.setWorkerConfiguration(minWorkers, maxWorkers, 100.0f); + + FileOutputFormat.setOutputPath(bspJob, + new Path(cmd.getOptionValue('o'))); + boolean verbose = false; + if (cmd.hasOption('v')) { + verbose = true; + } + if (cmd.hasOption('s')) { + getConf().setInt(SUPERSTEP_COUNT, + Integer.parseInt(cmd.getOptionValue('s'))); + } + if (bspJob.run(verbose)) { + return 0; + } else { + return -1; + } + } + + /** + * Executable from the command line. + * + * @param args Command line args. + * @throws Exception + */ + public static void main(String[] args) throws Exception { + System.exit(ToolRunner.run(new SimpleCheckpointVertex(), args)); + } + + @Override + public Configuration getConf() { + return conf; + } + + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } } diff --git a/src/main/java/org/apache/giraph/examples/SimpleCombinerVertex.java b/src/main/java/org/apache/giraph/examples/SimpleCombinerVertex.java index 1f96c5d85..193e8e7a2 100644 --- a/src/main/java/org/apache/giraph/examples/SimpleCombinerVertex.java +++ b/src/main/java/org/apache/giraph/examples/SimpleCombinerVertex.java @@ -23,6 +23,7 @@ import org.apache.hadoop.io.FloatWritable; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.LongWritable; +import org.apache.log4j.Logger; import org.apache.giraph.graph.EdgeListVertex; @@ -30,36 +31,38 @@ * Test whether messages can go through a combiner. */ public class SimpleCombinerVertex extends - EdgeListVertex { - @Override - public void compute(Iterator msgIterator) { - if (getVertexId().equals(new LongWritable(2))) { - sendMsg(new LongWritable(1), new IntWritable(101)); - sendMsg(new LongWritable(1), new IntWritable(102)); - sendMsg(new LongWritable(1), new IntWritable(103)); - } - if (!getVertexId().equals(new LongWritable(1))) { - voteToHalt(); - } - else { - // Check the messages - int sum = 0; - int num = 0; - while (msgIterator != null && msgIterator.hasNext()) { - sum += msgIterator.next().get(); - num++; - } - System.out.println("TestCombinerVertex: Received a sum of " + sum + - " (should have 306 with a single message value)"); + EdgeListVertex { + /** Class logger */ + private static Logger LOG = Logger.getLogger(SimpleCombinerVertex.class); - if (num == 1 && sum == 306) { - voteToHalt(); - } - } - if (getSuperstep() > 3) { - throw new IllegalStateException( - "TestCombinerVertex: Vertex 1 failed to receive " + - "messages in time"); - } + @Override + public void compute(Iterator msgIterator) { + if (getVertexId().equals(new LongWritable(2))) { + sendMsg(new LongWritable(1), new IntWritable(101)); + sendMsg(new LongWritable(1), new IntWritable(102)); + sendMsg(new LongWritable(1), new IntWritable(103)); } + if (!getVertexId().equals(new LongWritable(1))) { + voteToHalt(); + } else { + // Check the messages + int sum = 0; + int num = 0; + while (msgIterator != null && msgIterator.hasNext()) { + sum += msgIterator.next().get(); + num++; + } + LOG.info("TestCombinerVertex: Received a sum of " + sum + + " (should have 306 with a single message value)"); + + if (num == 1 && sum == 306) { + voteToHalt(); + } + } + if (getSuperstep() > 3) { + throw new IllegalStateException( + "TestCombinerVertex: Vertex 1 failed to receive " + + "messages in time"); + } + } } diff --git a/src/main/java/org/apache/giraph/examples/SimpleFailVertex.java b/src/main/java/org/apache/giraph/examples/SimpleFailVertex.java index 71117c060..105333b36 100644 --- a/src/main/java/org/apache/giraph/examples/SimpleFailVertex.java +++ b/src/main/java/org/apache/giraph/examples/SimpleFailVertex.java @@ -22,47 +22,50 @@ import org.apache.hadoop.io.DoubleWritable; import org.apache.hadoop.io.FloatWritable; import org.apache.hadoop.io.LongWritable; +import org.apache.log4j.Logger; import java.util.Iterator; /** * Vertex to allow unit testing of failure detection */ -public class SimpleFailVertex extends - EdgeListVertex { +public class SimpleFailVertex extends EdgeListVertex< + LongWritable, DoubleWritable, FloatWritable, DoubleWritable> { + /** Class logger */ + private static Logger LOG = Logger.getLogger(SimpleFailVertex.class); + /** TODO: Change this behavior to WorkerContext */ + private static long SUPERSTEP = 0; - static long superstep = 0; - - @Override - public void compute(Iterator msgIterator) { - if (getSuperstep() >= 1) { - double sum = 0; - while (msgIterator.hasNext()) { - sum += msgIterator.next().get(); - } - DoubleWritable vertexValue = - new DoubleWritable((0.15f / getNumVertices()) + 0.85f * sum); - setVertexValue(vertexValue); - if (getSuperstep() < 30) { - if (getSuperstep() == 20) { - if (getVertexId().get() == 10L) { - try { - Thread.sleep(2000); - } catch (InterruptedException e) { - } - System.exit(1); - } else if (getSuperstep() - superstep > 10) { - return; - } - } - long edges = getNumOutEdges(); - sendMsgToAllEdges( - new DoubleWritable(getVertexValue().get() / edges)); - } else { - voteToHalt(); + @Override + public void compute(Iterator msgIterator) { + if (getSuperstep() >= 1) { + double sum = 0; + while (msgIterator.hasNext()) { + sum += msgIterator.next().get(); + } + DoubleWritable vertexValue = + new DoubleWritable((0.15f / getNumVertices()) + 0.85f * sum); + setVertexValue(vertexValue); + if (getSuperstep() < 30) { + if (getSuperstep() == 20) { + if (getVertexId().get() == 10L) { + try { + Thread.sleep(2000); + } catch (InterruptedException e) { + LOG.info("Sleep interrupted ", e); } - superstep = getSuperstep(); + System.exit(1); + } else if (getSuperstep() - SUPERSTEP > 10) { + return; + } } + long edges = getNumOutEdges(); + sendMsgToAllEdges( + new DoubleWritable(getVertexValue().get() / edges)); + } else { + voteToHalt(); + } + SUPERSTEP = getSuperstep(); } + } } diff --git a/src/main/java/org/apache/giraph/examples/SimpleMsgVertex.java b/src/main/java/org/apache/giraph/examples/SimpleMsgVertex.java index 83a35bcd8..2f2848d38 100644 --- a/src/main/java/org/apache/giraph/examples/SimpleMsgVertex.java +++ b/src/main/java/org/apache/giraph/examples/SimpleMsgVertex.java @@ -23,6 +23,7 @@ import org.apache.hadoop.io.FloatWritable; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.LongWritable; +import org.apache.log4j.Logger; import org.apache.giraph.graph.EdgeListVertex; @@ -30,34 +31,35 @@ * Test whether messages can be sent and received by vertices. */ public class SimpleMsgVertex extends - EdgeListVertex { - @Override - public void compute(Iterator msgIterator) { - if (getVertexId().equals(new LongWritable(2))) { - sendMsg(new LongWritable(1), new IntWritable(101)); - sendMsg(new LongWritable(1), new IntWritable(102)); - sendMsg(new LongWritable(1), new IntWritable(103)); - } - if (!getVertexId().equals(new LongWritable(1))) { - voteToHalt(); - } - else { - /* Check the messages */ - int sum = 0; - while (msgIterator != null && msgIterator.hasNext()) { - sum += msgIterator.next().get(); - } - System.out.println("TestMsgVertex: Received a sum of " + sum + - " (will stop on 306)"); + EdgeListVertex { + /** Class logger */ + private static Logger LOG = Logger.getLogger(SimpleMsgVertex.class); + @Override + public void compute(Iterator msgIterator) { + if (getVertexId().equals(new LongWritable(2))) { + sendMsg(new LongWritable(1), new IntWritable(101)); + sendMsg(new LongWritable(1), new IntWritable(102)); + sendMsg(new LongWritable(1), new IntWritable(103)); + } + if (!getVertexId().equals(new LongWritable(1))) { + voteToHalt(); + } else { + /* Check the messages */ + int sum = 0; + while (msgIterator != null && msgIterator.hasNext()) { + sum += msgIterator.next().get(); + } + LOG.info("TestMsgVertex: Received a sum of " + sum + + " (will stop on 306)"); - if (sum == 306) { - voteToHalt(); - } - } - if (getSuperstep() > 3) { - System.err.println("TestMsgVertex: Vertex 1 failed to receive " + - "messages in time"); - voteToHalt(); - } + if (sum == 306) { + voteToHalt(); + } + } + if (getSuperstep() > 3) { + System.err.println("TestMsgVertex: Vertex 1 failed to receive " + + "messages in time"); + voteToHalt(); } + } } diff --git a/src/main/java/org/apache/giraph/examples/SimpleMutateGraphVertex.java b/src/main/java/org/apache/giraph/examples/SimpleMutateGraphVertex.java index ce23af53b..1fd5e43d3 100644 --- a/src/main/java/org/apache/giraph/examples/SimpleMutateGraphVertex.java +++ b/src/main/java/org/apache/giraph/examples/SimpleMutateGraphVertex.java @@ -34,163 +34,171 @@ /** * Vertex to allow unit testing of graph mutations. */ -public class SimpleMutateGraphVertex extends - EdgeListVertex { - /** Maximum number of ranges for vertex ids */ - private long maxRanges = 100; - /** Class logger */ - private static Logger LOG = - Logger.getLogger(SimpleMutateGraphVertex.class); - - /** - * Unless we create a ridiculous number of vertices , we should not - * collide within a vertex range defined by this method. - * - * @return Starting vertex id of the range - */ - private long rangeVertexIdStart(int range) { - return (Long.MAX_VALUE / maxRanges) * range; +public class SimpleMutateGraphVertex extends EdgeListVertex< + LongWritable, DoubleWritable, FloatWritable, DoubleWritable> { + /** Class logger */ + private static Logger LOG = + Logger.getLogger(SimpleMutateGraphVertex.class); + /** Maximum number of ranges for vertex ids */ + private long maxRanges = 100; + + + /** + * Unless we create a ridiculous number of vertices , we should not + * collide within a vertex range defined by this method. + * + * @param range Range index + * @return Starting vertex id of the range + */ + private long rangeVertexIdStart(int range) { + return (Long.MAX_VALUE / maxRanges) * range; + } + + @Override + public void compute(Iterator msgIterator) + throws IOException { + SimpleMutateGraphVertexWorkerContext workerContext = + (SimpleMutateGraphVertexWorkerContext) getWorkerContext(); + if (getSuperstep() == 0) { + LOG.debug("Reached superstep " + getSuperstep()); + } else if (getSuperstep() == 1) { + // Send messages to vertices that are sure not to exist + // (creating them) + LongWritable destVertexId = + new LongWritable(rangeVertexIdStart(1) + getVertexId().get()); + sendMsg(destVertexId, new DoubleWritable(0.0)); + } else if (getSuperstep() == 2) { + LOG.debug("Reached superstep " + getSuperstep()); + } else if (getSuperstep() == 3) { + long vertexCount = workerContext.getVertexCount(); + if (vertexCount * 2 != getNumVertices()) { + throw new IllegalStateException( + "Impossible to have " + getNumVertices() + + " vertices when should have " + vertexCount * 2 + + " on superstep " + getSuperstep()); + } + long edgeCount = workerContext.getEdgeCount(); + if (edgeCount != getNumEdges()) { + throw new IllegalStateException( + "Impossible to have " + getNumEdges() + + " edges when should have " + edgeCount + + " on superstep " + getSuperstep()); + } + // Create vertices that are sure not to exist (doubling vertices) + LongWritable vertexIndex = + new LongWritable(rangeVertexIdStart(3) + getVertexId().get()); + BasicVertex vertex = + instantiateVertex(vertexIndex, null, null, null); + addVertexRequest(vertex); + // Add edges to those remote vertices as well + addEdgeRequest(vertexIndex, + new Edge( + getVertexId(), new FloatWritable(0.0f))); + } else if (getSuperstep() == 4) { + LOG.debug("Reached superstep " + getSuperstep()); + } else if (getSuperstep() == 5) { + long vertexCount = workerContext.getVertexCount(); + if (vertexCount * 2 != getNumVertices()) { + throw new IllegalStateException( + "Impossible to have " + getNumVertices() + + " when should have " + vertexCount * 2 + + " on superstep " + getSuperstep()); + } + long edgeCount = workerContext.getEdgeCount(); + if (edgeCount + vertexCount != getNumEdges()) { + throw new IllegalStateException( + "Impossible to have " + getNumEdges() + + " edges when should have " + edgeCount + vertexCount + + " on superstep " + getSuperstep()); + } + // Remove the edges created in superstep 3 + LongWritable vertexIndex = + new LongWritable(rangeVertexIdStart(3) + getVertexId().get()); + workerContext.increaseEdgesRemoved(); + removeEdgeRequest(vertexIndex, getVertexId()); + } else if (getSuperstep() == 6) { + // Remove all the vertices created in superstep 3 + if (getVertexId().compareTo( + new LongWritable(rangeVertexIdStart(3))) >= 0) { + removeVertexRequest(getVertexId()); + } + } else if (getSuperstep() == 7) { + long origEdgeCount = workerContext.getOrigEdgeCount(); + if (origEdgeCount != getNumEdges()) { + throw new IllegalStateException( + "Impossible to have " + getNumEdges() + + " edges when should have " + origEdgeCount + + " on superstep " + getSuperstep()); + } + } else if (getSuperstep() == 8) { + long vertexCount = workerContext.getVertexCount(); + if (vertexCount / 2 != getNumVertices()) { + throw new IllegalStateException( + "Impossible to have " + getNumVertices() + + " vertices when should have " + vertexCount / 2 + + " on superstep " + getSuperstep()); + } + } else { + voteToHalt(); } + } + + /** + * Worker context used with {@link SimpleMutateGraphVertex}. + */ + public static class SimpleMutateGraphVertexWorkerContext + extends WorkerContext { + /** Cached vertex count */ + private long vertexCount; + /** Cached edge count */ + private long edgeCount; + /** Original number of edges */ + private long origEdgeCount; + /** Number of edges removed during superstep */ + private int edgesRemoved = 0; + + @Override + public void preApplication() + throws InstantiationException, IllegalAccessException { } + + @Override + public void postApplication() { } + + @Override + public void preSuperstep() { } @Override - public void compute(Iterator msgIterator) - throws IOException { - - SimpleMutateGraphVertexWorkerContext workerContext = - (SimpleMutateGraphVertexWorkerContext) getWorkerContext(); - if (getSuperstep() == 0) { - } else if (getSuperstep() == 1) { - // Send messages to vertices that are sure not to exist - // (creating them) - LongWritable destVertexId = - new LongWritable(rangeVertexIdStart(1) + getVertexId().get()); - sendMsg(destVertexId, new DoubleWritable(0.0)); - } else if (getSuperstep() == 2) { - } else if (getSuperstep() == 3) { - long vertexCount = workerContext.getVertexCount(); - if (vertexCount * 2 != getNumVertices()) { - throw new IllegalStateException( - "Impossible to have " + getNumVertices() + - " vertices when should have " + vertexCount * 2 + - " on superstep " + getSuperstep()); - } - long edgeCount = workerContext.getEdgeCount(); - if (edgeCount != getNumEdges()) { - throw new IllegalStateException( - "Impossible to have " + getNumEdges() + - " edges when should have " + edgeCount + - " on superstep " + getSuperstep()); - } - // Create vertices that are sure not to exist (doubling vertices) - LongWritable vertexIndex = - new LongWritable(rangeVertexIdStart(3) + getVertexId().get()); - BasicVertex vertex = - instantiateVertex(vertexIndex, null, null, null); - addVertexRequest(vertex); - // Add edges to those remote vertices as well - addEdgeRequest(vertexIndex, - new Edge( - getVertexId(), new FloatWritable(0.0f))); - } else if (getSuperstep() == 4) { - } else if (getSuperstep() == 5) { - long vertexCount = workerContext.getVertexCount(); - if (vertexCount * 2 != getNumVertices()) { - throw new IllegalStateException( - "Impossible to have " + getNumVertices() + - " when should have " + vertexCount * 2 + - " on superstep " + getSuperstep()); - } - long edgeCount = workerContext.getEdgeCount(); - if (edgeCount + vertexCount != getNumEdges()) { - throw new IllegalStateException( - "Impossible to have " + getNumEdges() + - " edges when should have " + edgeCount + vertexCount + - " on superstep " + getSuperstep()); - } - // Remove the edges created in superstep 3 - LongWritable vertexIndex = - new LongWritable(rangeVertexIdStart(3) + getVertexId().get()); - workerContext.increaseEdgesRemoved(); - removeEdgeRequest(vertexIndex, getVertexId()); - } else if (getSuperstep() == 6) { - // Remove all the vertices created in superstep 3 - if (getVertexId().compareTo( - new LongWritable(rangeVertexIdStart(3))) >= 0) { - removeVertexRequest(getVertexId()); - } - } else if (getSuperstep() == 7) { - long orig_edge_count = workerContext.getOrigEdgeCount(); - if (orig_edge_count != getNumEdges()) { - throw new IllegalStateException( - "Impossible to have " + getNumEdges() + - " edges when should have " + orig_edge_count + - " on superstep " + getSuperstep()); - } - } else if (getSuperstep() == 8) { - long vertex_count = workerContext.getVertexCount(); - if (vertex_count / 2 != getNumVertices()) { - throw new IllegalStateException( - "Impossible to have " + getNumVertices() + - " vertices when should have " + vertex_count / 2 + - " on superstep " + getSuperstep()); - } - } - else { - voteToHalt(); - } + public void postSuperstep() { + vertexCount = getNumVertices(); + edgeCount = getNumEdges(); + if (getSuperstep() == 1) { + origEdgeCount = edgeCount; + } + LOG.info("Got " + vertexCount + " vertices, " + + edgeCount + " edges on superstep " + + getSuperstep()); + LOG.info("Removed " + edgesRemoved); + edgesRemoved = 0; } - public static class SimpleMutateGraphVertexWorkerContext - extends WorkerContext { - /** Cached vertex count */ - private long vertexCount; - /** Cached edge count */ - private long edgeCount; - /** Original number of edges */ - private long origEdgeCount; - /** Number of edges removed during superstep */ - private int edgesRemoved = 0; - - @Override - public void preApplication() - throws InstantiationException, IllegalAccessException { } - - @Override - public void postApplication() { } - - @Override - public void preSuperstep() { } - - @Override - public void postSuperstep() { - vertexCount = getNumVertices(); - edgeCount = getNumEdges(); - if (getSuperstep() == 1) { - origEdgeCount = edgeCount; - } - LOG.info("Got " + vertexCount + " vertices, " + - edgeCount + " edges on superstep " + - getSuperstep()); - LOG.info("Removed " + edgesRemoved); - edgesRemoved = 0; - } - - public long getVertexCount() { - return vertexCount; - } - - public long getEdgeCount() { - return edgeCount; - } - - public long getOrigEdgeCount() { - return origEdgeCount; - } - - public void increaseEdgesRemoved() { - this.edgesRemoved++; - } + public long getVertexCount() { + return vertexCount; + } + + public long getEdgeCount() { + return edgeCount; + } + + public long getOrigEdgeCount() { + return origEdgeCount; + } + + /** + * Increase the number of edges removed by one. + */ + public void increaseEdgesRemoved() { + this.edgesRemoved++; } + } } diff --git a/src/main/java/org/apache/giraph/examples/SimplePageRankVertex.java b/src/main/java/org/apache/giraph/examples/SimplePageRankVertex.java index 5e37075d7..82089d516 100644 --- a/src/main/java/org/apache/giraph/examples/SimplePageRankVertex.java +++ b/src/main/java/org/apache/giraph/examples/SimplePageRankVertex.java @@ -45,205 +45,230 @@ * Demonstrates the basic Pregel PageRank implementation. */ public class SimplePageRankVertex extends LongDoubleFloatDoubleVertex { - /** Number of supersteps for this test */ - public static final int MAX_SUPERSTEPS = 30; - /** Logger */ - private static final Logger LOG = - Logger.getLogger(SimplePageRankVertex.class); + /** Number of supersteps for this test */ + public static final int MAX_SUPERSTEPS = 30; + /** Logger */ + private static final Logger LOG = + Logger.getLogger(SimplePageRankVertex.class); - @Override - public void compute(Iterator msgIterator) { - LongSumAggregator sumAggreg = (LongSumAggregator) getAggregator("sum"); - MinAggregator minAggreg = (MinAggregator) getAggregator("min"); - MaxAggregator maxAggreg = (MaxAggregator) getAggregator("max"); - if (getSuperstep() >= 1) { - double sum = 0; - while (msgIterator.hasNext()) { - sum += msgIterator.next().get(); - } - DoubleWritable vertexValue = - new DoubleWritable((0.15f / getNumVertices()) + 0.85f * sum); - setVertexValue(vertexValue); - maxAggreg.aggregate(vertexValue); - minAggreg.aggregate(vertexValue); - sumAggreg.aggregate(1L); - LOG.info(getVertexId() + ": PageRank=" + vertexValue + - " max=" + maxAggreg.getAggregatedValue() + - " min=" + minAggreg.getAggregatedValue()); - } + @Override + public void compute(Iterator msgIterator) { + LongSumAggregator sumAggreg = (LongSumAggregator) getAggregator("sum"); + MinAggregator minAggreg = (MinAggregator) getAggregator("min"); + MaxAggregator maxAggreg = (MaxAggregator) getAggregator("max"); + if (getSuperstep() >= 1) { + double sum = 0; + while (msgIterator.hasNext()) { + sum += msgIterator.next().get(); + } + DoubleWritable vertexValue = + new DoubleWritable((0.15f / getNumVertices()) + 0.85f * sum); + setVertexValue(vertexValue); + maxAggreg.aggregate(vertexValue); + minAggreg.aggregate(vertexValue); + sumAggreg.aggregate(1L); + LOG.info(getVertexId() + ": PageRank=" + vertexValue + + " max=" + maxAggreg.getAggregatedValue() + + " min=" + minAggreg.getAggregatedValue()); + } - if (getSuperstep() < MAX_SUPERSTEPS) { - long edges = getNumOutEdges(); - sendMsgToAllEdges( - new DoubleWritable(getVertexValue().get() / edges)); - } else { - voteToHalt(); - } + if (getSuperstep() < MAX_SUPERSTEPS) { + long edges = getNumOutEdges(); + sendMsgToAllEdges( + new DoubleWritable(getVertexValue().get() / edges)); + } else { + voteToHalt(); } + } + + /** + * Worker context used with {@link SimplePageRankVertex}. + */ + public static class SimplePageRankVertexWorkerContext extends + WorkerContext { + /** Final max value for verification for local jobs */ + private static double FINAL_MAX; + /** Final min value for verification for local jobs */ + private static double FINAL_MIN; + /** Final sum value for verification for local jobs */ + private static long FINAL_SUM; - public static class SimplePageRankVertexWorkerContext extends - WorkerContext { - - public static double finalMax, finalMin; - public static long finalSum; - - @Override - public void preApplication() - throws InstantiationException, IllegalAccessException { - - registerAggregator("sum", LongSumAggregator.class); - registerAggregator("min", MinAggregator.class); - registerAggregator("max", MaxAggregator.class); - } - - @Override - public void postApplication() { - - LongSumAggregator sumAggreg = - (LongSumAggregator) getAggregator("sum"); - MinAggregator minAggreg = - (MinAggregator) getAggregator("min"); - MaxAggregator maxAggreg = - (MaxAggregator) getAggregator("max"); - - finalSum = sumAggreg.getAggregatedValue().get(); - finalMax = maxAggreg.getAggregatedValue().get(); - finalMin = minAggreg.getAggregatedValue().get(); - - LOG.info("aggregatedNumVertices=" + finalSum); - LOG.info("aggregatedMaxPageRank=" + finalMax); - LOG.info("aggregatedMinPageRank=" + finalMin); - } - - @Override - public void preSuperstep() { - - LongSumAggregator sumAggreg = - (LongSumAggregator) getAggregator("sum"); - MinAggregator minAggreg = - (MinAggregator) getAggregator("min"); - MaxAggregator maxAggreg = - (MaxAggregator) getAggregator("max"); - - if (getSuperstep() >= 3) { - LOG.info("aggregatedNumVertices=" + - sumAggreg.getAggregatedValue() + - " NumVertices=" + getNumVertices()); - if (sumAggreg.getAggregatedValue().get() != getNumVertices()) { - throw new RuntimeException("wrong value of SumAggreg: " + - sumAggreg.getAggregatedValue() + ", should be: " + - getNumVertices()); - } - DoubleWritable maxPagerank = - (DoubleWritable) maxAggreg.getAggregatedValue(); - LOG.info("aggregatedMaxPageRank=" + maxPagerank.get()); - DoubleWritable minPagerank = - (DoubleWritable) minAggreg.getAggregatedValue(); - LOG.info("aggregatedMinPageRank=" + minPagerank.get()); - } - useAggregator("sum"); - useAggregator("min"); - useAggregator("max"); - sumAggreg.setAggregatedValue(new LongWritable(0L)); - } - - @Override - public void postSuperstep() { } + public static double getFinalMax() { + return FINAL_MAX; } - - /** - * Simple VertexReader that supports {@link SimplePageRankVertex} - */ - public static class SimplePageRankVertexReader extends - GeneratedVertexReader { - /** Class logger */ - private static final Logger LOG = - Logger.getLogger(SimplePageRankVertexReader.class); - - public SimplePageRankVertexReader() { - super(); - } - @Override - public boolean nextVertex() { - return totalRecords > recordsRead; - } + public static double getFinalMin() { + return FINAL_MIN; + } - @Override - public BasicVertex - getCurrentVertex() throws IOException { - BasicVertex - vertex = BspUtils.createVertex(configuration); - - LongWritable vertexId = new LongWritable( - (inputSplit.getSplitIndex() * totalRecords) + recordsRead); - DoubleWritable vertexValue = new DoubleWritable(vertexId.get() * 10d); - long destVertexId = - (vertexId.get() + 1) % - (inputSplit.getNumSplits() * totalRecords); - float edgeValue = vertexId.get() * 100f; - Map edges = Maps.newHashMap(); - edges.put(new LongWritable(destVertexId), new FloatWritable(edgeValue)); - vertex.initialize(vertexId, vertexValue, edges, null); - ++recordsRead; - if (LOG.isInfoEnabled()) { - LOG.info("next: Return vertexId=" + vertex.getVertexId().get() + - ", vertexValue=" + vertex.getVertexValue() + - ", destinationId=" + destVertexId + ", edgeValue=" + edgeValue); - } - return vertex; - } + public static long getFinalSum() { + return FINAL_SUM; } - /** - * Simple VertexInputFormat that supports {@link SimplePageRankVertex} - */ - public static class SimplePageRankVertexInputFormat extends - GeneratedVertexInputFormat { - @Override - public VertexReader - createVertexReader(InputSplit split, - TaskAttemptContext context) - throws IOException { - return new SimplePageRankVertexReader(); + @Override + public void preApplication() + throws InstantiationException, IllegalAccessException { + registerAggregator("sum", LongSumAggregator.class); + registerAggregator("min", MinAggregator.class); + registerAggregator("max", MaxAggregator.class); + } + + @Override + public void postApplication() { + + LongSumAggregator sumAggreg = + (LongSumAggregator) getAggregator("sum"); + MinAggregator minAggreg = + (MinAggregator) getAggregator("min"); + MaxAggregator maxAggreg = + (MaxAggregator) getAggregator("max"); + + FINAL_SUM = sumAggreg.getAggregatedValue().get(); + FINAL_MAX = maxAggreg.getAggregatedValue().get(); + FINAL_MIN = minAggreg.getAggregatedValue().get(); + + LOG.info("aggregatedNumVertices=" + FINAL_SUM); + LOG.info("aggregatedMaxPageRank=" + FINAL_MAX); + LOG.info("aggregatedMinPageRank=" + FINAL_MIN); + } + + @Override + public void preSuperstep() { + + LongSumAggregator sumAggreg = + (LongSumAggregator) getAggregator("sum"); + MinAggregator minAggreg = + (MinAggregator) getAggregator("min"); + MaxAggregator maxAggreg = + (MaxAggregator) getAggregator("max"); + + if (getSuperstep() >= 3) { + LOG.info("aggregatedNumVertices=" + + sumAggreg.getAggregatedValue() + + " NumVertices=" + getNumVertices()); + if (sumAggreg.getAggregatedValue().get() != getNumVertices()) { + throw new RuntimeException("wrong value of SumAggreg: " + + sumAggreg.getAggregatedValue() + ", should be: " + + getNumVertices()); } + DoubleWritable maxPagerank = + (DoubleWritable) maxAggreg.getAggregatedValue(); + LOG.info("aggregatedMaxPageRank=" + maxPagerank.get()); + DoubleWritable minPagerank = + (DoubleWritable) minAggreg.getAggregatedValue(); + LOG.info("aggregatedMinPageRank=" + minPagerank.get()); + } + useAggregator("sum"); + useAggregator("min"); + useAggregator("max"); + sumAggreg.setAggregatedValue(new LongWritable(0L)); } + @Override + public void postSuperstep() { } + } + + /** + * Simple VertexReader that supports {@link SimplePageRankVertex} + */ + public static class SimplePageRankVertexReader extends + GeneratedVertexReader { + /** Class logger */ + private static final Logger LOG = + Logger.getLogger(SimplePageRankVertexReader.class); + /** - * Simple VertexWriter that supports {@link SimplePageRankVertex} + * Constructor. */ - public static class SimplePageRankVertexWriter extends - TextVertexWriter { - public SimplePageRankVertexWriter( - RecordWriter lineRecordWriter) { - super(lineRecordWriter); - } + public SimplePageRankVertexReader() { + super(); + } - @Override - public void writeVertex( - BasicVertex vertex) - throws IOException, InterruptedException { - getRecordWriter().write( - new Text(vertex.getVertexId().toString()), - new Text(vertex.getVertexValue().toString())); - } + @Override + public boolean nextVertex() { + return totalRecords > recordsRead; + } + + @Override + public BasicVertex + getCurrentVertex() throws IOException { + BasicVertex + vertex = BspUtils.createVertex(configuration); + + LongWritable vertexId = new LongWritable( + (inputSplit.getSplitIndex() * totalRecords) + recordsRead); + DoubleWritable vertexValue = new DoubleWritable(vertexId.get() * 10d); + long destVertexId = + (vertexId.get() + 1) % + (inputSplit.getNumSplits() * totalRecords); + float edgeValue = vertexId.get() * 100f; + Map edges = Maps.newHashMap(); + edges.put(new LongWritable(destVertexId), new FloatWritable(edgeValue)); + vertex.initialize(vertexId, vertexValue, edges, null); + ++recordsRead; + if (LOG.isInfoEnabled()) { + LOG.info("next: Return vertexId=" + vertex.getVertexId().get() + + ", vertexValue=" + vertex.getVertexValue() + + ", destinationId=" + destVertexId + ", edgeValue=" + edgeValue); + } + return vertex; + } + } + + /** + * Simple VertexInputFormat that supports {@link SimplePageRankVertex} + */ + public static class SimplePageRankVertexInputFormat extends + GeneratedVertexInputFormat { + @Override + public VertexReader createVertexReader(InputSplit split, + TaskAttemptContext context) + throws IOException { + return new SimplePageRankVertexReader(); } + } + /** + * Simple VertexWriter that supports {@link SimplePageRankVertex} + */ + public static class SimplePageRankVertexWriter extends + TextVertexWriter { /** - * Simple VertexOutputFormat that supports {@link SimplePageRankVertex} + * Constructor with line writer. + * + * @param lineRecordWriter Line writer that will do the writing. */ - public static class SimplePageRankVertexOutputFormat extends - TextVertexOutputFormat { - - @Override - public VertexWriter - createVertexWriter(TaskAttemptContext context) - throws IOException, InterruptedException { - RecordWriter recordWriter = - textOutputFormat.getRecordWriter(context); - return new SimplePageRankVertexWriter(recordWriter); - } + public SimplePageRankVertexWriter( + RecordWriter lineRecordWriter) { + super(lineRecordWriter); + } + + @Override + public void writeVertex( + BasicVertex vertex) + throws IOException, InterruptedException { + getRecordWriter().write( + new Text(vertex.getVertexId().toString()), + new Text(vertex.getVertexValue().toString())); + } + } + + /** + * Simple VertexOutputFormat that supports {@link SimplePageRankVertex} + */ + public static class SimplePageRankVertexOutputFormat extends + TextVertexOutputFormat { + @Override + public VertexWriter + createVertexWriter(TaskAttemptContext context) + throws IOException, InterruptedException { + RecordWriter recordWriter = + textOutputFormat.getRecordWriter(context); + return new SimplePageRankVertexWriter(recordWriter); } + } } diff --git a/src/main/java/org/apache/giraph/examples/SimpleShortestPathsVertex.java b/src/main/java/org/apache/giraph/examples/SimpleShortestPathsVertex.java index 71253cfb8..4fa84f372 100644 --- a/src/main/java/org/apache/giraph/examples/SimpleShortestPathsVertex.java +++ b/src/main/java/org/apache/giraph/examples/SimpleShortestPathsVertex.java @@ -56,215 +56,230 @@ * Demonstrates the basic Pregel shortest paths implementation. */ public class SimpleShortestPathsVertex extends - EdgeListVertex implements Tool { - /** Configuration */ - private Configuration conf; - /** Class logger */ - private static final Logger LOG = - Logger.getLogger(SimpleShortestPathsVertex.class); - /** The shortest paths id */ - public static String SOURCE_ID = "SimpleShortestPathsVertex.sourceId"; - /** Default shortest paths id */ - public static long SOURCE_ID_DEFAULT = 1; + EdgeListVertex implements Tool { + /** The shortest paths id */ + public static final String SOURCE_ID = "SimpleShortestPathsVertex.sourceId"; + /** Default shortest paths id */ + public static final long SOURCE_ID_DEFAULT = 1; + /** Class logger */ + private static final Logger LOG = + Logger.getLogger(SimpleShortestPathsVertex.class); + /** Configuration */ + private Configuration conf; - /** - * Is this vertex the source id? - * - * @return True if the source id - */ - private boolean isSource() { - return (getVertexId().get() == - getContext().getConfiguration().getLong(SOURCE_ID, - SOURCE_ID_DEFAULT)); - } + /** + * Is this vertex the source id? + * + * @return True if the source id + */ + private boolean isSource() { + return getVertexId().get() == + getContext().getConfiguration().getLong(SOURCE_ID, + SOURCE_ID_DEFAULT); + } - @Override - public void compute(Iterator msgIterator) { - if (getSuperstep() == 0) { - setVertexValue(new DoubleWritable(Double.MAX_VALUE)); - } - double minDist = isSource() ? 0d : Double.MAX_VALUE; - while (msgIterator.hasNext()) { - minDist = Math.min(minDist, msgIterator.next().get()); - } + @Override + public void compute(Iterator msgIterator) { + if (getSuperstep() == 0) { + setVertexValue(new DoubleWritable(Double.MAX_VALUE)); + } + double minDist = isSource() ? 0d : Double.MAX_VALUE; + while (msgIterator.hasNext()) { + minDist = Math.min(minDist, msgIterator.next().get()); + } + if (LOG.isDebugEnabled()) { + LOG.debug("Vertex " + getVertexId() + " got minDist = " + minDist + + " vertex value = " + getVertexValue()); + } + if (minDist < getVertexValue().get()) { + setVertexValue(new DoubleWritable(minDist)); + for (LongWritable targetVertexId : this) { + FloatWritable edgeValue = getEdgeValue(targetVertexId); if (LOG.isDebugEnabled()) { - LOG.debug("Vertex " + getVertexId() + " got minDist = " + minDist + - " vertex value = " + getVertexValue()); + LOG.debug("Vertex " + getVertexId() + " sent to " + + targetVertexId + " = " + + (minDist + edgeValue.get())); } - if (minDist < getVertexValue().get()) { - setVertexValue(new DoubleWritable(minDist)); - for (LongWritable targetVertexId : this) { - FloatWritable edgeValue = getEdgeValue(targetVertexId); - if (LOG.isDebugEnabled()) { - LOG.debug("Vertex " + getVertexId() + " sent to " + - targetVertexId + " = " + - (minDist + edgeValue.get())); - } - sendMsg(targetVertexId, - new DoubleWritable(minDist + edgeValue.get())); - } - } - voteToHalt(); + sendMsg(targetVertexId, + new DoubleWritable(minDist + edgeValue.get())); + } } + voteToHalt(); + } - /** - * VertexInputFormat that supports {@link SimpleShortestPathsVertex} - */ - public static class SimpleShortestPathsVertexInputFormat extends - TextVertexInputFormat { - @Override - public VertexReader - createVertexReader(InputSplit split, - TaskAttemptContext context) - throws IOException { - return new SimpleShortestPathsVertexReader( - textInputFormat.createRecordReader(split, context)); - } + /** + * VertexInputFormat that supports {@link SimpleShortestPathsVertex} + */ + public static class SimpleShortestPathsVertexInputFormat extends + TextVertexInputFormat { + @Override + public VertexReader createVertexReader(InputSplit split, + TaskAttemptContext context) + throws IOException { + return new SimpleShortestPathsVertexReader( + textInputFormat.createRecordReader(split, context)); } + } + + /** + * VertexReader that supports {@link SimpleShortestPathsVertex}. In this + * case, the edge values are not used. The files should be in the + * following JSON format: + * JSONArray(, , + * JSONArray(JSONArray(, ), ...)) + * Here is an example with vertex id 1, vertex value 4.3, and two edges. + * First edge has a destination vertex 2, edge value 2.1. + * Second edge has a destination vertex 3, edge value 0.7. + * [1,4.3,[[2,2.1],[3,0.7]]] + */ + public static class SimpleShortestPathsVertexReader extends + TextVertexReader { /** - * VertexReader that supports {@link SimpleShortestPathsVertex}. In this - * case, the edge values are not used. The files should be in the - * following JSON format: - * JSONArray(, , - * JSONArray(JSONArray(, ), ...)) - * Here is an example with vertex id 1, vertex value 4.3, and two edges. - * First edge has a destination vertex 2, edge value 2.1. - * Second edge has a destination vertex 3, edge value 0.7. - * [1,4.3,[[2,2.1],[3,0.7]]] + * Constructor with the line record reader. + * + * @param lineRecordReader Will read from this line. */ - public static class SimpleShortestPathsVertexReader extends - TextVertexReader { - - public SimpleShortestPathsVertexReader( - RecordReader lineRecordReader) { - super(lineRecordReader); - } - - @Override - public BasicVertex getCurrentVertex() - throws IOException, InterruptedException { - BasicVertex vertex = BspUtils.createVertex(getContext().getConfiguration()); + public SimpleShortestPathsVertexReader( + RecordReader lineRecordReader) { + super(lineRecordReader); + } - Text line = getRecordReader().getCurrentValue(); - try { - JSONArray jsonVertex = new JSONArray(line.toString()); - LongWritable vertexId = new LongWritable(jsonVertex.getLong(0)); - DoubleWritable vertexValue = new DoubleWritable(jsonVertex.getDouble(1)); - Map edges = Maps.newHashMap(); - JSONArray jsonEdgeArray = jsonVertex.getJSONArray(2); - for (int i = 0; i < jsonEdgeArray.length(); ++i) { - JSONArray jsonEdge = jsonEdgeArray.getJSONArray(i); - edges.put(new LongWritable(jsonEdge.getLong(0)), - new FloatWritable((float) jsonEdge.getDouble(1))); - } - vertex.initialize(vertexId, vertexValue, edges, null); - } catch (JSONException e) { - throw new IllegalArgumentException( - "next: Couldn't get vertex from line " + line, e); - } - return vertex; - } + @Override + public BasicVertex getCurrentVertex() + throws IOException, InterruptedException { + BasicVertex vertex = + BspUtils.createVertex(getContext().getConfiguration()); - @Override - public boolean nextVertex() throws IOException, InterruptedException { - return getRecordReader().nextKeyValue(); + Text line = getRecordReader().getCurrentValue(); + try { + JSONArray jsonVertex = new JSONArray(line.toString()); + LongWritable vertexId = new LongWritable(jsonVertex.getLong(0)); + DoubleWritable vertexValue = + new DoubleWritable(jsonVertex.getDouble(1)); + Map edges = Maps.newHashMap(); + JSONArray jsonEdgeArray = jsonVertex.getJSONArray(2); + for (int i = 0; i < jsonEdgeArray.length(); ++i) { + JSONArray jsonEdge = jsonEdgeArray.getJSONArray(i); + edges.put(new LongWritable(jsonEdge.getLong(0)), + new FloatWritable((float) jsonEdge.getDouble(1))); } + vertex.initialize(vertexId, vertexValue, edges, null); + } catch (JSONException e) { + throw new IllegalArgumentException( + "next: Couldn't get vertex from line " + line, e); + } + return vertex; } - /** - * VertexOutputFormat that supports {@link SimpleShortestPathsVertex} - */ - public static class SimpleShortestPathsVertexOutputFormat extends - TextVertexOutputFormat { + @Override + public boolean nextVertex() throws IOException, InterruptedException { + return getRecordReader().nextKeyValue(); + } + } - @Override - public VertexWriter - createVertexWriter(TaskAttemptContext context) - throws IOException, InterruptedException { - RecordWriter recordWriter = - textOutputFormat.getRecordWriter(context); - return new SimpleShortestPathsVertexWriter(recordWriter); - } + /** + * VertexOutputFormat that supports {@link SimpleShortestPathsVertex} + */ + public static class SimpleShortestPathsVertexOutputFormat extends + TextVertexOutputFormat { + @Override + public VertexWriter + createVertexWriter(TaskAttemptContext context) + throws IOException, InterruptedException { + RecordWriter recordWriter = + textOutputFormat.getRecordWriter(context); + return new SimpleShortestPathsVertexWriter(recordWriter); } + } + /** + * VertexWriter that supports {@link SimpleShortestPathsVertex} + */ + public static class SimpleShortestPathsVertexWriter extends + TextVertexWriter { /** - * VertexWriter that supports {@link SimpleShortestPathsVertex} + * Vertex writer with the internal line writer. + * + * @param lineRecordWriter Wil actually be written to. */ - public static class SimpleShortestPathsVertexWriter extends - TextVertexWriter { - public SimpleShortestPathsVertexWriter( - RecordWriter lineRecordWriter) { - super(lineRecordWriter); - } - - @Override - public void writeVertex(BasicVertex vertex) - throws IOException, InterruptedException { - JSONArray jsonVertex = new JSONArray(); - try { - jsonVertex.put(vertex.getVertexId().get()); - jsonVertex.put(vertex.getVertexValue().get()); - JSONArray jsonEdgeArray = new JSONArray(); - for (LongWritable targetVertexId : vertex) { - JSONArray jsonEdge = new JSONArray(); - jsonEdge.put(targetVertexId.get()); - jsonEdge.put(vertex.getEdgeValue(targetVertexId).get()); - jsonEdgeArray.put(jsonEdge); - } - jsonVertex.put(jsonEdgeArray); - } catch (JSONException e) { - throw new IllegalArgumentException( - "writeVertex: Couldn't write vertex " + vertex); - } - getRecordWriter().write(new Text(jsonVertex.toString()), null); - } + public SimpleShortestPathsVertexWriter( + RecordWriter lineRecordWriter) { + super(lineRecordWriter); } @Override - public Configuration getConf() { - return conf; + public void writeVertex(BasicVertex vertex) + throws IOException, InterruptedException { + JSONArray jsonVertex = new JSONArray(); + try { + jsonVertex.put(vertex.getVertexId().get()); + jsonVertex.put(vertex.getVertexValue().get()); + JSONArray jsonEdgeArray = new JSONArray(); + for (LongWritable targetVertexId : vertex) { + JSONArray jsonEdge = new JSONArray(); + jsonEdge.put(targetVertexId.get()); + jsonEdge.put(vertex.getEdgeValue(targetVertexId).get()); + jsonEdgeArray.put(jsonEdge); + } + jsonVertex.put(jsonEdgeArray); + } catch (JSONException e) { + throw new IllegalArgumentException( + "writeVertex: Couldn't write vertex " + vertex); + } + getRecordWriter().write(new Text(jsonVertex.toString()), null); } + } - @Override - public void setConf(Configuration conf) { - this.conf = conf; - } + @Override + public Configuration getConf() { + return conf; + } - @Override - public int run(String[] argArray) throws Exception { - Preconditions.checkArgument(argArray.length == 4, - "run: Must have 4 arguments " + - " <# of workers>"); + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } - GiraphJob job = new GiraphJob(getConf(), getClass().getName()); - job.setVertexClass(getClass()); - job.setVertexInputFormatClass( - SimpleShortestPathsVertexInputFormat.class); - job.setVertexOutputFormatClass( - SimpleShortestPathsVertexOutputFormat.class); - FileInputFormat.addInputPath(job, new Path(argArray[0])); - FileOutputFormat.setOutputPath(job, new Path(argArray[1])); - job.getConfiguration().setLong(SimpleShortestPathsVertex.SOURCE_ID, - Long.parseLong(argArray[2])); - job.setWorkerConfiguration(Integer.parseInt(argArray[3]), - Integer.parseInt(argArray[3]), - 100.0f); + @Override + public int run(String[] argArray) throws Exception { + Preconditions.checkArgument(argArray.length == 4, + "run: Must have 4 arguments " + + " <# of workers>"); - return job.run(true) ? 0 : -1; - } + GiraphJob job = new GiraphJob(getConf(), getClass().getName()); + job.setVertexClass(getClass()); + job.setVertexInputFormatClass( + SimpleShortestPathsVertexInputFormat.class); + job.setVertexOutputFormatClass( + SimpleShortestPathsVertexOutputFormat.class); + FileInputFormat.addInputPath(job, new Path(argArray[0])); + FileOutputFormat.setOutputPath(job, new Path(argArray[1])); + job.getConfiguration().setLong(SimpleShortestPathsVertex.SOURCE_ID, + Long.parseLong(argArray[2])); + job.setWorkerConfiguration(Integer.parseInt(argArray[3]), + Integer.parseInt(argArray[3]), + 100.0f); - public static void main(String[] args) throws Exception { - System.exit(ToolRunner.run(new SimpleShortestPathsVertex(), args)); - } + return job.run(true) ? 0 : -1; + } + + /** + * Can be used for command line execution. + * + * @param args Command line arguments. + * @throws Exception + */ + public static void main(String[] args) throws Exception { + System.exit(ToolRunner.run(new SimpleShortestPathsVertex(), args)); + } } diff --git a/src/main/java/org/apache/giraph/examples/SimpleSumCombiner.java b/src/main/java/org/apache/giraph/examples/SimpleSumCombiner.java index 139885fcb..1f003454e 100644 --- a/src/main/java/org/apache/giraph/examples/SimpleSumCombiner.java +++ b/src/main/java/org/apache/giraph/examples/SimpleSumCombiner.java @@ -31,18 +31,18 @@ * Test whether combiner is called by summing up the messages. */ public class SimpleSumCombiner - extends VertexCombiner { + extends VertexCombiner { - @Override - public Iterable combine(LongWritable vertexIndex, - Iterable messages) throws IOException { - int sum = 0; - for (IntWritable msg : messages) { - sum += msg.get(); - } - List value = new ArrayList(); - value.add(new IntWritable(sum)); - - return value; + @Override + public Iterable combine(LongWritable vertexIndex, + Iterable messages) throws IOException { + int sum = 0; + for (IntWritable msg : messages) { + sum += msg.get(); } + List value = new ArrayList(); + value.add(new IntWritable(sum)); + + return value; + } } diff --git a/src/main/java/org/apache/giraph/examples/SimpleSuperstepVertex.java b/src/main/java/org/apache/giraph/examples/SimpleSuperstepVertex.java index 82aeb1bac..7ed736cf1 100644 --- a/src/main/java/org/apache/giraph/examples/SimpleSuperstepVertex.java +++ b/src/main/java/org/apache/giraph/examples/SimpleSuperstepVertex.java @@ -44,115 +44,119 @@ * finishes. */ public class SimpleSuperstepVertex extends - EdgeListVertex { - @Override - public void compute(Iterator msgIterator) { - if (getSuperstep() > 3) { - voteToHalt(); - } + EdgeListVertex { + @Override + public void compute(Iterator msgIterator) { + if (getSuperstep() > 3) { + voteToHalt(); } + } + /** + * Simple VertexReader that supports {@link SimpleSuperstepVertex} + */ + public static class SimpleSuperstepVertexReader extends + GeneratedVertexReader { + /** Class logger */ + private static final Logger LOG = + Logger.getLogger(SimpleSuperstepVertexReader.class); /** - * Simple VertexReader that supports {@link SimpleSuperstepVertex} + * Constructor. */ - public static class SimpleSuperstepVertexReader extends - GeneratedVertexReader { - /** Class logger */ - private static final Logger LOG = - Logger.getLogger(SimpleSuperstepVertexReader.class); - @Override - public boolean nextVertex() throws IOException, InterruptedException { - return totalRecords > recordsRead; - } + public SimpleSuperstepVertexReader() { + super(); + } - public SimpleSuperstepVertexReader() { - super(); - } + @Override + public boolean nextVertex() throws IOException, InterruptedException { + return totalRecords > recordsRead; + } - @Override - public BasicVertex getCurrentVertex() - throws IOException, InterruptedException { - BasicVertex vertex = - BspUtils.createVertex( - configuration); - long tmpId = reverseIdOrder ? - ((inputSplit.getSplitIndex() + 1) * totalRecords) - - recordsRead - 1 : - (inputSplit.getSplitIndex() * totalRecords) + recordsRead; - LongWritable vertexId = new LongWritable(tmpId); - IntWritable vertexValue = - new IntWritable((int) (vertexId.get() * 10)); - Map edgeMap = Maps.newHashMap(); - long destVertexId = - (vertexId.get() + 1) % - (inputSplit.getNumSplits() * totalRecords); - float edgeValue = vertexId.get() * 100f; - edgeMap.put(new LongWritable(destVertexId), - new FloatWritable(edgeValue)); - vertex.initialize(vertexId, vertexValue, edgeMap, null); - ++recordsRead; - if (LOG.isInfoEnabled()) { - LOG.info("next: Return vertexId=" + vertex.getVertexId().get() + - ", vertexValue=" + vertex.getVertexValue() + - ", destinationId=" + destVertexId + - ", edgeValue=" + edgeValue); - } - return vertex; - } + @Override + public BasicVertex getCurrentVertex() + throws IOException, InterruptedException { + BasicVertex vertex = + BspUtils.createVertex(configuration); + long tmpId = reverseIdOrder ? + ((inputSplit.getSplitIndex() + 1) * totalRecords) - + recordsRead - 1 : + (inputSplit.getSplitIndex() * totalRecords) + recordsRead; + LongWritable vertexId = new LongWritable(tmpId); + IntWritable vertexValue = + new IntWritable((int) (vertexId.get() * 10)); + Map edgeMap = Maps.newHashMap(); + long destVertexId = + (vertexId.get() + 1) % + (inputSplit.getNumSplits() * totalRecords); + float edgeValue = vertexId.get() * 100f; + edgeMap.put(new LongWritable(destVertexId), + new FloatWritable(edgeValue)); + vertex.initialize(vertexId, vertexValue, edgeMap, null); + ++recordsRead; + if (LOG.isInfoEnabled()) { + LOG.info("next: Return vertexId=" + vertex.getVertexId().get() + + ", vertexValue=" + vertex.getVertexValue() + + ", destinationId=" + destVertexId + + ", edgeValue=" + edgeValue); + } + return vertex; } + } - /** - * Simple VertexInputFormat that supports {@link SimpleSuperstepVertex} - */ - public static class SimpleSuperstepVertexInputFormat extends - GeneratedVertexInputFormat { - @Override - public VertexReader - createVertexReader(InputSplit split, - TaskAttemptContext context) - throws IOException { - return new SimpleSuperstepVertexReader(); - } + /** + * Simple VertexInputFormat that supports {@link SimpleSuperstepVertex} + */ + public static class SimpleSuperstepVertexInputFormat extends + GeneratedVertexInputFormat { + @Override + public VertexReader + createVertexReader(InputSplit split, TaskAttemptContext context) + throws IOException { + return new SimpleSuperstepVertexReader(); } + } + /** + * Simple VertexWriter that supports {@link SimpleSuperstepVertex} + */ + public static class SimpleSuperstepVertexWriter extends + TextVertexWriter { /** - * Simple VertexWriter that supports {@link SimpleSuperstepVertex} + * Constructor with the line record writer. + * + * @param lineRecordWriter Writer to write to. */ - public static class SimpleSuperstepVertexWriter extends - TextVertexWriter { - public SimpleSuperstepVertexWriter( - RecordWriter lineRecordWriter) { - super(lineRecordWriter); - } - - @Override - public void writeVertex( - BasicVertex vertex) - throws IOException, InterruptedException { - getRecordWriter().write( - new Text(vertex.getVertexId().toString()), - new Text(vertex.getVertexValue().toString())); - } + public SimpleSuperstepVertexWriter( + RecordWriter lineRecordWriter) { + super(lineRecordWriter); } - /** - * Simple VertexOutputFormat that supports {@link SimpleSuperstepVertex} - */ - public static class SimpleSuperstepVertexOutputFormat extends - TextVertexOutputFormat { + @Override + public void writeVertex(BasicVertex vertex) throws IOException, InterruptedException { + getRecordWriter().write( + new Text(vertex.getVertexId().toString()), + new Text(vertex.getVertexValue().toString())); + } + } - @Override - public VertexWriter - createVertexWriter(TaskAttemptContext context) - throws IOException, InterruptedException { - RecordWriter recordWriter = - textOutputFormat.getRecordWriter(context); - return new SimpleSuperstepVertexWriter(recordWriter); - } + /** + * Simple VertexOutputFormat that supports {@link SimpleSuperstepVertex} + */ + public static class SimpleSuperstepVertexOutputFormat extends + TextVertexOutputFormat { + @Override + public VertexWriter + createVertexWriter(TaskAttemptContext context) + throws IOException, InterruptedException { + RecordWriter recordWriter = + textOutputFormat.getRecordWriter(context); + return new SimpleSuperstepVertexWriter(recordWriter); } + } } diff --git a/src/main/java/org/apache/giraph/examples/SimpleTextVertexOutputFormat.java b/src/main/java/org/apache/giraph/examples/SimpleTextVertexOutputFormat.java index 8f652f034..2527f684b 100644 --- a/src/main/java/org/apache/giraph/examples/SimpleTextVertexOutputFormat.java +++ b/src/main/java/org/apache/giraph/examples/SimpleTextVertexOutputFormat.java @@ -33,39 +33,38 @@ * Simple text based vertex output format example. */ public class SimpleTextVertexOutputFormat extends - TextVertexOutputFormat { + TextVertexOutputFormat { + /** + * Simple text based vertex writer + */ + private static class SimpleTextVertexWriter + extends TextVertexWriter { /** - * Simple text based vertex writer + * Initialize with the LineRecordWriter. + * + * @param lineRecordWriter Line record writer from TextOutputFormat */ - private static class SimpleTextVertexWriter - extends TextVertexWriter { - - /** - * Initialize with the LineRecordWriter. - * - * @param lineRecordWriter Line record writer from TextOutputFormat - */ - public SimpleTextVertexWriter( - RecordWriter lineRecordWriter) { - super(lineRecordWriter); - } - - @Override - public void writeVertex( - BasicVertex vertex) - throws IOException, InterruptedException { - getRecordWriter().write( - new Text(vertex.getVertexId().toString()), - new Text(vertex.getVertexValue().toString())); - } + public SimpleTextVertexWriter( + RecordWriter lineRecordWriter) { + super(lineRecordWriter); } @Override - public VertexWriter - createVertexWriter(TaskAttemptContext context) - throws IOException, InterruptedException { - RecordWriter recordWriter = - textOutputFormat.getRecordWriter(context); - return new SimpleTextVertexWriter(recordWriter); + public void writeVertex( + BasicVertex vertex) + throws IOException, InterruptedException { + getRecordWriter().write( + new Text(vertex.getVertexId().toString()), + new Text(vertex.getVertexValue().toString())); } + } + + @Override + public VertexWriter + createVertexWriter(TaskAttemptContext context) + throws IOException, InterruptedException { + RecordWriter recordWriter = + textOutputFormat.getRecordWriter(context); + return new SimpleTextVertexWriter(recordWriter); + } } diff --git a/src/main/java/org/apache/giraph/examples/SimpleVertexWithWorkerContext.java b/src/main/java/org/apache/giraph/examples/SimpleVertexWithWorkerContext.java index 1caa2ff94..ccfcf6c58 100644 --- a/src/main/java/org/apache/giraph/examples/SimpleVertexWithWorkerContext.java +++ b/src/main/java/org/apache/giraph/examples/SimpleVertexWithWorkerContext.java @@ -22,7 +22,8 @@ import java.io.IOException; import java.util.Iterator; -import org.apache.giraph.examples.SimpleSuperstepVertex.SimpleSuperstepVertexInputFormat; +import org.apache.giraph.examples.SimpleSuperstepVertex. + SimpleSuperstepVertexInputFormat; import org.apache.giraph.graph.GiraphJob; import org.apache.giraph.graph.EdgeListVertex; import org.apache.giraph.graph.WorkerContext; @@ -43,123 +44,139 @@ * computation. */ public class SimpleVertexWithWorkerContext extends - EdgeListVertex - implements Tool { - - public static final String OUTPUTDIR = "svwwc.outputdir"; - private static final int TESTLENGTH = 30; + EdgeListVertex + implements Tool { + /** Directory name of where to write. */ + public static final String OUTPUTDIR = "svwwc.outputdir"; + /** Halting condition for the number of supersteps */ + private static final int TESTLENGTH = 30; + + @Override + public void compute(Iterator msgIterator) + throws IOException { + + long superstep = getSuperstep(); + + if (superstep < TESTLENGTH) { + EmitterWorkerContext emitter = + (EmitterWorkerContext) getWorkerContext(); + emitter.emit("vertexId=" + getVertexId() + + " superstep=" + superstep + "\n"); + } else { + voteToHalt(); + } + } + + /** + * Example worker context to emit data as part of a superstep. + */ + @SuppressWarnings("rawtypes") + public static class EmitterWorkerContext extends WorkerContext { + /** File name prefix */ + private static final String FILENAME = "emitter_"; + /** Output stream to dump the strings. */ + private DataOutputStream out; @Override - public void compute(Iterator msgIterator) - throws IOException { - - long superstep = getSuperstep(); - - if (superstep < TESTLENGTH) { - EmitterWorkerContext emitter = - (EmitterWorkerContext) getWorkerContext(); - emitter.emit("vertexId=" + getVertexId() + - " superstep=" + superstep + "\n"); - } else { - voteToHalt(); + public void preApplication() { + Context context = getContext(); + FileSystem fs; + + try { + fs = FileSystem.get(context.getConfiguration()); + + String p = context.getConfiguration() + .get(SimpleVertexWithWorkerContext.OUTPUTDIR); + if (p == null) { + throw new IllegalArgumentException( + SimpleVertexWithWorkerContext.OUTPUTDIR + + " undefined!"); } - } - @SuppressWarnings("rawtypes") - public static class EmitterWorkerContext extends WorkerContext { - - private static final String FILENAME = "emitter_"; - private DataOutputStream out; - - @Override - public void preApplication() { - Context context = getContext(); - FileSystem fs; - - try { - fs = FileSystem.get(context.getConfiguration()); - - String p = context.getConfiguration() - .get(SimpleVertexWithWorkerContext.OUTPUTDIR); - if (p == null) { - throw new IllegalArgumentException( - SimpleVertexWithWorkerContext.OUTPUTDIR + - " undefined!"); - } - - Path path = new Path(p); - if (!fs.exists(path)) { - throw new IllegalArgumentException(path + - " doesn't exist"); - } - - Path outF = new Path(path, FILENAME + - context.getTaskAttemptID()); - if (fs.exists(outF)) { - throw new IllegalArgumentException(outF + - " aready exists"); - } - - out = fs.create(outF); - } catch (IOException e) { - throw new RuntimeException( - "can't initialize WorkerContext", e); - } + Path path = new Path(p); + if (!fs.exists(path)) { + throw new IllegalArgumentException(path + + " doesn't exist"); } - @Override - public void postApplication() { - if (out != null) { - try { - out.flush(); - out.close(); - } catch (IOException e) { - throw new RuntimeException( - "can't finalize WorkerContext", e); - } - out = null; - } + Path outF = new Path(path, FILENAME + + context.getTaskAttemptID()); + if (fs.exists(outF)) { + throw new IllegalArgumentException(outF + + " aready exists"); } - @Override - public void preSuperstep() { } - - @Override - public void postSuperstep() { } + out = fs.create(outF); + } catch (IOException e) { + throw new RuntimeException( + "can't initialize WorkerContext", e); + } + } - public void emit(String s) { - try { - out.writeUTF(s); - } catch (IOException e) { - throw new RuntimeException("can't emit", e); - } + @Override + public void postApplication() { + if (out != null) { + try { + out.flush(); + out.close(); + } catch (IOException e) { + throw new RuntimeException( + "can't finalize WorkerContext", e); } + out = null; + } } @Override - public int run(String[] args) throws Exception { - if (args.length != 2) { - throw new IllegalArgumentException( - "run: Must have 2 arguments <# of workers>"); - } - GiraphJob job = new GiraphJob(getConf(), getClass().getName()); - job.setVertexClass(getClass()); - job.setVertexInputFormatClass( - SimpleSuperstepVertexInputFormat.class); - job.setWorkerContextClass(EmitterWorkerContext.class); - Configuration conf = job.getConfiguration(); - conf.set(SimpleVertexWithWorkerContext.OUTPUTDIR, args[0]); - job.setWorkerConfiguration(Integer.parseInt(args[1]), - Integer.parseInt(args[1]), - 100.0f); - if (job.run(true) == true) { - return 0; - } else { - return -1; - } + public void preSuperstep() { } + + @Override + public void postSuperstep() { } + + /** + * Write this string to the output stream. + * + * @param s String to dump. + */ + public void emit(String s) { + try { + out.writeUTF(s); + } catch (IOException e) { + throw new RuntimeException("can't emit", e); + } } + } - public static void main(String[] args) throws Exception { - System.exit(ToolRunner.run(new SimpleVertexWithWorkerContext(), args)); + @Override + public int run(String[] args) throws Exception { + if (args.length != 2) { + throw new IllegalArgumentException( + "run: Must have 2 arguments <# of workers>"); + } + GiraphJob job = new GiraphJob(getConf(), getClass().getName()); + job.setVertexClass(getClass()); + job.setVertexInputFormatClass( + SimpleSuperstepVertexInputFormat.class); + job.setWorkerContextClass(EmitterWorkerContext.class); + Configuration conf = job.getConfiguration(); + conf.set(SimpleVertexWithWorkerContext.OUTPUTDIR, args[0]); + job.setWorkerConfiguration(Integer.parseInt(args[1]), + Integer.parseInt(args[1]), + 100.0f); + if (job.run(true)) { + return 0; + } else { + return -1; } -} \ No newline at end of file + } + + /** + * Executable from the command line. + * + * @param args Command line arguments. + * @throws Exception + */ + public static void main(String[] args) throws Exception { + System.exit(ToolRunner.run(new SimpleVertexWithWorkerContext(), args)); + } +} diff --git a/src/main/java/org/apache/giraph/examples/SumAggregator.java b/src/main/java/org/apache/giraph/examples/SumAggregator.java index 6536b5ea9..1180b4c40 100644 --- a/src/main/java/org/apache/giraph/examples/SumAggregator.java +++ b/src/main/java/org/apache/giraph/examples/SumAggregator.java @@ -24,30 +24,38 @@ /** * Aggregator for summing up values. - * */ public class SumAggregator implements Aggregator { - + /** Aggregated sum */ private double sum = 0; + /** + * Aggregate a double. + * + * @param value Value to aggregate. + */ public void aggregate(double value) { - sum += value; + sum += value; } + @Override public void aggregate(DoubleWritable value) { - sum += value.get(); + sum += value.get(); } + @Override public void setAggregatedValue(DoubleWritable value) { - sum = value.get(); + sum = value.get(); } + @Override public DoubleWritable getAggregatedValue() { - return new DoubleWritable(sum); + return new DoubleWritable(sum); } + @Override public DoubleWritable createAggregatedValue() { - return new DoubleWritable(); + return new DoubleWritable(); } } diff --git a/src/main/java/org/apache/giraph/examples/VerifyMessage.java b/src/main/java/org/apache/giraph/examples/VerifyMessage.java index 7553b7154..fad4d3e12 100644 --- a/src/main/java/org/apache/giraph/examples/VerifyMessage.java +++ b/src/main/java/org/apache/giraph/examples/VerifyMessage.java @@ -18,7 +18,8 @@ package org.apache.giraph.examples; -import org.apache.giraph.graph.*; +import org.apache.giraph.graph.EdgeListVertex; +import org.apache.giraph.graph.WorkerContext; import org.apache.hadoop.io.FloatWritable; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.LongWritable; @@ -36,153 +37,174 @@ * appropriate location and superstep. */ public class VerifyMessage { - public static class VerifiableMessage implements Writable { - /** Superstep sent on */ - public long superstep; - /** Source vertex id */ - public long sourceVertexId; - /** Value */ - public float value; - - public VerifiableMessage() {} - - public VerifiableMessage( - long superstep, long sourceVertexId, float value) { - this.superstep = superstep; - this.sourceVertexId = sourceVertexId; - this.value = value; - } + /** + * Message that will be sent in {@link VerifyMessageVertex}. + */ + public static class VerifiableMessage implements Writable { + /** Superstep sent on */ + private long superstep; + /** Source vertex id */ + private long sourceVertexId; + /** Value */ + private float value; + + /** + * Default constructor used with reflection. + */ + public VerifiableMessage() { } + + /** + * Constructor with verifiable arguments. + * @param superstep Superstep this message was created on. + * @param sourceVertexId Who send this message. + * @param value A value associated with this message. + */ + public VerifiableMessage( + long superstep, long sourceVertexId, float value) { + this.superstep = superstep; + this.sourceVertexId = sourceVertexId; + this.value = value; + } - @Override - public void readFields(DataInput input) throws IOException { - superstep = input.readLong(); - sourceVertexId = input.readLong(); - value = input.readFloat(); - } + @Override + public void readFields(DataInput input) throws IOException { + superstep = input.readLong(); + sourceVertexId = input.readLong(); + value = input.readFloat(); + } - @Override - public void write(DataOutput output) throws IOException { - output.writeLong(superstep); - output.writeLong(sourceVertexId); - output.writeFloat(value); - } + @Override + public void write(DataOutput output) throws IOException { + output.writeLong(superstep); + output.writeLong(sourceVertexId); + output.writeFloat(value); + } - @Override - public String toString() { - return "(superstep=" + superstep + ",sourceVertexId=" + - sourceVertexId + ",value=" + value + ")"; - } + @Override + public String toString() { + return "(superstep=" + superstep + ",sourceVertexId=" + + sourceVertexId + ",value=" + value + ")"; + } + } + + /** + * Send and verify messages. + */ + public static class VerifyMessageVertex extends + EdgeListVertex { + /** Dynamically set number of SUPERSTEPS */ + public static final String SUPERSTEP_COUNT = + "verifyMessageVertex.superstepCount"; + /** User can access this after the application finishes if local */ + private static long FINAL_SUM; + /** Number of SUPERSTEPS to run (6 by default) */ + private static int SUPERSTEPS = 6; + /** Class logger */ + private static Logger LOG = Logger.getLogger(VerifyMessageVertex.class); + + public static long getFinalSum() { + return FINAL_SUM; } - public static class VerifyMessageVertex extends - EdgeListVertex { - /** User can access this after the application finishes if local */ - public static long finalSum; - /** Number of supersteps to run (6 by default) */ - private static int supersteps = 6; - /** Class logger */ - private static Logger LOG = Logger.getLogger(VerifyMessageVertex.class); - - /** Dynamically set number of supersteps */ - public static final String SUPERSTEP_COUNT = - "verifyMessageVertex.superstepCount"; - - public static class VerifyMessageVertexWorkerContext extends - WorkerContext { - @Override - public void preApplication() throws InstantiationException, - IllegalAccessException { - registerAggregator(LongSumAggregator.class.getName(), - LongSumAggregator.class); - LongSumAggregator sumAggregator = (LongSumAggregator) - getAggregator(LongSumAggregator.class.getName()); - sumAggregator.setAggregatedValue(new LongWritable(0)); - supersteps = getContext().getConfiguration().getInt( - SUPERSTEP_COUNT, supersteps); - } - - @Override - public void postApplication() { - LongSumAggregator sumAggregator = (LongSumAggregator) - getAggregator(LongSumAggregator.class.getName()); - finalSum = sumAggregator.getAggregatedValue().get(); - } - - @Override - public void preSuperstep() { - useAggregator(LongSumAggregator.class.getName()); - } - - @Override - public void postSuperstep() {} - } + /** + * Worker context used with {@link VerifyMessageVertex}. + */ + public static class VerifyMessageVertexWorkerContext extends + WorkerContext { + @Override + public void preApplication() throws InstantiationException, + IllegalAccessException { + registerAggregator(LongSumAggregator.class.getName(), + LongSumAggregator.class); + LongSumAggregator sumAggregator = (LongSumAggregator) + getAggregator(LongSumAggregator.class.getName()); + sumAggregator.setAggregatedValue(new LongWritable(0)); + SUPERSTEPS = getContext().getConfiguration().getInt( + SUPERSTEP_COUNT, SUPERSTEPS); + } + + @Override + public void postApplication() { + LongSumAggregator sumAggregator = (LongSumAggregator) + getAggregator(LongSumAggregator.class.getName()); + FINAL_SUM = sumAggregator.getAggregatedValue().get(); + } + + @Override + public void preSuperstep() { + useAggregator(LongSumAggregator.class.getName()); + } + + @Override + public void postSuperstep() { } + } - @Override - public void compute(Iterator msgIterator) { - LongSumAggregator sumAggregator = (LongSumAggregator) - getAggregator(LongSumAggregator.class.getName()); - if (getSuperstep() > supersteps) { - voteToHalt(); - return; - } - if (LOG.isDebugEnabled()) { - LOG.debug("compute: " + sumAggregator); - } - sumAggregator.aggregate(getVertexId().get()); - if (LOG.isDebugEnabled()) { - LOG.debug("compute: sum = " + - sumAggregator.getAggregatedValue().get() + - " for vertex " + getVertexId()); - } - float msgValue = 0.0f; - while (msgIterator.hasNext()) { - VerifiableMessage msg = msgIterator.next(); - msgValue += msg.value; - if (LOG.isDebugEnabled()) { - LOG.debug("compute: got msg = " + msg + - " for vertex id " + getVertexId() + - ", vertex value " + getVertexValue() + - " on superstep " + getSuperstep()); - } - if (msg.superstep != getSuperstep() - 1) { - throw new IllegalStateException( - "compute: Impossible to not get a messsage from " + - "the previous superstep, current superstep = " + - getSuperstep()); - } - if ((msg.sourceVertexId != getVertexId().get() - 1) && - (getVertexId().get() != 0)) { - throw new IllegalStateException( - "compute: Impossible that this message didn't come " + - "from the previous vertex and came from " + - msg.sourceVertexId); - } - } - int vertexValue = getVertexValue().get(); - setVertexValue(new IntWritable(vertexValue + (int) msgValue)); - if (LOG.isDebugEnabled()) { - LOG.debug("compute: vertex " + getVertexId() + - " has value " + getVertexValue() + - " on superstep " + getSuperstep()); - } - for (LongWritable targetVertexId : this) { - FloatWritable edgeValue = getEdgeValue(targetVertexId); - if (LOG.isDebugEnabled()) { - LOG.debug("compute: vertex " + getVertexId() + - " sending edgeValue " + edgeValue + - " vertexValue " + vertexValue + - " total " + - (edgeValue.get() + (float) vertexValue) + - " to vertex " + targetVertexId + - " on superstep " + getSuperstep()); - } - edgeValue.set(edgeValue.get() + (float) vertexValue); - addEdge(targetVertexId, edgeValue); - sendMsg(targetVertexId, - new VerifiableMessage( - getSuperstep(), getVertexId().get(), edgeValue.get())); - } + @Override + public void compute(Iterator msgIterator) { + LongSumAggregator sumAggregator = (LongSumAggregator) + getAggregator(LongSumAggregator.class.getName()); + if (getSuperstep() > SUPERSTEPS) { + voteToHalt(); + return; + } + if (LOG.isDebugEnabled()) { + LOG.debug("compute: " + sumAggregator); + } + sumAggregator.aggregate(getVertexId().get()); + if (LOG.isDebugEnabled()) { + LOG.debug("compute: sum = " + + sumAggregator.getAggregatedValue().get() + + " for vertex " + getVertexId()); + } + float msgValue = 0.0f; + while (msgIterator.hasNext()) { + VerifiableMessage msg = msgIterator.next(); + msgValue += msg.value; + if (LOG.isDebugEnabled()) { + LOG.debug("compute: got msg = " + msg + + " for vertex id " + getVertexId() + + ", vertex value " + getVertexValue() + + " on superstep " + getSuperstep()); + } + if (msg.superstep != getSuperstep() - 1) { + throw new IllegalStateException( + "compute: Impossible to not get a messsage from " + + "the previous superstep, current superstep = " + + getSuperstep()); + } + if ((msg.sourceVertexId != getVertexId().get() - 1) && + (getVertexId().get() != 0)) { + throw new IllegalStateException( + "compute: Impossible that this message didn't come " + + "from the previous vertex and came from " + + msg.sourceVertexId); + } + } + int vertexValue = getVertexValue().get(); + setVertexValue(new IntWritable(vertexValue + (int) msgValue)); + if (LOG.isDebugEnabled()) { + LOG.debug("compute: vertex " + getVertexId() + + " has value " + getVertexValue() + + " on superstep " + getSuperstep()); + } + for (LongWritable targetVertexId : this) { + FloatWritable edgeValue = getEdgeValue(targetVertexId); + if (LOG.isDebugEnabled()) { + LOG.debug("compute: vertex " + getVertexId() + + " sending edgeValue " + edgeValue + + " vertexValue " + vertexValue + + " total " + + (edgeValue.get() + (float) vertexValue) + + " to vertex " + targetVertexId + + " on superstep " + getSuperstep()); } + edgeValue.set(edgeValue.get() + (float) vertexValue); + addEdge(targetVertexId, edgeValue); + sendMsg(targetVertexId, + new VerifiableMessage( + getSuperstep(), getVertexId().get(), edgeValue.get())); + } } + } } diff --git a/src/main/java/org/apache/giraph/examples/VertexWithComponentTextOutputFormat.java b/src/main/java/org/apache/giraph/examples/VertexWithComponentTextOutputFormat.java index 17e6fa9b1..6fbf1d955 100644 --- a/src/main/java/org/apache/giraph/examples/VertexWithComponentTextOutputFormat.java +++ b/src/main/java/org/apache/giraph/examples/VertexWithComponentTextOutputFormat.java @@ -1,20 +1,20 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one -* or more contributor license agreements. See the NOTICE file -* distributed with this work for additional information -* regarding copyright ownership. The ASF licenses this file -* to you under the Apache License, Version 2.0 (the -* "License"); you may not use this file except in compliance -* with the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.giraph.examples; @@ -33,39 +33,44 @@ * Text-based {@link org.apache.giraph.graph.VertexOutputFormat} for usage with * {@link ConnectedComponentsVertex} * - * Each line consists of a vertex and its associated component (represented by the smallest - * vertex id in the component) + * Each line consists of a vertex and its associated component (represented + * by the smallest vertex id in the component) */ public class VertexWithComponentTextOutputFormat extends - TextVertexOutputFormat { + TextVertexOutputFormat { + @Override + public VertexWriter + createVertexWriter(TaskAttemptContext context) + throws IOException, InterruptedException { + RecordWriter recordWriter = + textOutputFormat.getRecordWriter(context); + return new VertexWithComponentWriter(recordWriter); + } - @Override - public VertexWriter - createVertexWriter(TaskAttemptContext context) - throws IOException, InterruptedException { - RecordWriter recordWriter = - textOutputFormat.getRecordWriter(context); - return new VertexWithComponentWriter(recordWriter); + /** + * Vertex writer used with {@link VertexWithComponentTextOutputFormat}. + */ + public static class VertexWithComponentWriter extends + TextVertexOutputFormat.TextVertexWriter { + /** + * Constructor with record writer. + * + * @param writer Where the vertices will finally be written. + */ + public VertexWithComponentWriter(RecordWriter writer) { + super(writer); } - public static class VertexWithComponentWriter extends - TextVertexOutputFormat.TextVertexWriter { - - public VertexWithComponentWriter(RecordWriter writer) { - super(writer); - } - - @Override - public void writeVertex(BasicVertex vertex) throws IOException, - InterruptedException { - StringBuilder output = new StringBuilder(); - output.append(vertex.getVertexId().get()); - output.append('\t'); - output.append(vertex.getVertexValue().get()); - getRecordWriter().write(new Text(output.toString()), null); - } - + @Override + public void writeVertex(BasicVertex vertex) throws IOException, + InterruptedException { + StringBuilder output = new StringBuilder(); + output.append(vertex.getVertexId().get()); + output.append('\t'); + output.append(vertex.getVertexValue().get()); + getRecordWriter().write(new Text(output.toString()), null); } -} \ No newline at end of file + } +} diff --git a/src/main/java/org/apache/giraph/examples/package-info.java b/src/main/java/org/apache/giraph/examples/package-info.java new file mode 100644 index 000000000..3ebb72b5d --- /dev/null +++ b/src/main/java/org/apache/giraph/examples/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/** + * Package of Giraph examples. + */ +package org.apache.giraph.examples; diff --git a/src/main/java/org/apache/giraph/graph/Aggregator.java b/src/main/java/org/apache/giraph/graph/Aggregator.java index 3b0bc98a8..8cda4af11 100644 --- a/src/main/java/org/apache/giraph/graph/Aggregator.java +++ b/src/main/java/org/apache/giraph/graph/Aggregator.java @@ -24,39 +24,39 @@ * Interface for Aggregator. Allows aggregate operations for all vertices * in a given superstep. * - * @param Aggregated value + * @param Aggregated value */ public interface Aggregator { - /** - * Add a new value. - * Needs to be commutative and associative - * - * @param value - */ - void aggregate(A value); + /** + * Add a new value. + * Needs to be commutative and associative + * + * @param value Value to be aggregated. + */ + void aggregate(A value); - /** - * Set aggregated value. - * Can be used for initialization or reset. - * - * @param value - */ - void setAggregatedValue(A value); + /** + * Set aggregated value. + * Can be used for initialization or reset. + * + * @param value Value to be set. + */ + void setAggregatedValue(A value); - /** - * Return current aggregated value. - * Needs to be initialized if aggregate or setAggregatedValue - * have not been called before. - * - * @return A - */ - A getAggregatedValue(); + /** + * Return current aggregated value. + * Needs to be initialized if aggregate or setAggregatedValue + * have not been called before. + * + * @return Aggregated + */ + A getAggregatedValue(); - /** - * Return new aggregated value. - * Must be changeable without affecting internals of Aggregator - * - * @return Writable - */ - A createAggregatedValue(); + /** + * Return new aggregated value. + * Must be changeable without affecting internals of Aggregator + * + * @return Writable + */ + A createAggregatedValue(); } diff --git a/src/main/java/org/apache/giraph/graph/AggregatorUsage.java b/src/main/java/org/apache/giraph/graph/AggregatorUsage.java index 37ce57cdb..e66f7a394 100644 --- a/src/main/java/org/apache/giraph/graph/AggregatorUsage.java +++ b/src/main/java/org/apache/giraph/graph/AggregatorUsage.java @@ -24,34 +24,35 @@ * Vertex classes can use this interface to register and use aggregators */ public interface AggregatorUsage { - /** - * Register an aggregator in preSuperstep() and/or preApplication(). - * - * @param name of aggregator - * @param aggregatorClass Class type of the aggregator - * @return created Aggregator or null when already registered - */ - public Aggregator registerAggregator( - String name, - Class> aggregatorClass) - throws InstantiationException, IllegalAccessException; + /** + * Register an aggregator in preSuperstep() and/or preApplication(). + * + * @param Aggregator type + * @param name of aggregator + * @param aggregatorClass Class type of the aggregator + * @return created Aggregator or null when already registered + */ + Aggregator registerAggregator( + String name, + Class> aggregatorClass) + throws InstantiationException, IllegalAccessException; - /** - * Get a registered aggregator. - * - * @param name Name of aggregator - * @return Aggregator (null when not registered) - */ - public Aggregator getAggregator(String name); + /** + * Get a registered aggregator. + * + * @param name Name of aggregator + * @return Aggregator (null when not registered) + */ + Aggregator getAggregator(String name); - /** - * Use a registered aggregator in current superstep. - * Even when the same aggregator should be used in the next - * superstep, useAggregator needs to be called at the beginning - * of that superstep in preSuperstep(). - * - * @param name Name of aggregator - * @return boolean (false when not registered) - */ - public boolean useAggregator(String name); + /** + * Use a registered aggregator in current superstep. + * Even when the same aggregator should be used in the next + * superstep, useAggregator needs to be called at the beginning + * of that superstep in preSuperstep(). + * + * @param name Name of aggregator + * @return boolean (false when not registered) + */ + boolean useAggregator(String name); } diff --git a/src/main/java/org/apache/giraph/graph/AggregatorWriter.java b/src/main/java/org/apache/giraph/graph/AggregatorWriter.java index 9a2a097ee..c998ccc65 100644 --- a/src/main/java/org/apache/giraph/graph/AggregatorWriter.java +++ b/src/main/java/org/apache/giraph/graph/AggregatorWriter.java @@ -25,49 +25,49 @@ import org.apache.hadoop.mapreduce.Mapper.Context; /** - * An AggregatorWriter is used to export Aggregators during or at the end of + * An AggregatorWriter is used to export Aggregators during or at the end of * each computation. It runs on the master and it's called at the end of each - * superstep. The special signal {@link AggregatorWriter#LAST_SUPERSTEP} is - * passed to {@link AggregatorWriter#writeAggregator(Map, long)} as the + * superstep. The special signal {@link AggregatorWriter#LAST_SUPERSTEP} is + * passed to {@link AggregatorWriter#writeAggregator(Map, long)} as the * superstep value to signal the end of computation. */ public interface AggregatorWriter { - /** Signal for last superstep */ - public static final int LAST_SUPERSTEP = -1; + /** Signal for last superstep */ + int LAST_SUPERSTEP = -1; - /** - * The method is called at the initialization of the AggregatorWriter. - * More precisely, the aggregatorWriter is initialized each time a new - * master is elected. - * - * @param context Mapper Context where the master is running on - * @param applicationAttempt ID of the applicationAttempt, used to - * disambiguate aggregator writes for different attempts - * @throws IOException - */ - @SuppressWarnings("rawtypes") - void initialize(Context context, long applicationAttempt) throws IOException; + /** + * The method is called at the initialization of the AggregatorWriter. + * More precisely, the aggregatorWriter is initialized each time a new + * master is elected. + * + * @param context Mapper Context where the master is running on + * @param applicationAttempt ID of the applicationAttempt, used to + * disambiguate aggregator writes for different attempts + * @throws IOException + */ + @SuppressWarnings("rawtypes") + void initialize(Context context, long applicationAttempt) throws IOException; - /** - * The method is called at the end of each superstep. The user might decide - * whether to write the aggregators values for the current superstep. For - * the last superstep, {@link AggregatorWriter#LAST_SUPERSTEP} is passed. - * - * @param aggregatorMap Map of aggregators to write - * @param superstep Current superstep - * @throws IOException - */ - void writeAggregator( - Map> aggregatorMap, - long superstep) throws IOException; + /** + * The method is called at the end of each superstep. The user might decide + * whether to write the aggregators values for the current superstep. For + * the last superstep, {@link AggregatorWriter#LAST_SUPERSTEP} is passed. + * + * @param aggregatorMap Map of aggregators to write + * @param superstep Current superstep + * @throws IOException + */ + void writeAggregator( + Map> aggregatorMap, + long superstep) throws IOException; - /** - * The method is called at the end of a successful computation. The method - * is not called when the job fails and a new master is elected. For this - * reason it's advised to flush data at the end of - * {@link AggregatorWriter#writeAggregator(Map, long)}. - * - * @throws IOException - */ - void close() throws IOException; + /** + * The method is called at the end of a successful computation. The method + * is not called when the job fails and a new master is elected. For this + * reason it's advised to flush data at the end of + * {@link AggregatorWriter#writeAggregator(Map, long)}. + * + * @throws IOException + */ + void close() throws IOException; } diff --git a/src/main/java/org/apache/giraph/graph/BasicVertex.java b/src/main/java/org/apache/giraph/graph/BasicVertex.java index 96efc3e3e..cd42ac668 100644 --- a/src/main/java/org/apache/giraph/graph/BasicVertex.java +++ b/src/main/java/org/apache/giraph/graph/BasicVertex.java @@ -1,4 +1,4 @@ - /* +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -28,249 +28,257 @@ import java.util.Iterator; import java.util.Map; - /** +/** * Basic interface for writing a BSP application for computation. * - * @param vertex id - * @param vertex data - * @param edge data - * @param message data + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data */ @SuppressWarnings("rawtypes") public abstract class BasicVertex - implements AggregatorUsage, Iterable, Writable, Configurable { - /** Global graph state **/ - private GraphState graphState; - /** Configuration */ - private Configuration conf; - /** If true, do not do anymore computation on this vertex. */ - boolean halt = false; - - /** - * This method must be called after instantiation of a vertex with BspUtils - * unless deserialization from readFields() is called. - * - * @param vertexId Will be the vertex id - * @param vertexValue Will be the vertex value - * @param edges A map of destination edge ids to edge values (can be null) - * @param messages Initial messages for this vertex (can be null) - */ - public abstract void initialize( - I vertexId, V vertexValue, Map edges, Iterable messages); - - /** - * Must be defined by user to do computation on a single Vertex. - * - * @param msgIterator Iterator to the messages that were sent to this - * vertex in the previous superstep - * @throws IOException - */ - public abstract void compute(Iterator msgIterator) throws IOException; - - /** - * Retrieves the current superstep. - * - * @return Current superstep - */ - public long getSuperstep() { - return getGraphState().getSuperstep(); - } - - /** - * Get the vertex id - */ - public abstract I getVertexId(); - - /** - * Get the vertex value (data stored with vertex) - * - * @return Vertex value - */ - public abstract V getVertexValue(); - - /** - * Set the vertex data (immediately visible in the computation) - * - * @param vertexValue Vertex data to be set - */ - public abstract void setVertexValue(V vertexValue); - - /** - * Get the total (all workers) number of vertices that - * existed in the previous superstep. - * - * @return Total number of vertices (-1 if first superstep) - */ - public long getNumVertices() { - return getGraphState().getNumVertices(); - } - - /** - * Get the total (all workers) number of edges that - * existed in the previous superstep. - * - * @return Total number of edges (-1 if first superstep) - */ - public long getNumEdges() { - return getGraphState().getNumEdges(); - } - - /** - * Get a read-only view of the out-edges of this vertex. - * - * @return the out edges (sort order determined by subclass implementation). - */ - @Override - public abstract Iterator iterator(); - - /** - * Get the edge value associated with a target vertex id. - * - * @param targetVertexId Target vertex id to check - * - * @return the value of the edge to targetVertexId (or null if there - * is no edge to it) - */ - public abstract E getEdgeValue(I targetVertexId); - - /** - * Does an edge with the target vertex id exist? - * - * @param targetVertexId Target vertex id to check - * @return true if there is an edge to the target - */ - public abstract boolean hasEdge(I targetVertexId); - - /** - * Get the number of outgoing edges on this vertex. - * - * @return the total number of outbound edges from this vertex - */ - public abstract int getNumOutEdges(); - - /** - * Send a message to a vertex id. The message should not be mutated after - * this method returns or else undefined results could occur. - * - * @param id Vertex id to send the message to - * @param msg Message data to send. Note that after the message is sent, - * the user should not modify the object. - */ - public void sendMsg(I id, M msg) { - if (msg == null) { - throw new IllegalArgumentException( - "sendMsg: Cannot send null message to " + id); - } - getGraphState().getWorkerCommunications(). - sendMessageReq(id, msg); - } - - /** - * Send a message to all edges. - */ - public abstract void sendMsgToAllEdges(M msg); - - /** - * After this is called, the compute() code will no longer be called for - * this vertex unless a message is sent to it. Then the compute() code - * will be called once again until this function is called. The - * application finishes only when all vertices vote to halt. - */ - public void voteToHalt() { - halt = true; - } - - /** - * Is this vertex done? - */ - public boolean isHalted() { - return halt; - } - - /** - * Get the list of incoming messages from the previous superstep. Same as - * the message iterator passed to compute(). - */ - public abstract Iterable getMessages(); - - /** - * Copy the messages this vertex should process in the current superstep - * - * @param messages the messages sent to this vertex in the previous superstep - */ - abstract void putMessages(Iterable messages); - - /** - * Release unnecessary resources (will be called after vertex returns from - * {@link #compute()}) - */ - abstract void releaseResources(); - - /** - * Get the graph state for all workers. - * - * @return Graph state for all workers - */ - GraphState getGraphState() { - return graphState; - } - - /** - * Set the graph state for all workers - * - * @param graphState Graph state for all workers - */ - void setGraphState(GraphState graphState) { - this.graphState = graphState; - } - - /** - * Get the mapper context - * - * @return Mapper context - */ - public Mapper.Context getContext() { - return getGraphState().getContext(); - } - - /** - * Get the worker context - * - * @return WorkerContext context - */ - public WorkerContext getWorkerContext() { - return getGraphState().getGraphMapper().getWorkerContext(); - } - - @Override - public final Aggregator registerAggregator( - String name, - Class> aggregatorClass) - throws InstantiationException, IllegalAccessException { - return getGraphState().getGraphMapper().getAggregatorUsage(). - registerAggregator(name, aggregatorClass); - } - - @Override - public final Aggregator getAggregator(String name) { - return getGraphState().getGraphMapper().getAggregatorUsage(). - getAggregator(name); - } - - @Override - public final boolean useAggregator(String name) { - return getGraphState().getGraphMapper().getAggregatorUsage(). - useAggregator(name); - } - - @Override - public Configuration getConf() { - return conf; - } - - @Override - public void setConf(Configuration conf) { - this.conf = conf; + V extends Writable, E extends Writable, M extends Writable> + implements AggregatorUsage, Iterable, Writable, Configurable { + /** If true, do not do anymore computation on this vertex. */ + protected boolean halt = false; + /** Global graph state **/ + private GraphState graphState; + /** Configuration */ + private Configuration conf; + + + /** + * This method must be called after instantiation of a vertex with BspUtils + * unless deserialization from readFields() is called. + * + * @param vertexId Will be the vertex id + * @param vertexValue Will be the vertex value + * @param edges A map of destination edge ids to edge values (can be null) + * @param messages Initial messages for this vertex (can be null) + */ + public abstract void initialize( + I vertexId, V vertexValue, Map edges, Iterable messages); + + /** + * Must be defined by user to do computation on a single Vertex. + * + * @param msgIterator Iterator to the messages that were sent to this + * vertex in the previous superstep + * @throws IOException + */ + public abstract void compute(Iterator msgIterator) throws IOException; + + /** + * Retrieves the current superstep. + * + * @return Current superstep + */ + public long getSuperstep() { + return getGraphState().getSuperstep(); + } + + /** + * Get the vertex id. + * + * @return My vertex id. + */ + public abstract I getVertexId(); + + /** + * Get the vertex value (data stored with vertex) + * + * @return Vertex value + */ + public abstract V getVertexValue(); + + /** + * Set the vertex data (immediately visible in the computation) + * + * @param vertexValue Vertex data to be set + */ + public abstract void setVertexValue(V vertexValue); + + /** + * Get the total (all workers) number of vertices that + * existed in the previous superstep. + * + * @return Total number of vertices (-1 if first superstep) + */ + public long getNumVertices() { + return getGraphState().getNumVertices(); + } + + /** + * Get the total (all workers) number of edges that + * existed in the previous superstep. + * + * @return Total number of edges (-1 if first superstep) + */ + public long getNumEdges() { + return getGraphState().getNumEdges(); + } + + /** + * Get a read-only view of the out-edges of this vertex. + * + * @return the out edges (sort order determined by subclass implementation). + */ + @Override + public abstract Iterator iterator(); + + /** + * Get the edge value associated with a target vertex id. + * + * @param targetVertexId Target vertex id to check + * + * @return the value of the edge to targetVertexId (or null if there + * is no edge to it) + */ + public abstract E getEdgeValue(I targetVertexId); + + /** + * Does an edge with the target vertex id exist? + * + * @param targetVertexId Target vertex id to check + * @return true if there is an edge to the target + */ + public abstract boolean hasEdge(I targetVertexId); + + /** + * Get the number of outgoing edges on this vertex. + * + * @return the total number of outbound edges from this vertex + */ + public abstract int getNumOutEdges(); + + /** + * Send a message to a vertex id. The message should not be mutated after + * this method returns or else undefined results could occur. + * + * @param id Vertex id to send the message to + * @param msg Message data to send. Note that after the message is sent, + * the user should not modify the object. + */ + public void sendMsg(I id, M msg) { + if (msg == null) { + throw new IllegalArgumentException( + "sendMsg: Cannot send null message to " + id); } + getGraphState().getWorkerCommunications(). + sendMessageReq(id, msg); + } + + /** + * Send a message to all edges. + * + * @param msg Message sent to all edges. + */ + public abstract void sendMsgToAllEdges(M msg); + + /** + * After this is called, the compute() code will no longer be called for + * this vertex unless a message is sent to it. Then the compute() code + * will be called once again until this function is called. The + * application finishes only when all vertices vote to halt. + */ + public void voteToHalt() { + halt = true; + } + + /** + * Is this vertex done? + * + * @return True if halted, false otherwise. + */ + public boolean isHalted() { + return halt; + } + + /** + * Get the list of incoming messages from the previous superstep. Same as + * the message iterator passed to compute(). + * + * @return Iterator of messages. + */ + public abstract Iterable getMessages(); + + /** + * Copy the messages this vertex should process in the current superstep + * + * @param messages the messages sent to this vertex in the previous superstep + */ + abstract void putMessages(Iterable messages); + + /** + * Release unnecessary resources (will be called after vertex returns from + * {@link #compute()}) + */ + abstract void releaseResources(); + + /** + * Get the graph state for all workers. + * + * @return Graph state for all workers + */ + GraphState getGraphState() { + return graphState; + } + + /** + * Set the graph state for all workers + * + * @param graphState Graph state for all workers + */ + void setGraphState(GraphState graphState) { + this.graphState = graphState; + } + + /** + * Get the mapper context + * + * @return Mapper context + */ + public Mapper.Context getContext() { + return getGraphState().getContext(); + } + + /** + * Get the worker context + * + * @return WorkerContext context + */ + public WorkerContext getWorkerContext() { + return getGraphState().getGraphMapper().getWorkerContext(); + } + + @Override + public final Aggregator registerAggregator( + String name, Class> aggregatorClass) + throws InstantiationException, IllegalAccessException { + return getGraphState().getGraphMapper().getAggregatorUsage(). + registerAggregator(name, aggregatorClass); + } + + @Override + public final Aggregator getAggregator(String name) { + return getGraphState().getGraphMapper().getAggregatorUsage(). + getAggregator(name); + } + + @Override + public final boolean useAggregator(String name) { + return getGraphState().getGraphMapper().getAggregatorUsage(). + useAggregator(name); + } + + @Override + public Configuration getConf() { + return conf; + } + + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } } diff --git a/src/main/java/org/apache/giraph/graph/BasicVertexResolver.java b/src/main/java/org/apache/giraph/graph/BasicVertexResolver.java index 780985421..51b43320f 100644 --- a/src/main/java/org/apache/giraph/graph/BasicVertexResolver.java +++ b/src/main/java/org/apache/giraph/graph/BasicVertexResolver.java @@ -24,36 +24,38 @@ /** * Handles all the situations that can arise upon creation/removal of * vertices and edges. + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data */ @SuppressWarnings("rawtypes") -public interface BasicVertexResolver< - I extends WritableComparable, - V extends Writable, - E extends Writable, - M extends Writable> { - /** - * A vertex may have been removed, created zero or more times and had - * zero or more messages sent to it. This method will handle all situations - * excluding the normal case (a vertex already exists and has zero or more - * messages sent it to). - * - * @param vertexId Vertex id (can be used for {@link BasicVertex}'s - * initialize()) - * @param vertex Original vertex or null if none - * @param vertexChanges Changes that happened to this vertex or null if none - * @param messages messages received in the last superstep or null if none - * @return Vertex to be returned, if null, and a vertex currently exists - * it will be removed - */ - BasicVertex resolve(I vertexId, - BasicVertex vertex, - VertexChanges vertexChanges, - Iterable messages); +public interface BasicVertexResolver { + /** + * A vertex may have been removed, created zero or more times and had + * zero or more messages sent to it. This method will handle all situations + * excluding the normal case (a vertex already exists and has zero or more + * messages sent it to). + * + * @param vertexId Vertex id (can be used for {@link BasicVertex}'s + * initialize()) + * @param vertex Original vertex or null if none + * @param vertexChanges Changes that happened to this vertex or null if none + * @param messages messages received in the last superstep or null if none + * @return Vertex to be returned, if null, and a vertex currently exists + * it will be removed + */ + BasicVertex resolve(I vertexId, + BasicVertex vertex, + VertexChanges vertexChanges, + Iterable messages); - /** - * Create a default vertex that can be used to return from resolve(). - * - * @return Newly instantiated vertex. - */ - BasicVertex instantiateVertex(); + /** + * Create a default vertex that can be used to return from resolve(). + * + * @return Newly instantiated vertex. + */ + BasicVertex instantiateVertex(); } diff --git a/src/main/java/org/apache/giraph/graph/BspService.java b/src/main/java/org/apache/giraph/graph/BspService.java index 0ee5da806..da9ab78a6 100644 --- a/src/main/java/org/apache/giraph/graph/BspService.java +++ b/src/main/java/org/apache/giraph/graph/BspService.java @@ -53,972 +53,1045 @@ /** * Zookeeper-based implementation of {@link CentralizedService}. + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data */ @SuppressWarnings("rawtypes") -public abstract class BspService < - I extends WritableComparable, - V extends Writable, - E extends Writable, - M extends Writable> - implements Watcher, CentralizedService { - /** Private ZooKeeper instance that implements the service */ - private final ZooKeeperExt zk; - /** Has the Connection occurred? */ - private final BspEvent connectedEvent = new PredicateLock(); - /** Has worker registration changed (either healthy or unhealthy) */ - private final BspEvent workerHealthRegistrationChanged = - new PredicateLock(); - /** InputSplits are ready for consumption by workers */ - private final BspEvent inputSplitsAllReadyChanged = - new PredicateLock(); - /** InputSplit reservation or finished notification and synchronization */ - private final BspEvent inputSplitsStateChanged = - new PredicateLock(); - /** InputSplits are done being processed by workers */ - private final BspEvent inputSplitsAllDoneChanged = - new PredicateLock(); - /** InputSplit done by a worker finished notification and synchronization */ - private final BspEvent inputSplitsDoneStateChanged = - new PredicateLock(); - /** Are the partition assignments to workers ready? */ - private final BspEvent partitionAssignmentsReadyChanged = - new PredicateLock(); - - /** Application attempt changed */ - private final BspEvent applicationAttemptChanged = - new PredicateLock(); - /** Superstep finished synchronization */ - private final BspEvent superstepFinished = - new PredicateLock(); - /** Master election changed for any waited on attempt */ - private final BspEvent masterElectionChildrenChanged = - new PredicateLock(); - /** Cleaned up directory children changed*/ - private final BspEvent cleanedUpChildrenChanged = - new PredicateLock(); - /** Registered list of BspEvents */ - private final List registeredBspEvents = - new ArrayList(); - /** Configuration of the job*/ - private final Configuration conf; - /** Job context (mainly for progress) */ - private final Mapper.Context context; - /** Cached superstep (from ZooKeeper) */ - private long cachedSuperstep = UNSET_SUPERSTEP; - /** Restarted from a checkpoint (manual or automatic) */ - private long restartedSuperstep = UNSET_SUPERSTEP; - /** Cached application attempt (from ZooKeeper) */ - private long cachedApplicationAttempt = UNSET_APPLICATION_ATTEMPT; - /** Job id, to ensure uniqueness */ - private final String jobId; - /** Task partition, to ensure uniqueness */ - private final int taskPartition; - /** My hostname */ - private final String hostname; - /** Combination of hostname '_' partition (unique id) */ - private final String hostnamePartitionId; - /** Graph partitioner */ - private final GraphPartitionerFactory graphPartitionerFactory; - /** Mapper that will do the graph computation */ - private final GraphMapper graphMapper; - /** Class logger */ - private static final Logger LOG = Logger.getLogger(BspService.class); - /** File system */ - private final FileSystem fs; - /** Checkpoint frequency */ - private int checkpointFrequency = -1; - /** Map of aggregators */ - private Map> aggregatorMap = - new TreeMap>(); - - /** Unset superstep */ - public static final long UNSET_SUPERSTEP = Long.MIN_VALUE; - /** Input superstep (superstep when loading the vertices happens) */ - public static final long INPUT_SUPERSTEP = -1; - /** Unset application attempt */ - public static final long UNSET_APPLICATION_ATTEMPT = Long.MIN_VALUE; - - private static final String BASE_DIR = "/_hadoopBsp"; - public static final String MASTER_JOB_STATE_NODE = "/_masterJobState"; - public static final String INPUT_SPLIT_DIR = "/_inputSplitDir"; - public static final String INPUT_SPLIT_DONE_DIR = "/_inputSplitDoneDir"; - public static final String INPUT_SPLIT_RESERVED_NODE = - "/_inputSplitReserved"; - public static final String INPUT_SPLIT_FINISHED_NODE = - "/_inputSplitFinished"; - public static final String INPUT_SPLITS_ALL_READY_NODE = - "/_inputSplitsAllReady"; - public static final String INPUT_SPLITS_ALL_DONE_NODE = - "/_inputSplitsAllDone"; - public static final String APPLICATION_ATTEMPTS_DIR = - "/_applicationAttemptsDir"; - public static final String MASTER_ELECTION_DIR = "/_masterElectionDir"; - public static final String SUPERSTEP_DIR = "/_superstepDir"; - public static final String MERGED_AGGREGATOR_DIR = - "/_mergedAggregatorDir"; - public static final String WORKER_HEALTHY_DIR = "/_workerHealthyDir"; - public static final String WORKER_UNHEALTHY_DIR = "/_workerUnhealthyDir"; - public static final String WORKER_FINISHED_DIR = "/_workerFinishedDir"; - public static final String PARTITION_ASSIGNMENTS_DIR = - "/_partitionAssignments"; - public static final String PARTITION_EXCHANGE_DIR = - "/_partitionExchangeDir"; - public static final String SUPERSTEP_FINISHED_NODE = "/_superstepFinished"; - public static final String CLEANED_UP_DIR = "/_cleanedUpDir"; - - public static final String JSONOBJ_AGGREGATOR_VALUE_ARRAY_KEY = - "_aggregatorValueArrayKey"; - public static final String JSONOBJ_PARTITION_STATS_KEY = - "_partitionStatsKey"; - public static final String JSONOBJ_FINISHED_VERTICES_KEY = - "_verticesFinishedKey"; - public static final String JSONOBJ_NUM_VERTICES_KEY = "_numVerticesKey"; - public static final String JSONOBJ_NUM_EDGES_KEY = "_numEdgesKey"; - public static final String JSONOBJ_NUM_MESSAGES_KEY = "_numMsgsKey"; - public static final String JSONOBJ_HOSTNAME_ID_KEY = "_hostnameIdKey"; - public static final String JSONOBJ_MAX_VERTEX_INDEX_KEY = - "_maxVertexIndexKey"; - public static final String JSONOBJ_HOSTNAME_KEY = "_hostnameKey"; - public static final String JSONOBJ_PORT_KEY = "_portKey"; - public static final String JSONOBJ_CHECKPOINT_FILE_PREFIX_KEY = - "_checkpointFilePrefixKey"; - public static final String JSONOBJ_PREVIOUS_HOSTNAME_KEY = - "_previousHostnameKey"; - public static final String JSONOBJ_PREVIOUS_PORT_KEY = "_previousPortKey"; - public static final String JSONOBJ_STATE_KEY = "_stateKey"; - public static final String JSONOBJ_APPLICATION_ATTEMPT_KEY = - "_applicationAttemptKey"; - public static final String JSONOBJ_SUPERSTEP_KEY = - "_superstepKey"; - public static final String AGGREGATOR_NAME_KEY = "_aggregatorNameKey"; - public static final String AGGREGATOR_CLASS_NAME_KEY = - "_aggregatorClassNameKey"; - public static final String AGGREGATOR_VALUE_KEY = "_aggregatorValueKey"; - - public static final String WORKER_SUFFIX = "_worker"; - public static final String MASTER_SUFFIX = "_master"; - - /** Path to the job's root */ - public final String BASE_PATH; - /** Path to the job state determined by the master (informative only) */ - public final String MASTER_JOB_STATE_PATH; - /** Path to the input splits written by the master */ - public final String INPUT_SPLIT_PATH; - /** Path to the input splits all ready to be processed by workers */ - public final String INPUT_SPLITS_ALL_READY_PATH; - /** Path to the input splits done */ - public final String INPUT_SPLIT_DONE_PATH; - /** Path to the input splits all done to notify the workers to proceed */ - public final String INPUT_SPLITS_ALL_DONE_PATH; - /** Path to the application attempts) */ - public final String APPLICATION_ATTEMPTS_PATH; - /** Path to the cleaned up notifications */ - public final String CLEANED_UP_PATH; - /** Path to the checkpoint's root (including job id) */ - public final String CHECKPOINT_BASE_PATH; - /** Path to the master election path */ - public final String MASTER_ELECTION_PATH; - - /** - * Get the superstep from a ZooKeeper path - * - * @param path Path to parse for the superstep - */ - public static long getSuperstepFromPath(String path) { - int foundSuperstepStart = path.indexOf(SUPERSTEP_DIR); - if (foundSuperstepStart == -1) { - throw new IllegalArgumentException( - "getSuperstepFromPath: Cannot find " + SUPERSTEP_DIR + - "from " + path); - } - foundSuperstepStart += SUPERSTEP_DIR.length() + 1; - int endIndex = foundSuperstepStart + - path.substring(foundSuperstepStart).indexOf("/"); - if (endIndex == -1) { - throw new IllegalArgumentException( - "getSuperstepFromPath: Cannot find end of superstep from " + - path); - } - if (LOG.isDebugEnabled()) { - LOG.debug("getSuperstepFromPath: Got path=" + path + - ", start=" + foundSuperstepStart + ", end=" + endIndex); - } - return Long.parseLong(path.substring(foundSuperstepStart, endIndex)); +public abstract class BspService + implements Watcher, CentralizedService { + /** Unset superstep */ + public static final long UNSET_SUPERSTEP = Long.MIN_VALUE; + /** Input superstep (superstep when loading the vertices happens) */ + public static final long INPUT_SUPERSTEP = -1; + /** Unset application attempt */ + public static final long UNSET_APPLICATION_ATTEMPT = Long.MIN_VALUE; + /** Base ZooKeeper directory */ + public static final String BASE_DIR = "/_hadoopBsp"; + /** Master job state znode above base dir */ + public static final String MASTER_JOB_STATE_NODE = "/_masterJobState"; + /** Input split directory about base dir */ + public static final String INPUT_SPLIT_DIR = "/_inputSplitDir"; + /** Input split done directory about base dir */ + public static final String INPUT_SPLIT_DONE_DIR = "/_inputSplitDoneDir"; + /** Denotes a reserved input split */ + public static final String INPUT_SPLIT_RESERVED_NODE = + "/_inputSplitReserved"; + /** Denotes a finished input split */ + public static final String INPUT_SPLIT_FINISHED_NODE = + "/_inputSplitFinished"; + /** Denotes that all the input splits are are ready for consumption */ + public static final String INPUT_SPLITS_ALL_READY_NODE = + "/_inputSplitsAllReady"; + /** Denotes that all the input splits are done. */ + public static final String INPUT_SPLITS_ALL_DONE_NODE = + "/_inputSplitsAllDone"; + /** Directory of attempts of this application */ + public static final String APPLICATION_ATTEMPTS_DIR = + "/_applicationAttemptsDir"; + /** Where the master election happens */ + public static final String MASTER_ELECTION_DIR = "/_masterElectionDir"; + /** Superstep scope */ + public static final String SUPERSTEP_DIR = "/_superstepDir"; + /** Where the merged aggregators are located */ + public static final String MERGED_AGGREGATOR_DIR = + "/_mergedAggregatorDir"; + /** Healthy workers register here. */ + public static final String WORKER_HEALTHY_DIR = "/_workerHealthyDir"; + /** Unhealthy workers register here. */ + public static final String WORKER_UNHEALTHY_DIR = "/_workerUnhealthyDir"; + /** Finished workers notify here */ + public static final String WORKER_FINISHED_DIR = "/_workerFinishedDir"; + /** Where the partition assignments are set */ + public static final String PARTITION_ASSIGNMENTS_DIR = + "/_partitionAssignments"; + /** Helps coordinate the partition exchnages */ + public static final String PARTITION_EXCHANGE_DIR = + "/_partitionExchangeDir"; + /** Denotes that the superstep is done */ + public static final String SUPERSTEP_FINISHED_NODE = "/_superstepFinished"; + /** Denotes which workers have been cleaned up */ + public static final String CLEANED_UP_DIR = "/_cleanedUpDir"; + /** JSON aggregator value array key */ + public static final String JSONOBJ_AGGREGATOR_VALUE_ARRAY_KEY = + "_aggregatorValueArrayKey"; + /** JSON partition stats key */ + public static final String JSONOBJ_PARTITION_STATS_KEY = + "_partitionStatsKey"; + /** JSON finished vertices key */ + public static final String JSONOBJ_FINISHED_VERTICES_KEY = + "_verticesFinishedKey"; + /** JSON vertex count key */ + public static final String JSONOBJ_NUM_VERTICES_KEY = "_numVerticesKey"; + /** JSON edge count key */ + public static final String JSONOBJ_NUM_EDGES_KEY = "_numEdgesKey"; + /** JSON message count key */ + public static final String JSONOBJ_NUM_MESSAGES_KEY = "_numMsgsKey"; + /** JSON hostname id key */ + public static final String JSONOBJ_HOSTNAME_ID_KEY = "_hostnameIdKey"; + /** JSON max vertex index key */ + public static final String JSONOBJ_MAX_VERTEX_INDEX_KEY = + "_maxVertexIndexKey"; + /** JSON hostname key */ + public static final String JSONOBJ_HOSTNAME_KEY = "_hostnameKey"; + /** JSON port key */ + public static final String JSONOBJ_PORT_KEY = "_portKey"; + /** JSON checkpoint file prefix key */ + public static final String JSONOBJ_CHECKPOINT_FILE_PREFIX_KEY = + "_checkpointFilePrefixKey"; + /** JSON previous hostname key */ + public static final String JSONOBJ_PREVIOUS_HOSTNAME_KEY = + "_previousHostnameKey"; + /** JSON previous port key */ + public static final String JSONOBJ_PREVIOUS_PORT_KEY = "_previousPortKey"; + /** JSON state key */ + public static final String JSONOBJ_STATE_KEY = "_stateKey"; + /** JSON application attempt key */ + public static final String JSONOBJ_APPLICATION_ATTEMPT_KEY = + "_applicationAttemptKey"; + /** JSON superstep key */ + public static final String JSONOBJ_SUPERSTEP_KEY = + "_superstepKey"; + /** Aggregator name key */ + public static final String AGGREGATOR_NAME_KEY = "_aggregatorNameKey"; + /** Aggregator class name key */ + public static final String AGGREGATOR_CLASS_NAME_KEY = + "_aggregatorClassNameKey"; + /** Aggregator value key */ + public static final String AGGREGATOR_VALUE_KEY = "_aggregatorValueKey"; + /** Suffix denotes a worker */ + public static final String WORKER_SUFFIX = "_worker"; + /** Suffix denotes a master */ + public static final String MASTER_SUFFIX = "_master"; + /** If at the end of a checkpoint file, indicates metadata */ + public static final String CHECKPOINT_METADATA_POSTFIX = ".metadata"; + /** + * If at the end of a checkpoint file, indicates vertices, edges, + * messages, etc. + */ + public static final String CHECKPOINT_VERTICES_POSTFIX = ".vertices"; + /** + * If at the end of a checkpoint file, indicates metadata and data is valid + * for the same filenames without .valid + */ + public static final String CHECKPOINT_VALID_POSTFIX = ".valid"; + /** + * If at the end of a checkpoint file, indicates the stitched checkpoint + * file prefixes. A checkpoint is not valid if this file does not exist. + */ + public static final String CHECKPOINT_FINALIZED_POSTFIX = ".finalized"; + /** Class logger */ + private static final Logger LOG = Logger.getLogger(BspService.class); + /** Path to the job's root */ + protected final String basePath; + /** Path to the job state determined by the master (informative only) */ + protected final String masterJobStatePath; + /** Path to the input splits written by the master */ + protected final String inputSplitsPath; + /** Path to the input splits all ready to be processed by workers */ + protected final String inputSplitsAllReadyPath; + /** Path to the input splits done */ + protected final String inputSplitsDonePath; + /** Path to the input splits all done to notify the workers to proceed */ + protected final String inputSplitsAllDonePath; + /** Path to the application attempts) */ + protected final String applicationAttemptsPath; + /** Path to the cleaned up notifications */ + protected final String cleanedUpPath; + /** Path to the checkpoint's root (including job id) */ + protected final String checkpointBasePath; + /** Path to the master election path */ + protected final String masterElectionPath; + /** Private ZooKeeper instance that implements the service */ + private final ZooKeeperExt zk; + /** Has the Connection occurred? */ + private final BspEvent connectedEvent = new PredicateLock(); + /** Has worker registration changed (either healthy or unhealthy) */ + private final BspEvent workerHealthRegistrationChanged = + new PredicateLock(); + /** InputSplits are ready for consumption by workers */ + private final BspEvent inputSplitsAllReadyChanged = + new PredicateLock(); + /** InputSplit reservation or finished notification and synchronization */ + private final BspEvent inputSplitsStateChanged = + new PredicateLock(); + /** InputSplits are done being processed by workers */ + private final BspEvent inputSplitsAllDoneChanged = + new PredicateLock(); + /** InputSplit done by a worker finished notification and synchronization */ + private final BspEvent inputSplitsDoneStateChanged = + new PredicateLock(); + /** Are the partition assignments to workers ready? */ + private final BspEvent partitionAssignmentsReadyChanged = + new PredicateLock(); + /** Application attempt changed */ + private final BspEvent applicationAttemptChanged = + new PredicateLock(); + /** Superstep finished synchronization */ + private final BspEvent superstepFinished = + new PredicateLock(); + /** Master election changed for any waited on attempt */ + private final BspEvent masterElectionChildrenChanged = + new PredicateLock(); + /** Cleaned up directory children changed*/ + private final BspEvent cleanedUpChildrenChanged = + new PredicateLock(); + /** Registered list of BspEvents */ + private final List registeredBspEvents = + new ArrayList(); + /** Configuration of the job*/ + private final Configuration conf; + /** Job context (mainly for progress) */ + private final Mapper.Context context; + /** Cached superstep (from ZooKeeper) */ + private long cachedSuperstep = UNSET_SUPERSTEP; + /** Restarted from a checkpoint (manual or automatic) */ + private long restartedSuperstep = UNSET_SUPERSTEP; + /** Cached application attempt (from ZooKeeper) */ + private long cachedApplicationAttempt = UNSET_APPLICATION_ATTEMPT; + /** Job id, to ensure uniqueness */ + private final String jobId; + /** Task partition, to ensure uniqueness */ + private final int taskPartition; + /** My hostname */ + private final String hostname; + /** Combination of hostname '_' partition (unique id) */ + private final String hostnamePartitionId; + /** Graph partitioner */ + private final GraphPartitionerFactory graphPartitionerFactory; + /** Mapper that will do the graph computation */ + private final GraphMapper graphMapper; + /** File system */ + private final FileSystem fs; + /** Checkpoint frequency */ + private int checkpointFrequency = -1; + /** Map of aggregators */ + private Map> aggregatorMap = + new TreeMap>(); + + /** + * Constructor. + * + * @param serverPortList ZooKeeper server port list + * @param sessionMsecTimeout ZooKeeper session timeount in milliseconds + * @param context Mapper context + * @param graphMapper Graph mapper reference + */ + public BspService(String serverPortList, + int sessionMsecTimeout, + Mapper.Context context, + GraphMapper graphMapper) { + registerBspEvent(connectedEvent); + registerBspEvent(workerHealthRegistrationChanged); + registerBspEvent(inputSplitsAllReadyChanged); + registerBspEvent(inputSplitsStateChanged); + registerBspEvent(partitionAssignmentsReadyChanged); + registerBspEvent(applicationAttemptChanged); + registerBspEvent(superstepFinished); + registerBspEvent(masterElectionChildrenChanged); + registerBspEvent(cleanedUpChildrenChanged); + + this.context = context; + this.graphMapper = graphMapper; + this.conf = context.getConfiguration(); + this.jobId = conf.get("mapred.job.id", "Unknown Job"); + this.taskPartition = conf.getInt("mapred.task.partition", -1); + this.restartedSuperstep = conf.getLong(GiraphJob.RESTART_SUPERSTEP, + UNSET_SUPERSTEP); + this.cachedSuperstep = restartedSuperstep; + if ((restartedSuperstep != UNSET_SUPERSTEP) && + (restartedSuperstep < 0)) { + throw new IllegalArgumentException( + "BspService: Invalid superstep to restart - " + + restartedSuperstep); } - - /** - * Get the hostname and id from a "healthy" worker path - */ - public static String getHealthyHostnameIdFromPath(String path) { - int foundWorkerHealthyStart = path.indexOf(WORKER_HEALTHY_DIR); - if (foundWorkerHealthyStart == -1) { - throw new IllegalArgumentException( - "getHealthyHostnameidFromPath: Couldn't find " + - WORKER_HEALTHY_DIR + " from " + path); - } - foundWorkerHealthyStart += WORKER_HEALTHY_DIR.length(); - return path.substring(foundWorkerHealthyStart); + try { + this.hostname = InetAddress.getLocalHost().getHostName(); + } catch (UnknownHostException e) { + throw new RuntimeException(e); } - - /** - * Generate the base superstep directory path for a given application - * attempt - * - * @param attempt application attempt number - * @return directory path based on the an attempt - */ - final public String getSuperstepPath(long attempt) { - return APPLICATION_ATTEMPTS_PATH + "/" + attempt + SUPERSTEP_DIR; + this.hostnamePartitionId = hostname + "_" + getTaskPartition(); + this.graphPartitionerFactory = + BspUtils.createGraphPartitioner(conf); + + this.checkpointFrequency = + conf.getInt(GiraphJob.CHECKPOINT_FREQUENCY, + GiraphJob.CHECKPOINT_FREQUENCY_DEFAULT); + + basePath = ZooKeeperManager.getBasePath(conf) + BASE_DIR + "/" + jobId; + masterJobStatePath = basePath + MASTER_JOB_STATE_NODE; + inputSplitsPath = basePath + INPUT_SPLIT_DIR; + inputSplitsAllReadyPath = basePath + INPUT_SPLITS_ALL_READY_NODE; + inputSplitsDonePath = basePath + INPUT_SPLIT_DONE_DIR; + inputSplitsAllDonePath = basePath + INPUT_SPLITS_ALL_DONE_NODE; + applicationAttemptsPath = basePath + APPLICATION_ATTEMPTS_DIR; + cleanedUpPath = basePath + CLEANED_UP_DIR; + checkpointBasePath = + getConfiguration().get( + GiraphJob.CHECKPOINT_DIRECTORY, + GiraphJob.CHECKPOINT_DIRECTORY_DEFAULT + "/" + getJobId()); + masterElectionPath = basePath + MASTER_ELECTION_DIR; + if (LOG.isInfoEnabled()) { + LOG.info("BspService: Connecting to ZooKeeper with job " + jobId + + ", " + getTaskPartition() + " on " + serverPortList); } - - /** - * Generate the worker information "healthy" directory path for a - * superstep - * - * @param attempt application attempt number - * @param superstep superstep to use - * @return directory path based on the a superstep - */ - final public String getWorkerInfoHealthyPath(long attempt, - long superstep) { - return APPLICATION_ATTEMPTS_PATH + "/" + attempt + - SUPERSTEP_DIR + "/" + superstep + WORKER_HEALTHY_DIR; + try { + this.zk = new ZooKeeperExt(serverPortList, sessionMsecTimeout, this); + connectedEvent.waitForever(); + this.fs = FileSystem.get(getConfiguration()); + } catch (IOException e) { + throw new RuntimeException(e); } - - /** - * Generate the worker information "unhealthy" directory path for a - * superstep - * - * @param attempt application attempt number - * @param superstep superstep to use - * @return directory path based on the a superstep - */ - final public String getWorkerInfoUnhealthyPath(long attempt, - long superstep) { - return APPLICATION_ATTEMPTS_PATH + "/" + attempt + - SUPERSTEP_DIR + "/" + superstep + WORKER_UNHEALTHY_DIR; + } + + + /** + * Get the superstep from a ZooKeeper path + * + * @param path Path to parse for the superstep + * @return Superstep from the path. + */ + public static long getSuperstepFromPath(String path) { + int foundSuperstepStart = path.indexOf(SUPERSTEP_DIR); + if (foundSuperstepStart == -1) { + throw new IllegalArgumentException( + "getSuperstepFromPath: Cannot find " + SUPERSTEP_DIR + + "from " + path); } - - /** - * Generate the worker "finished" directory path for a - * superstep - * - * @param attempt application attempt number - * @param superstep superstep to use - * @return directory path based on the a superstep - */ - final public String getWorkerFinishedPath(long attempt, long superstep) { - return APPLICATION_ATTEMPTS_PATH + "/" + attempt + - SUPERSTEP_DIR + "/" + superstep + WORKER_FINISHED_DIR; + foundSuperstepStart += SUPERSTEP_DIR.length() + 1; + int endIndex = foundSuperstepStart + + path.substring(foundSuperstepStart).indexOf("/"); + if (endIndex == -1) { + throw new IllegalArgumentException( + "getSuperstepFromPath: Cannot find end of superstep from " + + path); } - - /** - * Generate the "partiton assignments" directory path for a superstep - * - * @param attempt application attempt number - * @param superstep superstep to use - * @return directory path based on the a superstep - */ - final public String getPartitionAssignmentsPath(long attempt, - long superstep) { - return APPLICATION_ATTEMPTS_PATH + "/" + attempt + - SUPERSTEP_DIR + "/" + superstep + PARTITION_ASSIGNMENTS_DIR; + if (LOG.isDebugEnabled()) { + LOG.debug("getSuperstepFromPath: Got path=" + path + + ", start=" + foundSuperstepStart + ", end=" + endIndex); } - - /** - * Generate the "partition exchange" directory path for a superstep - * - * @param attempt application attempt number - * @param superstep superstep to use - * @return directory path based on the a superstep - */ - final public String getPartitionExchangePath(long attempt, - long superstep) { - return APPLICATION_ATTEMPTS_PATH + "/" + attempt + - SUPERSTEP_DIR + "/" + superstep + PARTITION_EXCHANGE_DIR; + return Long.parseLong(path.substring(foundSuperstepStart, endIndex)); + } + + /** + * Get the hostname and id from a "healthy" worker path + * + * @param path Path to check + * @return Hostname and id from path + */ + public static String getHealthyHostnameIdFromPath(String path) { + int foundWorkerHealthyStart = path.indexOf(WORKER_HEALTHY_DIR); + if (foundWorkerHealthyStart == -1) { + throw new IllegalArgumentException( + "getHealthyHostnameidFromPath: Couldn't find " + + WORKER_HEALTHY_DIR + " from " + path); } - - final public String getPartitionExchangeWorkerPath(long attempt, - long superstep, - WorkerInfo workerInfo) { - return getPartitionExchangePath(attempt, superstep) + - "/" + workerInfo.getHostnameId(); + foundWorkerHealthyStart += WORKER_HEALTHY_DIR.length(); + return path.substring(foundWorkerHealthyStart); + } + + /** + * Generate the base superstep directory path for a given application + * attempt + * + * @param attempt application attempt number + * @return directory path based on the an attempt + */ + public final String getSuperstepPath(long attempt) { + return applicationAttemptsPath + "/" + attempt + SUPERSTEP_DIR; + } + + /** + * Generate the worker information "healthy" directory path for a + * superstep + * + * @param attempt application attempt number + * @param superstep superstep to use + * @return directory path based on the a superstep + */ + public final String getWorkerInfoHealthyPath(long attempt, + long superstep) { + return applicationAttemptsPath + "/" + attempt + + SUPERSTEP_DIR + "/" + superstep + WORKER_HEALTHY_DIR; + } + + /** + * Generate the worker information "unhealthy" directory path for a + * superstep + * + * @param attempt application attempt number + * @param superstep superstep to use + * @return directory path based on the a superstep + */ + public final String getWorkerInfoUnhealthyPath(long attempt, + long superstep) { + return applicationAttemptsPath + "/" + attempt + + SUPERSTEP_DIR + "/" + superstep + WORKER_UNHEALTHY_DIR; + } + + /** + * Generate the worker "finished" directory path for a + * superstep + * + * @param attempt application attempt number + * @param superstep superstep to use + * @return directory path based on the a superstep + */ + public final String getWorkerFinishedPath(long attempt, long superstep) { + return applicationAttemptsPath + "/" + attempt + + SUPERSTEP_DIR + "/" + superstep + WORKER_FINISHED_DIR; + } + + /** + * Generate the "partiton assignments" directory path for a superstep + * + * @param attempt application attempt number + * @param superstep superstep to use + * @return directory path based on the a superstep + */ + public final String getPartitionAssignmentsPath(long attempt, + long superstep) { + return applicationAttemptsPath + "/" + attempt + + SUPERSTEP_DIR + "/" + superstep + PARTITION_ASSIGNMENTS_DIR; + } + + /** + * Generate the "partition exchange" directory path for a superstep + * + * @param attempt application attempt number + * @param superstep superstep to use + * @return directory path based on the a superstep + */ + public final String getPartitionExchangePath(long attempt, + long superstep) { + return applicationAttemptsPath + "/" + attempt + + SUPERSTEP_DIR + "/" + superstep + PARTITION_EXCHANGE_DIR; + } + + /** + * Based on the superstep, worker info, and attempt, get the appropriate + * worker path for the exchange. + * + * @param attempt Application attempt + * @param superstep Superstep + * @param workerInfo Worker info of the exchange. + * @return Path of the desired worker + */ + public final String getPartitionExchangeWorkerPath(long attempt, + long superstep, + WorkerInfo workerInfo) { + return getPartitionExchangePath(attempt, superstep) + + "/" + workerInfo.getHostnameId(); + } + + /** + * Generate the merged aggregator directory path for a superstep + * + * @param attempt application attempt number + * @param superstep superstep to use + * @return directory path based on the a superstep + */ + public final String getMergedAggregatorPath(long attempt, long superstep) { + return applicationAttemptsPath + "/" + attempt + + SUPERSTEP_DIR + "/" + superstep + MERGED_AGGREGATOR_DIR; + } + + /** + * Generate the "superstep finished" directory path for a superstep + * + * @param attempt application attempt number + * @param superstep superstep to use + * @return directory path based on the a superstep + */ + public final String getSuperstepFinishedPath(long attempt, long superstep) { + return applicationAttemptsPath + "/" + attempt + + SUPERSTEP_DIR + "/" + superstep + SUPERSTEP_FINISHED_NODE; + } + + /** + * Generate the base superstep directory path for a given application + * attempt + * + * @param superstep Superstep to use + * @return Directory path based on the a superstep + */ + public final String getCheckpointBasePath(long superstep) { + return checkpointBasePath + "/" + superstep; + } + + /** + * Get the checkpoint from a finalized checkpoint path + * + * @param finalizedPath Path of the finalized checkpoint + * @return Superstep referring to a checkpoint of the finalized path + */ + public static long getCheckpoint(Path finalizedPath) { + if (!finalizedPath.getName().endsWith(CHECKPOINT_FINALIZED_POSTFIX)) { + throw new InvalidParameterException( + "getCheckpoint: " + finalizedPath + "Doesn't end in " + + CHECKPOINT_FINALIZED_POSTFIX); } - - /** - * Generate the merged aggregator directory path for a superstep - * - * @param attempt application attempt number - * @param superstep superstep to use - * @return directory path based on the a superstep - */ - final public String getMergedAggregatorPath(long attempt, long superstep) { - return APPLICATION_ATTEMPTS_PATH + "/" + attempt + - SUPERSTEP_DIR + "/" + superstep + MERGED_AGGREGATOR_DIR; + String checkpointString = + finalizedPath.getName().replace(CHECKPOINT_FINALIZED_POSTFIX, ""); + return Long.parseLong(checkpointString); + } + + /** + * Get the ZooKeeperExt instance. + * + * @return ZooKeeperExt instance. + */ + public final ZooKeeperExt getZkExt() { + return zk; + } + + @Override + public final long getRestartedSuperstep() { + return restartedSuperstep; + } + + /** + * Set the restarted superstep + * + * @param superstep Set the manually restarted superstep + */ + public final void setRestartedSuperstep(long superstep) { + if (superstep < INPUT_SUPERSTEP) { + throw new IllegalArgumentException( + "setRestartedSuperstep: Bad argument " + superstep); } - - /** - * Generate the "superstep finished" directory path for a superstep - * - * @param attempt application attempt number - * @param superstep superstep to use - * @return directory path based on the a superstep - */ - final public String getSuperstepFinishedPath(long attempt, long superstep) { - return APPLICATION_ATTEMPTS_PATH + "/" + attempt + - SUPERSTEP_DIR + "/" + superstep + SUPERSTEP_FINISHED_NODE; + restartedSuperstep = superstep; + } + + /** + * Should checkpoint on this superstep? If checkpointing, always + * checkpoint the first user superstep. If restarting, the first + * checkpoint is after the frequency has been met. + * + * @param superstep Decide if checkpointing no this superstep + * @return True if this superstep should be checkpointed, false otherwise + */ + public final boolean checkpointFrequencyMet(long superstep) { + if (checkpointFrequency == 0) { + return false; } - - /** - * Generate the base superstep directory path for a given application - * attempt - * - * @param superstep Superstep to use - * @return Directory path based on the a superstep - */ - final public String getCheckpointBasePath(long superstep) { - return CHECKPOINT_BASE_PATH + "/" + superstep; + long firstCheckpoint = INPUT_SUPERSTEP + 1; + if (getRestartedSuperstep() != UNSET_SUPERSTEP) { + firstCheckpoint = getRestartedSuperstep() + checkpointFrequency; } - - /** If at the end of a checkpoint file, indicates metadata */ - public final String CHECKPOINT_METADATA_POSTFIX = ".metadata"; - - /** - * If at the end of a checkpoint file, indicates vertices, edges, - * messages, etc. - */ - public final String CHECKPOINT_VERTICES_POSTFIX = ".vertices"; - - /** - * If at the end of a checkpoint file, indicates metadata and data is valid - * for the same filenames without .valid - */ - public final String CHECKPOINT_VALID_POSTFIX = ".valid"; - - /** - * If at the end of a checkpoint file, indicates the stitched checkpoint - * file prefixes. A checkpoint is not valid if this file does not exist. - */ - public static final String CHECKPOINT_FINALIZED_POSTFIX = ".finalized"; - - /** - * Get the checkpoint from a finalized checkpoint path - * - * @param finalizedPath Path of the finalized checkpoint - * @return Superstep referring to a checkpoint of the finalized path - */ - public static long getCheckpoint(Path finalizedPath) { - if (!finalizedPath.getName().endsWith(CHECKPOINT_FINALIZED_POSTFIX)) { - throw new InvalidParameterException( - "getCheckpoint: " + finalizedPath + "Doesn't end in " + - CHECKPOINT_FINALIZED_POSTFIX); - } - String checkpointString = - finalizedPath.getName().replace(CHECKPOINT_FINALIZED_POSTFIX, ""); - return Long.parseLong(checkpointString); + if (superstep < firstCheckpoint) { + return false; + } else if (((superstep - firstCheckpoint) % checkpointFrequency) == 0) { + return true; } - - /** - * Get the ZooKeeperExt instance. - * - * @return ZooKeeperExt instance. - */ - final public ZooKeeperExt getZkExt() { - return zk; + return false; + } + + /** + * Get the file system + * + * @return file system + */ + public final FileSystem getFs() { + return fs; + } + + public final Configuration getConfiguration() { + return conf; + } + + public final Mapper.Context getContext() { + return context; + } + + public final String getHostname() { + return hostname; + } + + public final String getHostnamePartitionId() { + return hostnamePartitionId; + } + + public final int getTaskPartition() { + return taskPartition; + } + + public final GraphMapper getGraphMapper() { + return graphMapper; + } + + public final BspEvent getWorkerHealthRegistrationChangedEvent() { + return workerHealthRegistrationChanged; + } + + public final BspEvent getInputSplitsAllReadyEvent() { + return inputSplitsAllReadyChanged; + } + + public final BspEvent getInputSplitsStateChangedEvent() { + return inputSplitsStateChanged; + } + + public final BspEvent getInputSplitsAllDoneEvent() { + return inputSplitsAllDoneChanged; + } + + public final BspEvent getInputSplitsDoneStateChangedEvent() { + return inputSplitsDoneStateChanged; + } + + public final BspEvent getPartitionAssignmentsReadyChangedEvent() { + return partitionAssignmentsReadyChanged; + } + + + public final BspEvent getApplicationAttemptChangedEvent() { + return applicationAttemptChanged; + } + + public final BspEvent getSuperstepFinishedEvent() { + return superstepFinished; + } + + + public final BspEvent getMasterElectionChildrenChangedEvent() { + return masterElectionChildrenChanged; + } + + public final BspEvent getCleanedUpChildrenChangedEvent() { + return cleanedUpChildrenChanged; + } + + /** + * Get the master commanded job state as a JSONObject. Also sets the + * watches to see if the master commanded job state changes. + * + * @return Last job state or null if none + * @throws InterruptedException + * @throws KeeperException + */ + public final JSONObject getJobState() { + try { + getZkExt().createExt(masterJobStatePath, + null, + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT, + true); + } catch (KeeperException.NodeExistsException e) { + LOG.info("getJobState: Job state already exists (" + + masterJobStatePath + ")"); + } catch (KeeperException e) { + throw new IllegalStateException("Failed to create job state path " + + "due to KeeperException", e); + } catch (InterruptedException e) { + throw new IllegalStateException("Failed to create job state path " + + "due to InterruptedException", e); } - - @Override - final public long getRestartedSuperstep() { - return restartedSuperstep; + String jobState = null; + try { + List childList = + getZkExt().getChildrenExt( + masterJobStatePath, true, true, true); + if (childList.isEmpty()) { + return null; + } + jobState = + new String(getZkExt().getData( + childList.get(childList.size() - 1), true, null)); + } catch (KeeperException.NoNodeException e) { + LOG.info("getJobState: Job state path is empty! - " + + masterJobStatePath); + } catch (KeeperException e) { + throw new IllegalStateException("Failed to get job state path " + + "children due to KeeperException", e); + } catch (InterruptedException e) { + throw new IllegalStateException("Failed to get job state path " + + "children due to InterruptedException", e); } - - /** - * Set the restarted superstep - * - * @param superstep Set the manually restarted superstep - */ - final public void setRestartedSuperstep(long superstep) { - if (superstep < INPUT_SUPERSTEP) { - throw new IllegalArgumentException( - "setRestartedSuperstep: Bad argument " + superstep); - } - restartedSuperstep = superstep; + try { + return new JSONObject(jobState); + } catch (JSONException e) { + throw new RuntimeException( + "getJobState: Failed to parse job state " + jobState); } - - /** - * Should checkpoint on this superstep? If checkpointing, always - * checkpoint the first user superstep. If restarting, the first - * checkpoint is after the frequency has been met. - * - * @param superstep Decide if checkpointing no this superstep - * @return True if this superstep should be checkpointed, false otherwise - */ - final public boolean checkpointFrequencyMet(long superstep) { - if (checkpointFrequency == 0) { - return false; - } - long firstCheckpoint = INPUT_SUPERSTEP + 1; - if (getRestartedSuperstep() != UNSET_SUPERSTEP) { - firstCheckpoint = getRestartedSuperstep() + checkpointFrequency; - } - if (superstep < firstCheckpoint) { - return false; - } else if (((superstep - firstCheckpoint) % checkpointFrequency) == 0) { - return true; - } else { - return false; - } + } + + /** + * Get the job id + * + * @return job id + */ + public final String getJobId() { + return jobId; + } + + /** + * Get the latest application attempt and cache it. + * + * @return the latest application attempt + */ + public final long getApplicationAttempt() { + if (cachedApplicationAttempt != UNSET_APPLICATION_ATTEMPT) { + return cachedApplicationAttempt; } - - /** - * Get the file system - * - * @return file system - */ - final public FileSystem getFs() { - return fs; + try { + getZkExt().createExt(applicationAttemptsPath, + null, + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT, + true); + } catch (KeeperException.NodeExistsException e) { + LOG.info("getApplicationAttempt: Node " + + applicationAttemptsPath + " already exists!"); + } catch (KeeperException e) { + throw new IllegalStateException("Couldn't create application " + + "attempts path due to KeeperException", e); + } catch (InterruptedException e) { + throw new IllegalStateException("Couldn't create application " + + "attempts path due to InterruptedException", e); } - - final public Configuration getConfiguration() { - return conf; + try { + List attemptList = + getZkExt().getChildrenExt( + applicationAttemptsPath, true, false, false); + if (attemptList.isEmpty()) { + cachedApplicationAttempt = 0; + } else { + cachedApplicationAttempt = + Long.parseLong(Collections.max(attemptList)); + } + } catch (KeeperException e) { + throw new IllegalStateException("Couldn't get application " + + "attempts to KeeperException", e); + } catch (InterruptedException e) { + throw new IllegalStateException("Couldn't get application " + + "attempts to InterruptedException", e); } - final public Mapper.Context getContext() { - return context; + return cachedApplicationAttempt; + } + + /** + * Get the latest superstep and cache it. + * + * @return the latest superstep + * @throws InterruptedException + * @throws KeeperException + */ + public final long getSuperstep() { + if (cachedSuperstep != UNSET_SUPERSTEP) { + return cachedSuperstep; } - - final public String getHostname() { - return hostname; + String superstepPath = getSuperstepPath(getApplicationAttempt()); + try { + getZkExt().createExt(superstepPath, + null, + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT, + true); + } catch (KeeperException.NodeExistsException e) { + if (LOG.isInfoEnabled()) { + LOG.info("getApplicationAttempt: Node " + + applicationAttemptsPath + " already exists!"); + } + } catch (KeeperException e) { + throw new IllegalStateException( + "getSuperstep: KeeperException", e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "getSuperstep: InterruptedException", e); } - final public String getHostnamePartitionId() { - return hostnamePartitionId; + List superstepList; + try { + superstepList = + getZkExt().getChildrenExt(superstepPath, true, false, false); + } catch (KeeperException e) { + throw new IllegalStateException( + "getSuperstep: KeeperException", e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "getSuperstep: InterruptedException", e); } - - final public int getTaskPartition() { - return taskPartition; + if (superstepList.isEmpty()) { + cachedSuperstep = INPUT_SUPERSTEP; + } else { + cachedSuperstep = + Long.parseLong(Collections.max(superstepList)); } - final public GraphMapper getGraphMapper() { - return graphMapper; + return cachedSuperstep; + } + + /** + * Increment the cached superstep. Shouldn't be the initial value anymore. + */ + public final void incrCachedSuperstep() { + if (cachedSuperstep == UNSET_SUPERSTEP) { + throw new IllegalStateException( + "incrSuperstep: Invalid unset cached superstep " + + UNSET_SUPERSTEP); } - - final public BspEvent getWorkerHealthRegistrationChangedEvent() { - return workerHealthRegistrationChanged; + ++cachedSuperstep; + } + + /** + * Set the cached superstep (should only be used for loading checkpoints + * or recovering from failure). + * + * @param superstep will be used as the next superstep iteration + */ + public final void setCachedSuperstep(long superstep) { + cachedSuperstep = superstep; + } + + /** + * Set the cached application attempt (should only be used for restart from + * failure by the master) + * + * @param applicationAttempt Will denote the new application attempt + */ + public final void setApplicationAttempt(long applicationAttempt) { + cachedApplicationAttempt = applicationAttempt; + String superstepPath = getSuperstepPath(cachedApplicationAttempt); + try { + getZkExt().createExt(superstepPath, + null, + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT, + true); + } catch (KeeperException.NodeExistsException e) { + throw new IllegalArgumentException( + "setApplicationAttempt: Attempt already exists! - " + + superstepPath, e); + } catch (KeeperException e) { + throw new RuntimeException( + "setApplicationAttempt: KeeperException - " + + superstepPath, e); + } catch (InterruptedException e) { + throw new RuntimeException( + "setApplicationAttempt: InterruptedException - " + + superstepPath, e); } - - final public BspEvent getInputSplitsAllReadyEvent() { - return inputSplitsAllReadyChanged; + } + + /** + * Register an aggregator with name. + * + * @param Aggregator type + * @param name Name of the aggregator + * @param aggregatorClass Class of the aggregator + * @return Aggregator + * @throws IllegalAccessException + * @throws InstantiationException + */ + public final Aggregator registerAggregator( + String name, + Class> aggregatorClass) + throws InstantiationException, IllegalAccessException { + if (aggregatorMap.get(name) != null) { + return null; } - - final public BspEvent getInputSplitsStateChangedEvent() { - return inputSplitsStateChanged; + Aggregator aggregator = + (Aggregator) aggregatorClass.newInstance(); + @SuppressWarnings("unchecked") + Aggregator writableAggregator = + (Aggregator) aggregator; + aggregatorMap.put(name, writableAggregator); + if (LOG.isInfoEnabled()) { + LOG.info("registerAggregator: registered " + name); } - - final public BspEvent getInputSplitsAllDoneEvent() { - return inputSplitsAllDoneChanged; - } - - final public BspEvent getInputSplitsDoneStateChangedEvent() { - return inputSplitsDoneStateChanged; + return aggregator; + } + + /** + * Get aggregator by name. + * + * @param name Name of aggregator + * @return Aggregator or null when not registered + */ + public final Aggregator getAggregator(String name) { + return aggregatorMap.get(name); + } + + /** + * Get the aggregator map. + * + * @return Map of aggregator names to aggregator + */ + public Map> getAggregatorMap() { + return aggregatorMap; + } + + /** + * Register a BspEvent. Ensure that it will be signaled + * by catastrophic failure so that threads waiting on an event signal + * will be unblocked. + * + * @param event Event to be registered. + */ + public void registerBspEvent(BspEvent event) { + registeredBspEvents.add(event); + } + + /** + * Subclasses can use this to instantiate their respective partitioners + * + * @return Instantiated graph partitioner factory + */ + protected GraphPartitionerFactory getGraphPartitionerFactory() { + return graphPartitionerFactory; + } + + /** + * Derived classes that want additional ZooKeeper events to take action + * should override this. + * + * @param event Event that occurred + * @return true if the event was processed here, false otherwise + */ + protected boolean processEvent(WatchedEvent event) { + return false; + } + + @Override + public final void process(WatchedEvent event) { + // 1. Process all shared events + // 2. Process specific derived class events + if (LOG.isDebugEnabled()) { + LOG.debug("process: Got a new event, path = " + event.getPath() + + ", type = " + event.getType() + ", state = " + + event.getState()); } - final public BspEvent getPartitionAssignmentsReadyChangedEvent() { - return partitionAssignmentsReadyChanged; - } - - - final public BspEvent getApplicationAttemptChangedEvent() { - return applicationAttemptChanged; - } - - final public BspEvent getSuperstepFinishedEvent() { - return superstepFinished; - } - - - final public BspEvent getMasterElectionChildrenChangedEvent() { - return masterElectionChildrenChanged; - } - - final public BspEvent getCleanedUpChildrenChangedEvent() { - return cleanedUpChildrenChanged; - } - - /** - * Get the master commanded job state as a JSONObject. Also sets the - * watches to see if the master commanded job state changes. - * - * @return Last job state or null if none - */ - final public JSONObject getJobState() { - try { - getZkExt().createExt(MASTER_JOB_STATE_PATH, - null, - Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT, - true); - } catch (KeeperException.NodeExistsException e) { - LOG.info("getJobState: Job state already exists (" + - MASTER_JOB_STATE_PATH + ")"); - } catch (Exception e) { - throw new RuntimeException(e); + if ((event.getPath() == null) && (event.getType() == EventType.None)) { + if (event.getState() == KeeperState.Disconnected) { + // No way to recover from a disconnect event, signal all BspEvents + for (BspEvent bspEvent : registeredBspEvents) { + bspEvent.signal(); } - String jobState = null; - try { - List childList = - getZkExt().getChildrenExt( - MASTER_JOB_STATE_PATH, true, true, true); - if (childList.isEmpty()) { - return null; - } - jobState = - new String(getZkExt().getData( - childList.get(childList.size() - 1), true, null)); - } catch (KeeperException.NoNodeException e) { - LOG.info("getJobState: Job state path is empty! - " + - MASTER_JOB_STATE_PATH); - } catch (Exception e) { - throw new RuntimeException(e); - } - try { - return new JSONObject(jobState); - } catch (JSONException e) { - throw new RuntimeException( - "getJobState: Failed to parse job state " + jobState); - } - } - - public BspService(String serverPortList, - int sessionMsecTimeout, - Mapper.Context context, - GraphMapper graphMapper) { - registerBspEvent(connectedEvent); - registerBspEvent(workerHealthRegistrationChanged); - registerBspEvent(inputSplitsAllReadyChanged); - registerBspEvent(inputSplitsStateChanged); - registerBspEvent(partitionAssignmentsReadyChanged); - registerBspEvent(applicationAttemptChanged); - registerBspEvent(superstepFinished); - registerBspEvent(masterElectionChildrenChanged); - registerBspEvent(cleanedUpChildrenChanged); - - this.context = context; - this.graphMapper = graphMapper; - this.conf = context.getConfiguration(); - this.jobId = conf.get("mapred.job.id", "Unknown Job"); - this.taskPartition = conf.getInt("mapred.task.partition", -1); - this.restartedSuperstep = conf.getLong(GiraphJob.RESTART_SUPERSTEP, - UNSET_SUPERSTEP); - this.cachedSuperstep = restartedSuperstep; - if ((restartedSuperstep != UNSET_SUPERSTEP) && - (restartedSuperstep < 0)) { - throw new IllegalArgumentException( - "BspService: Invalid superstep to restart - " + - restartedSuperstep); - } - try { - this.hostname = InetAddress.getLocalHost().getHostName(); - } catch (UnknownHostException e) { - throw new RuntimeException(e); - } - this.hostnamePartitionId = hostname + "_" + getTaskPartition(); - this.graphPartitionerFactory = - BspUtils.createGraphPartitioner(conf); - - this.checkpointFrequency = - conf.getInt(GiraphJob.CHECKPOINT_FREQUENCY, - GiraphJob.CHECKPOINT_FREQUENCY_DEFAULT); - - BASE_PATH = ZooKeeperManager.getBasePath(conf) + BASE_DIR + "/" + jobId; - MASTER_JOB_STATE_PATH = BASE_PATH + MASTER_JOB_STATE_NODE; - INPUT_SPLIT_PATH = BASE_PATH + INPUT_SPLIT_DIR; - INPUT_SPLITS_ALL_READY_PATH = BASE_PATH + INPUT_SPLITS_ALL_READY_NODE; - INPUT_SPLIT_DONE_PATH = BASE_PATH + INPUT_SPLIT_DONE_DIR; - INPUT_SPLITS_ALL_DONE_PATH = BASE_PATH + INPUT_SPLITS_ALL_DONE_NODE; - APPLICATION_ATTEMPTS_PATH = BASE_PATH + APPLICATION_ATTEMPTS_DIR; - CLEANED_UP_PATH = BASE_PATH + CLEANED_UP_DIR; - CHECKPOINT_BASE_PATH = - getConfiguration().get( - GiraphJob.CHECKPOINT_DIRECTORY, - GiraphJob.CHECKPOINT_DIRECTORY_DEFAULT + "/" + getJobId()); - MASTER_ELECTION_PATH = BASE_PATH + MASTER_ELECTION_DIR; - if (LOG.isInfoEnabled()) { - LOG.info("BspService: Connecting to ZooKeeper with job " + jobId + - ", " + getTaskPartition() + " on " + serverPortList); - } - try { - this.zk = new ZooKeeperExt(serverPortList, sessionMsecTimeout, this); - connectedEvent.waitForever(); - this.fs = FileSystem.get(getConfiguration()); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - /** - * Get the job id - * - * @return job id - */ - final public String getJobId() { - return jobId; - } - - /** - * Get the latest application attempt and cache it. - * - * @return the latest application attempt - */ - final public long getApplicationAttempt() { - if (cachedApplicationAttempt != UNSET_APPLICATION_ATTEMPT) { - return cachedApplicationAttempt; - } - try { - getZkExt().createExt(APPLICATION_ATTEMPTS_PATH, - null, - Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT, - true); - } catch (KeeperException.NodeExistsException e) { - LOG.info("getApplicationAttempt: Node " + - APPLICATION_ATTEMPTS_PATH + " already exists!"); - } catch (Exception e) { - throw new RuntimeException(e); - } - try { - List attemptList = - getZkExt().getChildrenExt( - APPLICATION_ATTEMPTS_PATH, true, false, false); - if (attemptList.isEmpty()) { - cachedApplicationAttempt = 0; - } - else { - cachedApplicationAttempt = - Long.parseLong(Collections.max(attemptList)); - } - } catch (Exception e) { - throw new RuntimeException(e); - } - - return cachedApplicationAttempt; - } - - /** - * Get the latest superstep and cache it. - * - * @return the latest superstep - * @throws InterruptedException - * @throws KeeperException - */ - final public long getSuperstep() { - if (cachedSuperstep != UNSET_SUPERSTEP) { - return cachedSuperstep; - } - String superstepPath = getSuperstepPath(getApplicationAttempt()); - try { - getZkExt().createExt(superstepPath, - null, - Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT, - true); - } catch (KeeperException.NodeExistsException e) { - if (LOG.isInfoEnabled()) { - LOG.info("getApplicationAttempt: Node " + - APPLICATION_ATTEMPTS_PATH + " already exists!"); - } - } catch (KeeperException e) { - throw new IllegalStateException( - "getSuperstep: KeeperException", e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "getSuperstep: InterruptedException", e); - } - - List superstepList; - try { - superstepList = - getZkExt().getChildrenExt(superstepPath, true, false, false); - } catch (KeeperException e) { - throw new IllegalStateException( - "getSuperstep: KeeperException", e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "getSuperstep: InterruptedException", e); - } - if (superstepList.isEmpty()) { - cachedSuperstep = INPUT_SUPERSTEP; - } - else { - cachedSuperstep = - Long.parseLong(Collections.max(superstepList)); - } - - return cachedSuperstep; - } - - /** - * Increment the cached superstep. Shouldn't be the initial value anymore. - */ - final public void incrCachedSuperstep() { - if (cachedSuperstep == UNSET_SUPERSTEP) { - throw new IllegalStateException( - "incrSuperstep: Invalid unset cached superstep " + - UNSET_SUPERSTEP); - } - ++cachedSuperstep; - } - - /** - * Set the cached superstep (should only be used for loading checkpoints - * or recovering from failure). - * - * @param superstep will be used as the next superstep iteration - */ - final public void setCachedSuperstep(long superstep) { - cachedSuperstep = superstep; - } - - /** - * Set the cached application attempt (should only be used for restart from - * failure by the master) - * - * @param applicationAttempt Will denote the new application attempt - */ - final public void setApplicationAttempt(long applicationAttempt) { - cachedApplicationAttempt = applicationAttempt; - String superstepPath = getSuperstepPath(cachedApplicationAttempt); - try { - getZkExt().createExt(superstepPath, - null, - Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT, - true); - } catch (KeeperException.NodeExistsException e) { - throw new IllegalArgumentException( - "setApplicationAttempt: Attempt already exists! - " + - superstepPath, e); - } catch (KeeperException e) { - throw new RuntimeException( - "setApplicationAttempt: KeeperException - " + - superstepPath, e); - } catch (InterruptedException e) { - throw new RuntimeException( - "setApplicationAttempt: InterruptedException - " + - superstepPath, e); - } - } - - /** - * Register an aggregator with name. - * - * @param name Name of the aggregator - * @param aggregatorClass Class of the aggregator - * @return Aggregator - * @throws IllegalAccessException - * @throws InstantiationException - */ - public final Aggregator registerAggregator( - String name, - Class> aggregatorClass) - throws InstantiationException, IllegalAccessException { - if (aggregatorMap.get(name) != null) { - return null; - } - Aggregator aggregator = - (Aggregator) aggregatorClass.newInstance(); - @SuppressWarnings("unchecked") - Aggregator writableAggregator = - (Aggregator) aggregator; - aggregatorMap.put(name, writableAggregator); + throw new RuntimeException( + "process: Disconnected from ZooKeeper, cannot recover - " + + event); + } else if (event.getState() == KeeperState.SyncConnected) { if (LOG.isInfoEnabled()) { - LOG.info("registerAggregator: registered " + name); + LOG.info("process: Asynchronous connection complete."); } - return aggregator; - } - - /** - * Get aggregator by name. - * - * @param name - * @return Aggregator (null when not registered) - */ - public final Aggregator getAggregator(String name) { - return aggregatorMap.get(name); - } - - /** - * Get the aggregator map. - */ - public Map> getAggregatorMap() { - return aggregatorMap; + connectedEvent.signal(); + } else { + LOG.warn("process: Got unknown null path event " + event); + } + return; } - /** - * Register a BspEvent. Ensure that it will be signaled - * by catastrophic failure so that threads waiting on an event signal - * will be unblocked. - */ - public void registerBspEvent(BspEvent event) { - registeredBspEvents.add(event); + boolean eventProcessed = false; + if (event.getPath().startsWith(masterJobStatePath)) { + // This will cause all becomeMaster() MasterThreads to notice the + // change in job state and quit trying to become the master. + masterElectionChildrenChanged.signal(); + eventProcessed = true; + } else if ((event.getPath().contains(WORKER_HEALTHY_DIR) || + event.getPath().contains(WORKER_UNHEALTHY_DIR)) && + (event.getType() == EventType.NodeChildrenChanged)) { + if (LOG.isDebugEnabled()) { + LOG.debug("process: workerHealthRegistrationChanged " + + "(worker health reported - healthy/unhealthy )"); + } + workerHealthRegistrationChanged.signal(); + eventProcessed = true; + } else if (event.getPath().equals(inputSplitsAllReadyPath) && + (event.getType() == EventType.NodeCreated)) { + if (LOG.isInfoEnabled()) { + LOG.info("process: inputSplitsReadyChanged " + + "(input splits ready)"); + } + inputSplitsAllReadyChanged.signal(); + eventProcessed = true; + } else if (event.getPath().endsWith(INPUT_SPLIT_RESERVED_NODE) && + (event.getType() == EventType.NodeCreated)) { + if (LOG.isDebugEnabled()) { + LOG.debug("process: inputSplitsStateChanged " + + "(made a reservation)"); + } + inputSplitsStateChanged.signal(); + eventProcessed = true; + } else if (event.getPath().endsWith(INPUT_SPLIT_RESERVED_NODE) && + (event.getType() == EventType.NodeDeleted)) { + if (LOG.isInfoEnabled()) { + LOG.info("process: inputSplitsStateChanged " + + "(lost a reservation)"); + } + inputSplitsStateChanged.signal(); + eventProcessed = true; + } else if (event.getPath().endsWith(INPUT_SPLIT_FINISHED_NODE) && + (event.getType() == EventType.NodeCreated)) { + if (LOG.isDebugEnabled()) { + LOG.debug("process: inputSplitsStateChanged " + + "(finished inputsplit)"); + } + inputSplitsStateChanged.signal(); + eventProcessed = true; + } else if (event.getPath().endsWith(INPUT_SPLIT_DONE_DIR) && + (event.getType() == EventType.NodeChildrenChanged)) { + if (LOG.isDebugEnabled()) { + LOG.debug("process: inputSplitsDoneStateChanged " + + "(worker finished sending)"); + } + inputSplitsDoneStateChanged.signal(); + eventProcessed = true; + } else if (event.getPath().equals(inputSplitsAllDonePath) && + (event.getType() == EventType.NodeCreated)) { + if (LOG.isInfoEnabled()) { + LOG.info("process: inputSplitsAllDoneChanged " + + "(all vertices sent from input splits)"); + } + inputSplitsAllDoneChanged.signal(); + eventProcessed = true; + } else if (event.getPath().contains(PARTITION_ASSIGNMENTS_DIR) && + event.getType() == EventType.NodeCreated) { + if (LOG.isInfoEnabled()) { + LOG.info("process: partitionAssignmentsReadyChanged " + + "(partitions are assigned)"); + } + partitionAssignmentsReadyChanged.signal(); + eventProcessed = true; + } else if (event.getPath().contains(SUPERSTEP_FINISHED_NODE) && + event.getType() == EventType.NodeCreated) { + if (LOG.isInfoEnabled()) { + LOG.info("process: superstepFinished signaled"); + } + superstepFinished.signal(); + eventProcessed = true; + } else if (event.getPath().endsWith(applicationAttemptsPath) && + event.getType() == EventType.NodeChildrenChanged) { + if (LOG.isInfoEnabled()) { + LOG.info("process: applicationAttemptChanged signaled"); + } + applicationAttemptChanged.signal(); + eventProcessed = true; + } else if (event.getPath().contains(MASTER_ELECTION_DIR) && + event.getType() == EventType.NodeChildrenChanged) { + if (LOG.isInfoEnabled()) { + LOG.info("process: masterElectionChildrenChanged signaled"); + } + masterElectionChildrenChanged.signal(); + eventProcessed = true; + } else if (event.getPath().equals(cleanedUpPath) && + event.getType() == EventType.NodeChildrenChanged) { + if (LOG.isInfoEnabled()) { + LOG.info("process: cleanedUpChildrenChanged signaled"); + } + cleanedUpChildrenChanged.signal(); + eventProcessed = true; } - /** - * Subclasses can use this to instantiate their respective partitioners - * - * @return Instantiated graph partitioner factory - */ - protected GraphPartitionerFactory getGraphPartitionerFactory() { - return graphPartitionerFactory; - } - - /** - * Derived classes that want additional ZooKeeper events to take action - * should override this. - * - * @param event Event that occurred - * @return true if the event was processed here, false otherwise - */ - protected boolean processEvent(WatchedEvent event) { - return false; - } - - @Override - final public void process(WatchedEvent event) { - // 1. Process all shared events - // 2. Process specific derived class events - - if (LOG.isDebugEnabled()) { - LOG.debug("process: Got a new event, path = " + event.getPath() + - ", type = " + event.getType() + ", state = " + - event.getState()); - } - - if ((event.getPath() == null) && (event.getType() == EventType.None)) { - if (event.getState() == KeeperState.Disconnected) { - // No way to recover from a disconnect event, signal all BspEvents - for (BspEvent bspEvent : registeredBspEvents) { - bspEvent.signal(); - } - throw new RuntimeException( - "process: Disconnected from ZooKeeper, cannot recover - " + - event); - } else if (event.getState() == KeeperState.SyncConnected) { - if (LOG.isInfoEnabled()) { - LOG.info("process: Asynchronous connection complete."); - } - connectedEvent.signal(); - } else { - LOG.warn("process: Got unknown null path event " + event); - } - return; - } - - boolean eventProcessed = false; - if (event.getPath().startsWith(MASTER_JOB_STATE_PATH)) { - // This will cause all becomeMaster() MasterThreads to notice the - // change in job state and quit trying to become the master. - masterElectionChildrenChanged.signal(); - eventProcessed = true; - } else if ((event.getPath().contains(WORKER_HEALTHY_DIR) || - event.getPath().contains(WORKER_UNHEALTHY_DIR)) && - (event.getType() == EventType.NodeChildrenChanged)) { - if (LOG.isDebugEnabled()) { - LOG.debug("process: workerHealthRegistrationChanged " + - "(worker health reported - healthy/unhealthy )"); - } - workerHealthRegistrationChanged.signal(); - eventProcessed = true; - } else if (event.getPath().equals(INPUT_SPLITS_ALL_READY_PATH) && - (event.getType() == EventType.NodeCreated)) { - if (LOG.isInfoEnabled()) { - LOG.info("process: inputSplitsReadyChanged " + - "(input splits ready)"); - } - inputSplitsAllReadyChanged.signal(); - eventProcessed = true; - } else if (event.getPath().endsWith(INPUT_SPLIT_RESERVED_NODE) && - (event.getType() == EventType.NodeCreated)) { - if (LOG.isDebugEnabled()) { - LOG.debug("process: inputSplitsStateChanged "+ - "(made a reservation)"); - } - inputSplitsStateChanged.signal(); - eventProcessed = true; - } else if (event.getPath().endsWith(INPUT_SPLIT_RESERVED_NODE) && - (event.getType() == EventType.NodeDeleted)) { - if (LOG.isInfoEnabled()) { - LOG.info("process: inputSplitsStateChanged "+ - "(lost a reservation)"); - } - inputSplitsStateChanged.signal(); - eventProcessed = true; - } else if (event.getPath().endsWith(INPUT_SPLIT_FINISHED_NODE) && - (event.getType() == EventType.NodeCreated)) { - if (LOG.isDebugEnabled()) { - LOG.debug("process: inputSplitsStateChanged " + - "(finished inputsplit)"); - } - inputSplitsStateChanged.signal(); - eventProcessed = true; - } else if (event.getPath().endsWith(INPUT_SPLIT_DONE_DIR) && - (event.getType() == EventType.NodeChildrenChanged)) { - if (LOG.isDebugEnabled()) { - LOG.debug("process: inputSplitsDoneStateChanged " + - "(worker finished sending)"); - } - inputSplitsDoneStateChanged.signal(); - eventProcessed = true; - } else if (event.getPath().equals(INPUT_SPLITS_ALL_DONE_PATH) && - (event.getType() == EventType.NodeCreated)) { - if (LOG.isInfoEnabled()) { - LOG.info("process: inputSplitsAllDoneChanged " + - "(all vertices sent from input splits)"); - } - inputSplitsAllDoneChanged.signal(); - eventProcessed = true; - } else if (event.getPath().contains(PARTITION_ASSIGNMENTS_DIR) && - event.getType() == EventType.NodeCreated) { - if (LOG.isInfoEnabled()) { - LOG.info("process: partitionAssignmentsReadyChanged " + - "(partitions are assigned)"); - } - partitionAssignmentsReadyChanged.signal(); - eventProcessed = true; - } else if (event.getPath().contains(SUPERSTEP_FINISHED_NODE) && - event.getType() == EventType.NodeCreated) { - if (LOG.isInfoEnabled()) { - LOG.info("process: superstepFinished signaled"); - } - superstepFinished.signal(); - eventProcessed = true; - } else if (event.getPath().endsWith(APPLICATION_ATTEMPTS_PATH) && - event.getType() == EventType.NodeChildrenChanged) { - if (LOG.isInfoEnabled()) { - LOG.info("process: applicationAttemptChanged signaled"); - } - applicationAttemptChanged.signal(); - eventProcessed = true; - } else if (event.getPath().contains(MASTER_ELECTION_DIR) && - event.getType() == EventType.NodeChildrenChanged) { - if (LOG.isInfoEnabled()) { - LOG.info("process: masterElectionChildrenChanged signaled"); - } - masterElectionChildrenChanged.signal(); - eventProcessed = true; - } else if (event.getPath().equals(CLEANED_UP_PATH) && - event.getType() == EventType.NodeChildrenChanged) { - if (LOG.isInfoEnabled()) { - LOG.info("process: cleanedUpChildrenChanged signaled"); - } - cleanedUpChildrenChanged.signal(); - eventProcessed = true; - } - - if ((processEvent(event) == false) && (eventProcessed == false)) { - LOG.warn("process: Unknown and unprocessed event (path=" + - event.getPath() + ", type=" + event.getType() + - ", state=" + event.getState() + ")"); - } + if (!(processEvent(event)) && (!eventProcessed)) { + LOG.warn("process: Unknown and unprocessed event (path=" + + event.getPath() + ", type=" + event.getType() + + ", state=" + event.getState() + ")"); } + } } diff --git a/src/main/java/org/apache/giraph/graph/BspServiceMaster.java b/src/main/java/org/apache/giraph/graph/BspServiceMaster.java index c580bf34e..1848f5628 100644 --- a/src/main/java/org/apache/giraph/graph/BspServiceMaster.java +++ b/src/main/java/org/apache/giraph/graph/BspServiceMaster.java @@ -21,7 +21,6 @@ import net.iharder.Base64; import org.apache.giraph.bsp.ApplicationState; import org.apache.giraph.bsp.BspInputFormat; -import org.apache.giraph.bsp.CentralizedService; import org.apache.giraph.bsp.CentralizedServiceMaster; import org.apache.giraph.bsp.SuperstepState; import org.apache.giraph.graph.GraphMapper.MapFunctions; @@ -75,1663 +74,1678 @@ /** * ZooKeeper-based implementation of {@link CentralizedService}. + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data */ @SuppressWarnings("rawtypes") -public class BspServiceMaster< - I extends WritableComparable, - V extends Writable, - E extends Writable, M extends Writable> - extends BspService - implements CentralizedServiceMaster { - /** Class logger */ - private static final Logger LOG = Logger.getLogger(BspServiceMaster.class); - /** Superstep counter */ - private Counter superstepCounter = null; - /** Vertex counter */ - private Counter vertexCounter = null; - /** Finished vertex counter */ - private Counter finishedVertexCounter = null; - /** Edge counter */ - private Counter edgeCounter = null; - /** Sent messages counter */ - private Counter sentMessagesCounter = null; - /** Workers on this superstep */ - private Counter currentWorkersCounter = null; - /** Current master task partition */ - private Counter currentMasterTaskPartitionCounter = null; - /** Last checkpointed superstep */ - private Counter lastCheckpointedSuperstepCounter = null; - /** Am I the master? */ - private boolean isMaster = false; - /** Max number of workers */ - private final int maxWorkers; - /** Min number of workers */ - private final int minWorkers; - /** Min % responded workers */ - private final float minPercentResponded; - /** Poll period in msecs */ - private final int msecsPollPeriod; - /** Max number of poll attempts */ - private final int maxPollAttempts; - /** Min number of long tails before printing */ - private final int partitionLongTailMinPrint; - /** Last finalized checkpoint */ - private long lastCheckpointedSuperstep = -1; - /** State of the superstep changed */ - private final BspEvent superstepStateChanged = - new PredicateLock(); - /** Master graph partitioner */ - private final MasterGraphPartitioner masterGraphPartitioner; - /** All the partition stats from the last superstep */ - private final List allPartitionStatsList = - new ArrayList(); - /** Counter group name for the Giraph statistics */ - public String GIRAPH_STATS_COUNTER_GROUP_NAME = "Giraph Stats"; - /** Aggregator writer */ - public AggregatorWriter aggregatorWriter; - - public BspServiceMaster( - String serverPortList, - int sessionMsecTimeout, - Mapper.Context context, - GraphMapper graphMapper) { - super(serverPortList, sessionMsecTimeout, context, graphMapper); - registerBspEvent(superstepStateChanged); - - maxWorkers = - getConfiguration().getInt(GiraphJob.MAX_WORKERS, -1); - minWorkers = - getConfiguration().getInt(GiraphJob.MIN_WORKERS, -1); - minPercentResponded = - getConfiguration().getFloat(GiraphJob.MIN_PERCENT_RESPONDED, - 100.0f); - msecsPollPeriod = - getConfiguration().getInt(GiraphJob.POLL_MSECS, - GiraphJob.POLL_MSECS_DEFAULT); - maxPollAttempts = - getConfiguration().getInt(GiraphJob.POLL_ATTEMPTS, - GiraphJob.POLL_ATTEMPTS_DEFAULT); - partitionLongTailMinPrint = getConfiguration().getInt( - GiraphJob.PARTITION_LONG_TAIL_MIN_PRINT, - GiraphJob.PARTITION_LONG_TAIL_MIN_PRINT_DEFAULT); - masterGraphPartitioner = - getGraphPartitionerFactory().createMasterGraphPartitioner(); +public class BspServiceMaster + extends BspService + implements CentralizedServiceMaster { + /** Counter group name for the Giraph statistics */ + public static final String GIRAPH_STATS_COUNTER_GROUP_NAME = "Giraph Stats"; + /** Class logger */ + private static final Logger LOG = Logger.getLogger(BspServiceMaster.class); + /** Superstep counter */ + private Counter superstepCounter = null; + /** Vertex counter */ + private Counter vertexCounter = null; + /** Finished vertex counter */ + private Counter finishedVertexCounter = null; + /** Edge counter */ + private Counter edgeCounter = null; + /** Sent messages counter */ + private Counter sentMessagesCounter = null; + /** Workers on this superstep */ + private Counter currentWorkersCounter = null; + /** Current master task partition */ + private Counter currentMasterTaskPartitionCounter = null; + /** Last checkpointed superstep */ + private Counter lastCheckpointedSuperstepCounter = null; + /** Am I the master? */ + private boolean isMaster = false; + /** Max number of workers */ + private final int maxWorkers; + /** Min number of workers */ + private final int minWorkers; + /** Min % responded workers */ + private final float minPercentResponded; + /** Poll period in msecs */ + private final int msecsPollPeriod; + /** Max number of poll attempts */ + private final int maxPollAttempts; + /** Min number of long tails before printing */ + private final int partitionLongTailMinPrint; + /** Last finalized checkpoint */ + private long lastCheckpointedSuperstep = -1; + /** State of the superstep changed */ + private final BspEvent superstepStateChanged = + new PredicateLock(); + /** Master graph partitioner */ + private final MasterGraphPartitioner masterGraphPartitioner; + /** All the partition stats from the last superstep */ + private final List allPartitionStatsList = + new ArrayList(); + /** Aggregator writer */ + private AggregatorWriter aggregatorWriter; + + /** + * Constructor for setting up the master. + * + * @param serverPortList ZooKeeper server port list + * @param sessionMsecTimeout Msecs to timeout connecting to ZooKeeper + * @param context Mapper context + * @param graphMapper Graph mapper + */ + public BspServiceMaster( + String serverPortList, + int sessionMsecTimeout, + Mapper.Context context, + GraphMapper graphMapper) { + super(serverPortList, sessionMsecTimeout, context, graphMapper); + registerBspEvent(superstepStateChanged); + + maxWorkers = + getConfiguration().getInt(GiraphJob.MAX_WORKERS, -1); + minWorkers = + getConfiguration().getInt(GiraphJob.MIN_WORKERS, -1); + minPercentResponded = + getConfiguration().getFloat(GiraphJob.MIN_PERCENT_RESPONDED, + 100.0f); + msecsPollPeriod = + getConfiguration().getInt(GiraphJob.POLL_MSECS, + GiraphJob.POLL_MSECS_DEFAULT); + maxPollAttempts = + getConfiguration().getInt(GiraphJob.POLL_ATTEMPTS, + GiraphJob.POLL_ATTEMPTS_DEFAULT); + partitionLongTailMinPrint = getConfiguration().getInt( + GiraphJob.PARTITION_LONG_TAIL_MIN_PRINT, + GiraphJob.PARTITION_LONG_TAIL_MIN_PRINT_DEFAULT); + masterGraphPartitioner = + getGraphPartitionerFactory().createMasterGraphPartitioner(); + } + + @Override + public void setJobState(ApplicationState state, + long applicationAttempt, + long desiredSuperstep) { + JSONObject jobState = new JSONObject(); + try { + jobState.put(JSONOBJ_STATE_KEY, state.toString()); + jobState.put(JSONOBJ_APPLICATION_ATTEMPT_KEY, applicationAttempt); + jobState.put(JSONOBJ_SUPERSTEP_KEY, desiredSuperstep); + } catch (JSONException e) { + throw new RuntimeException("setJobState: Coudn't put " + + state.toString()); + } + if (LOG.isInfoEnabled()) { + LOG.info("setJobState: " + jobState.toString() + " on superstep " + + getSuperstep()); + } + try { + getZkExt().createExt(masterJobStatePath + "/jobState", + jobState.toString().getBytes(), + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT_SEQUENTIAL, + true); + } catch (KeeperException.NodeExistsException e) { + throw new IllegalStateException( + "setJobState: Imposible that " + + masterJobStatePath + " already exists!", e); + } catch (KeeperException e) { + throw new IllegalStateException( + "setJobState: Unknown KeeperException for " + + masterJobStatePath, e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "setJobState: Unknown InterruptedException for " + + masterJobStatePath, e); } - @Override - public void setJobState(ApplicationState state, - long applicationAttempt, - long desiredSuperstep) { - JSONObject jobState = new JSONObject(); - try { - jobState.put(JSONOBJ_STATE_KEY, state.toString()); - jobState.put(JSONOBJ_APPLICATION_ATTEMPT_KEY, applicationAttempt); - jobState.put(JSONOBJ_SUPERSTEP_KEY, desiredSuperstep); - } catch (JSONException e) { - throw new RuntimeException("setJobState: Coudn't put " + - state.toString()); - } + if (state == ApplicationState.FAILED) { + failJob(); + } + } + + /** + * Master uses this to calculate the {@link VertexInputFormat} + * input splits and write it to ZooKeeper. + * + * @param numWorkers Number of available workers + * @return List of input splits + * @throws InstantiationException + * @throws IllegalAccessException + * @throws IOException + * @throws InterruptedException + */ + private List generateInputSplits(int numWorkers) { + VertexInputFormat vertexInputFormat = + BspUtils.createVertexInputFormat(getConfiguration()); + List splits; + try { + splits = vertexInputFormat.getSplits(getContext(), numWorkers); + float samplePercent = + getConfiguration().getFloat( + GiraphJob.INPUT_SPLIT_SAMPLE_PERCENT, + GiraphJob.INPUT_SPLIT_SAMPLE_PERCENT_DEFAULT); + if (samplePercent != GiraphJob.INPUT_SPLIT_SAMPLE_PERCENT_DEFAULT) { + int lastIndex = (int) (samplePercent * splits.size() / 100f); + List sampleSplits = splits.subList(0, lastIndex); + LOG.warn("generateInputSplits: Using sampling - Processing " + + "only " + sampleSplits.size() + " instead of " + + splits.size() + " expected splits."); + return sampleSplits; + } else { if (LOG.isInfoEnabled()) { - LOG.info("setJobState: " + jobState.toString() + " on superstep " + - getSuperstep()); - } - try { - getZkExt().createExt(MASTER_JOB_STATE_PATH + "/jobState", - jobState.toString().getBytes(), - Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT_SEQUENTIAL, - true); - } catch (KeeperException.NodeExistsException e) { - throw new IllegalStateException( - "setJobState: Imposible that " + - MASTER_JOB_STATE_PATH + " already exists!", e); - } catch (KeeperException e) { - throw new IllegalStateException( - "setJobState: Unknown KeeperException for " + - MASTER_JOB_STATE_PATH, e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "setJobState: Unknown InterruptedException for " + - MASTER_JOB_STATE_PATH, e); - } - - if (state == ApplicationState.FAILED) { - failJob(); - } + LOG.info("generateInputSplits: Got " + splits.size() + + " input splits for " + numWorkers + " workers"); + } + return splits; + } + } catch (IOException e) { + throw new IllegalStateException( + "generateInputSplits: Got IOException", e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "generateInputSplits: Got InterruptedException", e); } - - /** - * Master uses this to calculate the {@link VertexInputFormat} - * input splits and write it to ZooKeeper. - * - * @param numWorkers Number of available workers - * @throws InstantiationException - * @throws IllegalAccessException - * @throws IOException - * @throws InterruptedException - */ - private List generateInputSplits(int numWorkers) { - VertexInputFormat vertexInputFormat = - BspUtils.createVertexInputFormat(getConfiguration()); - List splits; - try { - splits = vertexInputFormat.getSplits(getContext(), numWorkers); - float samplePercent = - getConfiguration().getFloat( - GiraphJob.INPUT_SPLIT_SAMPLE_PERCENT, - GiraphJob.INPUT_SPLIT_SAMPLE_PERCENT_DEFAULT); - if (samplePercent != GiraphJob.INPUT_SPLIT_SAMPLE_PERCENT_DEFAULT) { - int lastIndex = (int) (samplePercent * splits.size() / 100f); - List sampleSplits = splits.subList(0, lastIndex); - LOG.warn("generateInputSplits: Using sampling - Processing " + - "only " + sampleSplits.size() + " instead of " + - splits.size() + " expected splits."); - return sampleSplits; - } else { - if (LOG.isInfoEnabled()) { - LOG.info("generateInputSplits: Got " + splits.size() + - " input splits for " + numWorkers + " workers"); - } - return splits; - } - } catch (IOException e) { - throw new IllegalStateException( - "generateInputSplits: Got IOException", e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "generateInputSplits: Got InterruptedException", e); - } + } + + /** + * When there is no salvaging this job, fail it. + * + * @throws IOException + */ + private void failJob() { + LOG.fatal("failJob: Killing job " + getJobId()); + try { + @SuppressWarnings("deprecation") + org.apache.hadoop.mapred.JobClient jobClient = + new org.apache.hadoop.mapred.JobClient( + (org.apache.hadoop.mapred.JobConf) + getConfiguration()); + @SuppressWarnings("deprecation") + org.apache.hadoop.mapred.JobID jobId = + org.apache.hadoop.mapred.JobID.forName(getJobId()); + RunningJob job = jobClient.getJob(jobId); + job.killJob(); + } catch (IOException e) { + throw new RuntimeException(e); } - - /** - * When there is no salvaging this job, fail it. - * - * @throws IOException - */ - private void failJob() { - LOG.fatal("failJob: Killing job " + getJobId()); - try { - @SuppressWarnings("deprecation") - org.apache.hadoop.mapred.JobClient jobClient = - new org.apache.hadoop.mapred.JobClient( - (org.apache.hadoop.mapred.JobConf) - getConfiguration()); - @SuppressWarnings("deprecation") - org.apache.hadoop.mapred.JobID jobId = - org.apache.hadoop.mapred.JobID.forName(getJobId()); - RunningJob job = jobClient.getJob(jobId); - job.killJob(); - } catch (IOException e) { - throw new RuntimeException(e); - } + } + + /** + * Parse the {@link WorkerInfo} objects from a ZooKeeper path + * (and children). + * + * @param workerInfosPath Path where all the workers are children + * @param watch Watch or not? + * @return List of workers in that path + */ + private List getWorkerInfosFromPath(String workerInfosPath, + boolean watch) { + List workerInfoList = new ArrayList(); + List workerInfoPathList; + try { + workerInfoPathList = + getZkExt().getChildrenExt(workerInfosPath, watch, false, true); + } catch (KeeperException e) { + throw new IllegalStateException( + "getWorkers: Got KeeperException", e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "getWorkers: Got InterruptedStateException", e); } - - /** - * Parse the {@link WorkerInfo} objects from a ZooKeeper path - * (and children). - * - * @param workerInfosPath Path where all the workers are children - * @param watch Watch or not? - * @return List of workers in that path - */ - private List getWorkerInfosFromPath(String workerInfosPath, - boolean watch) { - List workerInfoList = new ArrayList(); - List workerInfoPathList; - try { - workerInfoPathList = - getZkExt().getChildrenExt(workerInfosPath, watch, false, true); - } catch (KeeperException e) { - throw new IllegalStateException( - "getWorkers: Got KeeperException", e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "getWorkers: Got InterruptedStateException", e); - } - for (String workerInfoPath : workerInfoPathList) { - WorkerInfo workerInfo = new WorkerInfo(); - WritableUtils.readFieldsFromZnode( - getZkExt(), workerInfoPath, true, null, workerInfo); - workerInfoList.add(workerInfo); - } - return workerInfoList; - } - - /** - * Get the healthy and unhealthy {@link WorkerInfo} objects for - * a superstep - * - * @param superstep superstep to check - * @param healthyWorkerInfoList filled in with current data - * @param unhealthyWorkerInfoList filled in with current data - */ - private void getAllWorkerInfos( - long superstep, - List healthyWorkerInfoList, - List unhealthyWorkerInfoList) { - String healthyWorkerInfoPath = - getWorkerInfoHealthyPath(getApplicationAttempt(), superstep); - String unhealthyWorkerInfoPath = - getWorkerInfoUnhealthyPath(getApplicationAttempt(), superstep); - - try { - getZkExt().createOnceExt(healthyWorkerInfoPath, - null, - Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT, - true); - } catch (KeeperException e) { - throw new IllegalStateException("getWorkers: KeeperException", e); - } catch (InterruptedException e) { - throw new IllegalStateException("getWorkers: IllegalStateException" - , e); - } - - try { - getZkExt().createOnceExt(unhealthyWorkerInfoPath, - null, - Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT, - true); - } catch (KeeperException e) { - throw new IllegalStateException("getWorkers: KeeperException", e); - } catch (InterruptedException e) { - throw new IllegalStateException("getWorkers: IllegalStateException" - , e); - } - - List currentHealthyWorkerInfoList = - getWorkerInfosFromPath(healthyWorkerInfoPath, true); - List currentUnhealthyWorkerInfoList = - getWorkerInfosFromPath(unhealthyWorkerInfoPath, false); - - healthyWorkerInfoList.clear(); - if (currentHealthyWorkerInfoList != null) { - for (WorkerInfo healthyWorkerInfo : - currentHealthyWorkerInfoList) { - healthyWorkerInfoList.add(healthyWorkerInfo); - } - } - - unhealthyWorkerInfoList.clear(); - if (currentUnhealthyWorkerInfoList != null) { - for (WorkerInfo unhealthyWorkerInfo : - currentUnhealthyWorkerInfoList) { - unhealthyWorkerInfoList.add(unhealthyWorkerInfo); - } - } + for (String workerInfoPath : workerInfoPathList) { + WorkerInfo workerInfo = new WorkerInfo(); + WritableUtils.readFieldsFromZnode( + getZkExt(), workerInfoPath, true, null, workerInfo); + workerInfoList.add(workerInfo); + } + return workerInfoList; + } + + /** + * Get the healthy and unhealthy {@link WorkerInfo} objects for + * a superstep + * + * @param superstep superstep to check + * @param healthyWorkerInfoList filled in with current data + * @param unhealthyWorkerInfoList filled in with current data + */ + private void getAllWorkerInfos( + long superstep, + List healthyWorkerInfoList, + List unhealthyWorkerInfoList) { + String healthyWorkerInfoPath = + getWorkerInfoHealthyPath(getApplicationAttempt(), superstep); + String unhealthyWorkerInfoPath = + getWorkerInfoUnhealthyPath(getApplicationAttempt(), superstep); + + try { + getZkExt().createOnceExt(healthyWorkerInfoPath, + null, + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT, + true); + } catch (KeeperException e) { + throw new IllegalStateException("getWorkers: KeeperException", e); + } catch (InterruptedException e) { + throw new IllegalStateException("getWorkers: IllegalStateException" + , e); } - /** - * Check all the {@link WorkerInfo} objects to ensure that a minimum - * number of good workers exists out of the total that have reported. - * - * @return List of of healthy workers such that the minimum has been - * met, otherwise null - */ - private List checkWorkers() { - boolean failJob = true; - int pollAttempt = 0; - List healthyWorkerInfoList = new ArrayList(); - List unhealthyWorkerInfoList = new ArrayList(); - int totalResponses = -1; - while (pollAttempt < maxPollAttempts) { - getAllWorkerInfos( - getSuperstep(), healthyWorkerInfoList, unhealthyWorkerInfoList); - totalResponses = healthyWorkerInfoList.size() + - unhealthyWorkerInfoList.size(); - if ((totalResponses * 100.0f / maxWorkers) >= - minPercentResponded) { - failJob = false; - break; - } - getContext().setStatus(getGraphMapper().getMapFunctions() + " " + - "checkWorkers: Only found " + - totalResponses + - " responses of " + maxWorkers + - " needed to start superstep " + - getSuperstep()); - if (getWorkerHealthRegistrationChangedEvent().waitMsecs( - msecsPollPeriod)) { - if (LOG.isDebugEnabled()) { - LOG.debug("checkWorkers: Got event that health " + - "registration changed, not using poll attempt"); - } - getWorkerHealthRegistrationChangedEvent().reset(); - continue; - } - if (LOG.isInfoEnabled()) { - LOG.info("checkWorkers: Only found " + totalResponses + - " responses of " + maxWorkers + - " needed to start superstep " + - getSuperstep() + ". Sleeping for " + - msecsPollPeriod + " msecs and used " + pollAttempt + - " of " + maxPollAttempts + " attempts."); - // Find the missing workers if there are only a few - if ((maxWorkers - totalResponses) <= - partitionLongTailMinPrint) { - Set partitionSet = new TreeSet(); - for (WorkerInfo workerInfo : healthyWorkerInfoList) { - partitionSet.add(workerInfo.getPartitionId()); - } - for (WorkerInfo workerInfo : unhealthyWorkerInfoList) { - partitionSet.add(workerInfo.getPartitionId()); - } - for (int i = 1; i <= maxWorkers; ++i) { - if (partitionSet.contains(new Integer(i))) { - continue; - } else if (i == getTaskPartition()) { - continue; - } else { - LOG.info("checkWorkers: No response from "+ - "partition " + i + " (could be master)"); - } - } - } - } - ++pollAttempt; - } - if (failJob) { - LOG.error("checkWorkers: Did not receive enough processes in " + - "time (only " + totalResponses + " of " + - minWorkers + " required). This occurs if you do not " + - "have enough map tasks available simultaneously on " + - "your Hadoop instance to fulfill the number of " + - "requested workers."); - return null; - } - - if (healthyWorkerInfoList.size() < minWorkers) { - LOG.error("checkWorkers: Only " + healthyWorkerInfoList.size() + - " available when " + minWorkers + " are required."); - return null; - } - - getContext().setStatus(getGraphMapper().getMapFunctions() + " " + - "checkWorkers: Done - Found " + totalResponses + - " responses of " + maxWorkers + " needed to start superstep " + - getSuperstep()); + try { + getZkExt().createOnceExt(unhealthyWorkerInfoPath, + null, + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT, + true); + } catch (KeeperException e) { + throw new IllegalStateException("getWorkers: KeeperException", e); + } catch (InterruptedException e) { + throw new IllegalStateException("getWorkers: IllegalStateException" + , e); + } - return healthyWorkerInfoList; + List currentHealthyWorkerInfoList = + getWorkerInfosFromPath(healthyWorkerInfoPath, true); + List currentUnhealthyWorkerInfoList = + getWorkerInfosFromPath(unhealthyWorkerInfoPath, false); + + healthyWorkerInfoList.clear(); + if (currentHealthyWorkerInfoList != null) { + for (WorkerInfo healthyWorkerInfo : + currentHealthyWorkerInfoList) { + healthyWorkerInfoList.add(healthyWorkerInfo); + } } - @Override - public int createInputSplits() { - // Only the 'master' should be doing this. Wait until the number of - // processes that have reported health exceeds the minimum percentage. - // If the minimum percentage is not met, fail the job. Otherwise - // generate the input splits - try { - if (getZkExt().exists(INPUT_SPLIT_PATH, false) != null) { - LOG.info(INPUT_SPLIT_PATH + - " already exists, no need to create"); - return Integer.parseInt( - new String( - getZkExt().getData(INPUT_SPLIT_PATH, false, null))); - } - } catch (KeeperException.NoNodeException e) { - if (LOG.isInfoEnabled()) { - LOG.info("createInputSplits: Need to create the " + - "input splits at " + INPUT_SPLIT_PATH); + unhealthyWorkerInfoList.clear(); + if (currentUnhealthyWorkerInfoList != null) { + for (WorkerInfo unhealthyWorkerInfo : + currentUnhealthyWorkerInfoList) { + unhealthyWorkerInfoList.add(unhealthyWorkerInfo); + } + } + } + + /** + * Check all the {@link WorkerInfo} objects to ensure that a minimum + * number of good workers exists out of the total that have reported. + * + * @return List of of healthy workers such that the minimum has been + * met, otherwise null + */ + private List checkWorkers() { + boolean failJob = true; + int pollAttempt = 0; + List healthyWorkerInfoList = new ArrayList(); + List unhealthyWorkerInfoList = new ArrayList(); + int totalResponses = -1; + while (pollAttempt < maxPollAttempts) { + getAllWorkerInfos( + getSuperstep(), healthyWorkerInfoList, unhealthyWorkerInfoList); + totalResponses = healthyWorkerInfoList.size() + + unhealthyWorkerInfoList.size(); + if ((totalResponses * 100.0f / maxWorkers) >= + minPercentResponded) { + failJob = false; + break; + } + getContext().setStatus(getGraphMapper().getMapFunctions() + " " + + "checkWorkers: Only found " + + totalResponses + + " responses of " + maxWorkers + + " needed to start superstep " + + getSuperstep()); + if (getWorkerHealthRegistrationChangedEvent().waitMsecs( + msecsPollPeriod)) { + if (LOG.isDebugEnabled()) { + LOG.debug("checkWorkers: Got event that health " + + "registration changed, not using poll attempt"); + } + getWorkerHealthRegistrationChangedEvent().reset(); + continue; + } + if (LOG.isInfoEnabled()) { + LOG.info("checkWorkers: Only found " + totalResponses + + " responses of " + maxWorkers + + " needed to start superstep " + + getSuperstep() + ". Sleeping for " + + msecsPollPeriod + " msecs and used " + pollAttempt + + " of " + maxPollAttempts + " attempts."); + // Find the missing workers if there are only a few + if ((maxWorkers - totalResponses) <= + partitionLongTailMinPrint) { + Set partitionSet = new TreeSet(); + for (WorkerInfo workerInfo : healthyWorkerInfoList) { + partitionSet.add(workerInfo.getPartitionId()); + } + for (WorkerInfo workerInfo : unhealthyWorkerInfoList) { + partitionSet.add(workerInfo.getPartitionId()); + } + for (int i = 1; i <= maxWorkers; ++i) { + if (partitionSet.contains(new Integer(i))) { + continue; + } else if (i == getTaskPartition()) { + continue; + } else { + LOG.info("checkWorkers: No response from " + + "partition " + i + " (could be master)"); } - } catch (KeeperException e) { - throw new IllegalStateException( - "createInputSplits: KeeperException", e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "createInputSplits: IllegalStateException", e); + } } + } + ++pollAttempt; + } + if (failJob) { + LOG.error("checkWorkers: Did not receive enough processes in " + + "time (only " + totalResponses + " of " + + minWorkers + " required). This occurs if you do not " + + "have enough map tasks available simultaneously on " + + "your Hadoop instance to fulfill the number of " + + "requested workers."); + return null; + } - // When creating znodes, in case the master has already run, resume - // where it left off. - List healthyWorkerInfoList = checkWorkers(); - if (healthyWorkerInfoList == null) { - setJobState(ApplicationState.FAILED, -1, -1); - return -1; - } + if (healthyWorkerInfoList.size() < minWorkers) { + LOG.error("checkWorkers: Only " + healthyWorkerInfoList.size() + + " available when " + minWorkers + " are required."); + return null; + } - // Note that the input splits may only be a sample if - // INPUT_SPLIT_SAMPLE_PERCENT is set to something other than 100 - List splitList = - generateInputSplits(healthyWorkerInfoList.size()); - if (healthyWorkerInfoList.size() > splitList.size()) { - LOG.warn("createInputSplits: Number of inputSplits=" - + splitList.size() + " < " + - healthyWorkerInfoList.size() + - "=number of healthy processes, " + - "some workers will be not used"); - } - String inputSplitPath = null; - for (int i = 0; i< splitList.size(); ++i) { - try { - ByteArrayOutputStream byteArrayOutputStream = - new ByteArrayOutputStream(); - DataOutput outputStream = - new DataOutputStream(byteArrayOutputStream); - InputSplit inputSplit = splitList.get(i); - Text.writeString(outputStream, - inputSplit.getClass().getName()); - ((Writable) inputSplit).write(outputStream); - inputSplitPath = INPUT_SPLIT_PATH + "/" + i; - getZkExt().createExt(inputSplitPath, - byteArrayOutputStream.toByteArray(), - Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT, - true); - if (LOG.isDebugEnabled()) { - LOG.debug("createInputSplits: Created input split " + - "with index " + i + " serialized as " + - byteArrayOutputStream.toString()); - } - } catch (KeeperException.NodeExistsException e) { - if (LOG.isInfoEnabled()) { - LOG.info("createInputSplits: Node " + - inputSplitPath + " already exists."); - } - } catch (KeeperException e) { - throw new IllegalStateException( - "createInputSplits: KeeperException", e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "createInputSplits: IllegalStateException", e); - } catch (IOException e) { - throw new IllegalStateException( - "createInputSplits: IOException", e); - } - } + getContext().setStatus(getGraphMapper().getMapFunctions() + " " + + "checkWorkers: Done - Found " + totalResponses + + " responses of " + maxWorkers + " needed to start superstep " + + getSuperstep()); + + return healthyWorkerInfoList; + } + + @Override + public int createInputSplits() { + // Only the 'master' should be doing this. Wait until the number of + // processes that have reported health exceeds the minimum percentage. + // If the minimum percentage is not met, fail the job. Otherwise + // generate the input splits + try { + if (getZkExt().exists(inputSplitsPath, false) != null) { + LOG.info(inputSplitsPath + + " already exists, no need to create"); + return Integer.parseInt( + new String( + getZkExt().getData(inputSplitsPath, false, null))); + } + } catch (KeeperException.NoNodeException e) { + if (LOG.isInfoEnabled()) { + LOG.info("createInputSplits: Need to create the " + + "input splits at " + inputSplitsPath); + } + } catch (KeeperException e) { + throw new IllegalStateException( + "createInputSplits: KeeperException", e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "createInputSplits: InterrtupedException", e); + } - // Let workers know they can start trying to load the input splits - try { - getZkExt().create(INPUT_SPLITS_ALL_READY_PATH, - null, - Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT); - } catch (KeeperException.NodeExistsException e) { - LOG.info("createInputSplits: Node " + - INPUT_SPLITS_ALL_READY_PATH + " already exists."); - } catch (KeeperException e) { - throw new IllegalStateException( - "createInputSplits: KeeperException", e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "createInputSplits: IllegalStateException", e); - } + // When creating znodes, in case the master has already run, resume + // where it left off. + List healthyWorkerInfoList = checkWorkers(); + if (healthyWorkerInfoList == null) { + setJobState(ApplicationState.FAILED, -1, -1); + return -1; + } - return splitList.size(); - } - - /** - * Read the finalized checkpoint file and associated metadata files for the - * checkpoint. Modifies the {@link PartitionOwner} objects to get the - * checkpoint prefixes. It is an optimization to prevent all workers from - * searching all the files. Also read in the aggregator data from the - * finalized checkpoint file and setting it. - * - * @param superstep Checkpoint set to examine. - * @param partitionOwners Partition owners to modify with checkpoint - * prefixes - * @throws IOException - * @throws InterruptedException - * @throws KeeperException - */ - private void prepareCheckpointRestart( - long superstep, - Collection partitionOwners) - throws IOException, KeeperException, InterruptedException { - FileSystem fs = getFs(); - List validMetadataPathList = new ArrayList(); - String finalizedCheckpointPath = - getCheckpointBasePath(superstep) + CHECKPOINT_FINALIZED_POSTFIX; - DataInputStream finalizedStream = - fs.open(new Path(finalizedCheckpointPath)); - int prefixFileCount = finalizedStream.readInt(); - for (int i = 0; i < prefixFileCount; ++i) { - String metadataFilePath = - finalizedStream.readUTF() + CHECKPOINT_METADATA_POSTFIX; - validMetadataPathList.add(new Path(metadataFilePath)); - } + // Note that the input splits may only be a sample if + // INPUT_SPLIT_SAMPLE_PERCENT is set to something other than 100 + List splitList = + generateInputSplits(healthyWorkerInfoList.size()); + if (healthyWorkerInfoList.size() > splitList.size()) { + LOG.warn("createInputSplits: Number of inputSplits=" + + splitList.size() + " < " + + healthyWorkerInfoList.size() + + "=number of healthy processes, " + + "some workers will be not used"); + } + String inputSplitPath = null; + for (int i = 0; i < splitList.size(); ++i) { + try { + ByteArrayOutputStream byteArrayOutputStream = + new ByteArrayOutputStream(); + DataOutput outputStream = + new DataOutputStream(byteArrayOutputStream); + InputSplit inputSplit = splitList.get(i); + Text.writeString(outputStream, + inputSplit.getClass().getName()); + ((Writable) inputSplit).write(outputStream); + inputSplitPath = inputSplitsPath + "/" + i; + getZkExt().createExt(inputSplitPath, + byteArrayOutputStream.toByteArray(), + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT, + true); + if (LOG.isDebugEnabled()) { + LOG.debug("createInputSplits: Created input split " + + "with index " + i + " serialized as " + + byteArrayOutputStream.toString()); + } + } catch (KeeperException.NodeExistsException e) { + if (LOG.isInfoEnabled()) { + LOG.info("createInputSplits: Node " + + inputSplitPath + " already exists."); + } + } catch (KeeperException e) { + throw new IllegalStateException( + "createInputSplits: KeeperException", e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "createInputSplits: IllegalStateException", e); + } catch (IOException e) { + throw new IllegalStateException( + "createInputSplits: IOException", e); + } + } - // Set the merged aggregator data if it exists. - int aggregatorDataSize = finalizedStream.readInt(); - if (aggregatorDataSize > 0) { - byte [] aggregatorZkData = new byte[aggregatorDataSize]; - int actualDataRead = - finalizedStream.read(aggregatorZkData, 0, aggregatorDataSize); - if (actualDataRead != aggregatorDataSize) { - throw new RuntimeException( - "prepareCheckpointRestart: Only read " + actualDataRead + - " of " + aggregatorDataSize + " aggregator bytes from " + - finalizedCheckpointPath); - } - String mergedAggregatorPath = - getMergedAggregatorPath(getApplicationAttempt(), superstep - 1); - if (LOG.isInfoEnabled()) { - LOG.info("prepareCheckpointRestart: Reloading merged " + - "aggregator " + "data '" + - Arrays.toString(aggregatorZkData) + - "' to previous checkpoint in path " + - mergedAggregatorPath); - } - if (getZkExt().exists(mergedAggregatorPath, false) == null) { - getZkExt().createExt(mergedAggregatorPath, - aggregatorZkData, - Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT, - true); - } - else { - getZkExt().setData(mergedAggregatorPath, aggregatorZkData, -1); - } - } - finalizedStream.close(); - - Map idOwnerMap = - new HashMap(); - for (PartitionOwner partitionOwner : partitionOwners) { - if (idOwnerMap.put(partitionOwner.getPartitionId(), - partitionOwner) != null) { - throw new IllegalStateException( - "prepareCheckpointRestart: Duplicate partition " + - partitionOwner); - } - } - // Reading the metadata files. Simply assign each partition owner - // the correct file prefix based on the partition id. - for (Path metadataPath : validMetadataPathList) { - String checkpointFilePrefix = metadataPath.toString(); - checkpointFilePrefix = - checkpointFilePrefix.substring( - 0, - checkpointFilePrefix.length() - - CHECKPOINT_METADATA_POSTFIX.length()); - DataInputStream metadataStream = fs.open(metadataPath); - long partitions = metadataStream.readInt(); - for (long i = 0; i < partitions; ++i) { - long dataPos = metadataStream.readLong(); - int partitionId = metadataStream.readInt(); - PartitionOwner partitionOwner = idOwnerMap.get(partitionId); - if (LOG.isInfoEnabled()) { - LOG.info("prepareSuperstepRestart: File " + metadataPath + - " with position " + dataPos + - ", partition id = " + partitionId + - " assigned to " + partitionOwner); - } - partitionOwner.setCheckpointFilesPrefix(checkpointFilePrefix); - } - metadataStream.close(); - } + // Let workers know they can start trying to load the input splits + try { + getZkExt().create(inputSplitsAllReadyPath, + null, + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + } catch (KeeperException.NodeExistsException e) { + LOG.info("createInputSplits: Node " + + inputSplitsAllReadyPath + " already exists."); + } catch (KeeperException e) { + throw new IllegalStateException( + "createInputSplits: KeeperException", e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "createInputSplits: IllegalStateException", e); } - @Override - public void setup() { - // Might have to manually load a checkpoint. - // In that case, the input splits are not set, they will be faked by - // the checkpoint files. Each checkpoint file will be an input split - // and the input split - superstepCounter = getContext().getCounter( - GIRAPH_STATS_COUNTER_GROUP_NAME, "Superstep"); - vertexCounter = getContext().getCounter( - GIRAPH_STATS_COUNTER_GROUP_NAME, "Aggregate vertices"); - finishedVertexCounter = getContext().getCounter( - GIRAPH_STATS_COUNTER_GROUP_NAME, "Aggregate finished vertices"); - edgeCounter = getContext().getCounter( - GIRAPH_STATS_COUNTER_GROUP_NAME, "Aggregate edges"); - sentMessagesCounter = getContext().getCounter( - GIRAPH_STATS_COUNTER_GROUP_NAME, "Sent messages"); - currentWorkersCounter = getContext().getCounter( - GIRAPH_STATS_COUNTER_GROUP_NAME, "Current workers"); - currentMasterTaskPartitionCounter = getContext().getCounter( - GIRAPH_STATS_COUNTER_GROUP_NAME, "Current master task partition"); - lastCheckpointedSuperstepCounter = getContext().getCounter( - GIRAPH_STATS_COUNTER_GROUP_NAME, "Last checkpointed superstep"); - if (getRestartedSuperstep() != UNSET_SUPERSTEP) { - superstepCounter.increment(getRestartedSuperstep()); - } + return splitList.size(); + } + + /** + * Read the finalized checkpoint file and associated metadata files for the + * checkpoint. Modifies the {@link PartitionOwner} objects to get the + * checkpoint prefixes. It is an optimization to prevent all workers from + * searching all the files. Also read in the aggregator data from the + * finalized checkpoint file and setting it. + * + * @param superstep Checkpoint set to examine. + * @param partitionOwners Partition owners to modify with checkpoint + * prefixes + * @throws IOException + * @throws InterruptedException + * @throws KeeperException + */ + private void prepareCheckpointRestart( + long superstep, + Collection partitionOwners) + throws IOException, KeeperException, InterruptedException { + FileSystem fs = getFs(); + List validMetadataPathList = new ArrayList(); + String finalizedCheckpointPath = + getCheckpointBasePath(superstep) + CHECKPOINT_FINALIZED_POSTFIX; + DataInputStream finalizedStream = + fs.open(new Path(finalizedCheckpointPath)); + int prefixFileCount = finalizedStream.readInt(); + for (int i = 0; i < prefixFileCount; ++i) { + String metadataFilePath = + finalizedStream.readUTF() + CHECKPOINT_METADATA_POSTFIX; + validMetadataPathList.add(new Path(metadataFilePath)); } - @Override - public boolean becomeMaster() { - // Create my bid to become the master, then try to become the worker - // or return false. - String myBid = null; - try { - myBid = - getZkExt().createExt(MASTER_ELECTION_PATH + - "/" + getHostnamePartitionId(), - null, - Ids.OPEN_ACL_UNSAFE, - CreateMode.EPHEMERAL_SEQUENTIAL, - true); - } catch (KeeperException e) { - throw new IllegalStateException( - "becomeMaster: KeeperException", e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "becomeMaster: IllegalStateException", e); - } - while (true) { - JSONObject jobState = getJobState(); - try { - if ((jobState != null) && - ApplicationState.valueOf( - jobState.getString(JSONOBJ_STATE_KEY)) == - ApplicationState.FINISHED) { - LOG.info("becomeMaster: Job is finished, " + - "give up trying to be the master!"); - isMaster = false; - return isMaster; - } - } catch (JSONException e) { - throw new IllegalStateException( - "becomeMaster: Couldn't get state from " + jobState, e); - } - try { - List masterChildArr = - getZkExt().getChildrenExt( - MASTER_ELECTION_PATH, true, true, true); - if (LOG.isInfoEnabled()) { - LOG.info("becomeMaster: First child is '" + - masterChildArr.get(0) + "' and my bid is '" + - myBid + "'"); - } - if (masterChildArr.get(0).equals(myBid)) { - currentMasterTaskPartitionCounter.increment( - getTaskPartition() - - currentMasterTaskPartitionCounter.getValue()); - aggregatorWriter = - BspUtils.createAggregatorWriter(getConfiguration()); - try { - aggregatorWriter.initialize(getContext(), - getApplicationAttempt()); - } catch (IOException e) { - throw new IllegalStateException("becomeMaster: " + - "Couldn't initialize aggregatorWriter", e); - } - LOG.info("becomeMaster: I am now the master!"); - isMaster = true; - return isMaster; - } - LOG.info("becomeMaster: Waiting to become the master..."); - getMasterElectionChildrenChangedEvent().waitForever(); - getMasterElectionChildrenChangedEvent().reset(); - } catch (KeeperException e) { - throw new IllegalStateException( - "becomeMaster: KeeperException", e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "becomeMaster: IllegalStateException", e); - } - } + // Set the merged aggregator data if it exists. + int aggregatorDataSize = finalizedStream.readInt(); + if (aggregatorDataSize > 0) { + byte [] aggregatorZkData = new byte[aggregatorDataSize]; + int actualDataRead = + finalizedStream.read(aggregatorZkData, 0, aggregatorDataSize); + if (actualDataRead != aggregatorDataSize) { + throw new RuntimeException( + "prepareCheckpointRestart: Only read " + actualDataRead + + " of " + aggregatorDataSize + " aggregator bytes from " + + finalizedCheckpointPath); + } + String mergedAggregatorPath = + getMergedAggregatorPath(getApplicationAttempt(), superstep - 1); + if (LOG.isInfoEnabled()) { + LOG.info("prepareCheckpointRestart: Reloading merged " + + "aggregator " + "data '" + + Arrays.toString(aggregatorZkData) + + "' to previous checkpoint in path " + + mergedAggregatorPath); + } + if (getZkExt().exists(mergedAggregatorPath, false) == null) { + getZkExt().createExt(mergedAggregatorPath, + aggregatorZkData, + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT, + true); + } else { + getZkExt().setData(mergedAggregatorPath, aggregatorZkData, -1); + } + } + finalizedStream.close(); + + Map idOwnerMap = + new HashMap(); + for (PartitionOwner partitionOwner : partitionOwners) { + if (idOwnerMap.put(partitionOwner.getPartitionId(), + partitionOwner) != null) { + throw new IllegalStateException( + "prepareCheckpointRestart: Duplicate partition " + + partitionOwner); + } + } + // Reading the metadata files. Simply assign each partition owner + // the correct file prefix based on the partition id. + for (Path metadataPath : validMetadataPathList) { + String checkpointFilePrefix = metadataPath.toString(); + checkpointFilePrefix = + checkpointFilePrefix.substring( + 0, + checkpointFilePrefix.length() - + CHECKPOINT_METADATA_POSTFIX.length()); + DataInputStream metadataStream = fs.open(metadataPath); + long partitions = metadataStream.readInt(); + for (long i = 0; i < partitions; ++i) { + long dataPos = metadataStream.readLong(); + int partitionId = metadataStream.readInt(); + PartitionOwner partitionOwner = idOwnerMap.get(partitionId); + if (LOG.isInfoEnabled()) { + LOG.info("prepareSuperstepRestart: File " + metadataPath + + " with position " + dataPos + + ", partition id = " + partitionId + + " assigned to " + partitionOwner); + } + partitionOwner.setCheckpointFilesPrefix(checkpointFilePrefix); + } + metadataStream.close(); + } + } + + @Override + public void setup() { + // Might have to manually load a checkpoint. + // In that case, the input splits are not set, they will be faked by + // the checkpoint files. Each checkpoint file will be an input split + // and the input split + superstepCounter = getContext().getCounter( + GIRAPH_STATS_COUNTER_GROUP_NAME, "Superstep"); + vertexCounter = getContext().getCounter( + GIRAPH_STATS_COUNTER_GROUP_NAME, "Aggregate vertices"); + finishedVertexCounter = getContext().getCounter( + GIRAPH_STATS_COUNTER_GROUP_NAME, "Aggregate finished vertices"); + edgeCounter = getContext().getCounter( + GIRAPH_STATS_COUNTER_GROUP_NAME, "Aggregate edges"); + sentMessagesCounter = getContext().getCounter( + GIRAPH_STATS_COUNTER_GROUP_NAME, "Sent messages"); + currentWorkersCounter = getContext().getCounter( + GIRAPH_STATS_COUNTER_GROUP_NAME, "Current workers"); + currentMasterTaskPartitionCounter = getContext().getCounter( + GIRAPH_STATS_COUNTER_GROUP_NAME, "Current master task partition"); + lastCheckpointedSuperstepCounter = getContext().getCounter( + GIRAPH_STATS_COUNTER_GROUP_NAME, "Last checkpointed superstep"); + if (getRestartedSuperstep() != UNSET_SUPERSTEP) { + superstepCounter.increment(getRestartedSuperstep()); + } + } + + @Override + public boolean becomeMaster() { + // Create my bid to become the master, then try to become the worker + // or return false. + String myBid = null; + try { + myBid = + getZkExt().createExt(masterElectionPath + + "/" + getHostnamePartitionId(), + null, + Ids.OPEN_ACL_UNSAFE, + CreateMode.EPHEMERAL_SEQUENTIAL, + true); + } catch (KeeperException e) { + throw new IllegalStateException( + "becomeMaster: KeeperException", e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "becomeMaster: IllegalStateException", e); + } + while (true) { + JSONObject jobState = getJobState(); + try { + if ((jobState != null) && + ApplicationState.valueOf( + jobState.getString(JSONOBJ_STATE_KEY)) == + ApplicationState.FINISHED) { + LOG.info("becomeMaster: Job is finished, " + + "give up trying to be the master!"); + isMaster = false; + return isMaster; + } + } catch (JSONException e) { + throw new IllegalStateException( + "becomeMaster: Couldn't get state from " + jobState, e); + } + try { + List masterChildArr = + getZkExt().getChildrenExt( + masterElectionPath, true, true, true); + if (LOG.isInfoEnabled()) { + LOG.info("becomeMaster: First child is '" + + masterChildArr.get(0) + "' and my bid is '" + + myBid + "'"); + } + if (masterChildArr.get(0).equals(myBid)) { + currentMasterTaskPartitionCounter.increment( + getTaskPartition() - + currentMasterTaskPartitionCounter.getValue()); + aggregatorWriter = + BspUtils.createAggregatorWriter(getConfiguration()); + try { + aggregatorWriter.initialize(getContext(), + getApplicationAttempt()); + } catch (IOException e) { + throw new IllegalStateException("becomeMaster: " + + "Couldn't initialize aggregatorWriter", e); + } + LOG.info("becomeMaster: I am now the master!"); + isMaster = true; + return isMaster; + } + LOG.info("becomeMaster: Waiting to become the master..."); + getMasterElectionChildrenChangedEvent().waitForever(); + getMasterElectionChildrenChangedEvent().reset(); + } catch (KeeperException e) { + throw new IllegalStateException( + "becomeMaster: KeeperException", e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "becomeMaster: IllegalStateException", e); + } + } + } + + /** + * Collect and aggregate the worker statistics for a particular superstep. + * + * @param superstep Superstep to aggregate on + * @return Global statistics aggregated on all worker statistics + */ + private GlobalStats aggregateWorkerStats(long superstep) { + Class partitionStatsClass = + masterGraphPartitioner.createPartitionStats().getClass(); + GlobalStats globalStats = new GlobalStats(); + // Get the stats from the all the worker selected nodes + String workerFinishedPath = + getWorkerFinishedPath(getApplicationAttempt(), superstep); + List workerFinishedPathList = null; + try { + workerFinishedPathList = + getZkExt().getChildrenExt( + workerFinishedPath, false, false, true); + } catch (KeeperException e) { + throw new IllegalStateException( + "aggregateWorkerStats: KeeperException", e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "aggregateWorkerStats: InterruptedException", e); } - /** - * Collect and aggregate the worker statistics for a particular superstep. - * - * @param superstep Superstep to aggregate on - * @return Global statistics aggregated on all worker statistics - */ - private GlobalStats aggregateWorkerStats(long superstep) { - Class partitionStatsClass = - masterGraphPartitioner.createPartitionStats().getClass(); - GlobalStats globalStats = new GlobalStats(); - // Get the stats from the all the worker selected nodes - String workerFinishedPath = - getWorkerFinishedPath(getApplicationAttempt(), superstep); - List workerFinishedPathList = null; - try { - workerFinishedPathList = - getZkExt().getChildrenExt( - workerFinishedPath, false, false, true); - } catch (KeeperException e) { - throw new IllegalStateException( - "aggregateWorkerStats: KeeperException", e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "aggregateWorkerStats: InterruptedException", e); - } + allPartitionStatsList.clear(); + for (String finishedPath : workerFinishedPathList) { + JSONObject workerFinishedInfoObj = null; + try { + byte [] zkData = + getZkExt().getData(finishedPath, false, null); + workerFinishedInfoObj = new JSONObject(new String(zkData)); + List writableList = + WritableUtils.readListFieldsFromByteArray( + Base64.decode(workerFinishedInfoObj.getString( + JSONOBJ_PARTITION_STATS_KEY)), + partitionStatsClass, + getConfiguration()); + for (Writable writable : writableList) { + globalStats.addPartitionStats((PartitionStats) writable); + globalStats.addMessageCount( + workerFinishedInfoObj.getLong( + JSONOBJ_NUM_MESSAGES_KEY)); + allPartitionStatsList.add((PartitionStats) writable); + } + } catch (JSONException e) { + throw new IllegalStateException( + "aggregateWorkerStats: JSONException", e); + } catch (KeeperException e) { + throw new IllegalStateException( + "aggregateWorkerStats: KeeperException", e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "aggregateWorkerStats: InterruptedException", e); + } catch (IOException e) { + throw new IllegalStateException( + "aggregateWorkerStats: IOException", e); + } + } - allPartitionStatsList.clear(); - for (String finishedPath : workerFinishedPathList) { - JSONObject workerFinishedInfoObj = null; - try { - byte [] zkData = - getZkExt().getData(finishedPath, false, null); - workerFinishedInfoObj = new JSONObject(new String(zkData)); - List writableList = - WritableUtils.readListFieldsFromByteArray( - Base64.decode(workerFinishedInfoObj.getString( - JSONOBJ_PARTITION_STATS_KEY)), - partitionStatsClass, - getConfiguration()); - for (Writable writable : writableList) { - globalStats.addPartitionStats((PartitionStats) writable); - globalStats.addMessageCount( - workerFinishedInfoObj.getLong( - JSONOBJ_NUM_MESSAGES_KEY)); - allPartitionStatsList.add((PartitionStats) writable); - } - } catch (JSONException e) { - throw new IllegalStateException( - "aggregateWorkerStats: JSONException", e); - } catch (KeeperException e) { - throw new IllegalStateException( - "aggregateWorkerStats: KeeperException", e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "aggregateWorkerStats: InterruptedException", e); - } catch (IOException e) { - throw new IllegalStateException( - "aggregateWorkerStats: IOException", e); - } - } + if (LOG.isInfoEnabled()) { + LOG.info("aggregateWorkerStats: Aggregation found " + globalStats + + " on superstep = " + getSuperstep()); + } + return globalStats; + } + + /** + * Get the aggregator values for a particular superstep, + * aggregate and save them. Does nothing on the INPUT_SUPERSTEP. + * + * @param superstep superstep to check + */ + private void collectAndProcessAggregatorValues(long superstep) { + if (superstep == INPUT_SUPERSTEP) { + // Nothing to collect on the input superstep + return; + } + Map> aggregatorMap = + new TreeMap>(); + String workerFinishedPath = + getWorkerFinishedPath(getApplicationAttempt(), superstep); + List hostnameIdPathList = null; + try { + hostnameIdPathList = + getZkExt().getChildrenExt( + workerFinishedPath, false, false, true); + } catch (KeeperException e) { + throw new IllegalStateException( + "collectAndProcessAggregatorValues: KeeperException", e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "collectAndProcessAggregatorValues: InterruptedException", e); + } - if (LOG.isInfoEnabled()) { - LOG.info("aggregateWorkerStats: Aggregation found " + globalStats + - " on superstep = " + getSuperstep()); - } - return globalStats; - } - - /** - * Get the aggregator values for a particular superstep, - * aggregate and save them. Does nothing on the INPUT_SUPERSTEP. - * - * @param superstep superstep to check - */ - private void collectAndProcessAggregatorValues(long superstep) { - if (superstep == INPUT_SUPERSTEP) { - // Nothing to collect on the input superstep - return; - } - Map> aggregatorMap = - new TreeMap>(); - String workerFinishedPath = - getWorkerFinishedPath(getApplicationAttempt(), superstep); - List hostnameIdPathList = null; + for (String hostnameIdPath : hostnameIdPathList) { + JSONObject workerFinishedInfoObj = null; + JSONArray aggregatorArray = null; + try { + byte [] zkData = + getZkExt().getData(hostnameIdPath, false, null); + workerFinishedInfoObj = new JSONObject(new String(zkData)); + } catch (KeeperException e) { + throw new IllegalStateException( + "collectAndProcessAggregatorValues: KeeperException", e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "collectAndProcessAggregatorValues: InterruptedException", + e); + } catch (JSONException e) { + throw new IllegalStateException( + "collectAndProcessAggregatorValues: JSONException", e); + } + try { + aggregatorArray = workerFinishedInfoObj.getJSONArray( + JSONOBJ_AGGREGATOR_VALUE_ARRAY_KEY); + } catch (JSONException e) { + if (LOG.isDebugEnabled()) { + LOG.debug("collectAndProcessAggregatorValues: " + + "No aggregators" + " for " + hostnameIdPath); + } + continue; + } + for (int i = 0; i < aggregatorArray.length(); ++i) { try { - hostnameIdPathList = - getZkExt().getChildrenExt( - workerFinishedPath, false, false, true); - } catch (KeeperException e) { - throw new IllegalStateException( - "collectAndProcessAggregatorValues: KeeperException", e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "collectAndProcessAggregatorValues: InterruptedException", e); - } - - for (String hostnameIdPath : hostnameIdPathList) { - JSONObject workerFinishedInfoObj = null; - JSONArray aggregatorArray = null; - try { - byte [] zkData = - getZkExt().getData(hostnameIdPath, false, null); - workerFinishedInfoObj = new JSONObject(new String(zkData)); - } catch (KeeperException e) { - throw new IllegalStateException( - "collectAndProcessAggregatorValues: KeeperException", e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "collectAndProcessAggregatorValues: InterruptedException", - e); - } catch (JSONException e) { - throw new IllegalStateException( - "collectAndProcessAggregatorValues: JSONException", e); - } - try { - aggregatorArray = workerFinishedInfoObj.getJSONArray( - JSONOBJ_AGGREGATOR_VALUE_ARRAY_KEY); - } catch (JSONException e) { - if (LOG.isDebugEnabled()) { - LOG.debug("collectAndProcessAggregatorValues: " + - "No aggregators" + " for " + hostnameIdPath); - } - continue; + if (LOG.isInfoEnabled()) { + LOG.info("collectAndProcessAggregatorValues: " + + "Getting aggregators from " + + aggregatorArray.getJSONObject(i)); + } + String aggregatorName = + aggregatorArray.getJSONObject(i).getString( + AGGREGATOR_NAME_KEY); + String aggregatorClassName = + aggregatorArray.getJSONObject(i).getString( + AGGREGATOR_CLASS_NAME_KEY); + @SuppressWarnings("unchecked") + Aggregator aggregator = + (Aggregator) aggregatorMap.get(aggregatorName); + boolean firstTime = false; + if (aggregator == null) { + @SuppressWarnings("unchecked") + Aggregator aggregatorWritable = + (Aggregator) getAggregator(aggregatorName); + aggregator = aggregatorWritable; + if (aggregator == null) { + @SuppressWarnings("unchecked") + Class> aggregatorClass = + (Class>) + Class.forName(aggregatorClassName); + aggregator = registerAggregator( + aggregatorName, + aggregatorClass); } - for (int i = 0; i < aggregatorArray.length(); ++i) { - try { - if (LOG.isInfoEnabled()) { - LOG.info("collectAndProcessAggregatorValues: " + - "Getting aggregators from " + - aggregatorArray.getJSONObject(i)); - } - String aggregatorName = - aggregatorArray.getJSONObject(i).getString( - AGGREGATOR_NAME_KEY); - String aggregatorClassName = - aggregatorArray.getJSONObject(i).getString( - AGGREGATOR_CLASS_NAME_KEY); - @SuppressWarnings("unchecked") - Aggregator aggregator = - (Aggregator) aggregatorMap.get(aggregatorName); - boolean firstTime = false; - if (aggregator == null) { - @SuppressWarnings("unchecked") - Aggregator aggregatorWritable = - (Aggregator) getAggregator(aggregatorName); - aggregator = aggregatorWritable; - if (aggregator == null) { - @SuppressWarnings("unchecked") - Class> aggregatorClass = - (Class>) - Class.forName(aggregatorClassName); - aggregator = registerAggregator( - aggregatorName, - aggregatorClass); - } - aggregatorMap.put(aggregatorName, aggregator); - firstTime = true; - } - Writable aggregatorValue = - aggregator.createAggregatedValue(); - InputStream input = - new ByteArrayInputStream( - Base64.decode( - aggregatorArray.getJSONObject(i). - getString(AGGREGATOR_VALUE_KEY))); - aggregatorValue.readFields(new DataInputStream(input)); - if (LOG.isDebugEnabled()) { - LOG.debug("collectAndProcessAggregatorValues: " + - "aggregator value size=" + input.available() + - " for aggregator=" + aggregatorName + - " value=" + aggregatorValue); - } - if (firstTime) { - aggregator.setAggregatedValue(aggregatorValue); - } else { - aggregator.aggregate(aggregatorValue); - } - } catch (IOException e) { - throw new IllegalStateException( - "collectAndProcessAggregatorValues: " + - "IOException when reading aggregator data " + - aggregatorArray, e); - } catch (JSONException e) { - throw new IllegalStateException( - "collectAndProcessAggregatorValues: " + - "JSONException when reading aggregator data " + - aggregatorArray, e); - } catch (ClassNotFoundException e) { - throw new IllegalStateException( - "collectAndProcessAggregatorValues: " + - "ClassNotFoundException when reading aggregator data " + - aggregatorArray, e); - } catch (InstantiationException e) { - throw new IllegalStateException( - "collectAndProcessAggregatorValues: " + - "InstantiationException when reading aggregator data " + - aggregatorArray, e); - } catch (IllegalAccessException e) { - throw new IllegalStateException( - "collectAndProcessAggregatorValues: " + - "IOException when reading aggregator data " + - aggregatorArray, e); - } - } - } - if (aggregatorMap.size() > 0) { - String mergedAggregatorPath = - getMergedAggregatorPath(getApplicationAttempt(), superstep); - byte [] zkData = null; - JSONArray aggregatorArray = new JSONArray(); - for (Map.Entry> entry : - aggregatorMap.entrySet()) { - try { - ByteArrayOutputStream outputStream = - new ByteArrayOutputStream(); - DataOutput output = new DataOutputStream(outputStream); - entry.getValue().getAggregatedValue().write(output); - - JSONObject aggregatorObj = new JSONObject(); - aggregatorObj.put(AGGREGATOR_NAME_KEY, - entry.getKey()); - aggregatorObj.put( - AGGREGATOR_VALUE_KEY, - Base64.encodeBytes(outputStream.toByteArray())); - aggregatorArray.put(aggregatorObj); - if (LOG.isInfoEnabled()) { - LOG.info("collectAndProcessAggregatorValues: " + - "Trying to add aggregatorObj " + - aggregatorObj + "(" + - entry.getValue().getAggregatedValue() + - ") to merged aggregator path " + - mergedAggregatorPath); - } - } catch (IOException e) { - throw new IllegalStateException( - "collectAndProcessAggregatorValues: " + - "IllegalStateException", e); - } catch (JSONException e) { - throw new IllegalStateException( - "collectAndProcessAggregatorValues: JSONException", e); - } - } - try { - zkData = aggregatorArray.toString().getBytes(); - getZkExt().createExt(mergedAggregatorPath, - zkData, - Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT, - true); - } catch (KeeperException.NodeExistsException e) { - LOG.warn("collectAndProcessAggregatorValues: " + - mergedAggregatorPath+ - " already exists!"); - } catch (KeeperException e) { - throw new IllegalStateException( - "collectAndProcessAggregatorValues: KeeperException", e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "collectAndProcessAggregatorValues: IllegalStateException", - e); - } - if (LOG.isInfoEnabled()) { - LOG.info("collectAndProcessAggregatorValues: Finished " + - "loading " + - mergedAggregatorPath+ " with aggregator values " + - aggregatorArray); - } - } + aggregatorMap.put(aggregatorName, aggregator); + firstTime = true; + } + Writable aggregatorValue = + aggregator.createAggregatedValue(); + InputStream input = + new ByteArrayInputStream( + Base64.decode( + aggregatorArray.getJSONObject(i). + getString(AGGREGATOR_VALUE_KEY))); + aggregatorValue.readFields(new DataInputStream(input)); + if (LOG.isDebugEnabled()) { + LOG.debug("collectAndProcessAggregatorValues: " + + "aggregator value size=" + input.available() + + " for aggregator=" + aggregatorName + + " value=" + aggregatorValue); + } + if (firstTime) { + aggregator.setAggregatedValue(aggregatorValue); + } else { + aggregator.aggregate(aggregatorValue); + } + } catch (IOException e) { + throw new IllegalStateException( + "collectAndProcessAggregatorValues: " + + "IOException when reading aggregator data " + + aggregatorArray, e); + } catch (JSONException e) { + throw new IllegalStateException( + "collectAndProcessAggregatorValues: " + + "JSONException when reading aggregator data " + + aggregatorArray, e); + } catch (ClassNotFoundException e) { + throw new IllegalStateException( + "collectAndProcessAggregatorValues: " + + "ClassNotFoundException when reading aggregator data " + + aggregatorArray, e); + } catch (InstantiationException e) { + throw new IllegalStateException( + "collectAndProcessAggregatorValues: " + + "InstantiationException when reading aggregator data " + + aggregatorArray, e); + } catch (IllegalAccessException e) { + throw new IllegalStateException( + "collectAndProcessAggregatorValues: " + + "IOException when reading aggregator data " + + aggregatorArray, e); + } + } } - - /** - * Finalize the checkpoint file prefixes by taking the chosen workers and - * writing them to a finalized file. Also write out the master - * aggregated aggregator array from the previous superstep. - * - * @param superstep superstep to finalize - * @param chosenWorkerList list of chosen workers that will be finalized - * @throws IOException - * @throws InterruptedException - * @throws KeeperException - */ - private void finalizeCheckpoint( - long superstep, - List chosenWorkerInfoList) - throws IOException, KeeperException, InterruptedException { - Path finalizedCheckpointPath = - new Path(getCheckpointBasePath(superstep) + - CHECKPOINT_FINALIZED_POSTFIX); + if (aggregatorMap.size() > 0) { + String mergedAggregatorPath = + getMergedAggregatorPath(getApplicationAttempt(), superstep); + byte [] zkData = null; + JSONArray aggregatorArray = new JSONArray(); + for (Map.Entry> entry : + aggregatorMap.entrySet()) { try { - getFs().delete(finalizedCheckpointPath, false); + ByteArrayOutputStream outputStream = + new ByteArrayOutputStream(); + DataOutput output = new DataOutputStream(outputStream); + entry.getValue().getAggregatedValue().write(output); + + JSONObject aggregatorObj = new JSONObject(); + aggregatorObj.put(AGGREGATOR_NAME_KEY, + entry.getKey()); + aggregatorObj.put( + AGGREGATOR_VALUE_KEY, + Base64.encodeBytes(outputStream.toByteArray())); + aggregatorArray.put(aggregatorObj); + if (LOG.isInfoEnabled()) { + LOG.info("collectAndProcessAggregatorValues: " + + "Trying to add aggregatorObj " + + aggregatorObj + "(" + + entry.getValue().getAggregatedValue() + + ") to merged aggregator path " + + mergedAggregatorPath); + } } catch (IOException e) { - LOG.warn("finalizedValidCheckpointPrefixes: Removed old file " + - finalizedCheckpointPath); - } - - // Format: - // - // ... - // - FSDataOutputStream finalizedOutputStream = - getFs().create(finalizedCheckpointPath); - finalizedOutputStream.writeInt(chosenWorkerInfoList.size()); - for (WorkerInfo chosenWorkerInfo : chosenWorkerInfoList) { - String chosenWorkerInfoPrefix = - getCheckpointBasePath(superstep) + "." + - chosenWorkerInfo.getHostnameId(); - finalizedOutputStream.writeUTF(chosenWorkerInfoPrefix); - } - String mergedAggregatorPath = - getMergedAggregatorPath(getApplicationAttempt(), superstep - 1); - if (getZkExt().exists(mergedAggregatorPath, false) != null) { - byte [] aggregatorZkData = - getZkExt().getData(mergedAggregatorPath, false, null); - finalizedOutputStream.writeInt(aggregatorZkData.length); - finalizedOutputStream.write(aggregatorZkData); - } - else { - finalizedOutputStream.writeInt(0); - } - finalizedOutputStream.close(); - lastCheckpointedSuperstep = superstep; - lastCheckpointedSuperstepCounter.increment(superstep - - lastCheckpointedSuperstepCounter.getValue()); - } - - /** - * Assign the partitions for this superstep. If there are changes, - * the workers will know how to do the exchange. If this was a restarted - * superstep, then make sure to provide information on where to find the - * checkpoint file. - * - * @param allPartitionStatsList All partition stats - * @param chosenWorkerInfoList All the chosen worker infos - * @param masterGraphPartitioner Master graph partitioner - */ - private void assignPartitionOwners( - List allPartitionStatsList, - List chosenWorkerInfoList, - MasterGraphPartitioner masterGraphPartitioner) { - Collection partitionOwners; - if (getSuperstep() == INPUT_SUPERSTEP || - getSuperstep() == getRestartedSuperstep()) { - partitionOwners = - masterGraphPartitioner.createInitialPartitionOwners( - chosenWorkerInfoList, maxWorkers); - if (partitionOwners.isEmpty()) { - throw new IllegalStateException( - "assignAndExchangePartitions: No partition owners set"); - } - } else { - partitionOwners = - masterGraphPartitioner.generateChangedPartitionOwners( - allPartitionStatsList, - chosenWorkerInfoList, - maxWorkers, - getSuperstep()); - - PartitionUtils.analyzePartitionStats(partitionOwners, - allPartitionStatsList); - } - - // If restarted, prepare the checkpoint restart - if (getRestartedSuperstep() == getSuperstep()) { - try { - prepareCheckpointRestart(getSuperstep(), partitionOwners); - } catch (IOException e) { - throw new IllegalStateException( - "assignPartitionOwners: IOException on preparing", e); - } catch (KeeperException e) { - throw new IllegalStateException( - "assignPartitionOwners: KeeperException on preparing", e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "assignPartitionOwners: InteruptedException on preparing", - e); - } - } + throw new IllegalStateException( + "collectAndProcessAggregatorValues: " + + "IllegalStateException", e); + } catch (JSONException e) { + throw new IllegalStateException( + "collectAndProcessAggregatorValues: JSONException", e); + } + } + try { + zkData = aggregatorArray.toString().getBytes(); + getZkExt().createExt(mergedAggregatorPath, + zkData, + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT, + true); + } catch (KeeperException.NodeExistsException e) { + LOG.warn("collectAndProcessAggregatorValues: " + + mergedAggregatorPath + " already exists!"); + } catch (KeeperException e) { + throw new IllegalStateException( + "collectAndProcessAggregatorValues: KeeperException", e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "collectAndProcessAggregatorValues: IllegalStateException", + e); + } + if (LOG.isInfoEnabled()) { + LOG.info("collectAndProcessAggregatorValues: Finished " + + "loading " + + mergedAggregatorPath + " with aggregator values " + + aggregatorArray); + } + } + } + + /** + * Finalize the checkpoint file prefixes by taking the chosen workers and + * writing them to a finalized file. Also write out the master + * aggregated aggregator array from the previous superstep. + * + * @param superstep superstep to finalize + * @param chosenWorkerInfoList list of chosen workers that will be finalized + * @throws IOException + * @throws InterruptedException + * @throws KeeperException + */ + private void finalizeCheckpoint(long superstep, + List chosenWorkerInfoList) + throws IOException, KeeperException, InterruptedException { + Path finalizedCheckpointPath = + new Path(getCheckpointBasePath(superstep) + + CHECKPOINT_FINALIZED_POSTFIX); + try { + getFs().delete(finalizedCheckpointPath, false); + } catch (IOException e) { + LOG.warn("finalizedValidCheckpointPrefixes: Removed old file " + + finalizedCheckpointPath); + } - // There will be some exchange of partitions - if (!partitionOwners.isEmpty()) { - String vertexExchangePath = - getPartitionExchangePath(getApplicationAttempt(), - getSuperstep()); - try { - getZkExt().createOnceExt(vertexExchangePath, - null, - Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT, - true); - } catch (KeeperException e) { - throw new IllegalStateException( - "assignPartitionOwners: KeeperException creating " + - vertexExchangePath); - } catch (InterruptedException e) { - throw new IllegalStateException( - "assignPartitionOwners: InterruptedException creating " + - vertexExchangePath); - } - } + // Format: + // + // ... + // + FSDataOutputStream finalizedOutputStream = + getFs().create(finalizedCheckpointPath); + finalizedOutputStream.writeInt(chosenWorkerInfoList.size()); + for (WorkerInfo chosenWorkerInfo : chosenWorkerInfoList) { + String chosenWorkerInfoPrefix = + getCheckpointBasePath(superstep) + "." + + chosenWorkerInfo.getHostnameId(); + finalizedOutputStream.writeUTF(chosenWorkerInfoPrefix); + } + String mergedAggregatorPath = + getMergedAggregatorPath(getApplicationAttempt(), superstep - 1); + if (getZkExt().exists(mergedAggregatorPath, false) != null) { + byte [] aggregatorZkData = + getZkExt().getData(mergedAggregatorPath, false, null); + finalizedOutputStream.writeInt(aggregatorZkData.length); + finalizedOutputStream.write(aggregatorZkData); + } else { + finalizedOutputStream.writeInt(0); + } + finalizedOutputStream.close(); + lastCheckpointedSuperstep = superstep; + lastCheckpointedSuperstepCounter.increment(superstep - + lastCheckpointedSuperstepCounter.getValue()); + } + + /** + * Assign the partitions for this superstep. If there are changes, + * the workers will know how to do the exchange. If this was a restarted + * superstep, then make sure to provide information on where to find the + * checkpoint file. + * + * @param allPartitionStatsList All partition stats + * @param chosenWorkerInfoList All the chosen worker infos + * @param masterGraphPartitioner Master graph partitioner + */ + private void assignPartitionOwners( + List allPartitionStatsList, + List chosenWorkerInfoList, + MasterGraphPartitioner masterGraphPartitioner) { + Collection partitionOwners; + if (getSuperstep() == INPUT_SUPERSTEP || + getSuperstep() == getRestartedSuperstep()) { + partitionOwners = + masterGraphPartitioner.createInitialPartitionOwners( + chosenWorkerInfoList, maxWorkers); + if (partitionOwners.isEmpty()) { + throw new IllegalStateException( + "assignAndExchangePartitions: No partition owners set"); + } + } else { + partitionOwners = + masterGraphPartitioner.generateChangedPartitionOwners( + allPartitionStatsList, + chosenWorkerInfoList, + maxWorkers, + getSuperstep()); + + PartitionUtils.analyzePartitionStats(partitionOwners, + allPartitionStatsList); + } - // Workers are waiting for these assignments - String partitionAssignmentsPath = - getPartitionAssignmentsPath(getApplicationAttempt(), - getSuperstep()); - WritableUtils.writeListToZnode( - getZkExt(), - partitionAssignmentsPath, - -1, - new ArrayList(partitionOwners)); - } - - /** - * Check whether the workers chosen for this superstep are still alive - * - * @param chosenWorkerHealthPath Path to the healthy workers in ZooKeeper - * @param chosenWorkerList List of the healthy workers - * @return true if they are all alive, false otherwise. - * @throws InterruptedException - * @throws KeeperException - */ - private boolean superstepChosenWorkerAlive( - String chosenWorkerInfoHealthPath, - List chosenWorkerInfoList) - throws KeeperException, InterruptedException { - List chosenWorkerInfoHealthyList = - getWorkerInfosFromPath(chosenWorkerInfoHealthPath, false); - Set chosenWorkerInfoHealthySet = - new HashSet(chosenWorkerInfoHealthyList); - boolean allChosenWorkersHealthy = true; - for (WorkerInfo chosenWorkerInfo : chosenWorkerInfoList) { - if (!chosenWorkerInfoHealthySet.contains(chosenWorkerInfo)) { - allChosenWorkersHealthy = false; - LOG.error("superstepChosenWorkerAlive: Missing chosen " + - "worker " + chosenWorkerInfo + - " on superstep " + getSuperstep()); - } - } - return allChosenWorkersHealthy; + // If restarted, prepare the checkpoint restart + if (getRestartedSuperstep() == getSuperstep()) { + try { + prepareCheckpointRestart(getSuperstep(), partitionOwners); + } catch (IOException e) { + throw new IllegalStateException( + "assignPartitionOwners: IOException on preparing", e); + } catch (KeeperException e) { + throw new IllegalStateException( + "assignPartitionOwners: KeeperException on preparing", e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "assignPartitionOwners: InteruptedException on preparing", + e); + } } - @Override - public void restartFromCheckpoint(long checkpoint) { - // Process: - // 1. Remove all old input split data - // 2. Increase the application attempt and set to the correct checkpoint - // 3. Send command to all workers to restart their tasks - try { - getZkExt().deleteExt(INPUT_SPLIT_PATH, -1, true); - } catch (InterruptedException e) { - throw new RuntimeException( - "retartFromCheckpoint: InterruptedException", e); - } catch (KeeperException e) { - throw new RuntimeException( - "retartFromCheckpoint: KeeperException", e); - } - setApplicationAttempt(getApplicationAttempt() + 1); - setCachedSuperstep(checkpoint); - setRestartedSuperstep(checkpoint); - setJobState(ApplicationState.START_SUPERSTEP, - getApplicationAttempt(), - checkpoint); - } - - /** - * Only get the finalized checkpoint files - */ - public static class FinalizedCheckpointPathFilter implements PathFilter { - @Override - public boolean accept(Path path) { - if (path.getName().endsWith( - BspService.CHECKPOINT_FINALIZED_POSTFIX)) { - return true; - } else { - return false; - } - } + // There will be some exchange of partitions + if (!partitionOwners.isEmpty()) { + String vertexExchangePath = + getPartitionExchangePath(getApplicationAttempt(), + getSuperstep()); + try { + getZkExt().createOnceExt(vertexExchangePath, + null, + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT, + true); + } catch (KeeperException e) { + throw new IllegalStateException( + "assignPartitionOwners: KeeperException creating " + + vertexExchangePath); + } catch (InterruptedException e) { + throw new IllegalStateException( + "assignPartitionOwners: InterruptedException creating " + + vertexExchangePath); + } } + // Workers are waiting for these assignments + String partitionAssignmentsPath = + getPartitionAssignmentsPath(getApplicationAttempt(), + getSuperstep()); + WritableUtils.writeListToZnode( + getZkExt(), + partitionAssignmentsPath, + -1, + new ArrayList(partitionOwners)); + } + + /** + * Check whether the workers chosen for this superstep are still alive + * + * @param chosenWorkerInfoHealthPath Path to the healthy workers in ZooKeeper + * @param chosenWorkerInfoList List of the healthy workers + * @return true if they are all alive, false otherwise. + * @throws InterruptedException + * @throws KeeperException + */ + private boolean superstepChosenWorkerAlive( + String chosenWorkerInfoHealthPath, + List chosenWorkerInfoList) + throws KeeperException, InterruptedException { + List chosenWorkerInfoHealthyList = + getWorkerInfosFromPath(chosenWorkerInfoHealthPath, false); + Set chosenWorkerInfoHealthySet = + new HashSet(chosenWorkerInfoHealthyList); + boolean allChosenWorkersHealthy = true; + for (WorkerInfo chosenWorkerInfo : chosenWorkerInfoList) { + if (!chosenWorkerInfoHealthySet.contains(chosenWorkerInfo)) { + allChosenWorkersHealthy = false; + LOG.error("superstepChosenWorkerAlive: Missing chosen " + + "worker " + chosenWorkerInfo + + " on superstep " + getSuperstep()); + } + } + return allChosenWorkersHealthy; + } + + @Override + public void restartFromCheckpoint(long checkpoint) { + // Process: + // 1. Remove all old input split data + // 2. Increase the application attempt and set to the correct checkpoint + // 3. Send command to all workers to restart their tasks + try { + getZkExt().deleteExt(inputSplitsPath, -1, true); + } catch (InterruptedException e) { + throw new RuntimeException( + "retartFromCheckpoint: InterruptedException", e); + } catch (KeeperException e) { + throw new RuntimeException( + "retartFromCheckpoint: KeeperException", e); + } + setApplicationAttempt(getApplicationAttempt() + 1); + setCachedSuperstep(checkpoint); + setRestartedSuperstep(checkpoint); + setJobState(ApplicationState.START_SUPERSTEP, + getApplicationAttempt(), + checkpoint); + } + + /** + * Only get the finalized checkpoint files + */ + public static class FinalizedCheckpointPathFilter implements PathFilter { @Override - public long getLastGoodCheckpoint() throws IOException { - // Find the last good checkpoint if none have been written to the - // knowledge of this master - if (lastCheckpointedSuperstep == -1) { - FileStatus[] fileStatusArray = - getFs().listStatus(new Path(CHECKPOINT_BASE_PATH), - new FinalizedCheckpointPathFilter()); - if (fileStatusArray == null) { - return -1; - } - Arrays.sort(fileStatusArray); - lastCheckpointedSuperstep = getCheckpoint( - fileStatusArray[fileStatusArray.length - 1].getPath()); - if (LOG.isInfoEnabled()) { - LOG.info("getLastGoodCheckpoint: Found last good checkpoint " + - lastCheckpointedSuperstep + " from " + - fileStatusArray[fileStatusArray.length - 1]. - getPath().toString()); - } - } - return lastCheckpointedSuperstep; - } - - /** - * Wait for a set of workers to signal that they are done with the - * barrier. - * - * @param finishedWorkerPath Path to where the workers will register their - * hostname and id - * @param workerInfoList List of the workers to wait for - * @return True if barrier was successful, false if there was a worker - * failure - */ - private boolean barrierOnWorkerList(String finishedWorkerPath, - List workerInfoList, - BspEvent event) { - try { - getZkExt().createOnceExt(finishedWorkerPath, - null, - Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT, - true); - } catch (KeeperException e) { - throw new IllegalStateException( - "barrierOnWorkerList: KeeperException - Couldn't create " + - finishedWorkerPath, e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "barrierOnWorkerList: InterruptedException - Couldn't create " + - finishedWorkerPath, e); - } - List hostnameIdList = - new ArrayList(workerInfoList.size()); - for (WorkerInfo workerInfo : workerInfoList) { - hostnameIdList.add(workerInfo.getHostnameId()); - } - String workerInfoHealthyPath = - getWorkerInfoHealthyPath(getApplicationAttempt(), getSuperstep()); - List finishedHostnameIdList; - long nextInfoMillis = System.currentTimeMillis(); - while (true) { - try { - finishedHostnameIdList = - getZkExt().getChildrenExt(finishedWorkerPath, - true, - false, - false); - } catch (KeeperException e) { - throw new IllegalStateException( - "barrierOnWorkerList: KeeperException - Couldn't get " + - "children of " + finishedWorkerPath, e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "barrierOnWorkerList: IllegalException - Couldn't get " + - "children of " + finishedWorkerPath, e); - } - if (LOG.isDebugEnabled()) { - LOG.debug("barrierOnWorkerList: Got finished worker list = " + - finishedHostnameIdList + ", size = " + - finishedHostnameIdList.size() + - ", worker list = " + - workerInfoList + ", size = " + - workerInfoList.size() + - " from " + finishedWorkerPath); - } - - if (LOG.isInfoEnabled() && - (System.currentTimeMillis() > nextInfoMillis)) { - nextInfoMillis = System.currentTimeMillis() + 30000; - LOG.info("barrierOnWorkerList: " + - finishedHostnameIdList.size() + - " out of " + workerInfoList.size() + - " workers finished on superstep " + - getSuperstep() + " on path " + finishedWorkerPath); - } - getContext().setStatus(getGraphMapper().getMapFunctions() + " - " + - finishedHostnameIdList.size() + - " finished out of " + - workerInfoList.size() + - " on superstep " + getSuperstep()); - if (finishedHostnameIdList.containsAll(hostnameIdList)) { - break; - } - - // Wait for a signal or no more than 60 seconds to progress - // or else will continue. - event.waitMsecs(60*1000); - event.reset(); - getContext().progress(); - - // Did a worker die? - try { - if ((getSuperstep() > 0) && - !superstepChosenWorkerAlive( - workerInfoHealthyPath, - workerInfoList)) { - return false; - } - } catch (KeeperException e) { - throw new IllegalStateException( - "barrierOnWorkerList: KeeperException - " + - "Couldn't get " + workerInfoHealthyPath, e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "barrierOnWorkerList: InterruptedException - " + - "Couldn't get " + workerInfoHealthyPath, e); - } - } - + public boolean accept(Path path) { + if (path.getName().endsWith( + BspService.CHECKPOINT_FINALIZED_POSTFIX)) { return true; + } + return false; + } + } + + @Override + public long getLastGoodCheckpoint() throws IOException { + // Find the last good checkpoint if none have been written to the + // knowledge of this master + if (lastCheckpointedSuperstep == -1) { + FileStatus[] fileStatusArray = + getFs().listStatus(new Path(checkpointBasePath), + new FinalizedCheckpointPathFilter()); + if (fileStatusArray == null) { + return -1; + } + Arrays.sort(fileStatusArray); + lastCheckpointedSuperstep = getCheckpoint( + fileStatusArray[fileStatusArray.length - 1].getPath()); + if (LOG.isInfoEnabled()) { + LOG.info("getLastGoodCheckpoint: Found last good checkpoint " + + lastCheckpointedSuperstep + " from " + + fileStatusArray[fileStatusArray.length - 1]. + getPath().toString()); + } + } + return lastCheckpointedSuperstep; + } + + /** + * Wait for a set of workers to signal that they are done with the + * barrier. + * + * @param finishedWorkerPath Path to where the workers will register their + * hostname and id + * @param workerInfoList List of the workers to wait for + * @param event Event to wait on for a chance to be done. + * @return True if barrier was successful, false if there was a worker + * failure + */ + private boolean barrierOnWorkerList(String finishedWorkerPath, + List workerInfoList, + BspEvent event) { + try { + getZkExt().createOnceExt(finishedWorkerPath, + null, + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT, + true); + } catch (KeeperException e) { + throw new IllegalStateException( + "barrierOnWorkerList: KeeperException - Couldn't create " + + finishedWorkerPath, e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "barrierOnWorkerList: InterruptedException - Couldn't create " + + finishedWorkerPath, e); + } + List hostnameIdList = + new ArrayList(workerInfoList.size()); + for (WorkerInfo workerInfo : workerInfoList) { + hostnameIdList.add(workerInfo.getHostnameId()); + } + String workerInfoHealthyPath = + getWorkerInfoHealthyPath(getApplicationAttempt(), getSuperstep()); + List finishedHostnameIdList; + long nextInfoMillis = System.currentTimeMillis(); + while (true) { + try { + finishedHostnameIdList = + getZkExt().getChildrenExt(finishedWorkerPath, + true, + false, + false); + } catch (KeeperException e) { + throw new IllegalStateException( + "barrierOnWorkerList: KeeperException - Couldn't get " + + "children of " + finishedWorkerPath, e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "barrierOnWorkerList: IllegalException - Couldn't get " + + "children of " + finishedWorkerPath, e); + } + if (LOG.isDebugEnabled()) { + LOG.debug("barrierOnWorkerList: Got finished worker list = " + + finishedHostnameIdList + ", size = " + + finishedHostnameIdList.size() + + ", worker list = " + + workerInfoList + ", size = " + + workerInfoList.size() + + " from " + finishedWorkerPath); + } + + if (LOG.isInfoEnabled() && + (System.currentTimeMillis() > nextInfoMillis)) { + nextInfoMillis = System.currentTimeMillis() + 30000; + LOG.info("barrierOnWorkerList: " + + finishedHostnameIdList.size() + + " out of " + workerInfoList.size() + + " workers finished on superstep " + + getSuperstep() + " on path " + finishedWorkerPath); + } + getContext().setStatus(getGraphMapper().getMapFunctions() + " - " + + finishedHostnameIdList.size() + + " finished out of " + + workerInfoList.size() + + " on superstep " + getSuperstep()); + if (finishedHostnameIdList.containsAll(hostnameIdList)) { + break; + } + + // Wait for a signal or no more than 60 seconds to progress + // or else will continue. + event.waitMsecs(60 * 1000); + event.reset(); + getContext().progress(); + + // Did a worker die? + try { + if ((getSuperstep() > 0) && + !superstepChosenWorkerAlive( + workerInfoHealthyPath, + workerInfoList)) { + return false; + } + } catch (KeeperException e) { + throw new IllegalStateException( + "barrierOnWorkerList: KeeperException - " + + "Couldn't get " + workerInfoHealthyPath, e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "barrierOnWorkerList: InterruptedException - " + + "Couldn't get " + workerInfoHealthyPath, e); + } } - - @Override - public SuperstepState coordinateSuperstep() throws - KeeperException, InterruptedException { - // 1. Get chosen workers and set up watches on them. - // 2. Assign partitions to the workers - // (possibly reloading from a superstep) - // 3. Wait for all workers to complete - // 4. Collect and process aggregators - // 5. Create superstep finished node - // 6. If the checkpoint frequency is met, finalize the checkpoint - List chosenWorkerInfoList = checkWorkers(); - if (chosenWorkerInfoList == null) { - LOG.fatal("coordinateSuperstep: Not enough healthy workers for " + - "superstep " + getSuperstep()); - setJobState(ApplicationState.FAILED, -1, -1); - } else { - for (WorkerInfo workerInfo : chosenWorkerInfoList) { - String workerInfoHealthyPath = - getWorkerInfoHealthyPath(getApplicationAttempt(), - getSuperstep()) + "/" + - workerInfo.getHostnameId(); - if (getZkExt().exists(workerInfoHealthyPath, true) == null) { - LOG.warn("coordinateSuperstep: Chosen worker " + - workerInfoHealthyPath + - " is no longer valid, failing superstep"); - } - } - } - - currentWorkersCounter.increment(chosenWorkerInfoList.size() - - currentWorkersCounter.getValue()); - assignPartitionOwners(allPartitionStatsList, - chosenWorkerInfoList, - masterGraphPartitioner); - - if (getSuperstep() == INPUT_SUPERSTEP) { - // Coordinate the workers finishing sending their vertices to the - // correct workers and signal when everything is done. - if (!barrierOnWorkerList(INPUT_SPLIT_DONE_PATH, - chosenWorkerInfoList, - getInputSplitsDoneStateChangedEvent())) { - throw new IllegalStateException( - "coordinateSuperstep: Worker failed during input split " + - "(currently not supported)"); - } - try { - getZkExt().create(INPUT_SPLITS_ALL_DONE_PATH, - null, - Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT); - } catch (KeeperException.NodeExistsException e) { - LOG.info("coordinateInputSplits: Node " + - INPUT_SPLITS_ALL_DONE_PATH + " already exists."); - } catch (KeeperException e) { - throw new IllegalStateException( - "coordinateInputSplits: KeeperException", e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "coordinateInputSplits: IllegalStateException", e); - } - } - - String finishedWorkerPath = - getWorkerFinishedPath(getApplicationAttempt(), getSuperstep()); - if (!barrierOnWorkerList(finishedWorkerPath, - chosenWorkerInfoList, - getSuperstepStateChangedEvent())) { - return SuperstepState.WORKER_FAILURE; - } - - collectAndProcessAggregatorValues(getSuperstep()); - GlobalStats globalStats = aggregateWorkerStats(getSuperstep()); - - // Let everyone know the aggregated application state through the - // superstep finishing znode. - String superstepFinishedNode = - getSuperstepFinishedPath(getApplicationAttempt(), getSuperstep()); - WritableUtils.writeToZnode( - getZkExt(), superstepFinishedNode, -1, globalStats); - vertexCounter.increment( - globalStats.getVertexCount() - - vertexCounter.getValue()); - finishedVertexCounter.increment( - globalStats.getFinishedVertexCount() - - finishedVertexCounter.getValue()); - edgeCounter.increment( - globalStats.getEdgeCount() - - edgeCounter.getValue()); - sentMessagesCounter.increment( - globalStats.getMessageCount() - - sentMessagesCounter.getValue()); - - // Finalize the valid checkpoint file prefixes and possibly - // the aggregators. - if (checkpointFrequencyMet(getSuperstep())) { - try { - finalizeCheckpoint(getSuperstep(), chosenWorkerInfoList); - } catch (IOException e) { - throw new IllegalStateException( - "coordinateSuperstep: IOException on finalizing checkpoint", - e); - } - } - - // Clean up the old supersteps (always keep this one) - long removeableSuperstep = getSuperstep() - 1; - if ((getConfiguration().getBoolean( - GiraphJob.KEEP_ZOOKEEPER_DATA, - GiraphJob.KEEP_ZOOKEEPER_DATA_DEFAULT) == false) && - (removeableSuperstep >= 0)) { - String oldSuperstepPath = - getSuperstepPath(getApplicationAttempt()) + "/" + - (removeableSuperstep); - try { - if (LOG.isInfoEnabled()) { - LOG.info("coordinateSuperstep: Cleaning up old Superstep " + - oldSuperstepPath); - } - getZkExt().deleteExt(oldSuperstepPath, - -1, - true); - } catch (KeeperException.NoNodeException e) { - LOG.warn("coordinateBarrier: Already cleaned up " + - oldSuperstepPath); - } catch (KeeperException e) { - throw new IllegalStateException( - "coordinateSuperstep: KeeperException on " + - "finalizing checkpoint", e); - } - } - incrCachedSuperstep(); - // Counter starts at zero, so no need to increment - if (getSuperstep() > 0) { - superstepCounter.increment(1); - } - SuperstepState superstepState; - if ((globalStats.getFinishedVertexCount() == - globalStats.getVertexCount()) && - globalStats.getMessageCount() == 0) { - superstepState = SuperstepState.ALL_SUPERSTEPS_DONE; - } else { - superstepState = SuperstepState.THIS_SUPERSTEP_DONE; - } - try { - aggregatorWriter.writeAggregator(getAggregatorMap(), - (superstepState == SuperstepState.ALL_SUPERSTEPS_DONE) ? - AggregatorWriter.LAST_SUPERSTEP : getSuperstep()); - } catch (IOException e) { - throw new IllegalStateException( - "coordinateSuperstep: IOException while " + - "writing aggregators data", e); - } - - return superstepState; + return true; + } + + + @Override + public SuperstepState coordinateSuperstep() throws + KeeperException, InterruptedException { + // 1. Get chosen workers and set up watches on them. + // 2. Assign partitions to the workers + // (possibly reloading from a superstep) + // 3. Wait for all workers to complete + // 4. Collect and process aggregators + // 5. Create superstep finished node + // 6. If the checkpoint frequency is met, finalize the checkpoint + List chosenWorkerInfoList = checkWorkers(); + if (chosenWorkerInfoList == null) { + LOG.fatal("coordinateSuperstep: Not enough healthy workers for " + + "superstep " + getSuperstep()); + setJobState(ApplicationState.FAILED, -1, -1); + } else { + for (WorkerInfo workerInfo : chosenWorkerInfoList) { + String workerInfoHealthyPath = + getWorkerInfoHealthyPath(getApplicationAttempt(), + getSuperstep()) + "/" + + workerInfo.getHostnameId(); + if (getZkExt().exists(workerInfoHealthyPath, true) == null) { + LOG.warn("coordinateSuperstep: Chosen worker " + + workerInfoHealthyPath + + " is no longer valid, failing superstep"); + } + } } - /** - * Need to clean up ZooKeeper nicely. Make sure all the masters and workers - * have reported ending their ZooKeeper connections. - */ - private void cleanUpZooKeeper() { - try { - getZkExt().createExt(CLEANED_UP_PATH, - null, - Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT, - true); - } catch (KeeperException.NodeExistsException e) { - if (LOG.isInfoEnabled()) { - LOG.info("cleanUpZooKeeper: Node " + CLEANED_UP_PATH + - " already exists, no need to create."); - } - } catch (KeeperException e) { - throw new IllegalStateException( - "cleanupZooKeeper: Got KeeperException", e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "cleanupZooKeeper: Got IllegalStateException", e); - } - // Need to wait for the number of workers and masters to complete - int maxTasks = BspInputFormat.getMaxTasks(getConfiguration()); - if ((getGraphMapper().getMapFunctions() == MapFunctions.ALL) || - (getGraphMapper().getMapFunctions() == - MapFunctions.ALL_EXCEPT_ZOOKEEPER)) { - maxTasks *= 2; - } - List cleanedUpChildrenList = null; - while (true) { - try { - cleanedUpChildrenList = - getZkExt().getChildrenExt( - CLEANED_UP_PATH, true, false, true); - if (LOG.isInfoEnabled()) { - LOG.info("cleanUpZooKeeper: Got " + - cleanedUpChildrenList.size() + " of " + - maxTasks + " desired children from " + - CLEANED_UP_PATH); - } - if (cleanedUpChildrenList.size() == maxTasks) { - break; - } - if (LOG.isInfoEnabled()) { - LOG.info("cleanedUpZooKeeper: Waiting for the " + - "children of " + CLEANED_UP_PATH + - " to change since only got " + - cleanedUpChildrenList.size() + " nodes."); - } - } - catch (Exception e) { - // We are in the cleanup phase -- just log the error - LOG.error("cleanUpZooKeeper: Got exception, but will continue", - e); - return; - } + currentWorkersCounter.increment(chosenWorkerInfoList.size() - + currentWorkersCounter.getValue()); + assignPartitionOwners(allPartitionStatsList, + chosenWorkerInfoList, + masterGraphPartitioner); + + if (getSuperstep() == INPUT_SUPERSTEP) { + // Coordinate the workers finishing sending their vertices to the + // correct workers and signal when everything is done. + if (!barrierOnWorkerList(inputSplitsDonePath, + chosenWorkerInfoList, + getInputSplitsDoneStateChangedEvent())) { + throw new IllegalStateException( + "coordinateSuperstep: Worker failed during input split " + + "(currently not supported)"); + } + try { + getZkExt().create(inputSplitsAllDonePath, + null, + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + } catch (KeeperException.NodeExistsException e) { + LOG.info("coordinateInputSplits: Node " + + inputSplitsAllDonePath + " already exists."); + } catch (KeeperException e) { + throw new IllegalStateException( + "coordinateInputSplits: KeeperException", e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "coordinateInputSplits: IllegalStateException", e); + } + } - getCleanedUpChildrenChangedEvent().waitForever(); - getCleanedUpChildrenChangedEvent().reset(); - } + String finishedWorkerPath = + getWorkerFinishedPath(getApplicationAttempt(), getSuperstep()); + if (!barrierOnWorkerList(finishedWorkerPath, + chosenWorkerInfoList, + getSuperstepStateChangedEvent())) { + return SuperstepState.WORKER_FAILURE; + } - // At this point, all processes have acknowledged the cleanup, - // and the master can do any final cleanup - try { - if (getConfiguration().getBoolean( - GiraphJob.KEEP_ZOOKEEPER_DATA, - GiraphJob.KEEP_ZOOKEEPER_DATA_DEFAULT) == false) { - if (LOG.isInfoEnabled()) { - LOG.info("cleanupZooKeeper: Removing the following path " + - "and all children - " + BASE_PATH); - } - getZkExt().deleteExt(BASE_PATH, -1, true); - } - } catch (Exception e) { - LOG.error("cleanupZooKeeper: Failed to do cleanup of " + - BASE_PATH, e); - } + collectAndProcessAggregatorValues(getSuperstep()); + GlobalStats globalStats = aggregateWorkerStats(getSuperstep()); + + // Let everyone know the aggregated application state through the + // superstep finishing znode. + String superstepFinishedNode = + getSuperstepFinishedPath(getApplicationAttempt(), getSuperstep()); + WritableUtils.writeToZnode( + getZkExt(), superstepFinishedNode, -1, globalStats); + vertexCounter.increment( + globalStats.getVertexCount() - + vertexCounter.getValue()); + finishedVertexCounter.increment( + globalStats.getFinishedVertexCount() - + finishedVertexCounter.getValue()); + edgeCounter.increment( + globalStats.getEdgeCount() - + edgeCounter.getValue()); + sentMessagesCounter.increment( + globalStats.getMessageCount() - + sentMessagesCounter.getValue()); + + // Finalize the valid checkpoint file prefixes and possibly + // the aggregators. + if (checkpointFrequencyMet(getSuperstep())) { + try { + finalizeCheckpoint(getSuperstep(), chosenWorkerInfoList); + } catch (IOException e) { + throw new IllegalStateException( + "coordinateSuperstep: IOException on finalizing checkpoint", + e); + } } - @Override - public void cleanup() throws IOException { - // All master processes should denote they are done by adding special - // znode. Once the number of znodes equals the number of partitions - // for workers and masters, the master will clean up the ZooKeeper - // znodes associated with this job. - String cleanedUpPath = CLEANED_UP_PATH + "/" + - getTaskPartition() + MASTER_SUFFIX; - try { - String finalFinishedPath = - getZkExt().createExt(cleanedUpPath, - null, - Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT, - true); - if (LOG.isInfoEnabled()) { - LOG.info("cleanup: Notifying master its okay to cleanup with " + - finalFinishedPath); - } - } catch (KeeperException.NodeExistsException e) { - if (LOG.isInfoEnabled()) { - LOG.info("cleanup: Couldn't create finished node '" + - cleanedUpPath); - } - } catch (KeeperException e) { - LOG.error("cleanup: Got KeeperException, continuing", e); - } catch (InterruptedException e) { - LOG.error("cleanup: Got InterruptedException, continuing", e); + // Clean up the old supersteps (always keep this one) + long removeableSuperstep = getSuperstep() - 1; + if (!(getConfiguration().getBoolean( + GiraphJob.KEEP_ZOOKEEPER_DATA, + GiraphJob.KEEP_ZOOKEEPER_DATA_DEFAULT)) && + (removeableSuperstep >= 0)) { + String oldSuperstepPath = + getSuperstepPath(getApplicationAttempt()) + "/" + + removeableSuperstep; + try { + if (LOG.isInfoEnabled()) { + LOG.info("coordinateSuperstep: Cleaning up old Superstep " + + oldSuperstepPath); } + getZkExt().deleteExt(oldSuperstepPath, + -1, + true); + } catch (KeeperException.NoNodeException e) { + LOG.warn("coordinateBarrier: Already cleaned up " + + oldSuperstepPath); + } catch (KeeperException e) { + throw new IllegalStateException( + "coordinateSuperstep: KeeperException on " + + "finalizing checkpoint", e); + } + } + incrCachedSuperstep(); + // Counter starts at zero, so no need to increment + if (getSuperstep() > 0) { + superstepCounter.increment(1); + } + SuperstepState superstepState; + if ((globalStats.getFinishedVertexCount() == + globalStats.getVertexCount()) && + globalStats.getMessageCount() == 0) { + superstepState = SuperstepState.ALL_SUPERSTEPS_DONE; + } else { + superstepState = SuperstepState.THIS_SUPERSTEP_DONE; + } + try { + aggregatorWriter.writeAggregator(getAggregatorMap(), + (superstepState == SuperstepState.ALL_SUPERSTEPS_DONE) ? + AggregatorWriter.LAST_SUPERSTEP : getSuperstep()); + } catch (IOException e) { + throw new IllegalStateException( + "coordinateSuperstep: IOException while " + + "writing aggregators data", e); + } - if (isMaster) { - cleanUpZooKeeper(); - // If desired, cleanup the checkpoint directory - if (getConfiguration().getBoolean( - GiraphJob.CLEANUP_CHECKPOINTS_AFTER_SUCCESS, - GiraphJob.CLEANUP_CHECKPOINTS_AFTER_SUCCESS_DEFAULT)) { - boolean success = - getFs().delete(new Path(CHECKPOINT_BASE_PATH), true); - if (LOG.isInfoEnabled()) { - LOG.info("cleanup: Removed HDFS checkpoint directory (" + - CHECKPOINT_BASE_PATH + ") with return = " + - success + " since this job succeeded "); - } - } - aggregatorWriter.close(); + return superstepState; + } + + /** + * Need to clean up ZooKeeper nicely. Make sure all the masters and workers + * have reported ending their ZooKeeper connections. + */ + private void cleanUpZooKeeper() { + try { + getZkExt().createExt(cleanedUpPath, + null, + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT, + true); + } catch (KeeperException.NodeExistsException e) { + if (LOG.isInfoEnabled()) { + LOG.info("cleanUpZooKeeper: Node " + cleanedUpPath + + " already exists, no need to create."); + } + } catch (KeeperException e) { + throw new IllegalStateException( + "cleanupZooKeeper: Got KeeperException", e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "cleanupZooKeeper: Got IllegalStateException", e); + } + // Need to wait for the number of workers and masters to complete + int maxTasks = BspInputFormat.getMaxTasks(getConfiguration()); + if ((getGraphMapper().getMapFunctions() == MapFunctions.ALL) || + (getGraphMapper().getMapFunctions() == + MapFunctions.ALL_EXCEPT_ZOOKEEPER)) { + maxTasks *= 2; + } + List cleanedUpChildrenList = null; + while (true) { + try { + cleanedUpChildrenList = + getZkExt().getChildrenExt( + cleanedUpPath, true, false, true); + if (LOG.isInfoEnabled()) { + LOG.info("cleanUpZooKeeper: Got " + + cleanedUpChildrenList.size() + " of " + + maxTasks + " desired children from " + + cleanedUpPath); } - - try { - getZkExt().close(); - } catch (InterruptedException e) { - // cleanup phase -- just log the error - LOG.error("cleanup: Zookeeper failed to close", e); + if (cleanedUpChildrenList.size() == maxTasks) { + break; } + if (LOG.isInfoEnabled()) { + LOG.info("cleanedUpZooKeeper: Waiting for the " + + "children of " + cleanedUpPath + + " to change since only got " + + cleanedUpChildrenList.size() + " nodes."); + } + } catch (KeeperException e) { + // We are in the cleanup phase -- just log the error + LOG.error("cleanUpZooKeeper: Got KeeperException, " + + "but will continue", e); + return; + } catch (InterruptedException e) { + // We are in the cleanup phase -- just log the error + LOG.error("cleanUpZooKeeper: Got InterruptedException, " + + "but will continue", e); + return; + } + + getCleanedUpChildrenChangedEvent().waitForever(); + getCleanedUpChildrenChangedEvent().reset(); } - /** - * Event that the master watches that denotes if a worker has done something - * that changes the state of a superstep (either a worker completed or died) - * - * @return Event that denotes a superstep state change - */ - final public BspEvent getSuperstepStateChangedEvent() { - return superstepStateChanged; - } - - /** - * Should this worker failure cause the current superstep to fail? - * - * @param failedWorkerPath Full path to the failed worker - */ - final private void checkHealthyWorkerFailure(String failedWorkerPath) { - if (getSuperstepFromPath(failedWorkerPath) < getSuperstep()) { - return; - } + // At this point, all processes have acknowledged the cleanup, + // and the master can do any final cleanup + try { + if (!getConfiguration().getBoolean( + GiraphJob.KEEP_ZOOKEEPER_DATA, + GiraphJob.KEEP_ZOOKEEPER_DATA_DEFAULT)) { + if (LOG.isInfoEnabled()) { + LOG.info("cleanupZooKeeper: Removing the following path " + + "and all children - " + basePath); + } + getZkExt().deleteExt(basePath, -1, true); + } + } catch (KeeperException e) { + LOG.error("cleanupZooKeeper: Failed to do cleanup of " + + basePath + " due to KeeperException", e); + } catch (InterruptedException e) { + LOG.error("cleanupZooKeeper: Failed to do cleanup of " + + basePath + " due to InterruptedException", e); + } + } + + @Override + public void cleanup() throws IOException { + // All master processes should denote they are done by adding special + // znode. Once the number of znodes equals the number of partitions + // for workers and masters, the master will clean up the ZooKeeper + // znodes associated with this job. + String masterCleanedUpPath = cleanedUpPath + "/" + + getTaskPartition() + MASTER_SUFFIX; + try { + String finalFinishedPath = + getZkExt().createExt(masterCleanedUpPath, + null, + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT, + true); + if (LOG.isInfoEnabled()) { + LOG.info("cleanup: Notifying master its okay to cleanup with " + + finalFinishedPath); + } + } catch (KeeperException.NodeExistsException e) { + if (LOG.isInfoEnabled()) { + LOG.info("cleanup: Couldn't create finished node '" + + masterCleanedUpPath); + } + } catch (KeeperException e) { + LOG.error("cleanup: Got KeeperException, continuing", e); + } catch (InterruptedException e) { + LOG.error("cleanup: Got InterruptedException, continuing", e); + } - Collection partitionOwners = - masterGraphPartitioner.getCurrentPartitionOwners(); - String hostnameId = - getHealthyHostnameIdFromPath(failedWorkerPath); - for (PartitionOwner partitionOwner : partitionOwners) { - WorkerInfo workerInfo = partitionOwner.getWorkerInfo(); - WorkerInfo previousWorkerInfo = - partitionOwner.getPreviousWorkerInfo(); - if (workerInfo.getHostnameId().equals(hostnameId) || - ((previousWorkerInfo != null) && - previousWorkerInfo.getHostnameId().equals(hostnameId))) { - LOG.warn("checkHealthyWorkerFailure: " + - "at least one healthy worker went down " + - "for superstep " + getSuperstep() + " - " + - hostnameId + ", will try to restart from " + - "checkpointed superstep " + - lastCheckpointedSuperstep); - superstepStateChanged.signal(); - } + if (isMaster) { + cleanUpZooKeeper(); + // If desired, cleanup the checkpoint directory + if (getConfiguration().getBoolean( + GiraphJob.CLEANUP_CHECKPOINTS_AFTER_SUCCESS, + GiraphJob.CLEANUP_CHECKPOINTS_AFTER_SUCCESS_DEFAULT)) { + boolean success = + getFs().delete(new Path(checkpointBasePath), true); + if (LOG.isInfoEnabled()) { + LOG.info("cleanup: Removed HDFS checkpoint directory (" + + checkpointBasePath + ") with return = " + + success + " since this job succeeded "); } + } + aggregatorWriter.close(); } - @Override - public boolean processEvent(WatchedEvent event) { - boolean foundEvent = false; - if (event.getPath().contains(WORKER_HEALTHY_DIR) && - (event.getType() == EventType.NodeDeleted)) { - if (LOG.isDebugEnabled()) { - LOG.debug("processEvent: Healthy worker died (node deleted) " + - "in " + event.getPath()); - } - checkHealthyWorkerFailure(event.getPath()); - superstepStateChanged.signal(); - foundEvent = true; - } else if (event.getPath().contains(WORKER_FINISHED_DIR) && - event.getType() == EventType.NodeChildrenChanged) { - if (LOG.isDebugEnabled()) { - LOG.debug("processEvent: Worker finished (node change) " + - "event - superstepStateChanged signaled"); - } - superstepStateChanged.signal(); - foundEvent = true; - } + try { + getZkExt().close(); + } catch (InterruptedException e) { + // cleanup phase -- just log the error + LOG.error("cleanup: Zookeeper failed to close", e); + } + } + + /** + * Event that the master watches that denotes if a worker has done something + * that changes the state of a superstep (either a worker completed or died) + * + * @return Event that denotes a superstep state change + */ + public final BspEvent getSuperstepStateChangedEvent() { + return superstepStateChanged; + } + + /** + * Should this worker failure cause the current superstep to fail? + * + * @param failedWorkerPath Full path to the failed worker + */ + private void checkHealthyWorkerFailure(String failedWorkerPath) { + if (getSuperstepFromPath(failedWorkerPath) < getSuperstep()) { + return; + } - return foundEvent; + Collection partitionOwners = + masterGraphPartitioner.getCurrentPartitionOwners(); + String hostnameId = + getHealthyHostnameIdFromPath(failedWorkerPath); + for (PartitionOwner partitionOwner : partitionOwners) { + WorkerInfo workerInfo = partitionOwner.getWorkerInfo(); + WorkerInfo previousWorkerInfo = + partitionOwner.getPreviousWorkerInfo(); + if (workerInfo.getHostnameId().equals(hostnameId) || + ((previousWorkerInfo != null) && + previousWorkerInfo.getHostnameId().equals(hostnameId))) { + LOG.warn("checkHealthyWorkerFailure: " + + "at least one healthy worker went down " + + "for superstep " + getSuperstep() + " - " + + hostnameId + ", will try to restart from " + + "checkpointed superstep " + + lastCheckpointedSuperstep); + superstepStateChanged.signal(); + } + } + } + + @Override + public boolean processEvent(WatchedEvent event) { + boolean foundEvent = false; + if (event.getPath().contains(WORKER_HEALTHY_DIR) && + (event.getType() == EventType.NodeDeleted)) { + if (LOG.isDebugEnabled()) { + LOG.debug("processEvent: Healthy worker died (node deleted) " + + "in " + event.getPath()); + } + checkHealthyWorkerFailure(event.getPath()); + superstepStateChanged.signal(); + foundEvent = true; + } else if (event.getPath().contains(WORKER_FINISHED_DIR) && + event.getType() == EventType.NodeChildrenChanged) { + if (LOG.isDebugEnabled()) { + LOG.debug("processEvent: Worker finished (node change) " + + "event - superstepStateChanged signaled"); + } + superstepStateChanged.signal(); + foundEvent = true; } + + return foundEvent; + } } diff --git a/src/main/java/org/apache/giraph/graph/BspServiceWorker.java b/src/main/java/org/apache/giraph/graph/BspServiceWorker.java index 27758194c..60ffd051c 100644 --- a/src/main/java/org/apache/giraph/graph/BspServiceWorker.java +++ b/src/main/java/org/apache/giraph/graph/BspServiceWorker.java @@ -59,7 +59,6 @@ import java.io.DataOutputStream; import java.io.IOException; import java.io.InputStream; -import java.net.UnknownHostException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -73,1412 +72,1470 @@ /** * ZooKeeper-based implementation of {@link CentralizedServiceWorker}. + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data */ @SuppressWarnings("rawtypes") -public class BspServiceWorker< - I extends WritableComparable, - V extends Writable, - E extends Writable, - M extends Writable> - extends BspService - implements CentralizedServiceWorker { - /** Number of input splits */ - private int inputSplitCount = -1; - /** My process health znode */ - private String myHealthZnode; - /** List of aggregators currently in use */ - private Set aggregatorInUse = new TreeSet(); - /** Worker info */ - private final WorkerInfo workerInfo; - /** Worker graph partitioner */ - private final WorkerGraphPartitioner workerGraphPartitioner; - /** Input split vertex cache (only used when loading from input split) */ - private final Map> - inputSplitCache = new HashMap>(); - /** Communication service */ - private final ServerInterface commService; - /** Structure to store the partitions on this worker */ - private final Map> workerPartitionMap = - new HashMap>(); - /** Have the partition exchange children (workers) changed? */ - private final BspEvent partitionExchangeChildrenChanged = - new PredicateLock(); - /** Max vertices per partition before sending */ - private final int maxVerticesPerPartition; - /** Worker Context */ - private final WorkerContext workerContext; - /** Total vertices loaded */ - private long totalVerticesLoaded = 0; - /** Total edges loaded */ - private long totalEdgesLoaded = 0; - /** Input split max vertices (-1 denotes all) */ - private final long inputSplitMaxVertices; - /** Class logger */ - private static final Logger LOG = Logger.getLogger(BspServiceWorker.class); - - public BspServiceWorker( - String serverPortList, - int sessionMsecTimeout, - Mapper.Context context, - GraphMapper graphMapper, - GraphState graphState) - throws UnknownHostException, IOException, InterruptedException { - super(serverPortList, sessionMsecTimeout, context, graphMapper); - registerBspEvent(partitionExchangeChildrenChanged); - int finalRpcPort = - getConfiguration().getInt(GiraphJob.RPC_INITIAL_PORT, - GiraphJob.RPC_INITIAL_PORT_DEFAULT) + - getTaskPartition(); - maxVerticesPerPartition = - getConfiguration().getInt( - GiraphJob.MAX_VERTICES_PER_PARTITION, - GiraphJob.MAX_VERTICES_PER_PARTITION_DEFAULT); - inputSplitMaxVertices = - getConfiguration().getLong( - GiraphJob.INPUT_SPLIT_MAX_VERTICES, - GiraphJob.INPUT_SPLIT_MAX_VERTICES_DEFAULT); - workerInfo = - new WorkerInfo(getHostname(), getTaskPartition(), finalRpcPort); - workerGraphPartitioner = - getGraphPartitionerFactory().createWorkerGraphPartitioner(); - commService = new RPCCommunications( - context, this, graphState); - graphState.setWorkerCommunications(commService); - this.workerContext = - BspUtils.createWorkerContext(getConfiguration(), - graphMapper.getGraphState()); - } - - public WorkerContext getWorkerContext() { - return workerContext; +public class BspServiceWorker + extends BspService + implements CentralizedServiceWorker { + /** Class logger */ + private static final Logger LOG = Logger.getLogger(BspServiceWorker.class); + /** Number of input splits */ + private int inputSplitCount = -1; + /** My process health znode */ + private String myHealthZnode; + /** List of aggregators currently in use */ + private Set aggregatorInUse = new TreeSet(); + /** Worker info */ + private final WorkerInfo workerInfo; + /** Worker graph partitioner */ + private final WorkerGraphPartitioner workerGraphPartitioner; + /** Input split vertex cache (only used when loading from input split) */ + private final Map> + inputSplitCache = new HashMap>(); + /** Communication service */ + private final ServerInterface commService; + /** Structure to store the partitions on this worker */ + private final Map> workerPartitionMap = + new HashMap>(); + /** Have the partition exchange children (workers) changed? */ + private final BspEvent partitionExchangeChildrenChanged = + new PredicateLock(); + /** Max vertices per partition before sending */ + private final int maxVerticesPerPartition; + /** Worker Context */ + private final WorkerContext workerContext; + /** Total vertices loaded */ + private long totalVerticesLoaded = 0; + /** Total edges loaded */ + private long totalEdgesLoaded = 0; + /** Input split max vertices (-1 denotes all) */ + private final long inputSplitMaxVertices; + + /** + * Constructor for setting up the worker. + * + * @param serverPortList ZooKeeper server port list + * @param sessionMsecTimeout Msecs to timeout connecting to ZooKeeper + * @param context Mapper context + * @param graphMapper Graph mapper + * @param graphState Global graph state + * @throws UnknownHostException + * @throws IOException + * @throws InterruptedException + */ + public BspServiceWorker( + String serverPortList, + int sessionMsecTimeout, + Mapper.Context context, + GraphMapper graphMapper, + GraphState graphState) + throws IOException, InterruptedException { + super(serverPortList, sessionMsecTimeout, context, graphMapper); + registerBspEvent(partitionExchangeChildrenChanged); + int finalRpcPort = + getConfiguration().getInt(GiraphJob.RPC_INITIAL_PORT, + GiraphJob.RPC_INITIAL_PORT_DEFAULT) + + getTaskPartition(); + maxVerticesPerPartition = + getConfiguration().getInt( + GiraphJob.MAX_VERTICES_PER_PARTITION, + GiraphJob.MAX_VERTICES_PER_PARTITION_DEFAULT); + inputSplitMaxVertices = + getConfiguration().getLong( + GiraphJob.INPUT_SPLIT_MAX_VERTICES, + GiraphJob.INPUT_SPLIT_MAX_VERTICES_DEFAULT); + workerInfo = + new WorkerInfo(getHostname(), getTaskPartition(), finalRpcPort); + workerGraphPartitioner = + getGraphPartitionerFactory().createWorkerGraphPartitioner(); + commService = new RPCCommunications( + context, this, graphState); + graphState.setWorkerCommunications(commService); + this.workerContext = + BspUtils.createWorkerContext(getConfiguration(), + graphMapper.getGraphState()); + } + + public WorkerContext getWorkerContext() { + return workerContext; + } + + /** + * Intended to check the health of the node. For instance, can it ssh, + * dmesg, etc. For now, does nothing. + * TODO: Make this check configurable by the user (i.e. search dmesg for + * problems). + * + * @return True if healthy (always in this case). + */ + public boolean isHealthy() { + return true; + } + + /** + * Use an aggregator in this superstep. + * + * @param name Name of aggregator (should be unique) + * @return boolean (false when aggregator not registered) + */ + public boolean useAggregator(String name) { + if (getAggregatorMap().get(name) == null) { + LOG.error("userAggregator: Aggregator=" + name + " not registered"); + return false; } - - /** - * Intended to check the health of the node. For instance, can it ssh, - * dmesg, etc. For now, does nothing. - */ - public boolean isHealthy() { - return true; + aggregatorInUse.add(name); + return true; + } + + /** + * Try to reserve an InputSplit for loading. While InputSplits exists that + * are not finished, wait until they are. + * + * @return reserved InputSplit or null if no unfinished InputSplits exist + * @throws KeeperException + * @throws InterruptedException + */ + private String reserveInputSplit() + throws KeeperException, InterruptedException { + List inputSplitPathList = null; + inputSplitPathList = + getZkExt().getChildrenExt(inputSplitsPath, false, false, true); + if (inputSplitCount == -1) { + inputSplitCount = inputSplitPathList.size(); } - /** - * Use an aggregator in this superstep. - * - * @param name - * @return boolean (false when aggregator not registered) - */ - public boolean useAggregator(String name) { - if (getAggregatorMap().get(name) == null) { - LOG.error("userAggregator: Aggregator=" + name + " not registered"); - return false; + String reservedInputSplitPath = null; + Stat reservedStat = null; + while (true) { + int finishedInputSplits = 0; + for (int i = 0; i < inputSplitPathList.size(); ++i) { + String tmpInputSplitFinishedPath = + inputSplitPathList.get(i) + INPUT_SPLIT_FINISHED_NODE; + reservedStat = + getZkExt().exists(tmpInputSplitFinishedPath, true); + if (reservedStat != null) { + ++finishedInputSplits; + continue; } - aggregatorInUse.add(name); - return true; - } - /** - * Try to reserve an InputSplit for loading. While InputSplits exists that - * are not finished, wait until they are. - * - * @return reserved InputSplit or null if no unfinished InputSplits exist - */ - private String reserveInputSplit() { - List inputSplitPathList = null; - try { - inputSplitPathList = - getZkExt().getChildrenExt(INPUT_SPLIT_PATH, false, false, true); - if (inputSplitCount == -1) { - inputSplitCount = inputSplitPathList.size(); - } - } catch (Exception e) { - throw new RuntimeException(e); - } - String reservedInputSplitPath = null; - Stat reservedStat = null; - while (true) { - int finishedInputSplits = 0; - for (int i = 0; i < inputSplitPathList.size(); ++i) { - String tmpInputSplitFinishedPath = - inputSplitPathList.get(i) + INPUT_SPLIT_FINISHED_NODE; - try { - reservedStat = - getZkExt().exists(tmpInputSplitFinishedPath, true); - } catch (Exception e) { - throw new RuntimeException(e); - } - if (reservedStat != null) { - ++finishedInputSplits; - continue; - } - - String tmpInputSplitReservedPath = - inputSplitPathList.get(i) + INPUT_SPLIT_RESERVED_NODE; - try { - reservedStat = - getZkExt().exists(tmpInputSplitReservedPath, true); - } catch (Exception e) { - throw new RuntimeException(e); - } - if (reservedStat == null) { - try { - // Attempt to reserve this InputSplit - getZkExt().createExt(tmpInputSplitReservedPath, - null, - Ids.OPEN_ACL_UNSAFE, - CreateMode.EPHEMERAL, - false); - reservedInputSplitPath = inputSplitPathList.get(i); - if (LOG.isInfoEnabled()) { - float percentFinished = - finishedInputSplits * 100.0f / - inputSplitPathList.size(); - LOG.info("reserveInputSplit: Reserved input " + - "split path " + reservedInputSplitPath + - ", overall roughly " + - + percentFinished + - "% input splits finished"); - } - return reservedInputSplitPath; - } catch (KeeperException.NodeExistsException e) { - LOG.info("reserveInputSplit: Couldn't reserve " + - "(already reserved) inputSplit" + - " at " + tmpInputSplitReservedPath); - } catch (KeeperException e) { - throw new IllegalStateException( - "reserveInputSplit: KeeperException on reserve", e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "reserveInputSplit: InterruptedException " + - "on reserve", e); - } - } - } + String tmpInputSplitReservedPath = + inputSplitPathList.get(i) + INPUT_SPLIT_RESERVED_NODE; + reservedStat = + getZkExt().exists(tmpInputSplitReservedPath, true); + if (reservedStat == null) { + try { + // Attempt to reserve this InputSplit + getZkExt().createExt(tmpInputSplitReservedPath, + null, + Ids.OPEN_ACL_UNSAFE, + CreateMode.EPHEMERAL, + false); + reservedInputSplitPath = inputSplitPathList.get(i); if (LOG.isInfoEnabled()) { - LOG.info("reserveInputSplit: reservedPath = " + - reservedInputSplitPath + ", " + finishedInputSplits + - " of " + inputSplitPathList.size() + - " InputSplits are finished."); - } - if (finishedInputSplits == inputSplitPathList.size()) { - return null; - } - // Wait for either a reservation to go away or a notification that - // an InputSplit has finished. - getInputSplitsStateChangedEvent().waitMsecs(60*1000); - getInputSplitsStateChangedEvent().reset(); - } - } - - - - /** - * Load the vertices from the user-defined VertexReader into our partitions - * of vertex ranges. Do this until all the InputSplits have been processed. - * All workers will try to do as many InputSplits as they can. The master - * will monitor progress and stop this once all the InputSplits have been - * loaded and check-pointed. Keep track of the last input split path to - * ensure the input split cache is flushed prior to marking the last input - * split complete. - * - * @throws IOException - * @throws IllegalAccessException - * @throws InstantiationException - * @throws ClassNotFoundException - * @throws InterruptedException - */ - private VertexEdgeCount loadVertices() throws IOException, - ClassNotFoundException, - InterruptedException, InstantiationException, - IllegalAccessException { - String inputSplitPath = null; - VertexEdgeCount vertexEdgeCount = new VertexEdgeCount(); - while ((inputSplitPath = reserveInputSplit()) != null) { - vertexEdgeCount = vertexEdgeCount.incrVertexEdgeCount( - loadVerticesFromInputSplit(inputSplitPath)); - } - - // Flush the remaining cached vertices - for (Entry> entry : - inputSplitCache.entrySet()) { - if (!entry.getValue().getVertices().isEmpty()) { - commService.sendPartitionReq(entry.getKey().getWorkerInfo(), - entry.getValue()); - entry.getValue().getVertices().clear(); + float percentFinished = + finishedInputSplits * 100.0f / + inputSplitPathList.size(); + LOG.info("reserveInputSplit: Reserved input " + + "split path " + reservedInputSplitPath + + ", overall roughly " + + + percentFinished + + "% input splits finished"); } - } - inputSplitCache.clear(); - - return vertexEdgeCount; - } - - /** - * Mark an input split path as completed by this worker. This notifies - * the master and the other workers that this input split has not only - * been reserved, but also marked processed. - * - * @param inputSplitPath Path to the input split. - */ - private void markInputSplitPathFinished(String inputSplitPath) { - String inputSplitFinishedPath = - inputSplitPath + INPUT_SPLIT_FINISHED_NODE; - try { - getZkExt().createExt(inputSplitFinishedPath, - null, - Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT, - true); - } catch (KeeperException.NodeExistsException e) { - LOG.warn("loadVertices: " + inputSplitFinishedPath + - " already exists!"); - } catch (KeeperException e) { + return reservedInputSplitPath; + } catch (KeeperException.NodeExistsException e) { + LOG.info("reserveInputSplit: Couldn't reserve " + + "(already reserved) inputSplit" + + " at " + tmpInputSplitReservedPath); + } catch (KeeperException e) { throw new IllegalStateException( - "loadVertices: KeeperException on " + - inputSplitFinishedPath, e); - } catch (InterruptedException e) { + "reserveInputSplit: KeeperException on reserve", e); + } catch (InterruptedException e) { throw new IllegalStateException( - "loadVertices: InterruptedException on " + - inputSplitFinishedPath, e); + "reserveInputSplit: InterruptedException " + + "on reserve", e); + } } + } + if (LOG.isInfoEnabled()) { + LOG.info("reserveInputSplit: reservedPath = " + + reservedInputSplitPath + ", " + finishedInputSplits + + " of " + inputSplitPathList.size() + + " InputSplits are finished."); + } + if (finishedInputSplits == inputSplitPathList.size()) { + return null; + } + // Wait for either a reservation to go away or a notification that + // an InputSplit has finished. + getInputSplitsStateChangedEvent().waitMsecs(60 * 1000); + getInputSplitsStateChangedEvent().reset(); } - - /** - * Extract vertices from input split, saving them into a mini cache of - * partitions. Periodically flush the cache of vertices when a limit is - * reached in readVerticeFromInputSplit. - * Mark the input split finished when done. - * - * @param inputSplitPath ZK location of input split - * @return Mapping of vertex indices and statistics, or null if no data read - * @throws IOException - * @throws ClassNotFoundException - * @throws InterruptedException - * @throws InstantiationException - * @throws IllegalAccessException - */ - private VertexEdgeCount loadVerticesFromInputSplit(String inputSplitPath) - throws IOException, ClassNotFoundException, InterruptedException, - InstantiationException, IllegalAccessException { - InputSplit inputSplit = getInputSplitForVertices(inputSplitPath); - VertexEdgeCount vertexEdgeCount = - readVerticesFromInputSplit(inputSplit); - if (LOG.isInfoEnabled()) { - LOG.info("loadVerticesFromInputSplit: Finished loading " + - inputSplitPath + " " + vertexEdgeCount); - } - markInputSplitPathFinished(inputSplitPath); - return vertexEdgeCount; + } + + /** + * Load the vertices from the user-defined VertexReader into our partitions + * of vertex ranges. Do this until all the InputSplits have been processed. + * All workers will try to do as many InputSplits as they can. The master + * will monitor progress and stop this once all the InputSplits have been + * loaded and check-pointed. Keep track of the last input split path to + * ensure the input split cache is flushed prior to marking the last input + * split complete. + * + * @return Statistics of the vertices loaded + * @throws IOException + * @throws IllegalAccessException + * @throws InstantiationException + * @throws ClassNotFoundException + * @throws InterruptedException + * @throws KeeperException + */ + private VertexEdgeCount loadVertices() throws IOException, + ClassNotFoundException, InterruptedException, InstantiationException, + IllegalAccessException, KeeperException { + String inputSplitPath = null; + VertexEdgeCount vertexEdgeCount = new VertexEdgeCount(); + while ((inputSplitPath = reserveInputSplit()) != null) { + vertexEdgeCount = vertexEdgeCount.incrVertexEdgeCount( + loadVerticesFromInputSplit(inputSplitPath)); } - /** - * Talk to ZooKeeper to convert the input split path to the actual - * InputSplit containing the vertices to read. - * - * @param inputSplitPath Location in ZK of input split - * @return instance of InputSplit containing vertices to read - * @throws IOException - * @throws ClassNotFoundException - */ - private InputSplit getInputSplitForVertices(String inputSplitPath) - throws IOException, ClassNotFoundException { - byte[] splitList; - try { - splitList = getZkExt().getData(inputSplitPath, false, null); - } catch (KeeperException e) { - throw new IllegalStateException( - "loadVertices: KeeperException on " + inputSplitPath, e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "loadVertices: IllegalStateException on " + inputSplitPath, e); + // Flush the remaining cached vertices + for (Entry> entry : + inputSplitCache.entrySet()) { + if (!entry.getValue().getVertices().isEmpty()) { + commService.sendPartitionReq(entry.getKey().getWorkerInfo(), + entry.getValue()); + entry.getValue().getVertices().clear(); + } + } + inputSplitCache.clear(); + + return vertexEdgeCount; + } + + /** + * Mark an input split path as completed by this worker. This notifies + * the master and the other workers that this input split has not only + * been reserved, but also marked processed. + * + * @param inputSplitPath Path to the input split. + */ + private void markInputSplitPathFinished(String inputSplitPath) { + String inputSplitFinishedPath = + inputSplitPath + INPUT_SPLIT_FINISHED_NODE; + try { + getZkExt().createExt(inputSplitFinishedPath, + null, + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT, + true); + } catch (KeeperException.NodeExistsException e) { + LOG.warn("loadVertices: " + inputSplitFinishedPath + + " already exists!"); + } catch (KeeperException e) { + throw new IllegalStateException( + "loadVertices: KeeperException on " + + inputSplitFinishedPath, e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "loadVertices: InterruptedException on " + + inputSplitFinishedPath, e); + } + } + + /** + * Extract vertices from input split, saving them into a mini cache of + * partitions. Periodically flush the cache of vertices when a limit is + * reached in readVerticeFromInputSplit. + * Mark the input split finished when done. + * + * @param inputSplitPath ZK location of input split + * @return Mapping of vertex indices and statistics, or null if no data read + * @throws IOException + * @throws ClassNotFoundException + * @throws InterruptedException + * @throws InstantiationException + * @throws IllegalAccessException + */ + private VertexEdgeCount loadVerticesFromInputSplit(String inputSplitPath) + throws IOException, ClassNotFoundException, InterruptedException, + InstantiationException, IllegalAccessException { + InputSplit inputSplit = getInputSplitForVertices(inputSplitPath); + VertexEdgeCount vertexEdgeCount = + readVerticesFromInputSplit(inputSplit); + if (LOG.isInfoEnabled()) { + LOG.info("loadVerticesFromInputSplit: Finished loading " + + inputSplitPath + " " + vertexEdgeCount); + } + markInputSplitPathFinished(inputSplitPath); + return vertexEdgeCount; + } + + /** + * Talk to ZooKeeper to convert the input split path to the actual + * InputSplit containing the vertices to read. + * + * @param inputSplitPath Location in ZK of input split + * @return instance of InputSplit containing vertices to read + * @throws IOException + * @throws ClassNotFoundException + */ + private InputSplit getInputSplitForVertices(String inputSplitPath) + throws IOException, ClassNotFoundException { + byte[] splitList; + try { + splitList = getZkExt().getData(inputSplitPath, false, null); + } catch (KeeperException e) { + throw new IllegalStateException( + "loadVertices: KeeperException on " + inputSplitPath, e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "loadVertices: IllegalStateException on " + inputSplitPath, e); + } + getContext().progress(); + + DataInputStream inputStream = + new DataInputStream(new ByteArrayInputStream(splitList)); + String inputSplitClass = Text.readString(inputStream); + InputSplit inputSplit = (InputSplit) + ReflectionUtils.newInstance( + getConfiguration().getClassByName(inputSplitClass), + getConfiguration()); + ((Writable) inputSplit).readFields(inputStream); + + if (LOG.isInfoEnabled()) { + LOG.info("getInputSplitForVertices: Reserved " + inputSplitPath + + " from ZooKeeper and got input split '" + + inputSplit.toString() + "'"); + } + return inputSplit; + } + + /** + * Read vertices from input split. If testing, the user may request a + * maximum number of vertices to be read from an input split. + * + * @param inputSplit Input split to process with vertex reader + * @return List of vertices. + * @throws IOException + * @throws InterruptedException + */ + private VertexEdgeCount readVerticesFromInputSplit( + InputSplit inputSplit) throws IOException, InterruptedException { + VertexInputFormat vertexInputFormat = + BspUtils.createVertexInputFormat(getConfiguration()); + VertexReader vertexReader = + vertexInputFormat.createVertexReader(inputSplit, getContext()); + vertexReader.initialize(inputSplit, getContext()); + long vertexCount = 0; + long edgeCount = 0; + while (vertexReader.nextVertex()) { + BasicVertex readerVertex = + vertexReader.getCurrentVertex(); + if (readerVertex.getVertexId() == null) { + throw new IllegalArgumentException( + "loadVertices: Vertex reader returned a vertex " + + "without an id! - " + readerVertex); + } + if (readerVertex.getVertexValue() == null) { + readerVertex.setVertexValue( + BspUtils.createVertexValue(getConfiguration())); + } + PartitionOwner partitionOwner = + workerGraphPartitioner.getPartitionOwner( + readerVertex.getVertexId()); + Partition partition = + inputSplitCache.get(partitionOwner); + if (partition == null) { + partition = new Partition( + getConfiguration(), + partitionOwner.getPartitionId()); + inputSplitCache.put(partitionOwner, partition); + } + BasicVertex oldVertex = + partition.putVertex(readerVertex); + if (oldVertex != null) { + LOG.warn("readVertices: Replacing vertex " + oldVertex + + " with " + readerVertex); + } + if (partition.getVertices().size() >= maxVerticesPerPartition) { + commService.sendPartitionReq(partitionOwner.getWorkerInfo(), + partition); + partition.getVertices().clear(); + } + ++vertexCount; + edgeCount += readerVertex.getNumOutEdges(); + getContext().progress(); + + ++totalVerticesLoaded; + totalEdgesLoaded += readerVertex.getNumOutEdges(); + // Update status every half a million vertices + if ((totalVerticesLoaded % 500000) == 0) { + String status = "readVerticesFromInputSplit: Loaded " + + totalVerticesLoaded + " vertices and " + + totalEdgesLoaded + " edges " + + MemoryUtils.getRuntimeMemoryStats() + " " + + getGraphMapper().getMapFunctions().toString() + + " - Attempt=" + getApplicationAttempt() + + ", Superstep=" + getSuperstep(); + if (LOG.isInfoEnabled()) { + LOG.info(status); } - getContext().progress(); - - DataInputStream inputStream = - new DataInputStream(new ByteArrayInputStream(splitList)); - String inputSplitClass = Text.readString(inputStream); - InputSplit inputSplit = (InputSplit) - ReflectionUtils.newInstance( - getConfiguration().getClassByName(inputSplitClass), - getConfiguration()); - ((Writable) inputSplit).readFields(inputStream); + getContext().setStatus(status); + } + // For sampling, or to limit outlier input splits, the number of + // records per input split can be limited + if ((inputSplitMaxVertices > 0) && + (vertexCount >= inputSplitMaxVertices)) { if (LOG.isInfoEnabled()) { - LOG.info("getInputSplitForVertices: Reserved " + inputSplitPath + - " from ZooKeeper and got input split '" + - inputSplit.toString() + "'"); + LOG.info("readVerticesFromInputSplit: Leaving the input " + + "split early, reached maximum vertices " + + vertexCount); } - return inputSplit; + break; + } + } + vertexReader.close(); + + return new VertexEdgeCount(vertexCount, edgeCount); + } + + @Override + public void assignMessagesToVertex(BasicVertex vertex, + Iterable messageIterator) { + vertex.putMessages(messageIterator); + } + + @Override + public void setup() { + // Unless doing a restart, prepare for computation: + // 1. Start superstep INPUT_SUPERSTEP (no computation) + // 2. Wait until the INPUT_SPLIT_ALL_READY_PATH node has been created + // 3. Process input splits until there are no more. + // 4. Wait until the INPUT_SPLIT_ALL_DONE_PATH node has been created + // 5. Wait for superstep INPUT_SUPERSTEP to complete. + if (getRestartedSuperstep() != UNSET_SUPERSTEP) { + setCachedSuperstep(getRestartedSuperstep()); + return; } - /** - * Read vertices from input split. If testing, the user may request a - * maximum number of vertices to be read from an input split. - * - * @param inputSplit Input split to process with vertex reader - * @return List of vertices. - * @throws IOException - * @throws InterruptedException - */ - private VertexEdgeCount readVerticesFromInputSplit( - InputSplit inputSplit) throws IOException, InterruptedException { - VertexInputFormat vertexInputFormat = - BspUtils.createVertexInputFormat(getConfiguration()); - VertexReader vertexReader = - vertexInputFormat.createVertexReader(inputSplit, getContext()); - vertexReader.initialize(inputSplit, getContext()); - long vertexCount = 0; - long edgeCount = 0; - while (vertexReader.nextVertex()) { - BasicVertex readerVertex = - vertexReader.getCurrentVertex(); - if (readerVertex.getVertexId() == null) { - throw new IllegalArgumentException( - "loadVertices: Vertex reader returned a vertex " + - "without an id! - " + readerVertex); - } - if (readerVertex.getVertexValue() == null) { - readerVertex.setVertexValue( - BspUtils.createVertexValue(getConfiguration())); - } - PartitionOwner partitionOwner = - workerGraphPartitioner.getPartitionOwner( - readerVertex.getVertexId()); - Partition partition = - inputSplitCache.get(partitionOwner); - if (partition == null) { - partition = new Partition( - getConfiguration(), - partitionOwner.getPartitionId()); - inputSplitCache.put(partitionOwner, partition); - } - BasicVertex oldVertex = - partition.putVertex(readerVertex); - if (oldVertex != null) { - LOG.warn("readVertices: Replacing vertex " + oldVertex + - " with " + readerVertex); - } - if (partition.getVertices().size() >= maxVerticesPerPartition) { - commService.sendPartitionReq(partitionOwner.getWorkerInfo(), - partition); - partition.getVertices().clear(); - } - ++vertexCount; - edgeCount += readerVertex.getNumOutEdges(); - getContext().progress(); - - ++totalVerticesLoaded; - totalEdgesLoaded += readerVertex.getNumOutEdges(); - // Update status every half a million vertices - if ((totalVerticesLoaded % 500000) == 0) { - String status = "readVerticesFromInputSplit: Loaded " + - totalVerticesLoaded + " vertices and " + - totalEdgesLoaded + " edges " + - MemoryUtils.getRuntimeMemoryStats() + " " + - getGraphMapper().getMapFunctions().toString() + - " - Attempt=" + getApplicationAttempt() + - ", Superstep=" + getSuperstep(); - if (LOG.isInfoEnabled()) { - LOG.info(status); - } - getContext().setStatus(status); - } - - // For sampling, or to limit outlier input splits, the number of - // records per input split can be limited - if ((inputSplitMaxVertices > 0) && - (vertexCount >= inputSplitMaxVertices)) { - if (LOG.isInfoEnabled()) { - LOG.info("readVerticesFromInputSplit: Leaving the input " + - "split early, reached maximum vertices " + - vertexCount); - } - break; - } + JSONObject jobState = getJobState(); + if (jobState != null) { + try { + if ((ApplicationState.valueOf(jobState.getString(JSONOBJ_STATE_KEY)) == + ApplicationState.START_SUPERSTEP) && + jobState.getLong(JSONOBJ_SUPERSTEP_KEY) == + getSuperstep()) { + if (LOG.isInfoEnabled()) { + LOG.info("setup: Restarting from an automated " + + "checkpointed superstep " + + getSuperstep() + ", attempt " + + getApplicationAttempt()); + } + setRestartedSuperstep(getSuperstep()); + return; } - vertexReader.close(); - - return new VertexEdgeCount(vertexCount, edgeCount); + } catch (JSONException e) { + throw new RuntimeException( + "setup: Failed to get key-values from " + + jobState.toString(), e); + } } - @Override - public void assignMessagesToVertex(BasicVertex vertex, - Iterable messageIterator) { - vertex.putMessages(messageIterator); + // Add the partitions for that this worker owns + Collection masterSetPartitionOwners = + startSuperstep(); + workerGraphPartitioner.updatePartitionOwners( + getWorkerInfo(), masterSetPartitionOwners, getPartitionMap()); + + commService.setup(); + + // Ensure the InputSplits are ready for processing before processing + while (true) { + Stat inputSplitsReadyStat; + try { + inputSplitsReadyStat = + getZkExt().exists(inputSplitsAllReadyPath, true); + } catch (KeeperException e) { + throw new IllegalStateException( + "setup: KeeperException waiting on input splits", e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "setup: InterruptedException waiting on input splits", e); + } + if (inputSplitsReadyStat != null) { + break; + } + getInputSplitsAllReadyEvent().waitForever(); + getInputSplitsAllReadyEvent().reset(); } - @Override - public void setup() { - // Unless doing a restart, prepare for computation: - // 1. Start superstep INPUT_SUPERSTEP (no computation) - // 2. Wait until the INPUT_SPLIT_ALL_READY_PATH node has been created - // 3. Process input splits until there are no more. - // 4. Wait until the INPUT_SPLIT_ALL_DONE_PATH node has been created - // 5. Wait for superstep INPUT_SUPERSTEP to complete. - if (getRestartedSuperstep() != UNSET_SUPERSTEP) { - setCachedSuperstep(getRestartedSuperstep()); - return; - } - - JSONObject jobState = getJobState(); - if (jobState != null) { - try { - if ((ApplicationState.valueOf(jobState.getString(JSONOBJ_STATE_KEY)) == - ApplicationState.START_SUPERSTEP) && - jobState.getLong(JSONOBJ_SUPERSTEP_KEY) == - getSuperstep()) { - if (LOG.isInfoEnabled()) { - LOG.info("setup: Restarting from an automated " + - "checkpointed superstep " + - getSuperstep() + ", attempt " + - getApplicationAttempt()); - } - setRestartedSuperstep(getSuperstep()); - return; - } - } catch (JSONException e) { - throw new RuntimeException( - "setup: Failed to get key-values from " + - jobState.toString(), e); - } - } - - // Add the partitions for that this worker owns - Collection masterSetPartitionOwners = - startSuperstep(); - workerGraphPartitioner.updatePartitionOwners( - getWorkerInfo(), masterSetPartitionOwners, getPartitionMap()); - - commService.setup(); - - // Ensure the InputSplits are ready for processing before processing - while (true) { - Stat inputSplitsReadyStat; - try { - inputSplitsReadyStat = - getZkExt().exists(INPUT_SPLITS_ALL_READY_PATH, true); - } catch (KeeperException e) { - throw new IllegalStateException( - "setup: KeeperException waiting on input splits", e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "setup: InterruptedException waiting on input splits", e); - } - if (inputSplitsReadyStat != null) { - break; - } - getInputSplitsAllReadyEvent().waitForever(); - getInputSplitsAllReadyEvent().reset(); - } - - getContext().progress(); - - try { - VertexEdgeCount vertexEdgeCount = loadVertices(); - if (LOG.isInfoEnabled()) { - LOG.info("setup: Finally loaded a total of " + - vertexEdgeCount); - } - } catch (Exception e) { - LOG.error("setup: loadVertices failed - ", e); - throw new IllegalStateException("setup: loadVertices failed", e); - } - getContext().progress(); - - // Workers wait for each other to finish, coordinated by master - String workerDonePath = - INPUT_SPLIT_DONE_PATH + "/" + getWorkerInfo().getHostnameId(); - try { - getZkExt().createExt(workerDonePath, - null, - Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT, - true); - } catch (KeeperException e) { - throw new IllegalStateException( - "setup: KeeperException creating worker done splits", e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "setup: InterruptedException creating worker done splits", e); - } - while (true) { - Stat inputSplitsDoneStat; - try { - inputSplitsDoneStat = - getZkExt().exists(INPUT_SPLITS_ALL_DONE_PATH, true); - } catch (KeeperException e) { - throw new IllegalStateException( - "setup: KeeperException waiting on worker done splits", e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "setup: InterruptedException waiting on worker " + - "done splits", e); - } - if (inputSplitsDoneStat != null) { - break; - } - getInputSplitsAllDoneEvent().waitForever(); - getInputSplitsAllDoneEvent().reset(); - } - - // At this point all vertices have been sent to their destinations. - // Move them to the worker, creating creating the empty partitions - movePartitionsToWorker(commService); - for (PartitionOwner partitionOwner : masterSetPartitionOwners) { - if (partitionOwner.getWorkerInfo().equals(getWorkerInfo()) && - !getPartitionMap().containsKey( - partitionOwner.getPartitionId())) { - Partition partition = - new Partition(getConfiguration(), - partitionOwner.getPartitionId()); - getPartitionMap().put(partitionOwner.getPartitionId(), - partition); - } - } - - // Generate the partition stats for the input superstep and process - // if necessary - List partitionStatsList = - new ArrayList(); - for (Partition partition : getPartitionMap().values()) { - PartitionStats partitionStats = - new PartitionStats(partition.getPartitionId(), - partition.getVertices().size(), - 0, - partition.getEdgeCount()); - partitionStatsList.add(partitionStats); - } - workerGraphPartitioner.finalizePartitionStats( - partitionStatsList, workerPartitionMap); - - finishSuperstep(partitionStatsList); + getContext().progress(); + + try { + VertexEdgeCount vertexEdgeCount = loadVertices(); + if (LOG.isInfoEnabled()) { + LOG.info("setup: Finally loaded a total of " + + vertexEdgeCount); + } + } catch (IOException e) { + throw new IllegalStateException("setup: loadVertices failed due to " + + "IOException", e); + } catch (ClassNotFoundException e) { + throw new IllegalStateException("setup: loadVertices failed due to " + + "ClassNotFoundException", e); + } catch (InterruptedException e) { + throw new IllegalStateException("setup: loadVertices failed due to " + + "InterruptedException", e); + } catch (InstantiationException e) { + throw new IllegalStateException("setup: loadVertices failed due to " + + "InstantiationException", e); + } catch (IllegalAccessException e) { + throw new IllegalStateException("setup: loadVertices failed due to " + + "IllegalAccessException", e); + } catch (KeeperException e) { + throw new IllegalStateException("setup: loadVertices failed due to " + + "KeeperException", e); + } + getContext().progress(); + + // Workers wait for each other to finish, coordinated by master + String workerDonePath = + inputSplitsDonePath + "/" + getWorkerInfo().getHostnameId(); + try { + getZkExt().createExt(workerDonePath, + null, + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT, + true); + } catch (KeeperException e) { + throw new IllegalStateException( + "setup: KeeperException creating worker done splits", e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "setup: InterruptedException creating worker done splits", e); + } + while (true) { + Stat inputSplitsDoneStat; + try { + inputSplitsDoneStat = + getZkExt().exists(inputSplitsAllDonePath, true); + } catch (KeeperException e) { + throw new IllegalStateException( + "setup: KeeperException waiting on worker done splits", e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "setup: InterruptedException waiting on worker " + + "done splits", e); + } + if (inputSplitsDoneStat != null) { + break; + } + getInputSplitsAllDoneEvent().waitForever(); + getInputSplitsAllDoneEvent().reset(); } - /** - * Marshal the aggregator values of to a JSONArray that will later be - * aggregated by master. Reset the 'use' of aggregators in the next - * superstep - * - * @param superstep - */ - private JSONArray marshalAggregatorValues(long superstep) { - JSONArray aggregatorArray = new JSONArray(); - if ((superstep == INPUT_SUPERSTEP) || (aggregatorInUse.size() == 0)) { - return aggregatorArray; - } - - for (String name : aggregatorInUse) { - try { - Aggregator aggregator = getAggregatorMap().get(name); - ByteArrayOutputStream outputStream = - new ByteArrayOutputStream(); - DataOutput output = new DataOutputStream(outputStream); - aggregator.getAggregatedValue().write(output); - - JSONObject aggregatorObj = new JSONObject(); - aggregatorObj.put(AGGREGATOR_NAME_KEY, name); - aggregatorObj.put(AGGREGATOR_CLASS_NAME_KEY, - aggregator.getClass().getName()); - aggregatorObj.put( - AGGREGATOR_VALUE_KEY, - Base64.encodeBytes(outputStream.toByteArray())); - aggregatorArray.put(aggregatorObj); - LOG.info("marshalAggregatorValues: " + - "Found aggregatorObj " + - aggregatorObj + ", value (" + - aggregator.getAggregatedValue() + ")"); - } catch (Exception e) { - throw new RuntimeException(e); - } - } + // At this point all vertices have been sent to their destinations. + // Move them to the worker, creating creating the empty partitions + movePartitionsToWorker(commService); + for (PartitionOwner partitionOwner : masterSetPartitionOwners) { + if (partitionOwner.getWorkerInfo().equals(getWorkerInfo()) && + !getPartitionMap().containsKey( + partitionOwner.getPartitionId())) { + Partition partition = + new Partition(getConfiguration(), + partitionOwner.getPartitionId()); + getPartitionMap().put(partitionOwner.getPartitionId(), + partition); + } + } - if (LOG.isInfoEnabled()) { - LOG.info("marshalAggregatorValues: Finished assembling " + - "aggregator values in JSONArray - " + aggregatorArray); - } - aggregatorInUse.clear(); - return aggregatorArray; + // Generate the partition stats for the input superstep and process + // if necessary + List partitionStatsList = + new ArrayList(); + for (Partition partition : getPartitionMap().values()) { + PartitionStats partitionStats = + new PartitionStats(partition.getPartitionId(), + partition.getVertices().size(), + 0, + partition.getEdgeCount()); + partitionStatsList.add(partitionStats); + } + workerGraphPartitioner.finalizePartitionStats( + partitionStatsList, workerPartitionMap); + + finishSuperstep(partitionStatsList); + } + + /** + * Marshal the aggregator values of to a JSONArray that will later be + * aggregated by master. Reset the 'use' of aggregators in the next + * superstep + * + * @param superstep Superstep to marshall on + * @return JSON array of the aggreagtor values + */ + private JSONArray marshalAggregatorValues(long superstep) { + JSONArray aggregatorArray = new JSONArray(); + if ((superstep == INPUT_SUPERSTEP) || (aggregatorInUse.size() == 0)) { + return aggregatorArray; } - /** - * Get values of aggregators aggregated by master in previous superstep. - * - * @param superstep Superstep to get the aggregated values from - */ - private void getAggregatorValues(long superstep) { - if (superstep <= (INPUT_SUPERSTEP + 1)) { - return; - } - String mergedAggregatorPath = - getMergedAggregatorPath(getApplicationAttempt(), superstep - 1); - JSONArray aggregatorArray = null; - try { - byte[] zkData = - getZkExt().getData(mergedAggregatorPath, false, null); - aggregatorArray = new JSONArray(new String(zkData)); - } catch (KeeperException.NoNodeException e) { - LOG.info("getAggregatorValues: no aggregators in " + - mergedAggregatorPath + " on superstep " + superstep); - return; - } catch (Exception e) { - throw new RuntimeException(e); - } - for (int i = 0; i < aggregatorArray.length(); ++i) { - try { - if (LOG.isDebugEnabled()) { - LOG.debug("getAggregatorValues: " + - "Getting aggregators from " + - aggregatorArray.getJSONObject(i)); - } - String aggregatorName = aggregatorArray.getJSONObject(i). - getString(AGGREGATOR_NAME_KEY); - Aggregator aggregator = - getAggregatorMap().get(aggregatorName); - if (aggregator == null) { - continue; - } - Writable aggregatorValue = aggregator.getAggregatedValue(); - InputStream input = - new ByteArrayInputStream( - Base64.decode(aggregatorArray.getJSONObject(i). - getString(AGGREGATOR_VALUE_KEY))); - aggregatorValue.readFields( - new DataInputStream(input)); - aggregator.setAggregatedValue(aggregatorValue); - if (LOG.isDebugEnabled()) { - LOG.debug("getAggregatorValues: " + - "Got aggregator=" + aggregatorName + " value=" + - aggregatorValue); - } - } catch (Exception e) { - throw new RuntimeException(e); - } - } + for (String name : aggregatorInUse) { + try { + Aggregator aggregator = getAggregatorMap().get(name); + ByteArrayOutputStream outputStream = + new ByteArrayOutputStream(); + DataOutput output = new DataOutputStream(outputStream); + aggregator.getAggregatedValue().write(output); + + JSONObject aggregatorObj = new JSONObject(); + aggregatorObj.put(AGGREGATOR_NAME_KEY, name); + aggregatorObj.put(AGGREGATOR_CLASS_NAME_KEY, + aggregator.getClass().getName()); + aggregatorObj.put( + AGGREGATOR_VALUE_KEY, + Base64.encodeBytes(outputStream.toByteArray())); + aggregatorArray.put(aggregatorObj); if (LOG.isInfoEnabled()) { - LOG.info("getAggregatorValues: Finished loading " + - mergedAggregatorPath + " with aggregator values " + - aggregatorArray); + LOG.info("marshalAggregatorValues: " + + "Found aggregatorObj " + + aggregatorObj + ", value (" + + aggregator.getAggregatedValue() + ")"); } + } catch (JSONException e) { + throw new IllegalStateException("Failed to marshall aggregator " + + "with JSONException " + name, e); + } catch (IOException e) { + throw new IllegalStateException("Failed to marshall aggregator " + + "with IOException " + name, e); + } } - /** - * Register the health of this worker for a given superstep - * - * @param superstep Superstep to register health on - */ - private void registerHealth(long superstep) { - JSONArray hostnamePort = new JSONArray(); - hostnamePort.put(getHostname()); - - hostnamePort.put(workerInfo.getPort()); - - String myHealthPath = null; - if (isHealthy()) { - myHealthPath = getWorkerInfoHealthyPath(getApplicationAttempt(), - getSuperstep()); + if (LOG.isInfoEnabled()) { + LOG.info("marshalAggregatorValues: Finished assembling " + + "aggregator values in JSONArray - " + aggregatorArray); + } + aggregatorInUse.clear(); + return aggregatorArray; + } + + /** + * Get values of aggregators aggregated by master in previous superstep. + * + * @param superstep Superstep to get the aggregated values from + */ + private void getAggregatorValues(long superstep) { + if (superstep <= (INPUT_SUPERSTEP + 1)) { + return; + } + String mergedAggregatorPath = + getMergedAggregatorPath(getApplicationAttempt(), superstep - 1); + JSONArray aggregatorArray = null; + try { + byte[] zkData = + getZkExt().getData(mergedAggregatorPath, false, null); + aggregatorArray = new JSONArray(new String(zkData)); + } catch (KeeperException.NoNodeException e) { + LOG.info("getAggregatorValues: no aggregators in " + + mergedAggregatorPath + " on superstep " + superstep); + return; + } catch (KeeperException e) { + throw new IllegalStateException("Failed to get data for " + + mergedAggregatorPath + " with KeeperException", e); + } catch (InterruptedException e) { + throw new IllegalStateException("Failed to get data for " + + mergedAggregatorPath + " with InterruptedException", e); + } catch (JSONException e) { + throw new IllegalStateException("Failed to get data for " + + mergedAggregatorPath + " with JSONException", e); + } + for (int i = 0; i < aggregatorArray.length(); ++i) { + try { + if (LOG.isDebugEnabled()) { + LOG.debug("getAggregatorValues: " + + "Getting aggregators from " + + aggregatorArray.getJSONObject(i)); } - else { - myHealthPath = getWorkerInfoUnhealthyPath(getApplicationAttempt(), - getSuperstep()); + String aggregatorName = aggregatorArray.getJSONObject(i). + getString(AGGREGATOR_NAME_KEY); + Aggregator aggregator = + getAggregatorMap().get(aggregatorName); + if (aggregator == null) { + continue; } - myHealthPath = myHealthPath + "/" + workerInfo.getHostnameId(); - try { - myHealthZnode = getZkExt().createExt( - myHealthPath, - WritableUtils.writeToByteArray(workerInfo), - Ids.OPEN_ACL_UNSAFE, - CreateMode.EPHEMERAL, - true); - } catch (KeeperException.NodeExistsException e) { - LOG.warn("registerHealth: myHealthPath already exists (likely " + - "from previous failure): " + myHealthPath + - ". Waiting for change in attempts " + - "to re-join the application"); - getApplicationAttemptChangedEvent().waitForever(); - if (LOG.isInfoEnabled()) { - LOG.info("registerHealth: Got application " + - "attempt changed event, killing self"); - } - throw new RuntimeException( - "registerHealth: Trying " + - "to get the new application attempt by killing self", e); - } catch (Exception e) { - throw new RuntimeException(e); - } - if (LOG.isInfoEnabled()) { - LOG.info("registerHealth: Created my health node for attempt=" + - getApplicationAttempt() + ", superstep=" + - getSuperstep() + " with " + myHealthZnode + - " and workerInfo= " + workerInfo); + Writable aggregatorValue = aggregator.getAggregatedValue(); + InputStream input = + new ByteArrayInputStream( + Base64.decode(aggregatorArray.getJSONObject(i). + getString(AGGREGATOR_VALUE_KEY))); + aggregatorValue.readFields( + new DataInputStream(input)); + aggregator.setAggregatedValue(aggregatorValue); + if (LOG.isDebugEnabled()) { + LOG.debug("getAggregatorValues: " + + "Got aggregator=" + aggregatorName + " value=" + + aggregatorValue); } + } catch (JSONException e) { + throw new IllegalStateException("Failed to decode data for index " + + i + " with KeeperException", e); + } catch (IOException e) { + throw new IllegalStateException("Failed to decode data for index " + + i + " with KeeperException", e); + } } - - /** - * Do this to help notify the master quicker that this worker has failed. - */ - private void unregisterHealth() { - LOG.error("unregisterHealth: Got failure, unregistering health on " + - myHealthZnode + " on superstep " + getSuperstep()); - try { - getZkExt().delete(myHealthZnode, -1); - } catch (InterruptedException e) { - throw new IllegalStateException( - "unregisterHealth: InterruptedException - Couldn't delete " + - myHealthZnode, e); - } catch (KeeperException e) { - throw new IllegalStateException( - "unregisterHealth: KeeperException - Couldn't delete " + - myHealthZnode, e); - } + if (LOG.isInfoEnabled()) { + LOG.info("getAggregatorValues: Finished loading " + + mergedAggregatorPath + " with aggregator values " + + aggregatorArray); } - - @Override - public void failureCleanup() { - unregisterHealth(); + } + + /** + * Register the health of this worker for a given superstep + * + * @param superstep Superstep to register health on + */ + private void registerHealth(long superstep) { + JSONArray hostnamePort = new JSONArray(); + hostnamePort.put(getHostname()); + + hostnamePort.put(workerInfo.getPort()); + + String myHealthPath = null; + if (isHealthy()) { + myHealthPath = getWorkerInfoHealthyPath(getApplicationAttempt(), + getSuperstep()); + } else { + myHealthPath = getWorkerInfoUnhealthyPath(getApplicationAttempt(), + getSuperstep()); + } + myHealthPath = myHealthPath + "/" + workerInfo.getHostnameId(); + try { + myHealthZnode = getZkExt().createExt( + myHealthPath, + WritableUtils.writeToByteArray(workerInfo), + Ids.OPEN_ACL_UNSAFE, + CreateMode.EPHEMERAL, + true); + } catch (KeeperException.NodeExistsException e) { + LOG.warn("registerHealth: myHealthPath already exists (likely " + + "from previous failure): " + myHealthPath + + ". Waiting for change in attempts " + + "to re-join the application"); + getApplicationAttemptChangedEvent().waitForever(); + if (LOG.isInfoEnabled()) { + LOG.info("registerHealth: Got application " + + "attempt changed event, killing self"); + } + throw new IllegalStateException( + "registerHealth: Trying " + + "to get the new application attempt by killing self", e); + } catch (KeeperException e) { + throw new IllegalStateException("Creating " + myHealthPath + + " failed with KeeperException", e); + } catch (InterruptedException e) { + throw new IllegalStateException("Creating " + myHealthPath + + " failed with InterruptedException", e); + } + if (LOG.isInfoEnabled()) { + LOG.info("registerHealth: Created my health node for attempt=" + + getApplicationAttempt() + ", superstep=" + + getSuperstep() + " with " + myHealthZnode + + " and workerInfo= " + workerInfo); + } + } + + /** + * Do this to help notify the master quicker that this worker has failed. + */ + private void unregisterHealth() { + LOG.error("unregisterHealth: Got failure, unregistering health on " + + myHealthZnode + " on superstep " + getSuperstep()); + try { + getZkExt().delete(myHealthZnode, -1); + } catch (InterruptedException e) { + throw new IllegalStateException( + "unregisterHealth: InterruptedException - Couldn't delete " + + myHealthZnode, e); + } catch (KeeperException e) { + throw new IllegalStateException( + "unregisterHealth: KeeperException - Couldn't delete " + + myHealthZnode, e); + } + } + + @Override + public void failureCleanup() { + unregisterHealth(); + } + + @Override + public Collection startSuperstep() { + // Algorithm: + // 1. Communication service will combine message from previous + // superstep + // 2. Register my health for the next superstep. + // 3. Wait until the partition assignment is complete and get it + // 4. Get the aggregator values from the previous superstep + if (getSuperstep() != INPUT_SUPERSTEP) { + commService.prepareSuperstep(); } - @Override - public Collection startSuperstep() { - // Algorithm: - // 1. Communication service will combine message from previous - // superstep - // 2. Register my health for the next superstep. - // 3. Wait until the partition assignment is complete and get it - // 4. Get the aggregator values from the previous superstep - if (getSuperstep() != INPUT_SUPERSTEP) { - commService.prepareSuperstep(); - } - - registerHealth(getSuperstep()); + registerHealth(getSuperstep()); + + String partitionAssignmentsNode = + getPartitionAssignmentsPath(getApplicationAttempt(), + getSuperstep()); + Collection masterSetPartitionOwners; + try { + while (getZkExt().exists(partitionAssignmentsNode, true) == + null) { + getPartitionAssignmentsReadyChangedEvent().waitForever(); + getPartitionAssignmentsReadyChangedEvent().reset(); + } + List writableList = + WritableUtils.readListFieldsFromZnode( + getZkExt(), + partitionAssignmentsNode, + false, + null, + workerGraphPartitioner.createPartitionOwner().getClass(), + getConfiguration()); + + @SuppressWarnings("unchecked") + Collection castedWritableList = + (Collection) writableList; + masterSetPartitionOwners = castedWritableList; + } catch (KeeperException e) { + throw new IllegalStateException( + "startSuperstep: KeeperException getting assignments", e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "startSuperstep: InterruptedException getting assignments", e); + } - String partitionAssignmentsNode = - getPartitionAssignmentsPath(getApplicationAttempt(), - getSuperstep()); - Collection masterSetPartitionOwners; - try { - while (getZkExt().exists(partitionAssignmentsNode, true) == - null) { - getPartitionAssignmentsReadyChangedEvent().waitForever(); - getPartitionAssignmentsReadyChangedEvent().reset(); - } - List writableList = - WritableUtils.readListFieldsFromZnode( - getZkExt(), - partitionAssignmentsNode, - false, - null, - workerGraphPartitioner.createPartitionOwner().getClass(), - getConfiguration()); - - @SuppressWarnings("unchecked") - Collection castedWritableList = - (Collection) writableList; - masterSetPartitionOwners = castedWritableList; - } catch (KeeperException e) { - throw new IllegalStateException( - "startSuperstep: KeeperException getting assignments", e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "startSuperstep: InterruptedException getting assignments", e); - } + if (LOG.isInfoEnabled()) { + LOG.info("startSuperstep: Ready for computation on superstep " + + getSuperstep() + " since worker " + + "selection and vertex range assignments are done in " + + partitionAssignmentsNode); + } - if (LOG.isInfoEnabled()) { - LOG.info("startSuperstep: Ready for computation on superstep " + - getSuperstep() + " since worker " + - "selection and vertex range assignments are done in " + - partitionAssignmentsNode); - } + if (getSuperstep() != INPUT_SUPERSTEP) { + getAggregatorValues(getSuperstep()); + } + getContext().setStatus("startSuperstep: " + + getGraphMapper().getMapFunctions().toString() + + " - Attempt=" + getApplicationAttempt() + + ", Superstep=" + getSuperstep()); + return masterSetPartitionOwners; + } + + @Override + public boolean finishSuperstep(List partitionStatsList) { + // This barrier blocks until success (or the master signals it to + // restart). + // + // Master will coordinate the barriers and aggregate "doneness" of all + // the vertices. Each worker will: + // 1. Flush the unsent messages + // 2. Execute user postSuperstep() if necessary. + // 3. Save aggregator values that are in use. + // 4. Report the statistics (vertices, edges, messages, etc.) + // of this worker + // 5. Let the master know it is finished. + // 6. Wait for the master's global stats, and check if done + long workerSentMessages = 0; + try { + workerSentMessages = commService.flush(getContext()); + } catch (IOException e) { + throw new IllegalStateException( + "finishSuperstep: flush failed", e); + } - if (getSuperstep() != INPUT_SUPERSTEP) { - getAggregatorValues(getSuperstep()); - } - getContext().setStatus("startSuperstep: " + - getGraphMapper().getMapFunctions().toString() + - " - Attempt=" + getApplicationAttempt() + - ", Superstep=" + getSuperstep()); - return masterSetPartitionOwners; + if (getSuperstep() != INPUT_SUPERSTEP) { + getWorkerContext().postSuperstep(); + getContext().progress(); } - @Override - public boolean finishSuperstep(List partitionStatsList) { - // This barrier blocks until success (or the master signals it to - // restart). - // - // Master will coordinate the barriers and aggregate "doneness" of all - // the vertices. Each worker will: - // 1. Flush the unsent messages - // 2. Execute user postSuperstep() if necessary. - // 3. Save aggregator values that are in use. - // 4. Report the statistics (vertices, edges, messages, etc.) - // of this worker - // 5. Let the master know it is finished. - // 6. Wait for the master's global stats, and check if done - long workerSentMessages = 0; - try { - workerSentMessages = commService.flush(getContext()); - } catch (IOException e) { - throw new IllegalStateException( - "finishSuperstep: flush failed", e); - } + if (LOG.isInfoEnabled()) { + LOG.info("finishSuperstep: Superstep " + getSuperstep() + " " + + MemoryUtils.getRuntimeMemoryStats()); + } - if (getSuperstep() != INPUT_SUPERSTEP) { - getWorkerContext().postSuperstep(); - getContext().progress(); - } + JSONArray aggregatorValueArray = + marshalAggregatorValues(getSuperstep()); + Collection finalizedPartitionStats = + workerGraphPartitioner.finalizePartitionStats( + partitionStatsList, workerPartitionMap); + List finalizedPartitionStatsList = + new ArrayList(finalizedPartitionStats); + byte [] partitionStatsBytes = + WritableUtils.writeListToByteArray(finalizedPartitionStatsList); + JSONObject workerFinishedInfoObj = new JSONObject(); + try { + workerFinishedInfoObj.put(JSONOBJ_AGGREGATOR_VALUE_ARRAY_KEY, + aggregatorValueArray); + workerFinishedInfoObj.put(JSONOBJ_PARTITION_STATS_KEY, + Base64.encodeBytes(partitionStatsBytes)); + workerFinishedInfoObj.put(JSONOBJ_NUM_MESSAGES_KEY, + workerSentMessages); + } catch (JSONException e) { + throw new RuntimeException(e); + } + String finishedWorkerPath = + getWorkerFinishedPath(getApplicationAttempt(), getSuperstep()) + + "/" + getHostnamePartitionId(); + try { + getZkExt().createExt(finishedWorkerPath, + workerFinishedInfoObj.toString().getBytes(), + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT, + true); + } catch (KeeperException.NodeExistsException e) { + LOG.warn("finishSuperstep: finished worker path " + + finishedWorkerPath + " already exists!"); + } catch (KeeperException e) { + throw new IllegalStateException("Creating " + finishedWorkerPath + + " failed with KeeperException", e); + } catch (InterruptedException e) { + throw new IllegalStateException("Creating " + finishedWorkerPath + + " failed with InterruptedException", e); + } + getContext().setStatus("finishSuperstep: (waiting for rest " + + "of workers) " + + getGraphMapper().getMapFunctions().toString() + + " - Attempt=" + getApplicationAttempt() + + ", Superstep=" + getSuperstep()); + + String superstepFinishedNode = + getSuperstepFinishedPath(getApplicationAttempt(), getSuperstep()); + try { + while (getZkExt().exists(superstepFinishedNode, true) == null) { + getSuperstepFinishedEvent().waitForever(); + getSuperstepFinishedEvent().reset(); + } + } catch (KeeperException e) { + throw new IllegalStateException( + "finishSuperstep: Failed while waiting for master to " + + "signal completion of superstep " + getSuperstep(), e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "finishSuperstep: Failed while waiting for master to " + + "signal completion of superstep " + getSuperstep(), e); + } + GlobalStats globalStats = new GlobalStats(); + WritableUtils.readFieldsFromZnode( + getZkExt(), superstepFinishedNode, false, null, globalStats); + if (LOG.isInfoEnabled()) { + LOG.info("finishSuperstep: Completed superstep " + getSuperstep() + + " with global stats " + globalStats); + } + incrCachedSuperstep(); + getContext().setStatus("finishSuperstep: (all workers done) " + + getGraphMapper().getMapFunctions().toString() + + " - Attempt=" + getApplicationAttempt() + + ", Superstep=" + getSuperstep()); + getGraphMapper().getGraphState(). + setNumEdges(globalStats.getEdgeCount()). + setNumVertices(globalStats.getVertexCount()); + return (globalStats.getFinishedVertexCount() == + globalStats.getVertexCount()) && + (globalStats.getMessageCount() == 0); + } + + /** + * Save the vertices using the user-defined VertexOutputFormat from our + * vertexArray based on the split. + * @throws InterruptedException + */ + private void saveVertices() throws IOException, InterruptedException { + if (getConfiguration().get(GiraphJob.VERTEX_OUTPUT_FORMAT_CLASS) == + null) { + LOG.warn("saveVertices: " + GiraphJob.VERTEX_OUTPUT_FORMAT_CLASS + + " not specified -- there will be no saved output"); + return; + } - if (LOG.isInfoEnabled()) { - LOG.info("finishSuperstep: Superstep " + getSuperstep() + " " + - MemoryUtils.getRuntimeMemoryStats()); - } + VertexOutputFormat vertexOutputFormat = + BspUtils.createVertexOutputFormat(getConfiguration()); + VertexWriter vertexWriter = + vertexOutputFormat.createVertexWriter(getContext()); + vertexWriter.initialize(getContext()); + for (Partition partition : workerPartitionMap.values()) { + for (BasicVertex vertex : partition.getVertices()) { + vertexWriter.writeVertex(vertex); + } + } + vertexWriter.close(getContext()); + } + + @Override + public void cleanup() throws IOException, InterruptedException { + commService.closeConnections(); + setCachedSuperstep(getSuperstep() - 1); + saveVertices(); + // All worker processes should denote they are done by adding special + // znode. Once the number of znodes equals the number of partitions + // for workers and masters, the master will clean up the ZooKeeper + // znodes associated with this job. + String workerCleanedUpPath = cleanedUpPath + "/" + + getTaskPartition() + WORKER_SUFFIX; + try { + String finalFinishedPath = + getZkExt().createExt(workerCleanedUpPath, + null, + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT, + true); + if (LOG.isInfoEnabled()) { + LOG.info("cleanup: Notifying master its okay to cleanup with " + + finalFinishedPath); + } + } catch (KeeperException.NodeExistsException e) { + if (LOG.isInfoEnabled()) { + LOG.info("cleanup: Couldn't create finished node '" + + workerCleanedUpPath); + } + } catch (KeeperException e) { + // Cleaning up, it's okay to fail after cleanup is successful + LOG.error("cleanup: Got KeeperException on notifcation " + + "to master about cleanup", e); + } catch (InterruptedException e) { + // Cleaning up, it's okay to fail after cleanup is successful + LOG.error("cleanup: Got InterruptedException on notifcation " + + "to master about cleanup", e); + } + try { + getZkExt().close(); + } catch (InterruptedException e) { + // cleanup phase -- just log the error + LOG.error("cleanup: Zookeeper failed to close with " + e); + } - JSONArray aggregatorValueArray = - marshalAggregatorValues(getSuperstep()); - Collection finalizedPartitionStats = - workerGraphPartitioner.finalizePartitionStats( - partitionStatsList, workerPartitionMap); - List finalizedPartitionStatsList = - new ArrayList(finalizedPartitionStats); - byte [] partitionStatsBytes = - WritableUtils.writeListToByteArray(finalizedPartitionStatsList); - JSONObject workerFinishedInfoObj = new JSONObject(); - try { - workerFinishedInfoObj.put(JSONOBJ_AGGREGATOR_VALUE_ARRAY_KEY, - aggregatorValueArray); - workerFinishedInfoObj.put(JSONOBJ_PARTITION_STATS_KEY, - Base64.encodeBytes(partitionStatsBytes)); - workerFinishedInfoObj.put(JSONOBJ_NUM_MESSAGES_KEY, - workerSentMessages); - } catch (JSONException e) { - throw new RuntimeException(e); - } - String finishedWorkerPath = - getWorkerFinishedPath(getApplicationAttempt(), getSuperstep()) + - "/" + getHostnamePartitionId(); - try { - getZkExt().createExt(finishedWorkerPath, - workerFinishedInfoObj.toString().getBytes(), - Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT, - true); - } catch (KeeperException.NodeExistsException e) { - LOG.warn("finishSuperstep: finished worker path " + - finishedWorkerPath + " already exists!"); - } catch (Exception e) { - throw new RuntimeException(e); - } + // Preferably would shut down the service only after + // all clients have disconnected (or the exceptions on the + // client side ignored). + commService.close(); + } + + @Override + public void storeCheckpoint() throws IOException { + getContext().setStatus("storeCheckpoint: Starting checkpoint " + + getGraphMapper().getMapFunctions().toString() + + " - Attempt=" + getApplicationAttempt() + + ", Superstep=" + getSuperstep()); + + // Algorithm: + // For each partition, dump vertices and messages + Path metadataFilePath = + new Path(getCheckpointBasePath(getSuperstep()) + "." + + getHostnamePartitionId() + + CHECKPOINT_METADATA_POSTFIX); + Path verticesFilePath = + new Path(getCheckpointBasePath(getSuperstep()) + "." + + getHostnamePartitionId() + + CHECKPOINT_VERTICES_POSTFIX); + Path validFilePath = + new Path(getCheckpointBasePath(getSuperstep()) + "." + + getHostnamePartitionId() + + CHECKPOINT_VALID_POSTFIX); + + // Remove these files if they already exist (shouldn't though, unless + // of previous failure of this worker) + if (getFs().delete(validFilePath, false)) { + LOG.warn("storeCheckpoint: Removed valid file " + + validFilePath); + } + if (getFs().delete(metadataFilePath, false)) { + LOG.warn("storeCheckpoint: Removed metadata file " + + metadataFilePath); + } + if (getFs().delete(verticesFilePath, false)) { + LOG.warn("storeCheckpoint: Removed file " + verticesFilePath); + } - getContext().setStatus("finishSuperstep: (waiting for rest " + - "of workers) " + - getGraphMapper().getMapFunctions().toString() + - " - Attempt=" + getApplicationAttempt() + - ", Superstep=" + getSuperstep()); + FSDataOutputStream verticesOutputStream = + getFs().create(verticesFilePath); + ByteArrayOutputStream metadataByteStream = new ByteArrayOutputStream(); + DataOutput metadataOutput = new DataOutputStream(metadataByteStream); + for (Partition partition : workerPartitionMap.values()) { + long startPos = verticesOutputStream.getPos(); + partition.write(verticesOutputStream); + // Write the metadata for this partition + // Format: + // + // + // + metadataOutput.writeLong(startPos); + metadataOutput.writeInt(partition.getPartitionId()); + if (LOG.isDebugEnabled()) { + LOG.debug("storeCheckpoint: Vertex file starting " + + "offset = " + startPos + ", length = " + + (verticesOutputStream.getPos() - startPos) + + ", partition = " + partition.toString()); + } + } + // Metadata is buffered and written at the end since it's small and + // needs to know how many partitions this worker owns + FSDataOutputStream metadataOutputStream = + getFs().create(metadataFilePath); + metadataOutputStream.writeInt(workerPartitionMap.size()); + metadataOutputStream.write(metadataByteStream.toByteArray()); + metadataOutputStream.close(); + verticesOutputStream.close(); + if (LOG.isInfoEnabled()) { + LOG.info("storeCheckpoint: Finished metadata (" + + metadataFilePath + ") and vertices (" + verticesFilePath + ")."); + } - String superstepFinishedNode = - getSuperstepFinishedPath(getApplicationAttempt(), getSuperstep()); + getFs().createNewFile(validFilePath); + } + + @Override + public void loadCheckpoint(long superstep) { + // Algorithm: + // Examine all the partition owners and load the ones + // that match my hostname and id from the master designated checkpoint + // prefixes. + long startPos = 0; + int loadedPartitions = 0; + for (PartitionOwner partitionOwner : + workerGraphPartitioner.getPartitionOwners()) { + if (partitionOwner.getWorkerInfo().equals(getWorkerInfo())) { + String metadataFile = + partitionOwner.getCheckpointFilesPrefix() + + CHECKPOINT_METADATA_POSTFIX; + String partitionsFile = + partitionOwner.getCheckpointFilesPrefix() + + CHECKPOINT_VERTICES_POSTFIX; try { - while (getZkExt().exists(superstepFinishedNode, true) == null) { - getSuperstepFinishedEvent().waitForever(); - getSuperstepFinishedEvent().reset(); + int partitionId = -1; + DataInputStream metadataStream = + getFs().open(new Path(metadataFile)); + int partitions = metadataStream.readInt(); + for (int i = 0; i < partitions; ++i) { + startPos = metadataStream.readLong(); + partitionId = metadataStream.readInt(); + if (partitionId == partitionOwner.getPartitionId()) { + break; } - } catch (KeeperException e) { + } + if (partitionId != partitionOwner.getPartitionId()) { throw new IllegalStateException( - "finishSuperstep: Failed while waiting for master to " + - "signal completion of superstep " + getSuperstep(), e); - } catch (InterruptedException e) { + "loadCheckpoint: " + partitionOwner + + " not found!"); + } + metadataStream.close(); + Partition partition = + new Partition( + getConfiguration(), + partitionId); + DataInputStream partitionsStream = + getFs().open(new Path(partitionsFile)); + if (partitionsStream.skip(startPos) != startPos) { throw new IllegalStateException( - "finishSuperstep: Failed while waiting for master to " + - "signal completion of superstep " + getSuperstep(), e); - } - GlobalStats globalStats = new GlobalStats(); - WritableUtils.readFieldsFromZnode( - getZkExt(), superstepFinishedNode, false, null, globalStats); - if (LOG.isInfoEnabled()) { - LOG.info("finishSuperstep: Completed superstep " + getSuperstep() + - " with global stats " + globalStats); - } - incrCachedSuperstep(); - getContext().setStatus("finishSuperstep: (all workers done) " + - getGraphMapper().getMapFunctions().toString() + - " - Attempt=" + getApplicationAttempt() + - ", Superstep=" + getSuperstep()); - getGraphMapper().getGraphState(). - setNumEdges(globalStats.getEdgeCount()). - setNumVertices(globalStats.getVertexCount()); - return ((globalStats.getFinishedVertexCount() == - globalStats.getVertexCount()) && - (globalStats.getMessageCount() == 0)); - } - - /** - * Save the vertices using the user-defined VertexOutputFormat from our - * vertexArray based on the split. - * @throws InterruptedException - */ - private void saveVertices() throws IOException, InterruptedException { - if (getConfiguration().get(GiraphJob.VERTEX_OUTPUT_FORMAT_CLASS) - == null) { - LOG.warn("saveVertices: " + GiraphJob.VERTEX_OUTPUT_FORMAT_CLASS + - " not specified -- there will be no saved output"); - return; - } - - VertexOutputFormat vertexOutputFormat = - BspUtils.createVertexOutputFormat(getConfiguration()); - VertexWriter vertexWriter = - vertexOutputFormat.createVertexWriter(getContext()); - vertexWriter.initialize(getContext()); - for (Partition partition : workerPartitionMap.values()) { - for (BasicVertex vertex : partition.getVertices()) { - vertexWriter.writeVertex(vertex); - } + "loadCheckpoint: Failed to skip " + startPos + + " on " + partitionsFile); + } + partition.readFields(partitionsStream); + partitionsStream.close(); + if (LOG.isInfoEnabled()) { + LOG.info("loadCheckpoint: Loaded partition " + + partition); + } + if (getPartitionMap().put(partitionId, partition) != null) { + throw new IllegalStateException( + "loadCheckpoint: Already has partition owner " + + partitionOwner); + } + ++loadedPartitions; + } catch (IOException e) { + throw new RuntimeException( + "loadCheckpoing: Failed to get partition owner " + + partitionOwner, e); } - vertexWriter.close(getContext()); + } } - - @Override - public void cleanup() throws IOException, InterruptedException { - commService.closeConnections(); - setCachedSuperstep(getSuperstep() - 1); - saveVertices(); - // All worker processes should denote they are done by adding special - // znode. Once the number of znodes equals the number of partitions - // for workers and masters, the master will clean up the ZooKeeper - // znodes associated with this job. - String cleanedUpPath = CLEANED_UP_PATH + "/" + - getTaskPartition() + WORKER_SUFFIX; - try { - String finalFinishedPath = - getZkExt().createExt(cleanedUpPath, - null, - Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT, - true); - if (LOG.isInfoEnabled()) { - LOG.info("cleanup: Notifying master its okay to cleanup with " + - finalFinishedPath); - } - } catch (KeeperException.NodeExistsException e) { - if (LOG.isInfoEnabled()) { - LOG.info("cleanup: Couldn't create finished node '" + - cleanedUpPath); - } - } catch (KeeperException e) { - // Cleaning up, it's okay to fail after cleanup is successful - LOG.error("cleanup: Got KeeperException on notifcation " + - "to master about cleanup", e); - } catch (InterruptedException e) { - // Cleaning up, it's okay to fail after cleanup is successful - LOG.error("cleanup: Got InterruptedException on notifcation " + - "to master about cleanup", e); - } - try { - getZkExt().close(); - } catch (InterruptedException e) { - // cleanup phase -- just log the error - LOG.error("cleanup: Zookeeper failed to close with " + e); - } - - // Preferably would shut down the service only after - // all clients have disconnected (or the exceptions on the - // client side ignored). - commService.close(); + if (LOG.isInfoEnabled()) { + LOG.info("loadCheckpoint: Loaded " + loadedPartitions + + " partitions of out " + + workerGraphPartitioner.getPartitionOwners().size() + + " total."); } - - @Override - public void storeCheckpoint() throws IOException { - getContext().setStatus("storeCheckpoint: Starting checkpoint " + - getGraphMapper().getMapFunctions().toString() + - " - Attempt=" + getApplicationAttempt() + - ", Superstep=" + getSuperstep()); - - // Algorithm: - // For each partition, dump vertices and messages - Path metadataFilePath = - new Path(getCheckpointBasePath(getSuperstep()) + "." + - getHostnamePartitionId() + - CHECKPOINT_METADATA_POSTFIX); - Path verticesFilePath = - new Path(getCheckpointBasePath(getSuperstep()) + "." + - getHostnamePartitionId() + - CHECKPOINT_VERTICES_POSTFIX); - Path validFilePath = - new Path(getCheckpointBasePath(getSuperstep()) + "." + - getHostnamePartitionId() + - CHECKPOINT_VALID_POSTFIX); - - // Remove these files if they already exist (shouldn't though, unless - // of previous failure of this worker) - if (getFs().delete(validFilePath, false)) { - LOG.warn("storeCheckpoint: Removed valid file " + - validFilePath); - } - if (getFs().delete(metadataFilePath, false)) { - LOG.warn("storeCheckpoint: Removed metadata file " + - metadataFilePath); - } - if (getFs().delete(verticesFilePath, false)) { - LOG.warn("storeCheckpoint: Removed file " + verticesFilePath); - } - - FSDataOutputStream verticesOutputStream = - getFs().create(verticesFilePath); - ByteArrayOutputStream metadataByteStream = new ByteArrayOutputStream(); - DataOutput metadataOutput = new DataOutputStream(metadataByteStream); - for (Partition partition : workerPartitionMap.values()) { - long startPos = verticesOutputStream.getPos(); - partition.write(verticesOutputStream); - // Write the metadata for this partition - // Format: - // - // - // - metadataOutput.writeLong(startPos); - metadataOutput.writeInt(partition.getPartitionId()); - if (LOG.isDebugEnabled()) { - LOG.debug("storeCheckpoint: Vertex file starting " + - "offset = " + startPos + ", length = " + - (verticesOutputStream.getPos() - startPos) + - ", partition = " + partition.toString()); - } + // Communication service needs to setup the connections prior to + // processing vertices + commService.setup(); + } + + /** + * Send the worker partitions to their destination workers + * + * @param workerPartitionMap Map of worker info to the partitions stored + * on this worker to be sent + */ + private void sendWorkerPartitions( + Map> workerPartitionMap) { + List>> randomEntryList = + new ArrayList>>( + workerPartitionMap.entrySet()); + Collections.shuffle(randomEntryList); + for (Entry> workerPartitionList : + randomEntryList) { + for (Integer partitionId : workerPartitionList.getValue()) { + Partition partition = + getPartitionMap().get(partitionId); + if (partition == null) { + throw new IllegalStateException( + "sendWorkerPartitions: Couldn't find partition " + + partitionId + " to send to " + + workerPartitionList.getKey()); } - // Metadata is buffered and written at the end since it's small and - // needs to know how many partitions this worker owns - FSDataOutputStream metadataOutputStream = - getFs().create(metadataFilePath); - metadataOutputStream.writeInt(workerPartitionMap.size()); - metadataOutputStream.write(metadataByteStream.toByteArray()); - metadataOutputStream.close(); - verticesOutputStream.close(); if (LOG.isInfoEnabled()) { - LOG.info("storeCheckpoint: Finished metadata (" + - metadataFilePath + ") and vertices (" + verticesFilePath - + ")."); + LOG.info("sendWorkerPartitions: Sending worker " + + workerPartitionList.getKey() + " partition " + + partitionId); } + getGraphMapper().getGraphState().getWorkerCommunications(). + sendPartitionReq(workerPartitionList.getKey(), + partition); + getPartitionMap().remove(partitionId); + } + } - getFs().createNewFile(validFilePath); + String myPartitionExchangeDonePath = + getPartitionExchangeWorkerPath( + getApplicationAttempt(), getSuperstep(), getWorkerInfo()); + try { + getZkExt().createExt(myPartitionExchangeDonePath, + null, + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT, + true); + } catch (KeeperException e) { + throw new IllegalStateException( + "sendWorkerPartitions: KeeperException to create " + + myPartitionExchangeDonePath, e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "sendWorkerPartitions: InterruptedException to create " + + myPartitionExchangeDonePath, e); + } + if (LOG.isInfoEnabled()) { + LOG.info("sendWorkerPartitions: Done sending all my partitions."); } + } + + @Override + public final void exchangeVertexPartitions( + Collection masterSetPartitionOwners) { + // 1. Fix the addresses of the partition ids if they have changed. + // 2. Send all the partitions to their destination workers in a random + // fashion. + // 3. Notify completion with a ZooKeeper stamp + // 4. Wait for all my dependencies to be done (if any) + // 5. Add the partitions to myself. + PartitionExchange partitionExchange = + workerGraphPartitioner.updatePartitionOwners( + getWorkerInfo(), masterSetPartitionOwners, getPartitionMap()); + commService.fixPartitionIdToSocketAddrMap(); - @Override - public void loadCheckpoint(long superstep) { - // Algorithm: - // Examine all the partition owners and load the ones - // that match my hostname and id from the master designated checkpoint - // prefixes. - long startPos = 0; - int loadedPartitions = 0; - for (PartitionOwner partitionOwner : - workerGraphPartitioner.getPartitionOwners()) { - if (partitionOwner.getWorkerInfo().equals(getWorkerInfo())) { - String metadataFile = - partitionOwner.getCheckpointFilesPrefix() + - CHECKPOINT_METADATA_POSTFIX; - String partitionsFile = - partitionOwner.getCheckpointFilesPrefix() + - CHECKPOINT_VERTICES_POSTFIX; - try { - int partitionId = -1; - DataInputStream metadataStream = - getFs().open(new Path(metadataFile)); - int partitions = metadataStream.readInt(); - for (int i = 0; i < partitions; ++i) { - startPos = metadataStream.readLong(); - partitionId = metadataStream.readInt(); - if (partitionId == partitionOwner.getPartitionId()) { - break; - } - } - if (partitionId != partitionOwner.getPartitionId()) { - throw new IllegalStateException( - "loadCheckpoint: " + partitionOwner + - " not found!"); - } - metadataStream.close(); - Partition partition = - new Partition( - getConfiguration(), - partitionId); - DataInputStream partitionsStream = - getFs().open(new Path(partitionsFile)); - if (partitionsStream.skip(startPos) != startPos) { - throw new IllegalStateException( - "loadCheckpoint: Failed to skip " + startPos + - " on " + partitionsFile); - } - partition.readFields(partitionsStream); - partitionsStream.close(); - if (LOG.isInfoEnabled()) { - LOG.info("loadCheckpoint: Loaded partition " + - partition); - } - if (getPartitionMap().put(partitionId, partition) != null) { - throw new IllegalStateException( - "loadCheckpoint: Already has partition owner " + - partitionOwner); - } - ++loadedPartitions; - } catch (IOException e) { - throw new RuntimeException( - "loadCheckpoing: Failed to get partition owner " + - partitionOwner, e); - } - } - } - if (LOG.isInfoEnabled()) { - LOG.info("loadCheckpoint: Loaded " + loadedPartitions + - " partitions of out " + - workerGraphPartitioner.getPartitionOwners().size() + - " total."); - } - // Communication service needs to setup the connections prior to - // processing vertices - commService.setup(); + Map> sendWorkerPartitionMap = + partitionExchange.getSendWorkerPartitionMap(); + if (!workerPartitionMap.isEmpty()) { + sendWorkerPartitions(sendWorkerPartitionMap); } - /** - * Send the worker partitions to their destination workers - * - * @param workerPartitionMap Map of worker info to the partitions stored - * on this worker to be sent - */ - private void sendWorkerPartitions( - Map> workerPartitionMap) { - List>> randomEntryList = - new ArrayList>>( - workerPartitionMap.entrySet()); - Collections.shuffle(randomEntryList); - for (Entry> workerPartitionList : - randomEntryList) { - for (Integer partitionId : workerPartitionList.getValue()) { - Partition partition = - getPartitionMap().get(partitionId); - if (partition == null) { - throw new IllegalStateException( - "sendWorkerPartitions: Couldn't find partition " + - partitionId + " to send to " + - workerPartitionList.getKey()); - } - if (LOG.isInfoEnabled()) { - LOG.info("sendWorkerPartitions: Sending worker " + - workerPartitionList.getKey() + " partition " + - partitionId); - } - getGraphMapper().getGraphState().getWorkerCommunications(). - sendPartitionReq(workerPartitionList.getKey(), - partition); - getPartitionMap().remove(partitionId); - } - } + Set myDependencyWorkerSet = + partitionExchange.getMyDependencyWorkerSet(); + Set workerIdSet = new HashSet(); + for (WorkerInfo tmpWorkerInfo : myDependencyWorkerSet) { + if (!workerIdSet.add(tmpWorkerInfo.getHostnameId())) { + throw new IllegalStateException( + "exchangeVertexPartitions: Duplicate entry " + tmpWorkerInfo); + } + } + if (myDependencyWorkerSet.isEmpty() && workerPartitionMap.isEmpty()) { + if (LOG.isInfoEnabled()) { + LOG.info("exchangeVertexPartitions: Nothing to exchange, " + + "exiting early"); + } + return; + } - String myPartitionExchangeDonePath = - getPartitionExchangeWorkerPath( - getApplicationAttempt(), getSuperstep(), getWorkerInfo()); - try { - getZkExt().createExt(myPartitionExchangeDonePath, - null, - Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT, - true); - } catch (KeeperException e) { - throw new IllegalStateException( - "sendWorkerPartitions: KeeperException to create " + - myPartitionExchangeDonePath, e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "sendWorkerPartitions: InterruptedException to create " + - myPartitionExchangeDonePath, e); + String vertexExchangePath = + getPartitionExchangePath(getApplicationAttempt(), getSuperstep()); + List workerDoneList; + try { + while (true) { + workerDoneList = getZkExt().getChildrenExt( + vertexExchangePath, true, false, false); + workerIdSet.removeAll(workerDoneList); + if (workerIdSet.isEmpty()) { + break; } if (LOG.isInfoEnabled()) { - LOG.info("sendWorkerPartitions: Done sending all my partitions."); + LOG.info("exchangeVertexPartitions: Waiting for workers " + + workerIdSet); } + getPartitionExchangeChildrenChangedEvent().waitForever(); + getPartitionExchangeChildrenChangedEvent().reset(); + } + } catch (KeeperException e) { + throw new RuntimeException(e); + } catch (InterruptedException e) { + throw new RuntimeException(e); } - @Override - public final void exchangeVertexPartitions( - Collection masterSetPartitionOwners) { - // 1. Fix the addresses of the partition ids if they have changed. - // 2. Send all the partitions to their destination workers in a random - // fashion. - // 3. Notify completion with a ZooKeeper stamp - // 4. Wait for all my dependencies to be done (if any) - // 5. Add the partitions to myself. - PartitionExchange partitionExchange = - workerGraphPartitioner.updatePartitionOwners( - getWorkerInfo(), masterSetPartitionOwners, getPartitionMap()); - commService.fixPartitionIdToSocketAddrMap(); - - Map> workerPartitionMap = - partitionExchange.getSendWorkerPartitionMap(); - if (!workerPartitionMap.isEmpty()) { - sendWorkerPartitions(workerPartitionMap); - } + if (LOG.isInfoEnabled()) { + LOG.info("exchangeVertexPartitions: Done with exchange."); + } - Set myDependencyWorkerSet = - partitionExchange.getMyDependencyWorkerSet(); - Set workerIdSet = new HashSet(); - for (WorkerInfo workerInfo : myDependencyWorkerSet) { - if (workerIdSet.add(workerInfo.getHostnameId()) != true) { - throw new IllegalStateException( - "exchangeVertexPartitions: Duplicate entry " + workerInfo); - } - } - if (myDependencyWorkerSet.isEmpty() && workerPartitionMap.isEmpty()) { - if (LOG.isInfoEnabled()) { - LOG.info("exchangeVertexPartitions: Nothing to exchange, " + - "exiting early"); - } - return; + // Add the partitions sent earlier + movePartitionsToWorker(commService); + } + + /** + * Partitions that are exchanged need to be moved from the communication + * service to the worker. + * + * @param commService Communication service where the partitions are + * temporarily stored. + */ + private void movePartitionsToWorker( + ServerInterface commService) { + Map>> inPartitionVertexMap = + commService.getInPartitionVertexMap(); + synchronized (inPartitionVertexMap) { + for (Entry>> entry : + inPartitionVertexMap.entrySet()) { + if (getPartitionMap().containsKey(entry.getKey())) { + throw new IllegalStateException( + "moveVerticesToWorker: Already has partition " + + getPartitionMap().get(entry.getKey()) + + ", cannot receive vertex list of size " + + entry.getValue().size()); } - String vertexExchangePath = - getPartitionExchangePath(getApplicationAttempt(), getSuperstep()); - List workerDoneList; - try { - while (true) { - workerDoneList = getZkExt().getChildrenExt( - vertexExchangePath, true, false, false); - workerIdSet.removeAll(workerDoneList); - if (workerIdSet.isEmpty()) { - break; - } - if (LOG.isInfoEnabled()) { - LOG.info("exchangeVertexPartitions: Waiting for workers " + - workerIdSet); - } - getPartitionExchangeChildrenChangedEvent().waitForever(); - getPartitionExchangeChildrenChangedEvent().reset(); - } - } catch (KeeperException e) { - throw new RuntimeException(e); - } catch (InterruptedException e) { - throw new RuntimeException(e); + Partition tmpPartition = + new Partition(getConfiguration(), + entry.getKey()); + for (BasicVertex vertex : entry.getValue()) { + if (tmpPartition.putVertex(vertex) != null) { + throw new IllegalStateException( + "moveVerticesToWorker: Vertex " + vertex + + " already exists!"); + } } - if (LOG.isInfoEnabled()) { - LOG.info("exchangeVertexPartitions: Done with exchange."); + LOG.info("moveVerticesToWorker: Adding " + + entry.getValue().size() + + " vertices for partition id " + entry.getKey()); } - - // Add the partitions sent earlier - movePartitionsToWorker(commService); + getPartitionMap().put(tmpPartition.getPartitionId(), + tmpPartition); + entry.getValue().clear(); + } + inPartitionVertexMap.clear(); } - - /** - * Partitions that are exchanged need to be moved from the communication - * service to the worker. - * - * @param commService Communication service where the partitions are - * temporarily stored. - */ - private void movePartitionsToWorker( - ServerInterface commService) { - Map>> inPartitionVertexMap = - commService.getInPartitionVertexMap(); - synchronized (inPartitionVertexMap) { - for (Entry>> entry : - inPartitionVertexMap.entrySet()) { - if (getPartitionMap().containsKey(entry.getKey())) { - throw new IllegalStateException( - "moveVerticesToWorker: Already has partition " + - getPartitionMap().get(entry.getKey()) + - ", cannot receive vertex list of size " + - entry.getValue().size()); - } - - Partition tmpPartition = - new Partition(getConfiguration(), - entry.getKey()); - for (BasicVertex vertex : entry.getValue()) { - if (tmpPartition.putVertex(vertex) != null) { - throw new IllegalStateException( - "moveVerticesToWorker: Vertex " + vertex + - " already exists!"); - } - } - if (LOG.isInfoEnabled()) { - LOG.info("moveVerticesToWorker: Adding " + - entry.getValue().size() + - " vertices for partition id " + entry.getKey()); - } - getPartitionMap().put(tmpPartition.getPartitionId(), - tmpPartition); - entry.getValue().clear(); - } - inPartitionVertexMap.clear(); + } + + /** + * Get event when the state of a partition exchange has changed. + * + * @return Event to check. + */ + public final BspEvent getPartitionExchangeChildrenChangedEvent() { + return partitionExchangeChildrenChanged; + } + + @Override + protected boolean processEvent(WatchedEvent event) { + boolean foundEvent = false; + if (event.getPath().startsWith(masterJobStatePath) && + (event.getType() == EventType.NodeChildrenChanged)) { + if (LOG.isInfoEnabled()) { + LOG.info("processEvent: Job state changed, checking " + + "to see if it needs to restart"); + } + JSONObject jsonObj = getJobState(); + try { + if ((ApplicationState.valueOf(jsonObj.getString(JSONOBJ_STATE_KEY)) == + ApplicationState.START_SUPERSTEP) && + jsonObj.getLong(JSONOBJ_APPLICATION_ATTEMPT_KEY) != + getApplicationAttempt()) { + LOG.fatal("processEvent: Worker will restart " + + "from command - " + jsonObj.toString()); + System.exit(-1); } + } catch (JSONException e) { + throw new RuntimeException( + "processEvent: Couldn't properly get job state from " + + jsonObj.toString()); + } + foundEvent = true; + } else if (event.getPath().contains(PARTITION_EXCHANGE_DIR) && + event.getType() == EventType.NodeChildrenChanged) { + if (LOG.isInfoEnabled()) { + LOG.info("processEvent : partitionExchangeChildrenChanged " + + "(at least one worker is done sending partitions)"); + } + partitionExchangeChildrenChanged.signal(); + foundEvent = true; } - final public BspEvent getPartitionExchangeChildrenChangedEvent() { - return partitionExchangeChildrenChanged; - } - - @Override - protected boolean processEvent(WatchedEvent event) { - boolean foundEvent = false; - if (event.getPath().startsWith(MASTER_JOB_STATE_PATH) && - (event.getType() == EventType.NodeChildrenChanged)) { - if (LOG.isInfoEnabled()) { - LOG.info("processEvent: Job state changed, checking " + - "to see if it needs to restart"); - } - JSONObject jsonObj = getJobState(); - try { - if ((ApplicationState.valueOf(jsonObj.getString(JSONOBJ_STATE_KEY)) == - ApplicationState.START_SUPERSTEP) && - jsonObj.getLong(JSONOBJ_APPLICATION_ATTEMPT_KEY) != - getApplicationAttempt()) { - LOG.fatal("processEvent: Worker will restart " + - "from command - " + jsonObj.toString()); - System.exit(-1); - } - } catch (JSONException e) { - throw new RuntimeException( - "processEvent: Couldn't properly get job state from " + - jsonObj.toString()); - } - foundEvent = true; - } else if (event.getPath().contains(PARTITION_EXCHANGE_DIR) && - event.getType() == EventType.NodeChildrenChanged) { - if (LOG.isInfoEnabled()) { - LOG.info("processEvent : partitionExchangeChildrenChanged " + - "(at least one worker is done sending partitions)"); - } - partitionExchangeChildrenChanged.signal(); - foundEvent = true; - } - - return foundEvent; - } - - @Override - public WorkerInfo getWorkerInfo() { - return workerInfo; - } - - @Override - public Map> getPartitionMap() { - return workerPartitionMap; - } - - @Override - public Collection getPartitionOwners() { - return workerGraphPartitioner.getPartitionOwners(); - } - - @Override - public PartitionOwner getVertexPartitionOwner(I vertexIndex) { - return workerGraphPartitioner.getPartitionOwner(vertexIndex); - } - - public Partition getPartition(I vertexIndex) { - PartitionOwner partitionOwner = getVertexPartitionOwner(vertexIndex); - return workerPartitionMap.get(partitionOwner.getPartitionId()); - } - - @Override - public BasicVertex getVertex(I vertexIndex) { - PartitionOwner partitionOwner = getVertexPartitionOwner(vertexIndex); - if (workerPartitionMap.containsKey(partitionOwner.getPartitionId())) { - return workerPartitionMap.get( - partitionOwner.getPartitionId()).getVertex(vertexIndex); - } else { - return null; - } + return foundEvent; + } + + @Override + public WorkerInfo getWorkerInfo() { + return workerInfo; + } + + @Override + public Map> getPartitionMap() { + return workerPartitionMap; + } + + @Override + public Collection getPartitionOwners() { + return workerGraphPartitioner.getPartitionOwners(); + } + + @Override + public PartitionOwner getVertexPartitionOwner(I vertexIndex) { + return workerGraphPartitioner.getPartitionOwner(vertexIndex); + } + + /** + * Get the partition for a vertex index. + * + * @param vertexIndex Vertex index to search for the partition. + * @return Partition that owns this vertex. + */ + public Partition getPartition(I vertexIndex) { + PartitionOwner partitionOwner = getVertexPartitionOwner(vertexIndex); + return workerPartitionMap.get(partitionOwner.getPartitionId()); + } + + @Override + public BasicVertex getVertex(I vertexIndex) { + PartitionOwner partitionOwner = getVertexPartitionOwner(vertexIndex); + if (workerPartitionMap.containsKey(partitionOwner.getPartitionId())) { + return workerPartitionMap.get( + partitionOwner.getPartitionId()).getVertex(vertexIndex); + } else { + return null; } + } } diff --git a/src/main/java/org/apache/giraph/graph/BspUtils.java b/src/main/java/org/apache/giraph/graph/BspUtils.java index 828f3255b..812bf05bb 100644 --- a/src/main/java/org/apache/giraph/graph/BspUtils.java +++ b/src/main/java/org/apache/giraph/graph/BspUtils.java @@ -31,424 +31,479 @@ * instantiate them. */ public class BspUtils { - /** - * Get the user's subclassed {@link GraphPartitionerFactory}. - * - * @param conf Configuration to check - * @return User's graph partitioner - */ - @SuppressWarnings({ "rawtypes", "unchecked" }) - public static - Class> - getGraphPartitionerClass(Configuration conf) { - return (Class>) - conf.getClass(GiraphJob.GRAPH_PARTITIONER_FACTORY_CLASS, - HashPartitionerFactory.class, - GraphPartitionerFactory.class); - } + /** + * Do not construct. + */ + private BspUtils() { } - /** - * Create a user graph partitioner class - * - * @param conf Configuration to check - * @return Instantiated user graph partitioner class - */ - @SuppressWarnings("rawtypes") - public static - GraphPartitionerFactory - createGraphPartitioner(Configuration conf) { - Class> - graphPartitionerFactoryClass = - getGraphPartitionerClass(conf); - return ReflectionUtils.newInstance(graphPartitionerFactoryClass, conf); - } + /** + * Get the user's subclassed {@link GraphPartitionerFactory}. + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data + * @param conf Configuration to check + * @return User's graph partitioner + */ + @SuppressWarnings({ "rawtypes", "unchecked" }) + public static + Class> + getGraphPartitionerClass(Configuration conf) { + return (Class>) + conf.getClass(GiraphJob.GRAPH_PARTITIONER_FACTORY_CLASS, + HashPartitionerFactory.class, + GraphPartitionerFactory.class); + } - /** - * Create a user graph partitioner partition stats class - * - * @param conf Configuration to check - * @return Instantiated user graph partition stats class - */ - @SuppressWarnings("rawtypes") - public static - PartitionStats createGraphPartitionStats(Configuration conf) { - GraphPartitionerFactory graphPartitioner = - createGraphPartitioner(conf); - return graphPartitioner.createMasterGraphPartitioner(). - createPartitionStats(); - } + /** + * Create a user graph partitioner class + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data + * @param conf Configuration to check + * @return Instantiated user graph partitioner class + */ + @SuppressWarnings("rawtypes") + public static + GraphPartitionerFactory + createGraphPartitioner(Configuration conf) { + Class> + graphPartitionerFactoryClass = getGraphPartitionerClass(conf); + return ReflectionUtils.newInstance(graphPartitionerFactoryClass, conf); + } - /** - * Get the user's subclassed {@link VertexInputFormat}. - * - * @param conf Configuration to check - * @return User's vertex input format class - */ - @SuppressWarnings({ "rawtypes", "unchecked" }) - public static - Class> - getVertexInputFormatClass(Configuration conf) { - return (Class>) - conf.getClass(GiraphJob.VERTEX_INPUT_FORMAT_CLASS, - null, - VertexInputFormat.class); - } + /** + * Create a user graph partitioner partition stats class + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data + * @param conf Configuration to check + * @return Instantiated user graph partition stats class + */ + @SuppressWarnings("rawtypes") + public static + PartitionStats createGraphPartitionStats(Configuration conf) { + GraphPartitionerFactory graphPartitioner = + createGraphPartitioner(conf); + return graphPartitioner.createMasterGraphPartitioner(). + createPartitionStats(); + } - /** - * Create a user vertex input format class - * - * @param conf Configuration to check - * @return Instantiated user vertex input format class - */ - @SuppressWarnings("rawtypes") - public static VertexInputFormat - createVertexInputFormat(Configuration conf) { - Class> vertexInputFormatClass = - getVertexInputFormatClass(conf); - VertexInputFormat inputFormat = - ReflectionUtils.newInstance(vertexInputFormatClass, conf); - return inputFormat; - } + /** + * Get the user's subclassed {@link VertexInputFormat}. + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data + * @param conf Configuration to check + * @return User's vertex input format class + */ + @SuppressWarnings({ "rawtypes", "unchecked" }) + public static + Class> + getVertexInputFormatClass(Configuration conf) { + return (Class>) + conf.getClass(GiraphJob.VERTEX_INPUT_FORMAT_CLASS, + null, + VertexInputFormat.class); + } - /** - * Get the user's subclassed {@link VertexOutputFormat}. - * - * @param conf Configuration to check - * @return User's vertex output format class - */ - @SuppressWarnings({ "rawtypes", "unchecked" }) - public static - Class> - getVertexOutputFormatClass(Configuration conf) { - return (Class>) - conf.getClass(GiraphJob.VERTEX_OUTPUT_FORMAT_CLASS, - null, - VertexOutputFormat.class); - } + /** + * Create a user vertex input format class + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data + * @param conf Configuration to check + * @return Instantiated user vertex input format class + */ + @SuppressWarnings("rawtypes") + public static VertexInputFormat + createVertexInputFormat(Configuration conf) { + Class> vertexInputFormatClass = + getVertexInputFormatClass(conf); + VertexInputFormat inputFormat = + ReflectionUtils.newInstance(vertexInputFormatClass, conf); + return inputFormat; + } - /** - * Create a user vertex output format class - * - * @param conf Configuration to check - * @return Instantiated user vertex output format class - */ - @SuppressWarnings("rawtypes") - public static VertexOutputFormat - createVertexOutputFormat(Configuration conf) { - Class> vertexOutputFormatClass = - getVertexOutputFormatClass(conf); - return ReflectionUtils.newInstance(vertexOutputFormatClass, conf); - } + /** + * Get the user's subclassed {@link VertexOutputFormat}. + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param conf Configuration to check + * @return User's vertex output format class + */ + @SuppressWarnings({ "rawtypes", "unchecked" }) + public static + Class> + getVertexOutputFormatClass(Configuration conf) { + return (Class>) + conf.getClass(GiraphJob.VERTEX_OUTPUT_FORMAT_CLASS, + null, + VertexOutputFormat.class); + } - /** - * Get the user's subclassed {@link AggregatorWriter}. - * - * @param conf Configuration to check - * @return User's aggregator writer class - */ - public static Class - getAggregatorWriterClass(Configuration conf) { - return conf.getClass(GiraphJob.AGGREGATOR_WRITER_CLASS, - TextAggregatorWriter.class, - AggregatorWriter.class); - } + /** + * Create a user vertex output format class + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param conf Configuration to check + * @return Instantiated user vertex output format class + */ + @SuppressWarnings("rawtypes") + public static VertexOutputFormat + createVertexOutputFormat(Configuration conf) { + Class> vertexOutputFormatClass = + getVertexOutputFormatClass(conf); + return ReflectionUtils.newInstance(vertexOutputFormatClass, conf); + } - /** - * Create a user aggregator output format class - * - * @param conf Configuration to check - * @return Instantiated user aggregator writer class - */ - public static AggregatorWriter - createAggregatorWriter(Configuration conf) { - Class aggregatorWriterClass = - getAggregatorWriterClass(conf); - return ReflectionUtils.newInstance(aggregatorWriterClass, conf); - } + /** + * Get the user's subclassed {@link AggregatorWriter}. + * + * @param conf Configuration to check + * @return User's aggregator writer class + */ + public static Class + getAggregatorWriterClass(Configuration conf) { + return conf.getClass(GiraphJob.AGGREGATOR_WRITER_CLASS, + TextAggregatorWriter.class, + AggregatorWriter.class); + } - /** - * Get the user's subclassed {@link VertexCombiner}. - * - * @param conf Configuration to check - * @return User's vertex combiner class - */ - @SuppressWarnings({ "rawtypes", "unchecked" }) - public static - Class> - getVertexCombinerClass(Configuration conf) { - return (Class>) - conf.getClass(GiraphJob.VERTEX_COMBINER_CLASS, - null, - VertexCombiner.class); - } + /** + * Create a user aggregator output format class + * + * @param conf Configuration to check + * @return Instantiated user aggregator writer class + */ + public static AggregatorWriter createAggregatorWriter(Configuration conf) { + Class aggregatorWriterClass = + getAggregatorWriterClass(conf); + return ReflectionUtils.newInstance(aggregatorWriterClass, conf); + } - /** - * Create a user vertex combiner class - * - * @param conf Configuration to check - * @return Instantiated user vertex combiner class - */ - @SuppressWarnings("rawtypes") - public static - VertexCombiner createVertexCombiner(Configuration conf) { - Class> vertexCombinerClass = - getVertexCombinerClass(conf); - return ReflectionUtils.newInstance(vertexCombinerClass, conf); - } + /** + * Get the user's subclassed {@link VertexCombiner}. + * + * @param Vertex id + * @param Message data + * @param conf Configuration to check + * @return User's vertex combiner class + */ + @SuppressWarnings({ "rawtypes", "unchecked" }) + public static + Class> + getVertexCombinerClass(Configuration conf) { + return (Class>) + conf.getClass(GiraphJob.VERTEX_COMBINER_CLASS, + null, + VertexCombiner.class); + } - /** - * Get the user's subclassed VertexResolver. - * - * @param conf Configuration to check - * @return User's vertex resolver class - */ - @SuppressWarnings({ "unchecked", "rawtypes" }) - public static - Class> - getVertexResolverClass(Configuration conf) { - return (Class>) - conf.getClass(GiraphJob.VERTEX_RESOLVER_CLASS, - VertexResolver.class, - VertexResolver.class); - } + /** + * Create a user vertex combiner class + * + * @param Vertex id + * @param Message data + * @param conf Configuration to check + * @return Instantiated user vertex combiner class + */ + @SuppressWarnings("rawtypes") + public static + VertexCombiner createVertexCombiner(Configuration conf) { + Class> vertexCombinerClass = + getVertexCombinerClass(conf); + return ReflectionUtils.newInstance(vertexCombinerClass, conf); + } - /** - * Create a user vertex revolver - * - * @param conf Configuration to check - * @return Instantiated user vertex resolver - */ - @SuppressWarnings("rawtypes") - public static VertexResolver - createVertexResolver(Configuration conf, - GraphState graphState) { - Class> vertexResolverClass = - getVertexResolverClass(conf); - VertexResolver resolver = - ReflectionUtils.newInstance(vertexResolverClass, conf); - resolver.setGraphState(graphState); - return resolver; - } + /** + * Get the user's subclassed VertexResolver. + * + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data + * @param conf Configuration to check + * @return User's vertex resolver class + */ + @SuppressWarnings({ "unchecked", "rawtypes" }) + public static + Class> + getVertexResolverClass(Configuration conf) { + return (Class>) + conf.getClass(GiraphJob.VERTEX_RESOLVER_CLASS, + VertexResolver.class, + VertexResolver.class); + } - /** - * Get the user's subclassed WorkerContext. - * - * @param conf Configuration to check - * @return User's worker context class - */ - public static Class - getWorkerContextClass(Configuration conf) { - return (Class) - conf.getClass(GiraphJob.WORKER_CONTEXT_CLASS, - DefaultWorkerContext.class, - WorkerContext.class); - } + /** + * Create a user vertex revolver + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data + * @param conf Configuration to check + * @param graphState State of the graph from the worker + * @return Instantiated user vertex resolver + */ + @SuppressWarnings("rawtypes") + public static VertexResolver + createVertexResolver(Configuration conf, + GraphState graphState) { + Class> vertexResolverClass = + getVertexResolverClass(conf); + VertexResolver resolver = + ReflectionUtils.newInstance(vertexResolverClass, conf); + resolver.setGraphState(graphState); + return resolver; + } - /** - * Create a user worker context - * - * @param conf Configuration to check - * @return Instantiated user worker context - */ - @SuppressWarnings("rawtypes") - public static - WorkerContext createWorkerContext(Configuration conf, - GraphState graphState) { - Class workerContextClass = - getWorkerContextClass(conf); - WorkerContext workerContext = - ReflectionUtils.newInstance(workerContextClass, conf); - workerContext.setGraphState(graphState); - return workerContext; - } + /** + * Get the user's subclassed WorkerContext. + * + * @param conf Configuration to check + * @return User's worker context class + */ + public static Class + getWorkerContextClass(Configuration conf) { + return (Class) + conf.getClass(GiraphJob.WORKER_CONTEXT_CLASS, + DefaultWorkerContext.class, + WorkerContext.class); + } + /** + * Create a user worker context + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data + * @param conf Configuration to check + * @param graphState State of the graph from the worker + * @return Instantiated user worker context + */ + @SuppressWarnings("rawtypes") + public static + WorkerContext createWorkerContext(Configuration conf, + GraphState graphState) { + Class workerContextClass = + getWorkerContextClass(conf); + WorkerContext workerContext = + ReflectionUtils.newInstance(workerContextClass, conf); + workerContext.setGraphState(graphState); + return workerContext; + } - /** - * Get the user's subclassed {@link BasicVertex} - * - * @param conf Configuration to check - * @return User's vertex class - */ - @SuppressWarnings({ "rawtypes", "unchecked" }) - public static - Class> - getVertexClass(Configuration conf) { - return (Class>) - conf.getClass(GiraphJob.VERTEX_CLASS, - null, - BasicVertex.class); - } - /** - * Create a user vertex - * - * @param conf Configuration to check - * @return Instantiated user vertex - */ - @SuppressWarnings("rawtypes") - public static BasicVertex - createVertex(Configuration conf) { - Class> vertexClass = - getVertexClass(conf); - BasicVertex vertex = - ReflectionUtils.newInstance(vertexClass, conf); - return vertex; - } + /** + * Get the user's subclassed {@link BasicVertex} + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data + * @param conf Configuration to check + * @return User's vertex class + */ + @SuppressWarnings({ "rawtypes", "unchecked" }) + public static + Class> getVertexClass(Configuration conf) { + return (Class>) + conf.getClass(GiraphJob.VERTEX_CLASS, + null, + BasicVertex.class); + } - /** - * Get the user's subclassed vertex index class. - * - * @param conf Configuration to check - * @return User's vertex index class - */ - @SuppressWarnings("unchecked") - public static Class - getVertexIndexClass(Configuration conf) { - return (Class) conf.getClass(GiraphJob.VERTEX_INDEX_CLASS, - WritableComparable.class); - } + /** + * Create a user vertex + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data + * @param conf Configuration to check + * @return Instantiated user vertex + */ + @SuppressWarnings("rawtypes") + public static BasicVertex + createVertex(Configuration conf) { + Class> vertexClass = getVertexClass(conf); + BasicVertex vertex = + ReflectionUtils.newInstance(vertexClass, conf); + return vertex; + } - /** - * Create a user vertex index - * - * @param conf Configuration to check - * @return Instantiated user vertex index - */ - @SuppressWarnings("rawtypes") - public static - I createVertexIndex(Configuration conf) { - Class vertexClass = getVertexIndexClass(conf); - try { - return vertexClass.newInstance(); - } catch (InstantiationException e) { - throw new IllegalArgumentException( - "createVertexIndex: Failed to instantiate", e); - } catch (IllegalAccessException e) { - throw new IllegalArgumentException( - "createVertexIndex: Illegally accessed", e); - } - } + /** + * Get the user's subclassed vertex index class. + * + * @param Vertex id + * @param conf Configuration to check + * @return User's vertex index class + */ + @SuppressWarnings("unchecked") + public static Class + getVertexIndexClass(Configuration conf) { + return (Class) conf.getClass(GiraphJob.VERTEX_INDEX_CLASS, + WritableComparable.class); + } - /** - * Get the user's subclassed vertex value class. - * - * @param conf Configuration to check - * @return User's vertex value class - */ - @SuppressWarnings("unchecked") - public static Class - getVertexValueClass(Configuration conf) { - return (Class) conf.getClass(GiraphJob.VERTEX_VALUE_CLASS, - Writable.class); + /** + * Create a user vertex index + * + * @param Vertex id + * @param conf Configuration to check + * @return Instantiated user vertex index + */ + @SuppressWarnings("rawtypes") + public static + I createVertexIndex(Configuration conf) { + Class vertexClass = getVertexIndexClass(conf); + try { + return vertexClass.newInstance(); + } catch (InstantiationException e) { + throw new IllegalArgumentException( + "createVertexIndex: Failed to instantiate", e); + } catch (IllegalAccessException e) { + throw new IllegalArgumentException( + "createVertexIndex: Illegally accessed", e); } + } - /** - * Create a user vertex value - * - * @param conf Configuration to check - * @return Instantiated user vertex value - */ - public static V - createVertexValue(Configuration conf) { - Class vertexValueClass = getVertexValueClass(conf); - try { - return vertexValueClass.newInstance(); - } catch (InstantiationException e) { - throw new IllegalArgumentException( - "createVertexValue: Failed to instantiate", e); - } catch (IllegalAccessException e) { - throw new IllegalArgumentException( - "createVertexValue: Illegally accessed", e); - } - } + /** + * Get the user's subclassed vertex value class. + * + * @param Vertex data + * @param conf Configuration to check + * @return User's vertex value class + */ + @SuppressWarnings("unchecked") + public static Class + getVertexValueClass(Configuration conf) { + return (Class) conf.getClass(GiraphJob.VERTEX_VALUE_CLASS, + Writable.class); + } - /** - * Get the user's subclassed edge value class. - * - * @param conf Configuration to check - * @return User's vertex edge value class - */ - @SuppressWarnings("unchecked") - public static Class - getEdgeValueClass(Configuration conf){ - return (Class) conf.getClass(GiraphJob.EDGE_VALUE_CLASS, - Writable.class); + /** + * Create a user vertex value + * + * @param Vertex data + * @param conf Configuration to check + * @return Instantiated user vertex value + */ + public static V + createVertexValue(Configuration conf) { + Class vertexValueClass = getVertexValueClass(conf); + try { + return vertexValueClass.newInstance(); + } catch (InstantiationException e) { + throw new IllegalArgumentException( + "createVertexValue: Failed to instantiate", e); + } catch (IllegalAccessException e) { + throw new IllegalArgumentException( + "createVertexValue: Illegally accessed", e); } + } - /** - * Create a user edge value - * - * @param conf Configuration to check - * @return Instantiated user edge value - */ - public static E - createEdgeValue(Configuration conf) { - Class edgeValueClass = getEdgeValueClass(conf); - try { - return edgeValueClass.newInstance(); - } catch (InstantiationException e) { - throw new IllegalArgumentException( - "createEdgeValue: Failed to instantiate", e); - } catch (IllegalAccessException e) { - throw new IllegalArgumentException( - "createEdgeValue: Illegally accessed", e); - } - } + /** + * Get the user's subclassed edge value class. + * + * @param Edge data + * @param conf Configuration to check + * @return User's vertex edge value class + */ + @SuppressWarnings("unchecked") + public static Class + getEdgeValueClass(Configuration conf) { + return (Class) conf.getClass(GiraphJob.EDGE_VALUE_CLASS, + Writable.class); + } - /** - * Get the user's subclassed vertex message value class. - * - * @param conf Configuration to check - * @return User's vertex message value class - */ - @SuppressWarnings("unchecked") - public static Class - getMessageValueClass(Configuration conf) { - return (Class) conf.getClass(GiraphJob.MESSAGE_VALUE_CLASS, - Writable.class); + /** + * Create a user edge value + * + * @param Edge data + * @param conf Configuration to check + * @return Instantiated user edge value + */ + public static E + createEdgeValue(Configuration conf) { + Class edgeValueClass = getEdgeValueClass(conf); + try { + return edgeValueClass.newInstance(); + } catch (InstantiationException e) { + throw new IllegalArgumentException( + "createEdgeValue: Failed to instantiate", e); + } catch (IllegalAccessException e) { + throw new IllegalArgumentException( + "createEdgeValue: Illegally accessed", e); } + } + + /** + * Get the user's subclassed vertex message value class. + * + * @param Message data + * @param conf Configuration to check + * @return User's vertex message value class + */ + @SuppressWarnings("unchecked") + public static Class + getMessageValueClass(Configuration conf) { + return (Class) conf.getClass(GiraphJob.MESSAGE_VALUE_CLASS, + Writable.class); + } - /** - * Create a user vertex message value - * - * @param conf Configuration to check - * @return Instantiated user vertex message value - */ - public static M - createMessageValue(Configuration conf) { - Class messageValueClass = getMessageValueClass(conf); - try { - return messageValueClass.newInstance(); - } catch (InstantiationException e) { - throw new IllegalArgumentException( - "createMessageValue: Failed to instantiate", e); - } catch (IllegalAccessException e) { - throw new IllegalArgumentException( - "createMessageValue: Illegally accessed", e); - } + /** + * Create a user vertex message value + * + * @param Message data + * @param conf Configuration to check + * @return Instantiated user vertex message value + */ + public static M + createMessageValue(Configuration conf) { + Class messageValueClass = getMessageValueClass(conf); + try { + return messageValueClass.newInstance(); + } catch (InstantiationException e) { + throw new IllegalArgumentException( + "createMessageValue: Failed to instantiate", e); + } catch (IllegalAccessException e) { + throw new IllegalArgumentException( + "createMessageValue: Illegally accessed", e); } + } } diff --git a/src/main/java/org/apache/giraph/graph/DefaultWorkerContext.java b/src/main/java/org/apache/giraph/graph/DefaultWorkerContext.java index 56360bfd7..b1042f7f9 100644 --- a/src/main/java/org/apache/giraph/graph/DefaultWorkerContext.java +++ b/src/main/java/org/apache/giraph/graph/DefaultWorkerContext.java @@ -25,17 +25,17 @@ */ public class DefaultWorkerContext extends WorkerContext { - @Override - public void preApplication() throws InstantiationException, - IllegalAccessException { - } + @Override + public void preApplication() + throws InstantiationException, IllegalAccessException { + } - @Override - public void postApplication() { } + @Override + public void postApplication() { } - @Override - public void preSuperstep() { } + @Override + public void preSuperstep() { } - @Override - public void postSuperstep() { } -} \ No newline at end of file + @Override + public void postSuperstep() { } +} diff --git a/src/main/java/org/apache/giraph/graph/Edge.java b/src/main/java/org/apache/giraph/graph/Edge.java index b276a8a3d..cd16822e7 100644 --- a/src/main/java/org/apache/giraph/graph/Edge.java +++ b/src/main/java/org/apache/giraph/graph/Edge.java @@ -36,133 +36,138 @@ */ @SuppressWarnings("rawtypes") public class Edge - implements WritableComparable>, Configurable { - /** Destination vertex id */ - private I destVertexId = null; - /** Edge value */ - private E edgeValue = null; - /** Configuration - Used to instantiate classes */ - private Configuration conf = null; - - /** - * Constructor for reflection - */ - public Edge() {} - - /** - * Create the edge with final values - * - * @param destVertexId - * @param edgeValue - */ - public Edge(I destVertexId, E edgeValue) { - this.destVertexId = destVertexId; - this.edgeValue = edgeValue; + implements WritableComparable>, Configurable { + /** Destination vertex id */ + private I destVertexId = null; + /** Edge value */ + private E edgeValue = null; + /** Configuration - Used to instantiate classes */ + private Configuration conf = null; + + /** + * Constructor for reflection + */ + public Edge() { } + + /** + * Create the edge with final values + * + * @param destVertexId Desination vertex id. + * @param edgeValue Value of the edge. + */ + public Edge(I destVertexId, E edgeValue) { + this.destVertexId = destVertexId; + this.edgeValue = edgeValue; + } + + /** + * Get the destination vertex index of this edge + * + * @return Destination vertex index of this edge + */ + public I getDestVertexId() { + return destVertexId; + } + + /** + * Get the edge value of the edge + * + * @return Edge value of this edge + */ + public E getEdgeValue() { + return edgeValue; + } + + /** + * Set the destination vertex index of this edge. + * + * @param destVertexId new destination vertex + */ + public void setDestVertexId(I destVertexId) { + this.destVertexId = destVertexId; + } + + /** + * Set the value for this edge. + * + * @param edgeValue new edge value + */ + public void setEdgeValue(E edgeValue) { + this.edgeValue = edgeValue; + } + + @Override + public String toString() { + return "(DestVertexIndex = " + destVertexId + + ", edgeValue = " + edgeValue + ")"; + } + + @SuppressWarnings("unchecked") + @Override + public void readFields(DataInput input) throws IOException { + destVertexId = (I) BspUtils.createVertexIndex(getConf()); + destVertexId.readFields(input); + edgeValue = (E) BspUtils.createEdgeValue(getConf()); + edgeValue.readFields(input); + } + + @Override + public void write(DataOutput output) throws IOException { + if (destVertexId == null) { + throw new IllegalStateException( + "write: Null destination vertex index"); } - - /** - * Get the destination vertex index of this edge - * - * @return Destination vertex index of this edge - */ - public I getDestVertexId() { - return destVertexId; - } - - /** - * Get the edge value of the edge - * - * @return Edge value of this edge - */ - public E getEdgeValue() { - return edgeValue; - } - - /** - * Set the destination vertex index of this edge. - * - * @param destVertexId new destination vertex - */ - public void setDestVertexId(I destVertexId) { - this.destVertexId = destVertexId; - } - - /** - * Set the value for this edge. - * - * @param edgeValue new edge value - */ - public void setEdgeValue(E edgeValue) { - this.edgeValue = edgeValue; - } - - @Override - public String toString() { - return "(DestVertexIndex = " + destVertexId + - ", edgeValue = " + edgeValue + ")"; + if (edgeValue == null) { + throw new IllegalStateException( + "write: Null edge value"); } - - @SuppressWarnings("unchecked") - @Override - public void readFields(DataInput input) throws IOException { - destVertexId = (I) BspUtils.createVertexIndex(getConf()); - destVertexId.readFields(input); - edgeValue = (E) BspUtils.createEdgeValue(getConf()); - edgeValue.readFields(input); + destVertexId.write(output); + edgeValue.write(output); + } + + @Override + public Configuration getConf() { + return conf; + } + + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } + + @SuppressWarnings("unchecked") + @Override + public int compareTo(Edge edge) { + return destVertexId.compareTo(edge.getDestVertexId()); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; } - - @Override - public void write(DataOutput output) throws IOException { - if (destVertexId == null) { - throw new IllegalStateException( - "write: Null destination vertex index"); - } - if (edgeValue == null) { - throw new IllegalStateException( - "write: Null edge value"); - } - destVertexId.write(output); - edgeValue.write(output); + if (o == null || getClass() != o.getClass()) { + return false; } - @Override - public Configuration getConf() { - return conf; - } + Edge edge = (Edge) o; - @Override - public void setConf(Configuration conf) { - this.conf = conf; + if (destVertexId != null ? !destVertexId.equals(edge.destVertexId) : + edge.destVertexId != null) { + return false; } - - @SuppressWarnings("unchecked") - @Override - public int compareTo(Edge edge) { - return destVertexId.compareTo(edge.getDestVertexId()); + if (edgeValue != null ? + !edgeValue.equals(edge.edgeValue) : edge.edgeValue != null) { + return false; } - @Override - public boolean equals(Object o) { - if (this == o) { return true; } - if (o == null || getClass() != o.getClass()) { return false; } - - Edge edge = (Edge) o; + return true; + } - if (destVertexId != null ? !destVertexId.equals(edge.destVertexId) : - edge.destVertexId != null) { - return false; - } - if (edgeValue != null ? !edgeValue.equals(edge.edgeValue) : edge.edgeValue != null) { - return false; - } - - return true; - } - - @Override - public int hashCode() { - int result = destVertexId != null ? destVertexId.hashCode() : 0; - result = 31 * result + (edgeValue != null ? edgeValue.hashCode() : 0); - return result; - } + @Override + public int hashCode() { + int result = destVertexId != null ? destVertexId.hashCode() : 0; + result = 31 * result + (edgeValue != null ? edgeValue.hashCode() : 0); + return result; + } } diff --git a/src/main/java/org/apache/giraph/graph/EdgeListVertex.java b/src/main/java/org/apache/giraph/graph/EdgeListVertex.java index 0e0d730ef..bfb40570e 100644 --- a/src/main/java/org/apache/giraph/graph/EdgeListVertex.java +++ b/src/main/java/org/apache/giraph/graph/EdgeListVertex.java @@ -48,265 +48,266 @@ */ @SuppressWarnings("rawtypes") public abstract class EdgeListVertex - extends MutableVertex { - /** Class logger */ - private static final Logger LOG = Logger.getLogger(EdgeListVertex.class); - /** Vertex id */ - private I vertexId = null; - /** Vertex value */ - private V vertexValue = null; - /** List of the dest edge indices */ - private List destEdgeIndexList; - /** List of the dest edge values */ - /** Map of destination vertices and their edge values */ - private List destEdgeValueList; - /** List of incoming messages from the previous superstep */ - private List msgList; + V extends Writable, E extends Writable, M extends Writable> + extends MutableVertex { + /** Class logger */ + private static final Logger LOG = Logger.getLogger(EdgeListVertex.class); + /** Vertex id */ + private I vertexId = null; + /** Vertex value */ + private V vertexValue = null; + /** List of the dest edge indices */ + private List destEdgeIndexList; + /** List of the dest edge values */ + private List destEdgeValueList; + /** List of incoming messages from the previous superstep */ + private List msgList; - @Override - public void initialize(I vertexId, V vertexValue, - Map edges, - Iterable messages) { - if (vertexId != null) { - setVertexId(vertexId); - } - if (vertexValue != null) { - setVertexValue(vertexValue); - } - if (edges != null && !edges.isEmpty()) { - destEdgeIndexList = Lists.newArrayListWithCapacity(edges.size()); - destEdgeValueList = Lists.newArrayListWithCapacity(edges.size()); - List sortedIndexList = new ArrayList(edges.keySet()); - Collections.sort(sortedIndexList, new VertexIdComparator()); - for (I index : sortedIndexList) { - destEdgeIndexList.add(index); - destEdgeValueList.add(edges.get(index)); - } - sortedIndexList.clear(); - } else { - destEdgeIndexList = Lists.newArrayListWithCapacity(0); - destEdgeValueList = Lists.newArrayListWithCapacity(0); - } - if (messages != null) { - msgList = Lists.newArrayListWithCapacity(Iterables.size(messages)); - Iterables.addAll(msgList, messages); - } else { - msgList = Lists.newArrayListWithCapacity(0); - } + @Override + public void initialize(I vertexId, V vertexValue, + Map edges, + Iterable messages) { + if (vertexId != null) { + setVertexId(vertexId); } - - @Override - public boolean equals(Object other) { - if (other instanceof EdgeListVertex) { - @SuppressWarnings("unchecked") - EdgeListVertex otherVertex = (EdgeListVertex) other; - if (!getVertexId().equals(otherVertex.getVertexId())) { - return false; - } - if (!getVertexValue().equals(otherVertex.getVertexValue())) { - return false; - } - if (!ComparisonUtils.equal(getMessages(), - otherVertex.getMessages())) { - return false; - } - return ComparisonUtils.equal(iterator(), otherVertex.iterator()); - } - return false; + if (vertexValue != null) { + setVertexValue(vertexValue); } - - /** - * Comparator for the vertex id - */ - private class VertexIdComparator implements Comparator { - @SuppressWarnings("unchecked") - @Override - public int compare(I index1, I index2) { - return index1.compareTo(index2); - } + if (edges != null && !edges.isEmpty()) { + destEdgeIndexList = Lists.newArrayListWithCapacity(edges.size()); + destEdgeValueList = Lists.newArrayListWithCapacity(edges.size()); + List sortedIndexList = new ArrayList(edges.keySet()); + Collections.sort(sortedIndexList, new VertexIdComparator()); + for (I index : sortedIndexList) { + destEdgeIndexList.add(index); + destEdgeValueList.add(edges.get(index)); + } + sortedIndexList.clear(); + } else { + destEdgeIndexList = Lists.newArrayListWithCapacity(0); + destEdgeValueList = Lists.newArrayListWithCapacity(0); } - - @Override - public final boolean addEdge(I targetVertexId, E edgeValue) { - System.out.println("addEdge: " + targetVertexId + " " + edgeValue + " " + destEdgeIndexList); - int pos = Collections.binarySearch(destEdgeIndexList, - targetVertexId, - new VertexIdComparator()); - if (pos < 0) { - destEdgeIndexList.add(-1 * (pos + 1), targetVertexId); - destEdgeValueList.add(-1 * (pos + 1), edgeValue); - return true; - } else { - LOG.warn("addEdge: Vertex=" + vertexId + - ": already added an edge value for dest vertex id " + - targetVertexId); - return false; - } + if (messages != null) { + msgList = Lists.newArrayListWithCapacity(Iterables.size(messages)); + Iterables.addAll(msgList, messages); + } else { + msgList = Lists.newArrayListWithCapacity(0); } + } - @Override - public long getSuperstep() { - return getGraphState().getSuperstep(); - } + @Override + public int hashCode() { + return vertexId.hashCode() * 37 + vertexValue.hashCode(); + } - @Override - public final void setVertexId(I vertexId) { - this.vertexId = vertexId; + @Override + public boolean equals(Object other) { + if (other instanceof EdgeListVertex) { + @SuppressWarnings("unchecked") + EdgeListVertex otherVertex = (EdgeListVertex) other; + if (!getVertexId().equals(otherVertex.getVertexId())) { + return false; + } + if (!getVertexValue().equals(otherVertex.getVertexValue())) { + return false; + } + if (!ComparisonUtils.equal(getMessages(), + otherVertex.getMessages())) { + return false; + } + return ComparisonUtils.equal(iterator(), otherVertex.iterator()); } + return false; + } + /** + * Comparator for the vertex id + */ + private class VertexIdComparator implements Comparator { + @SuppressWarnings("unchecked") @Override - public final I getVertexId() { - return vertexId; + public int compare(I index1, I index2) { + return index1.compareTo(index2); } + } - @Override - public final V getVertexValue() { - return vertexValue; + @Override + public final boolean addEdge(I targetVertexId, E edgeValue) { + int pos = Collections.binarySearch(destEdgeIndexList, + targetVertexId, + new VertexIdComparator()); + if (pos < 0) { + destEdgeIndexList.add(-1 * (pos + 1), targetVertexId); + destEdgeValueList.add(-1 * (pos + 1), edgeValue); + return true; + } else { + LOG.warn("addEdge: Vertex=" + vertexId + + ": already added an edge value for dest vertex id " + + targetVertexId); + return false; } + } - @Override - public final void setVertexValue(V vertexValue) { - this.vertexValue = vertexValue; - } + @Override + public long getSuperstep() { + return getGraphState().getSuperstep(); + } - @Override - public E getEdgeValue(I targetVertexId) { - int pos = Collections.binarySearch(destEdgeIndexList, - targetVertexId, - new VertexIdComparator()); - if (pos < 0) { - return null; - } else { - return destEdgeValueList.get(pos); - } - } + @Override + public final void setVertexId(I vertexId) { + this.vertexId = vertexId; + } - @Override - public boolean hasEdge(I targetVertexId) { - int pos = Collections.binarySearch(destEdgeIndexList, - targetVertexId, - new VertexIdComparator()); - if (pos < 0) { - return false; - } else { - return true; - } - } + @Override + public final I getVertexId() { + return vertexId; + } - /** - * Get an iterator to the edges on this vertex. - * - * @return A sorted iterator, as defined by the sort-order - * of the vertex ids - */ - @Override - public Iterator iterator() { - return destEdgeIndexList.iterator(); - } + @Override + public final V getVertexValue() { + return vertexValue; + } - @Override - public int getNumOutEdges() { - return destEdgeIndexList.size(); - } + @Override + public final void setVertexValue(V vertexValue) { + this.vertexValue = vertexValue; + } - @Override - public E removeEdge(I targetVertexId) { - int pos = Collections.binarySearch(destEdgeIndexList, - targetVertexId, - new VertexIdComparator()); - if (pos < 0) { - return null; - } else { - destEdgeIndexList.remove(pos); - return destEdgeValueList.remove(pos); - } + @Override + public E getEdgeValue(I targetVertexId) { + int pos = Collections.binarySearch(destEdgeIndexList, + targetVertexId, + new VertexIdComparator()); + if (pos < 0) { + return null; + } else { + return destEdgeValueList.get(pos); } + } - @Override - public final void sendMsgToAllEdges(M msg) { - if (msg == null) { - throw new IllegalArgumentException( - "sendMsgToAllEdges: Cannot send null message to all edges"); - } - for (I index : destEdgeIndexList) { - sendMsg(index, msg); - } + @Override + public boolean hasEdge(I targetVertexId) { + int pos = Collections.binarySearch(destEdgeIndexList, + targetVertexId, + new VertexIdComparator()); + if (pos < 0) { + return false; } + return true; + } - @Override - final public void readFields(DataInput in) throws IOException { - vertexId = BspUtils.createVertexIndex(getConf()); - vertexId.readFields(in); - boolean hasVertexValue = in.readBoolean(); - if (hasVertexValue) { - vertexValue = BspUtils.createVertexValue(getConf()); - vertexValue.readFields(in); - } - int edgeListCount = in.readInt(); - destEdgeIndexList = Lists.newArrayListWithCapacity(edgeListCount); - destEdgeValueList = Lists.newArrayListWithCapacity(edgeListCount); - for (int i = 0; i < edgeListCount; ++i) { - I vertexId = BspUtils.createVertexIndex(getConf()); - E edgeValue = BspUtils.createEdgeValue(getConf()); - vertexId.readFields(in); - edgeValue.readFields(in); - destEdgeIndexList.add(vertexId); - destEdgeValueList.add(edgeValue); - } - int msgListSize = in.readInt(); - msgList = Lists.newArrayListWithCapacity(msgListSize); - for (int i = 0; i < msgListSize; ++i) { - M msg = BspUtils.createMessageValue(getConf()); - msg.readFields(in); - msgList.add(msg); - } - halt = in.readBoolean(); - } + /** + * Get an iterator to the edges on this vertex. + * + * @return A sorted iterator, as defined by the sort-order + * of the vertex ids + */ + @Override + public Iterator iterator() { + return destEdgeIndexList.iterator(); + } - @Override - final public void write(DataOutput out) throws IOException { - vertexId.write(out); - out.writeBoolean(vertexValue != null); - if (vertexValue != null) { - vertexValue.write(out); - } - out.writeInt(destEdgeIndexList.size()); - for (int i = 0 ; i < destEdgeIndexList.size(); ++i) { - destEdgeIndexList.get(i).write(out); - destEdgeValueList.get(i).write(out); - } - out.writeInt(msgList.size()); - for (M msg : msgList) { - msg.write(out); - } - out.writeBoolean(halt); + @Override + public int getNumOutEdges() { + return destEdgeIndexList.size(); + } + + @Override + public E removeEdge(I targetVertexId) { + int pos = Collections.binarySearch(destEdgeIndexList, + targetVertexId, + new VertexIdComparator()); + if (pos < 0) { + return null; + } else { + destEdgeIndexList.remove(pos); + return destEdgeValueList.remove(pos); } + } - @Override - void putMessages(Iterable messages) { - msgList.clear(); - for (M message : messages) { - msgList.add(message); - } + @Override + public final void sendMsgToAllEdges(M msg) { + if (msg == null) { + throw new IllegalArgumentException( + "sendMsgToAllEdges: Cannot send null message to all edges"); + } + for (I index : destEdgeIndexList) { + sendMsg(index, msg); } + } - @Override - public Iterable getMessages() { - return Iterables.unmodifiableIterable(msgList); + @Override + public final void readFields(DataInput in) throws IOException { + vertexId = BspUtils.createVertexIndex(getConf()); + vertexId.readFields(in); + boolean hasVertexValue = in.readBoolean(); + if (hasVertexValue) { + vertexValue = BspUtils.createVertexValue(getConf()); + vertexValue.readFields(in); + } + int edgeListCount = in.readInt(); + destEdgeIndexList = Lists.newArrayListWithCapacity(edgeListCount); + destEdgeValueList = Lists.newArrayListWithCapacity(edgeListCount); + for (int i = 0; i < edgeListCount; ++i) { + I destVertexId = BspUtils.createVertexIndex(getConf()); + E edgeValue = BspUtils.createEdgeValue(getConf()); + destVertexId.readFields(in); + edgeValue.readFields(in); + destEdgeIndexList.add(destVertexId); + destEdgeValueList.add(edgeValue); + } + int msgListSize = in.readInt(); + msgList = Lists.newArrayListWithCapacity(msgListSize); + for (int i = 0; i < msgListSize; ++i) { + M msg = BspUtils.createMessageValue(getConf()); + msg.readFields(in); + msgList.add(msg); } + halt = in.readBoolean(); + } - @Override - void releaseResources() { - // Hint to GC to free the messages - msgList.clear(); + @Override + public final void write(DataOutput out) throws IOException { + vertexId.write(out); + out.writeBoolean(vertexValue != null); + if (vertexValue != null) { + vertexValue.write(out); + } + out.writeInt(destEdgeIndexList.size()); + for (int i = 0; i < destEdgeIndexList.size(); ++i) { + destEdgeIndexList.get(i).write(out); + destEdgeValueList.get(i).write(out); } + out.writeInt(msgList.size()); + for (M msg : msgList) { + msg.write(out); + } + out.writeBoolean(halt); + } - @Override - public String toString() { - return "Vertex(id=" + getVertexId() + ",value=" + getVertexValue() + - ",#edges=" + getNumOutEdges() + ")"; + @Override + void putMessages(Iterable messages) { + msgList.clear(); + for (M message : messages) { + msgList.add(message); } + } + + @Override + public Iterable getMessages() { + return Iterables.unmodifiableIterable(msgList); + } + + @Override + void releaseResources() { + // Hint to GC to free the messages + msgList.clear(); + } + + @Override + public String toString() { + return "Vertex(id=" + getVertexId() + ",value=" + getVertexValue() + + ",#edges=" + getNumOutEdges() + ")"; + } } diff --git a/src/main/java/org/apache/giraph/graph/GiraphJob.java b/src/main/java/org/apache/giraph/graph/GiraphJob.java index 8ddf102d9..b829ed0ea 100644 --- a/src/main/java/org/apache/giraph/graph/GiraphJob.java +++ b/src/main/java/org/apache/giraph/graph/GiraphJob.java @@ -33,570 +33,571 @@ * for our needs. For instance, our job should not have any reduce tasks. */ public class GiraphJob extends Job { - static { - Configuration.addDefaultResource("giraph-site.xml"); + static { + Configuration.addDefaultResource("giraph-site.xml"); + } + + /** Vertex class - required */ + public static final String VERTEX_CLASS = "giraph.vertexClass"; + /** VertexInputFormat class - required */ + public static final String VERTEX_INPUT_FORMAT_CLASS = + "giraph.vertexInputFormatClass"; + + /** VertexOutputFormat class - optional */ + public static final String VERTEX_OUTPUT_FORMAT_CLASS = + "giraph.vertexOutputFormatClass"; + /** Vertex combiner class - optional */ + public static final String VERTEX_COMBINER_CLASS = + "giraph.combinerClass"; + /** Vertex resolver class - optional */ + public static final String VERTEX_RESOLVER_CLASS = + "giraph.vertexResolverClass"; + /** Graph partitioner factory class - optional */ + public static final String GRAPH_PARTITIONER_FACTORY_CLASS = + "giraph.graphPartitionerFactoryClass"; + + /** Vertex index class */ + public static final String VERTEX_INDEX_CLASS = "giraph.vertexIndexClass"; + /** Vertex value class */ + public static final String VERTEX_VALUE_CLASS = "giraph.vertexValueClass"; + /** Edge value class */ + public static final String EDGE_VALUE_CLASS = "giraph.edgeValueClass"; + /** Message value class */ + public static final String MESSAGE_VALUE_CLASS = "giraph.messageValueClass"; + /** Worker context class */ + public static final String WORKER_CONTEXT_CLASS = + "giraph.workerContextClass"; + /** AggregatorWriter class - optional */ + public static final String AGGREGATOR_WRITER_CLASS = + "giraph.aggregatorWriterClass"; + + /** + * Minimum number of simultaneous workers before this job can run (int) + */ + public static final String MIN_WORKERS = "giraph.minWorkers"; + /** + * Maximum number of simultaneous worker tasks started by this job (int). + */ + public static final String MAX_WORKERS = "giraph.maxWorkers"; + + /** + * Separate the workers and the master tasks. This is required + * to support dynamic recovery. (boolean) + */ + public static final String SPLIT_MASTER_WORKER = + "giraph.SplitMasterWorker"; + /** + * Default on whether to separate the workers and the master tasks. + * Needs to be "true" to support dynamic recovery. + */ + public static final boolean SPLIT_MASTER_WORKER_DEFAULT = true; + + /** Indicates whether this job is run in an internal unit test */ + public static final String LOCAL_TEST_MODE = + "giraph.localTestMode"; + + /** not in local test mode per default */ + public static final boolean LOCAL_TEST_MODE_DEFAULT = false; + + /** + * Minimum percent of the maximum number of workers that have responded + * in order to continue progressing. (float) + */ + public static final String MIN_PERCENT_RESPONDED = + "giraph.minPercentResponded"; + /** Default 100% response rate for workers */ + public static final float MIN_PERCENT_RESPONDED_DEFAULT = 100.0f; + + /** Polling timeout to check on the number of responded tasks (int) */ + public static final String POLL_MSECS = "giraph.pollMsecs"; + /** Default poll msecs (30 seconds) */ + public static final int POLL_MSECS_DEFAULT = 30 * 1000; + + /** + * ZooKeeper comma-separated list (if not set, + * will start up ZooKeeper locally) + */ + public static final String ZOOKEEPER_LIST = "giraph.zkList"; + + /** ZooKeeper session millisecond timeout */ + public static final String ZOOKEEPER_SESSION_TIMEOUT = + "giraph.zkSessionMsecTimeout"; + /** Default Zookeeper session millisecond timeout */ + public static final int ZOOKEEPER_SESSION_TIMEOUT_DEFAULT = 60 * 1000; + + /** Polling interval to check for the final ZooKeeper server data */ + public static final String ZOOKEEPER_SERVERLIST_POLL_MSECS = + "giraph.zkServerlistPollMsecs"; + /** Default polling interval to check for the final ZooKeeper server data */ + public static final int ZOOKEEPER_SERVERLIST_POLL_MSECS_DEFAULT = + 3 * 1000; + + /** Number of nodes (not tasks) to run Zookeeper on */ + public static final String ZOOKEEPER_SERVER_COUNT = + "giraph.zkServerCount"; + /** Default number of nodes to run Zookeeper on */ + public static final int ZOOKEEPER_SERVER_COUNT_DEFAULT = 1; + + /** ZooKeeper port to use */ + public static final String ZOOKEEPER_SERVER_PORT = + "giraph.zkServerPort"; + /** Default ZooKeeper port to use */ + public static final int ZOOKEEPER_SERVER_PORT_DEFAULT = 22181; + + /** Location of the ZooKeeper jar - Used internally, not meant for users */ + public static final String ZOOKEEPER_JAR = "giraph.zkJar"; + + /** Local ZooKeeper directory to use */ + public static final String ZOOKEEPER_DIR = "giraph.zkDir"; + + /** Initial port to start using for the RPC communication */ + public static final String RPC_INITIAL_PORT = "giraph.rpcInitialPort"; + /** Default port to start using for the RPC communication */ + public static final int RPC_INITIAL_PORT_DEFAULT = 30000; + + /** Maximum bind attempts for different RPC ports */ + public static final String MAX_RPC_PORT_BIND_ATTEMPTS = + "giraph.maxRpcPortBindAttempts"; + /** Default maximum bind attempts for different RPC ports */ + public static final int MAX_RPC_PORT_BIND_ATTEMPTS_DEFAULT = 20; + + /** Maximum number of RPC handlers */ + public static final String RPC_NUM_HANDLERS = "giraph.rpcNumHandlers"; + /** Default maximum number of RPC handlers */ + public static final int RPC_NUM_HANDLERS_DEFAULT = 100; + + /** + * Maximum number of vertices per partition before sending. + * (input superstep only). + */ + public static final String MAX_VERTICES_PER_PARTITION = + "giraph.maxVerticesPerPartition"; + /** Default maximum number of vertices per partition before sending. */ + public static final int MAX_VERTICES_PER_PARTITION_DEFAULT = 100000; + + /** Maximum number of messages per peer before flush */ + public static final String MSG_SIZE = "giraph.msgSize"; + /** Default maximum number of messages per peer before flush */ + public static final int MSG_SIZE_DEFAULT = 1000; + + /** Maximum number of messages that can be bulk sent during a flush */ + public static final String MAX_MESSAGES_PER_FLUSH_PUT = + "giraph.maxMessagesPerFlushPut"; + /** Default number of messages that can be bulk sent during a flush */ + public static final int DEFAULT_MAX_MESSAGES_PER_FLUSH_PUT = 5000; + + /** Number of flush threads per peer */ + public static final String MSG_NUM_FLUSH_THREADS = + "giraph.msgNumFlushThreads"; + + /** Number of poll attempts prior to failing the job (int) */ + public static final String POLL_ATTEMPTS = "giraph.pollAttempts"; + /** Default poll attempts */ + public static final int POLL_ATTEMPTS_DEFAULT = 10; + + /** Number of minimum vertices in each vertex range */ + public static final String MIN_VERTICES_PER_RANGE = + "giraph.minVerticesPerRange"; + /** Default number of minimum vertices in each vertex range */ + public static final long MIN_VERTICES_PER_RANGE_DEFAULT = 3; + + /** Minimum stragglers of the superstep before printing them out */ + public static final String PARTITION_LONG_TAIL_MIN_PRINT = + "giraph.partitionLongTailMinPrint"; + /** Only print stragglers with one as a default */ + public static final int PARTITION_LONG_TAIL_MIN_PRINT_DEFAULT = 1; + + /** Use superstep counters? (boolean) */ + public static final String USE_SUPERSTEP_COUNTERS = + "giraph.useSuperstepCounters"; + /** Default is to use the superstep counters */ + public static final boolean USE_SUPERSTEP_COUNTERS_DEFAULT = true; + + /** + * Set the multiplicative factor of how many partitions to create from + * a single InputSplit based on the number of total InputSplits. For + * example, if there are 10 total InputSplits and this is set to 0.5, then + * you will get 0.5 * 10 = 5 partitions for every InputSplit (given that the + * minimum size is met). + */ + public static final String TOTAL_INPUT_SPLIT_MULTIPLIER = + "giraph.totalInputSplitMultiplier"; + /** Default total input split multiplier */ + public static final float TOTAL_INPUT_SPLIT_MULTIPLIER_DEFAULT = 0.5f; + + /** + * Input split sample percent - Used only for sampling and testing, rather + * than an actual job. The idea is that to test, you might only want a + * fraction of the actual input splits from your VertexInputFormat to + * load (values should be [0, 100]). + */ + public static final String INPUT_SPLIT_SAMPLE_PERCENT = + "giraph.inputSplitSamplePercent"; + /** Default is to use all the input splits */ + public static final float INPUT_SPLIT_SAMPLE_PERCENT_DEFAULT = 100f; + + /** + * To limit outlier input splits from producing too many vertices or to + * help with testing, the number of vertices loaded from an input split can + * be limited. By default, everything is loaded. + */ + public static final String INPUT_SPLIT_MAX_VERTICES = + "giraph.InputSplitMaxVertices"; + /** + * Default is that all the vertices are to be loaded from the input + * split + */ + public static final long INPUT_SPLIT_MAX_VERTICES_DEFAULT = -1; + + /** Java opts passed to ZooKeeper startup */ + public static final String ZOOKEEPER_JAVA_OPTS = + "giraph.zkJavaOpts"; + /** Default java opts passed to ZooKeeper startup */ + public static final String ZOOKEEPER_JAVA_OPTS_DEFAULT = + "-Xmx512m -XX:ParallelGCThreads=4 -XX:+UseConcMarkSweepGC " + + "-XX:CMSInitiatingOccupancyFraction=70 -XX:MaxGCPauseMillis=100"; + + /** + * How often to checkpoint (i.e. 0, means no checkpoint, + * 1 means every superstep, 2 is every two supersteps, etc.). + */ + public static final String CHECKPOINT_FREQUENCY = + "giraph.checkpointFrequency"; + + /** Default checkpointing frequency of every 2 supersteps. */ + public static final int CHECKPOINT_FREQUENCY_DEFAULT = 2; + + /** + * Delete checkpoints after a successful job run? + */ + public static final String CLEANUP_CHECKPOINTS_AFTER_SUCCESS = + "giraph.cleanupCheckpointsAfterSuccess"; + /** Default is to clean up the checkponts after a successful job */ + public static final boolean CLEANUP_CHECKPOINTS_AFTER_SUCCESS_DEFAULT = + true; + + /** + * An application can be restarted manually by selecting a superstep. The + * corresponding checkpoint must exist for this to work. The user should + * set a long value. Default is start from scratch. + */ + public static final String RESTART_SUPERSTEP = "giraph.restartSuperstep"; + + /** + * Base ZNode for Giraph's state in the ZooKeeper cluster. Must be a root + * znode on the cluster beginning with "/" + */ + public static final String BASE_ZNODE_KEY = "giraph.zkBaseZNode"; + + /** + * If ZOOKEEPER_LIST is not set, then use this directory to manage + * ZooKeeper + */ + public static final String ZOOKEEPER_MANAGER_DIRECTORY = + "giraph.zkManagerDirectory"; + /** + * Default ZooKeeper manager directory (where determining the servers in + * HDFS files will go). Final directory path will also have job number + * for uniqueness. + */ + public static final String ZOOKEEPER_MANAGER_DIR_DEFAULT = + "_bsp/_defaultZkManagerDir"; + + /** This directory has/stores the available checkpoint files in HDFS. */ + public static final String CHECKPOINT_DIRECTORY = + "giraph.checkpointDirectory"; + /** + * Default checkpoint directory (where checkpoing files go in HDFS). Final + * directory path will also have the job number for uniqueness + */ + public static final String CHECKPOINT_DIRECTORY_DEFAULT = + "_bsp/_checkpoints/"; + + /** Keep the zookeeper output for debugging? Default is to remove it. */ + public static final String KEEP_ZOOKEEPER_DATA = + "giraph.keepZooKeeperData"; + /** Default is to remove ZooKeeper data. */ + public static final Boolean KEEP_ZOOKEEPER_DATA_DEFAULT = false; + + /** Default ZooKeeper tick time. */ + public static final int DEFAULT_ZOOKEEPER_TICK_TIME = 6000; + /** Default ZooKeeper init limit (in ticks). */ + public static final int DEFAULT_ZOOKEEPER_INIT_LIMIT = 10; + /** Default ZooKeeper sync limit (in ticks). */ + public static final int DEFAULT_ZOOKEEPER_SYNC_LIMIT = 5; + /** Default ZooKeeper snap count. */ + public static final int DEFAULT_ZOOKEEPER_SNAP_COUNT = 50000; + /** Default ZooKeeper maximum client connections. */ + public static final int DEFAULT_ZOOKEEPER_MAX_CLIENT_CNXNS = 10000; + /** Default ZooKeeper minimum session timeout of 5 minutes (in msecs). */ + public static final int DEFAULT_ZOOKEEPER_MIN_SESSION_TIMEOUT = 300 * 1000; + /** Default ZooKeeper maximum session timeout of 10 minutes (in msecs). */ + public static final int DEFAULT_ZOOKEEPER_MAX_SESSION_TIMEOUT = 600 * 1000; + + /** Class logger */ + private static final Logger LOG = Logger.getLogger(GiraphJob.class); + + /** + * Constructor that will instantiate the configuration + * + * @param jobName User-defined job name + * @throws IOException + */ + public GiraphJob(String jobName) throws IOException { + super(new Configuration(), jobName); + } + + /** + * Constructor. + * + * @param conf User-defined configuration + * @param jobName User-defined job name + * @throws IOException + */ + public GiraphJob(Configuration conf, String jobName) throws IOException { + super(conf, jobName); + } + + /** + * Make sure the configuration is set properly by the user prior to + * submitting the job. + */ + private void checkConfiguration() { + if (conf.getInt(MAX_WORKERS, -1) < 0) { + throw new RuntimeException("No valid " + MAX_WORKERS); } - - /** Vertex class - required */ - public static final String VERTEX_CLASS = "giraph.vertexClass"; - /** VertexInputFormat class - required */ - public static final String VERTEX_INPUT_FORMAT_CLASS = - "giraph.vertexInputFormatClass"; - - /** VertexOutputFormat class - optional */ - public static final String VERTEX_OUTPUT_FORMAT_CLASS = - "giraph.vertexOutputFormatClass"; - /** Vertex combiner class - optional */ - public static final String VERTEX_COMBINER_CLASS = - "giraph.combinerClass"; - /** Vertex resolver class - optional */ - public static final String VERTEX_RESOLVER_CLASS = - "giraph.vertexResolverClass"; - /** Graph partitioner factory class - optional */ - public static final String GRAPH_PARTITIONER_FACTORY_CLASS = - "giraph.graphPartitionerFactoryClass"; - - /** Vertex index class */ - public static final String VERTEX_INDEX_CLASS = "giraph.vertexIndexClass"; - /** Vertex value class */ - public static final String VERTEX_VALUE_CLASS = "giraph.vertexValueClass"; - /** Edge value class */ - public static final String EDGE_VALUE_CLASS = "giraph.edgeValueClass"; - /** Message value class */ - public static final String MESSAGE_VALUE_CLASS = "giraph.messageValueClass"; - /** Worker context class */ - public static final String WORKER_CONTEXT_CLASS = - "giraph.workerContextClass"; - /** AggregatorWriter class - optional */ - public static final String AGGREGATOR_WRITER_CLASS = - "giraph.aggregatorWriterClass"; - - /** - * Minimum number of simultaneous workers before this job can run (int) - */ - public static final String MIN_WORKERS = "giraph.minWorkers"; - /** - * Maximum number of simultaneous worker tasks started by this job (int). - */ - public static final String MAX_WORKERS = "giraph.maxWorkers"; - - /** - * Separate the workers and the master tasks. This is required - * to support dynamic recovery. (boolean) - */ - public static final String SPLIT_MASTER_WORKER = - "giraph.SplitMasterWorker"; - /** - * Default on whether to separate the workers and the master tasks. - * Needs to be "true" to support dynamic recovery. - */ - public static final boolean SPLIT_MASTER_WORKER_DEFAULT = true; - - /** Indicates whether this job is run in an internal unit test */ - public static final String LOCAL_TEST_MODE = - "giraph.localTestMode"; - - /** not in local test mode per default */ - public static final boolean LOCAL_TEST_MODE_DEFAULT = false; - - /** - * Minimum percent of the maximum number of workers that have responded - * in order to continue progressing. (float) - */ - public static final String MIN_PERCENT_RESPONDED = - "giraph.minPercentResponded"; - /** Default 100% response rate for workers */ - public static final float MIN_PERCENT_RESPONDED_DEFAULT = 100.0f; - - /** Polling timeout to check on the number of responded tasks (int) */ - public static final String POLL_MSECS = "giraph.pollMsecs"; - /** Default poll msecs (30 seconds) */ - public static final int POLL_MSECS_DEFAULT = 30*1000; - - /** - * ZooKeeper comma-separated list (if not set, - * will start up ZooKeeper locally) - */ - public static final String ZOOKEEPER_LIST = "giraph.zkList"; - - /** ZooKeeper session millisecond timeout */ - public static final String ZOOKEEPER_SESSION_TIMEOUT = - "giraph.zkSessionMsecTimeout"; - /** Default Zookeeper session millisecond timeout */ - public static final int ZOOKEEPER_SESSION_TIMEOUT_DEFAULT = 60*1000; - - /** Polling interval to check for the final ZooKeeper server data */ - public static final String ZOOKEEPER_SERVERLIST_POLL_MSECS = - "giraph.zkServerlistPollMsecs"; - /** Default polling interval to check for the final ZooKeeper server data */ - public static final int ZOOKEEPER_SERVERLIST_POLL_MSECS_DEFAULT = - 3*1000; - - /** Number of nodes (not tasks) to run Zookeeper on */ - public static final String ZOOKEEPER_SERVER_COUNT = - "giraph.zkServerCount"; - /** Default number of nodes to run Zookeeper on */ - public static final int ZOOKEEPER_SERVER_COUNT_DEFAULT = 1; - - /** ZooKeeper port to use */ - public static final String ZOOKEEPER_SERVER_PORT = - "giraph.zkServerPort"; - /** Default ZooKeeper port to use */ - public static final int ZOOKEEPER_SERVER_PORT_DEFAULT = 22181; - - /** Location of the ZooKeeper jar - Used internally, not meant for users */ - public static final String ZOOKEEPER_JAR = "giraph.zkJar"; - - /** Local ZooKeeper directory to use */ - public static final String ZOOKEEPER_DIR = "giraph.zkDir"; - - /** Initial port to start using for the RPC communication */ - public static final String RPC_INITIAL_PORT = "giraph.rpcInitialPort"; - /** Default port to start using for the RPC communication */ - public static final int RPC_INITIAL_PORT_DEFAULT = 30000; - - /** Maximum bind attempts for different RPC ports */ - public static final String MAX_RPC_PORT_BIND_ATTEMPTS = - "giraph.maxRpcPortBindAttempts"; - /** Default maximum bind attempts for different RPC ports */ - public static final int MAX_RPC_PORT_BIND_ATTEMPTS_DEFAULT = 20; - - /** Maximum number of RPC handlers */ - public static final String RPC_NUM_HANDLERS = "giraph.rpcNumHandlers"; - /** Default maximum number of RPC handlers */ - public static final int RPC_NUM_HANDLERS_DEFAULT = 100; - - /** - * Maximum number of vertices per partition before sending. - * (input superstep only). - */ - public static final String MAX_VERTICES_PER_PARTITION = - "giraph.maxVerticesPerPartition"; - /** Default maximum number of vertices per partition before sending. */ - public static final int MAX_VERTICES_PER_PARTITION_DEFAULT = 100000; - - /** Maximum number of messages per peer before flush */ - public static final String MSG_SIZE = "giraph.msgSize"; - /** Default maximum number of messages per peer before flush */ - public static final int MSG_SIZE_DEFAULT = 1000; - - /** Maximum number of messages that can be bulk sent during a flush */ - public static final String MAX_MESSAGES_PER_FLUSH_PUT = - "giraph.maxMessagesPerFlushPut"; - /** Default number of messages that can be bulk sent during a flush */ - public static final int DEFAULT_MAX_MESSAGES_PER_FLUSH_PUT = 5000; - - /** Number of flush threads per peer */ - public static final String MSG_NUM_FLUSH_THREADS = - "giraph.msgNumFlushThreads"; - - /** Number of poll attempts prior to failing the job (int) */ - public static final String POLL_ATTEMPTS = "giraph.pollAttempts"; - /** Default poll attempts */ - public static final int POLL_ATTEMPTS_DEFAULT = 10; - - /** Number of minimum vertices in each vertex range */ - public static final String MIN_VERTICES_PER_RANGE = - "giraph.minVerticesPerRange"; - /** Default number of minimum vertices in each vertex range */ - public static final long MIN_VERTICES_PER_RANGE_DEFAULT = 3; - - /** Minimum stragglers of the superstep before printing them out */ - public static final String PARTITION_LONG_TAIL_MIN_PRINT = - "giraph.partitionLongTailMinPrint"; - /** Only print stragglers with one as a default */ - public static final int PARTITION_LONG_TAIL_MIN_PRINT_DEFAULT = 1; - - /** Use superstep counters? (boolean) */ - public static final String USE_SUPERSTEP_COUNTERS = - "giraph.useSuperstepCounters"; - /** Default is to use the superstep counters */ - public static final boolean USE_SUPERSTEP_COUNTERS_DEFAULT = true; - - /** - * Set the multiplicative factor of how many partitions to create from - * a single InputSplit based on the number of total InputSplits. For - * example, if there are 10 total InputSplits and this is set to 0.5, then - * you will get 0.5 * 10 = 5 partitions for every InputSplit (given that the - * minimum size is met). - */ - public static final String TOTAL_INPUT_SPLIT_MULTIPLIER = - "giraph.totalInputSplitMultiplier"; - /** Default total input split multiplier */ - public static final float TOTAL_INPUT_SPLIT_MULTIPLIER_DEFAULT = 0.5f; - - /** - * Input split sample percent - Used only for sampling and testing, rather - * than an actual job. The idea is that to test, you might only want a - * fraction of the actual input splits from your VertexInputFormat to - * load (values should be [0, 100]). - */ - public static final String INPUT_SPLIT_SAMPLE_PERCENT = - "giraph.inputSplitSamplePercent"; - /** Default is to use all the input splits */ - public static final float INPUT_SPLIT_SAMPLE_PERCENT_DEFAULT = 100f; - - /** - * To limit outlier input splits from producing too many vertices or to - * help with testing, the number of vertices loaded from an input split can - * be limited. By default, everything is loaded. - */ - public static final String INPUT_SPLIT_MAX_VERTICES = - "giraph.InputSplitMaxVertices"; - /** - * Default is that all the vertices are to be loaded from the input - * split - */ - public static final long INPUT_SPLIT_MAX_VERTICES_DEFAULT = -1; - - /** Java opts passed to ZooKeeper startup */ - public static final String ZOOKEEPER_JAVA_OPTS = - "giraph.zkJavaOpts"; - /** Default java opts passed to ZooKeeper startup */ - public static final String ZOOKEEPER_JAVA_OPTS_DEFAULT = - "-Xmx512m -XX:ParallelGCThreads=4 -XX:+UseConcMarkSweepGC " + - "-XX:CMSInitiatingOccupancyFraction=70 -XX:MaxGCPauseMillis=100"; - - /** - * How often to checkpoint (i.e. 0, means no checkpoint, - * 1 means every superstep, 2 is every two supersteps, etc.). - */ - public static final String CHECKPOINT_FREQUENCY = - "giraph.checkpointFrequency"; - - /** Default checkpointing frequency of every 2 supersteps. */ - public static final int CHECKPOINT_FREQUENCY_DEFAULT = 2; - - /** - * Delete checkpoints after a successful job run? - */ - public static final String CLEANUP_CHECKPOINTS_AFTER_SUCCESS = - "giraph.cleanupCheckpointsAfterSuccess"; - /** Default is to clean up the checkponts after a successful job */ - public static final boolean CLEANUP_CHECKPOINTS_AFTER_SUCCESS_DEFAULT = - true; - - /** - * An application can be restarted manually by selecting a superstep. The - * corresponding checkpoint must exist for this to work. The user should - * set a long value. Default is start from scratch. - */ - public static final String RESTART_SUPERSTEP = "giraph.restartSuperstep"; - - /** - * Base ZNode for Giraph's state in the ZooKeeper cluster. Must be a root - * znode on the cluster beginning with "/" - */ - public static final String BASE_ZNODE_KEY = "giraph.zkBaseZNode"; - - /** - * If ZOOKEEPER_LIST is not set, then use this directory to manage - * ZooKeeper - */ - public static final String ZOOKEEPER_MANAGER_DIRECTORY = - "giraph.zkManagerDirectory"; - /** - * Default ZooKeeper manager directory (where determining the servers in - * HDFS files will go). Final directory path will also have job number - * for uniqueness. - */ - public static final String ZOOKEEPER_MANAGER_DIR_DEFAULT = - "_bsp/_defaultZkManagerDir"; - - /** This directory has/stores the available checkpoint files in HDFS. */ - public static final String CHECKPOINT_DIRECTORY = - "giraph.checkpointDirectory"; - /** - * Default checkpoint directory (where checkpoing files go in HDFS). Final - * directory path will also have the job number for uniqueness - */ - public static final String CHECKPOINT_DIRECTORY_DEFAULT = - "_bsp/_checkpoints/"; - - /** Keep the zookeeper output for debugging? Default is to remove it. */ - public static final String KEEP_ZOOKEEPER_DATA = - "giraph.keepZooKeeperData"; - /** Default is to remove ZooKeeper data. */ - public static final Boolean KEEP_ZOOKEEPER_DATA_DEFAULT = false; - - /** Default ZooKeeper tick time. */ - public static final int DEFAULT_ZOOKEEPER_TICK_TIME = 6000; - /** Default ZooKeeper init limit (in ticks). */ - public static final int DEFAULT_ZOOKEEPER_INIT_LIMIT = 10; - /** Default ZooKeeper sync limit (in ticks). */ - public static final int DEFAULT_ZOOKEEPER_SYNC_LIMIT = 5; - /** Default ZooKeeper snap count. */ - public static final int DEFAULT_ZOOKEEPER_SNAP_COUNT = 50000; - /** Default ZooKeeper maximum client connections. */ - public static final int DEFAULT_ZOOKEEPER_MAX_CLIENT_CNXNS = 10000; - /** Default ZooKeeper minimum session timeout of 5 minutes (in msecs). */ - public static final int DEFAULT_ZOOKEEPER_MIN_SESSION_TIMEOUT = 300*1000; - /** Default ZooKeeper maximum session timeout of 10 minutes (in msecs). */ - public static final int DEFAULT_ZOOKEEPER_MAX_SESSION_TIMEOUT = 600*1000; - - /** Class logger */ - private static final Logger LOG = Logger.getLogger(GiraphJob.class); - - /** - * Constructor that will instantiate the configuration - * - * @param jobName User-defined job name - * @throws IOException - */ - public GiraphJob(String jobName) throws IOException { - super(new Configuration(), jobName); + if (conf.getFloat(MIN_PERCENT_RESPONDED, + MIN_PERCENT_RESPONDED_DEFAULT) <= 0.0f || + conf.getFloat(MIN_PERCENT_RESPONDED, + MIN_PERCENT_RESPONDED_DEFAULT) > 100.0f) { + throw new IllegalArgumentException( + "Invalid " + + conf.getFloat(MIN_PERCENT_RESPONDED, + MIN_PERCENT_RESPONDED_DEFAULT) + " for " + + MIN_PERCENT_RESPONDED); } - - /** - * Constructor. - * - * @param conf User-defined configuration - * @param jobName User-defined job name - * @throws IOException - */ - public GiraphJob(Configuration conf, String jobName) throws IOException { - super(conf, jobName); + if (conf.getInt(MIN_WORKERS, -1) < 0) { + throw new IllegalArgumentException("No valid " + MIN_WORKERS); } - - /** - * Make sure the configuration is set properly by the user prior to - * submitting the job. - */ - private void checkConfiguration() { - if (conf.getInt(MAX_WORKERS, -1) < 0) { - throw new RuntimeException("No valid " + MAX_WORKERS); - } - if (conf.getFloat(MIN_PERCENT_RESPONDED, - MIN_PERCENT_RESPONDED_DEFAULT) <= 0.0f || - conf.getFloat(MIN_PERCENT_RESPONDED, - MIN_PERCENT_RESPONDED_DEFAULT) > 100.0f) { - throw new IllegalArgumentException( - "Invalid " + - conf.getFloat(MIN_PERCENT_RESPONDED, - MIN_PERCENT_RESPONDED_DEFAULT) + " for " + - MIN_PERCENT_RESPONDED); - } - if (conf.getInt(MIN_WORKERS, -1) < 0) { - throw new IllegalArgumentException("No valid " + MIN_WORKERS); - } - if (BspUtils.getVertexClass(getConfiguration()) == null) { - throw new IllegalArgumentException("GiraphJob: Null VERTEX_CLASS"); - } - if (BspUtils.getVertexInputFormatClass(getConfiguration()) == null) { - throw new IllegalArgumentException( - "GiraphJob: Null VERTEX_INPUT_FORMAT_CLASS"); - } - if (BspUtils.getVertexResolverClass(getConfiguration()) == null) { - setVertexResolverClass(VertexResolver.class); - if (LOG.isInfoEnabled()) { - LOG.info("GiraphJob: No class found for " + - VERTEX_RESOLVER_CLASS + ", defaulting to " + - VertexResolver.class.getCanonicalName()); - } - } - } - - /** - * Set the vertex class (required) - * - * @param vertexClass Runs vertex computation - */ - final public void setVertexClass(Class vertexClass) { - getConfiguration().setClass(VERTEX_CLASS, vertexClass, BasicVertex.class); + if (BspUtils.getVertexClass(getConfiguration()) == null) { + throw new IllegalArgumentException("GiraphJob: Null VERTEX_CLASS"); } - - /** - * Set the vertex input format class (required) - * - * @param vertexInputFormatClass Determines how graph is input - */ - final public void setVertexInputFormatClass( - Class vertexInputFormatClass) { - getConfiguration().setClass(VERTEX_INPUT_FORMAT_CLASS, - vertexInputFormatClass, - VertexInputFormat.class); + if (BspUtils.getVertexInputFormatClass(getConfiguration()) == null) { + throw new IllegalArgumentException( + "GiraphJob: Null VERTEX_INPUT_FORMAT_CLASS"); } - - /** - * Set the vertex output format class (optional) - * - * @param vertexOutputFormatClass Determines how graph is output - */ - final public void setVertexOutputFormatClass( - Class vertexOutputFormatClass) { - getConfiguration().setClass(VERTEX_OUTPUT_FORMAT_CLASS, - vertexOutputFormatClass, - VertexOutputFormat.class); + if (BspUtils.getVertexResolverClass(getConfiguration()) == null) { + setVertexResolverClass(VertexResolver.class); + if (LOG.isInfoEnabled()) { + LOG.info("GiraphJob: No class found for " + + VERTEX_RESOLVER_CLASS + ", defaulting to " + + VertexResolver.class.getCanonicalName()); + } } - - /** - * Set the vertex combiner class (optional) - * - * @param vertexCombinerClass Determines how vertex messages are combined - */ - final public void setVertexCombinerClass(Class vertexCombinerClass) { - getConfiguration().setClass(VERTEX_COMBINER_CLASS, - vertexCombinerClass, - VertexCombiner.class); + } + + /** + * Set the vertex class (required) + * + * @param vertexClass Runs vertex computation + */ + public final void setVertexClass(Class vertexClass) { + getConfiguration().setClass(VERTEX_CLASS, vertexClass, BasicVertex.class); + } + + /** + * Set the vertex input format class (required) + * + * @param vertexInputFormatClass Determines how graph is input + */ + public final void setVertexInputFormatClass( + Class vertexInputFormatClass) { + getConfiguration().setClass(VERTEX_INPUT_FORMAT_CLASS, + vertexInputFormatClass, + VertexInputFormat.class); + } + + /** + * Set the vertex output format class (optional) + * + * @param vertexOutputFormatClass Determines how graph is output + */ + public final void setVertexOutputFormatClass( + Class vertexOutputFormatClass) { + getConfiguration().setClass(VERTEX_OUTPUT_FORMAT_CLASS, + vertexOutputFormatClass, + VertexOutputFormat.class); + } + + /** + * Set the vertex combiner class (optional) + * + * @param vertexCombinerClass Determines how vertex messages are combined + */ + public final void setVertexCombinerClass(Class vertexCombinerClass) { + getConfiguration().setClass(VERTEX_COMBINER_CLASS, + vertexCombinerClass, + VertexCombiner.class); + } + + /** + * Set the graph partitioner class (optional) + * + * @param graphPartitionerFactoryClass Determines how the graph is partitioned + */ + public final void setGraphPartitionerFactoryClass( + Class graphPartitionerFactoryClass) { + getConfiguration().setClass(GRAPH_PARTITIONER_FACTORY_CLASS, + graphPartitionerFactoryClass, + GraphPartitionerFactory.class); + } + + /** + * Set the vertex resolver class (optional) + * + * @param vertexResolverClass Determines how vertex mutations are resolved + */ + public final void setVertexResolverClass(Class vertexResolverClass) { + getConfiguration().setClass(VERTEX_RESOLVER_CLASS, + vertexResolverClass, + VertexResolver.class); + } + + /** + * Set the worker context class (optional) + * + * @param workerContextClass Determines what code is executed on a each + * worker before and after each superstep and computation + */ + public final void setWorkerContextClass(Class workerContextClass) { + getConfiguration().setClass(WORKER_CONTEXT_CLASS, + workerContextClass, + WorkerContext.class); + } + + /** + * Set the aggregator writer class (optional) + * + * @param aggregatorWriterClass Determines how the aggregators are + * written to file at the end of the job + */ + public final void setAggregatorWriterClass( + Class aggregatorWriterClass) { + getConfiguration().setClass(AGGREGATOR_WRITER_CLASS, + aggregatorWriterClass, + AggregatorWriter.class); + } + + /** + * Set worker configuration for determining what is required for + * a superstep. + * + * @param minWorkers Minimum workers to do a superstep + * @param maxWorkers Maximum workers to do a superstep + * (max map tasks in job) + * @param minPercentResponded 0 - 100 % of the workers required to + * have responded before continuing the superstep + */ + public final void setWorkerConfiguration(int minWorkers, + int maxWorkers, + float minPercentResponded) { + conf.setInt(MIN_WORKERS, minWorkers); + conf.setInt(MAX_WORKERS, maxWorkers); + conf.setFloat(MIN_PERCENT_RESPONDED, minPercentResponded); + } + + /** + * Utilize an existing ZooKeeper service. If this is not set, ZooKeeper + * will be dynamically started by Giraph for this job. + * + * @param serverList Comma separated list of servers and ports + * (i.e. zk1:2221,zk2:2221) + */ + public final void setZooKeeperConfiguration(String serverList) { + conf.set(ZOOKEEPER_LIST, serverList); + } + + /** + * Check if the configuration is local. If it is local, do additional + * checks due to the restrictions of LocalJobRunner. + * + * @param conf Configuration + */ + private static void checkLocalJobRunnerConfiguration( + Configuration conf) { + String jobTracker = conf.get("mapred.job.tracker", null); + if (!jobTracker.equals("local")) { + // Nothing to check + return; } - /** - * Set the graph partitioner class (optional) - * - * @param graphPartitionerFactoryClass Determines how the graph is partitioned - */ - final public void setGraphPartitionerFactoryClass( - Class graphPartitionerFactoryClass) { - getConfiguration().setClass(GRAPH_PARTITIONER_FACTORY_CLASS, - graphPartitionerFactoryClass, - GraphPartitionerFactory.class); + int maxWorkers = conf.getInt(MAX_WORKERS, -1); + if (maxWorkers != 1) { + throw new IllegalArgumentException( + "checkLocalJobRunnerConfiguration: When using " + + "LocalJobRunner, must have only one worker since " + + "only 1 task at a time!"); } - - /** - * Set the vertex resolver class (optional) - * - * @param vertexResolverClass Determines how vertex mutations are resolved - */ - final public void setVertexResolverClass(Class vertexResolverClass) { - getConfiguration().setClass(VERTEX_RESOLVER_CLASS, - vertexResolverClass, - VertexResolver.class); + if (conf.getBoolean(SPLIT_MASTER_WORKER, + SPLIT_MASTER_WORKER_DEFAULT)) { + throw new IllegalArgumentException( + "checkLocalJobRunnerConfiguration: When using " + + "LocalJobRunner, you cannot run in split master / worker " + + "mode since there is only 1 task at a time!"); } - - /** - * Set the worker context class (optional) - * - * @param workerContextClass Determines what code is executed on a each - * worker before and after each superstep and computation - */ - final public void setWorkerContextClass(Class workerContextClass) { - getConfiguration().setClass(WORKER_CONTEXT_CLASS, - workerContextClass, - WorkerContext.class); + } + + /** + * Check whether a specified int conf value is set and if not, set it. + * + * @param param Conf value to check + * @param defaultValue Assign to value if not set + */ + private void setIntConfIfDefault(String param, int defaultValue) { + if (conf.getInt(param, Integer.MIN_VALUE) == Integer.MIN_VALUE) { + conf.setInt(param, defaultValue); } - - /** - * Set the aggregator writer class (optional) - * - * @param aggregatorWriterClass Determines how the aggregators are - * written to file at the end of the job - */ - final public void setAggregatorWriterClass( - Class aggregatorWriterClass) { - getConfiguration().setClass(AGGREGATOR_WRITER_CLASS, - aggregatorWriterClass, - AggregatorWriter.class); - } - - /** - * Set worker configuration for determining what is required for - * a superstep. - * - * @param minWorkers Minimum workers to do a superstep - * @param maxWorkers Maximum workers to do a superstep - * (max map tasks in job) - * @param minPercentResponded 0 - 100 % of the workers required to - * have responded before continuing the superstep - */ - final public void setWorkerConfiguration(int minWorkers, - int maxWorkers, - float minPercentResponded) { - conf.setInt(MIN_WORKERS, minWorkers); - conf.setInt(MAX_WORKERS, maxWorkers); - conf.setFloat(MIN_PERCENT_RESPONDED, minPercentResponded); - } - - /** - * Utilize an existing ZooKeeper service. If this is not set, ZooKeeper - * will be dynamically started by Giraph for this job. - * - * @param serverList Comma separated list of servers and ports - * (i.e. zk1:2221,zk2:2221) - */ - final public void setZooKeeperConfiguration(String serverList) { - conf.set(ZOOKEEPER_LIST, serverList); - } - - /** - * Check if the configuration is local. If it is local, do additional - * checks due to the restrictions of LocalJobRunner. - * - * @param conf Configuration - */ - private static void checkLocalJobRunnerConfiguration( - Configuration conf) { - String jobTracker = conf.get("mapred.job.tracker", null); - if (!jobTracker.equals("local")) { - // Nothing to check - return; - } - - int maxWorkers = conf.getInt(MAX_WORKERS, -1); - if (maxWorkers != 1) { - throw new IllegalArgumentException( - "checkLocalJobRunnerConfiguration: When using " + - "LocalJobRunner, must have only one worker since " + - "only 1 task at a time!"); - } - if (conf.getBoolean(SPLIT_MASTER_WORKER, - SPLIT_MASTER_WORKER_DEFAULT)) { - throw new IllegalArgumentException( - "checkLocalJobRunnerConfiguration: When using " + - "LocalJobRunner, you cannot run in split master / worker " + - "mode since there is only 1 task at a time!"); - } - } - - /** - * Check whether a specified int conf value is set and if not, set it. - * - * @param param Conf value to check - * @param defaultValue Assign to value if not set - */ - private void setIntConfIfDefault(String param, int defaultValue) { - if (conf.getInt(param, Integer.MIN_VALUE) == Integer.MIN_VALUE) { - conf.setInt(param, defaultValue); - } - } - - /** - * Runs the actual graph application through Hadoop Map-Reduce. - * - * @param verbose If true, provide verbose output, false otherwise - * @throws ClassNotFoundException - * @throws InterruptedException - * @throws IOException - */ - final public boolean run(boolean verbose) - throws IOException, InterruptedException, ClassNotFoundException { - checkConfiguration(); - checkLocalJobRunnerConfiguration(conf); - setNumReduceTasks(0); - // Most users won't hit this hopefully and can set it higher if desired - setIntConfIfDefault("mapreduce.job.counters.limit", 512); - - // Capacity scheduler-specific settings. These should be enough for - // a reasonable Giraph job - setIntConfIfDefault("mapred.job.map.memory.mb", 1024); - setIntConfIfDefault("mapred.job.reduce.memory.mb", 1024); - - // Speculative execution doesn't make sense for Giraph - conf.setBoolean("mapred.map.tasks.speculative.execution", false); - - // Set the ping interval to 5 minutes instead of one minute - // (DEFAULT_PING_INTERVAL) - Client.setPingInterval(conf, 60000*5); - - if (getJar() == null) { - setJarByClass(GiraphJob.class); - } - // Should work in MAPREDUCE-1938 to let the user jars/classes - // get loaded first - conf.setBoolean("mapreduce.user.classpath.first", true); - - setMapperClass(GraphMapper.class); - setInputFormatClass(BspInputFormat.class); - setOutputFormatClass(BspOutputFormat.class); - return waitForCompletion(verbose); + } + + /** + * Runs the actual graph application through Hadoop Map-Reduce. + * + * @param verbose If true, provide verbose output, false otherwise + * @return True if success, false otherwise + * @throws ClassNotFoundException + * @throws InterruptedException + * @throws IOException + */ + public final boolean run(boolean verbose) + throws IOException, InterruptedException, ClassNotFoundException { + checkConfiguration(); + checkLocalJobRunnerConfiguration(conf); + setNumReduceTasks(0); + // Most users won't hit this hopefully and can set it higher if desired + setIntConfIfDefault("mapreduce.job.counters.limit", 512); + + // Capacity scheduler-specific settings. These should be enough for + // a reasonable Giraph job + setIntConfIfDefault("mapred.job.map.memory.mb", 1024); + setIntConfIfDefault("mapred.job.reduce.memory.mb", 1024); + + // Speculative execution doesn't make sense for Giraph + conf.setBoolean("mapred.map.tasks.speculative.execution", false); + + // Set the ping interval to 5 minutes instead of one minute + // (DEFAULT_PING_INTERVAL) + Client.setPingInterval(conf, 60000 * 5); + + if (getJar() == null) { + setJarByClass(GiraphJob.class); } + // Should work in MAPREDUCE-1938 to let the user jars/classes + // get loaded first + conf.setBoolean("mapreduce.user.classpath.first", true); + + setMapperClass(GraphMapper.class); + setInputFormatClass(BspInputFormat.class); + setOutputFormatClass(BspOutputFormat.class); + return waitForCompletion(verbose); + } } diff --git a/src/main/java/org/apache/giraph/graph/GlobalStats.java b/src/main/java/org/apache/giraph/graph/GlobalStats.java index db0389db3..e5a9b032c 100644 --- a/src/main/java/org/apache/giraph/graph/GlobalStats.java +++ b/src/main/java/org/apache/giraph/graph/GlobalStats.java @@ -29,57 +29,71 @@ * Aggregated stats by the master. */ public class GlobalStats implements Writable { - private long vertexCount = 0; - private long finishedVertexCount = 0; - private long edgeCount = 0; - private long messageCount = 0; + /** All vertices in the application */ + private long vertexCount = 0; + /** All finished vertices in the last superstep */ + private long finishedVertexCount = 0; + /** All edges in the last superstep */ + private long edgeCount = 0; + /** All messages sent in the last superstep */ + private long messageCount = 0; - public void addPartitionStats(PartitionStats partitionStats) { - this.vertexCount += partitionStats.getVertexCount(); - this.finishedVertexCount += partitionStats.getFinishedVertexCount(); - this.edgeCount += partitionStats.getEdgeCount(); - } + /** + * Add the stats of a partition to the global stats. + * + * @param partitionStats Partition stats to be added. + */ + public void addPartitionStats(PartitionStats partitionStats) { + this.vertexCount += partitionStats.getVertexCount(); + this.finishedVertexCount += partitionStats.getFinishedVertexCount(); + this.edgeCount += partitionStats.getEdgeCount(); + } - public long getVertexCount() { - return vertexCount; - } + public long getVertexCount() { + return vertexCount; + } - public long getFinishedVertexCount() { - return finishedVertexCount; - } + public long getFinishedVertexCount() { + return finishedVertexCount; + } - public long getEdgeCount() { - return edgeCount; - } + public long getEdgeCount() { + return edgeCount; + } - public long getMessageCount() { - return messageCount; - } + public long getMessageCount() { + return messageCount; + } - public void addMessageCount(long messageCount) { - this.messageCount += messageCount; - } + /** + * Add messages to the global stats. + * + * @param messageCount Number of messages to be added. + */ + public void addMessageCount(long messageCount) { + this.messageCount += messageCount; + } - @Override - public void readFields(DataInput input) throws IOException { - vertexCount = input.readLong(); - finishedVertexCount = input.readLong(); - edgeCount = input.readLong(); - messageCount = input.readLong(); - } + @Override + public void readFields(DataInput input) throws IOException { + vertexCount = input.readLong(); + finishedVertexCount = input.readLong(); + edgeCount = input.readLong(); + messageCount = input.readLong(); + } - @Override - public void write(DataOutput output) throws IOException { - output.writeLong(vertexCount); - output.writeLong(finishedVertexCount); - output.writeLong(edgeCount); - output.writeLong(messageCount); - } + @Override + public void write(DataOutput output) throws IOException { + output.writeLong(vertexCount); + output.writeLong(finishedVertexCount); + output.writeLong(edgeCount); + output.writeLong(messageCount); + } - @Override - public String toString() { - return "(vtx=" + vertexCount + ",finVtx=" + - finishedVertexCount + ",edges=" + edgeCount + ",msgCount=" + - messageCount + ")"; - } + @Override + public String toString() { + return "(vtx=" + vertexCount + ",finVtx=" + + finishedVertexCount + ",edges=" + edgeCount + ",msgCount=" + + messageCount + ")"; + } } diff --git a/src/main/java/org/apache/giraph/graph/GraphMapper.java b/src/main/java/org/apache/giraph/graph/GraphMapper.java index 2f28ee238..5e878f91b 100644 --- a/src/main/java/org/apache/giraph/graph/GraphMapper.java +++ b/src/main/java/org/apache/giraph/graph/GraphMapper.java @@ -48,598 +48,614 @@ * This mapper that will execute the BSP graph tasks. Since this mapper will * not be passing data by key-value pairs through the MR framework, the * types are irrelevant. + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data */ @SuppressWarnings("rawtypes") public class GraphMapper extends - Mapper { - /** Class logger */ - private static final Logger LOG = Logger.getLogger(GraphMapper.class); - /** Coordination service worker */ - CentralizedServiceWorker serviceWorker; - /** Coordination service master thread */ - Thread masterThread = null; - /** The map should be run exactly once, or else there is a problem. */ - boolean mapAlreadyRun = false; - /** Manages the ZooKeeper servers if necessary (dynamic startup) */ - private ZooKeeperManager zkManager; - /** Configuration */ - private Configuration conf; - /** Already complete? */ - private boolean done = false; - /** What kind of functions is this mapper doing? */ - private MapFunctions mapFunctions = MapFunctions.UNKNOWN; - /** - * Graph state for all vertices that is used for the duration of - * this mapper. - */ - private GraphState graphState = new GraphState(); - - /** What kinds of functions to run on this mapper */ - public enum MapFunctions { - UNKNOWN, - MASTER_ONLY, - MASTER_ZOOKEEPER_ONLY, - WORKER_ONLY, - ALL, - ALL_EXCEPT_ZOOKEEPER - } - - /** - * Get the map function enum - */ - public MapFunctions getMapFunctions() { - return mapFunctions; + E extends Writable, M extends Writable> extends + Mapper { + /** Class logger */ + private static final Logger LOG = Logger.getLogger(GraphMapper.class); + /** Coordination service worker */ + private CentralizedServiceWorker serviceWorker; + /** Coordination service master thread */ + private Thread masterThread = null; + /** The map should be run exactly once, or else there is a problem. */ + private boolean mapAlreadyRun = false; + /** Manages the ZooKeeper servers if necessary (dynamic startup) */ + private ZooKeeperManager zkManager; + /** Configuration */ + private Configuration conf; + /** Already complete? */ + private boolean done = false; + /** What kind of functions is this mapper doing? */ + private MapFunctions mapFunctions = MapFunctions.UNKNOWN; + /** + * Graph state for all vertices that is used for the duration of + * this mapper. + */ + private GraphState graphState = new GraphState(); + + /** What kinds of functions to run on this mapper */ + public enum MapFunctions { + /** Undecided yet */ + UNKNOWN, + /** Only be the master */ + MASTER_ONLY, + /** Only be the master and ZooKeeper */ + MASTER_ZOOKEEPER_ONLY, + /** Only be the worker */ + WORKER_ONLY, + /** Do master, worker, and ZooKeeper */ + ALL, + /** Do master and worker */ + ALL_EXCEPT_ZOOKEEPER + } + + /** + * Get the map function enum. + * + * @return Map functions of this mapper. + */ + public MapFunctions getMapFunctions() { + return mapFunctions; + } + + /** + * Get the aggregator usage, a subset of the functionality + * + * @return Aggregator usage interface + */ + public final AggregatorUsage getAggregatorUsage() { + return serviceWorker; + } + + public final WorkerContext getWorkerContext() { + return serviceWorker.getWorkerContext(); + } + + public final GraphState getGraphState() { + return graphState; + } + + /** + * Default handler for uncaught exceptions. + */ + class OverrideExceptionHandler implements Thread.UncaughtExceptionHandler { + @Override + public void uncaughtException(Thread t, Throwable e) { + LOG.fatal( + "uncaughtException: OverrideExceptionHandler on thread " + + t.getName() + ", msg = " + e.getMessage() + + ", exiting...", e); + System.exit(1); } - - /** - * Get the aggregator usage, a subset of the functionality - * - * @return Aggregator usage interface - */ - public final AggregatorUsage getAggregatorUsage() { - return serviceWorker; + } + + /** + * Copied from JobConf to get the location of this jar. Workaround for + * things like Oozie map-reduce jobs. + * + * @param myClass Class to search the class loader path for to locate + * the relevant jar file + * @return Location of the jar file containing myClass + */ + private static String findContainingJar(Class myClass) { + ClassLoader loader = myClass.getClassLoader(); + String classFile = + myClass.getName().replaceAll("\\.", "/") + ".class"; + try { + for (Enumeration itr = loader.getResources(classFile); + itr.hasMoreElements();) { + URL url = (URL) itr.nextElement(); + if ("jar".equals(url.getProtocol())) { + String toReturn = url.getPath(); + if (toReturn.startsWith("file:")) { + toReturn = toReturn.substring("file:".length()); + } + toReturn = URLDecoder.decode(toReturn, "UTF-8"); + return toReturn.replaceAll("!.*$", ""); + } + } + } catch (IOException e) { + throw new RuntimeException(e); } - - public final WorkerContext getWorkerContext() { - return serviceWorker.getWorkerContext(); + return null; + } + + /** + * Make sure that all registered classes have matching types. This + * is a little tricky due to type erasure, cannot simply get them from + * the class type arguments. Also, set the vertex index, vertex value, + * edge value and message value classes. + * + * @param conf Configuration to get the various classes + */ + public void determineClassTypes(Configuration conf) { + Class> vertexClass = + BspUtils.getVertexClass(conf); + List> classList = ReflectionUtils.getTypeArguments( + BasicVertex.class, vertexClass); + Type vertexIndexType = classList.get(0); + Type vertexValueType = classList.get(1); + Type edgeValueType = classList.get(2); + Type messageValueType = classList.get(3); + + Class> vertexInputFormatClass = + BspUtils.getVertexInputFormatClass(conf); + classList = ReflectionUtils.getTypeArguments( + VertexInputFormat.class, vertexInputFormatClass); + if (classList.get(0) == null) { + LOG.warn("Input format vertex index type is not known"); + } else if (!vertexIndexType.equals(classList.get(0))) { + throw new IllegalArgumentException( + "checkClassTypes: Vertex index types don't match, " + + "vertex - " + vertexIndexType + + ", vertex input format - " + classList.get(0)); } - - public final GraphState getGraphState() { - return graphState; + if (classList.get(1) == null) { + LOG.warn("Input format vertex value type is not known"); + } else if (!vertexValueType.equals(classList.get(1))) { + throw new IllegalArgumentException( + "checkClassTypes: Vertex value types don't match, " + + "vertex - " + vertexValueType + + ", vertex input format - " + classList.get(1)); } - - /** - * Default handler for uncaught exceptions. - */ - class OverrideExceptionHandler - implements Thread.UncaughtExceptionHandler { - public void uncaughtException(Thread t, Throwable e) { - LOG.fatal( - "uncaughtException: OverrideExceptionHandler on thread " + - t.getName() + ", msg = " + e.getMessage() + - ", exiting...", e); - System.exit(1); - } + if (classList.get(2) == null) { + LOG.warn("Input format edge value type is not known"); + } else if (!edgeValueType.equals(classList.get(2))) { + throw new IllegalArgumentException( + "checkClassTypes: Edge value types don't match, " + + "vertex - " + edgeValueType + + ", vertex input format - " + classList.get(2)); } - - /** - * Copied from JobConf to get the location of this jar. Workaround for - * things like Oozie map-reduce jobs. - * - * @param my_class Class to search the class loader path for to locate - * the relevant jar file - * @return Location of the jar file containing my_class - */ - private static String findContainingJar(Class my_class) { - ClassLoader loader = my_class.getClassLoader(); - String class_file = - my_class.getName().replaceAll("\\.", "/") + ".class"; - try { - for(Enumeration itr = loader.getResources(class_file); - itr.hasMoreElements();) { - URL url = (URL) itr.nextElement(); - if ("jar".equals(url.getProtocol())) { - String toReturn = url.getPath(); - if (toReturn.startsWith("file:")) { - toReturn = toReturn.substring("file:".length()); - } - toReturn = URLDecoder.decode(toReturn, "UTF-8"); - return toReturn.replaceAll("!.*$", ""); - } - } - } catch (IOException e) { - throw new RuntimeException(e); - } - return null; - } - - /** - * Make sure that all registered classes have matching types. This - * is a little tricky due to type erasure, cannot simply get them from - * the class type arguments. Also, set the vertex index, vertex value, - * edge value and message value classes. - * - * @param conf Configuration to get the various classes - */ - public void determineClassTypes(Configuration conf) { - Class> vertexClass = - BspUtils.getVertexClass(conf); - List> classList = ReflectionUtils.getTypeArguments( - BasicVertex.class, vertexClass); - Type vertexIndexType = classList.get(0); - Type vertexValueType = classList.get(1); - Type edgeValueType = classList.get(2); - Type messageValueType = classList.get(3); - - Class> vertexInputFormatClass = - BspUtils.getVertexInputFormatClass(conf); - classList = ReflectionUtils.getTypeArguments( - VertexInputFormat.class, vertexInputFormatClass); - if (classList.get(0) == null) { - LOG.warn("Input format vertex index type is not known"); - } else if (!vertexIndexType.equals(classList.get(0))) { - throw new IllegalArgumentException( - "checkClassTypes: Vertex index types don't match, " + + // If has vertex combiner class, check + Class> vertexCombinerClass = + BspUtils.getVertexCombinerClass(conf); + if (vertexCombinerClass != null) { + classList = ReflectionUtils.getTypeArguments( + VertexCombiner.class, vertexCombinerClass); + if (!vertexIndexType.equals(classList.get(0))) { + throw new IllegalArgumentException( + "checkClassTypes: Vertex index types don't match, " + "vertex - " + vertexIndexType + - ", vertex input format - " + classList.get(0)); - } - if (classList.get(1) == null) { - LOG.warn("Input format vertex value type is not known"); - } else if (!vertexValueType.equals(classList.get(1))) { - throw new IllegalArgumentException( - "checkClassTypes: Vertex value types don't match, " + + ", vertex combiner - " + classList.get(0)); + } + if (!messageValueType.equals(classList.get(1))) { + throw new IllegalArgumentException( + "checkClassTypes: Message value types don't match, " + "vertex - " + vertexValueType + - ", vertex input format - " + classList.get(1)); - } - if (classList.get(2) == null) { - LOG.warn("Input format edge value type is not known"); - } else if (!edgeValueType.equals(classList.get(2))) { - throw new IllegalArgumentException( - "checkClassTypes: Edge value types don't match, " + - "vertex - " + edgeValueType + - ", vertex input format - " + classList.get(2)); - } - // If has vertex combiner class, check - Class> vertexCombinerClass = - BspUtils.getVertexCombinerClass(conf); - if (vertexCombinerClass != null) { - classList = ReflectionUtils.getTypeArguments( - VertexCombiner.class, vertexCombinerClass); - if (!vertexIndexType.equals(classList.get(0))) { - throw new IllegalArgumentException( - "checkClassTypes: Vertex index types don't match, " + - "vertex - " + vertexIndexType + - ", vertex combiner - " + classList.get(0)); - } - if (!messageValueType.equals(classList.get(1))) { - throw new IllegalArgumentException( - "checkClassTypes: Message value types don't match, " + - "vertex - " + vertexValueType + - ", vertex combiner - " + classList.get(1)); - } - } - // If has vertex output format class, check - Class> - vertexOutputFormatClass = - BspUtils.getVertexOutputFormatClass(conf); - if (vertexOutputFormatClass != null) { - classList = - ReflectionUtils.getTypeArguments( - VertexOutputFormat.class, vertexOutputFormatClass); - if (classList.get(0) == null) { - LOG.warn("Output format vertex index type is not known"); - } else if (!vertexIndexType.equals(classList.get(0))) { - throw new IllegalArgumentException( - "checkClassTypes: Vertex index types don't match, " + - "vertex - " + vertexIndexType + - ", vertex output format - " + classList.get(0)); - } - if (classList.get(1) == null) { - LOG.warn("Output format vertex value type is not known"); - } else if (!vertexValueType.equals(classList.get(1))) { - throw new IllegalArgumentException( - "checkClassTypes: Vertex value types don't match, " + - "vertex - " + vertexValueType + - ", vertex output format - " + classList.get(1)); - } if (classList.get(2) == null) { - LOG.warn("Output format edge value type is not known"); - } else if (!edgeValueType.equals(classList.get(2))) { - throw new IllegalArgumentException( - "checkClassTypes: Edge value types don't match, " + - "vertex - " + vertexIndexType + - ", vertex output format - " + classList.get(2)); - } - } - // Vertex resolver might never select the types - Class> - vertexResolverClass = - BspUtils.getVertexResolverClass(conf); - classList = ReflectionUtils.getTypeArguments( - VertexResolver.class, vertexResolverClass); - if (classList.get(0) != null && - !vertexIndexType.equals(classList.get(0))) { - throw new IllegalArgumentException( - "checkClassTypes: Vertex index types don't match, " + + ", vertex combiner - " + classList.get(1)); + } + } + // If has vertex output format class, check + Class> + vertexOutputFormatClass = + BspUtils.getVertexOutputFormatClass(conf); + if (vertexOutputFormatClass != null) { + classList = + ReflectionUtils.getTypeArguments( + VertexOutputFormat.class, vertexOutputFormatClass); + if (classList.get(0) == null) { + LOG.warn("Output format vertex index type is not known"); + } else if (!vertexIndexType.equals(classList.get(0))) { + throw new IllegalArgumentException( + "checkClassTypes: Vertex index types don't match, " + "vertex - " + vertexIndexType + - ", vertex resolver - " + classList.get(0)); - } - if (classList.get(1) != null && - !vertexValueType.equals(classList.get(1))) { - throw new IllegalArgumentException( - "checkClassTypes: Vertex value types don't match, " + + ", vertex output format - " + classList.get(0)); + } + if (classList.get(1) == null) { + LOG.warn("Output format vertex value type is not known"); + } else if (!vertexValueType.equals(classList.get(1))) { + throw new IllegalArgumentException( + "checkClassTypes: Vertex value types don't match, " + "vertex - " + vertexValueType + - ", vertex resolver - " + classList.get(1)); - } - if (classList.get(2) != null && - !edgeValueType.equals(classList.get(2))) { - throw new IllegalArgumentException( - "checkClassTypes: Edge value types don't match, " + - "vertex - " + edgeValueType + - ", vertex resolver - " + classList.get(2)); - } - if (classList.get(3) != null && - !messageValueType.equals(classList.get(3))) { - throw new IllegalArgumentException( - "checkClassTypes: Message value types don't match, " + - "vertex - " + edgeValueType + - ", vertex resolver - " + classList.get(3)); - } - conf.setClass(GiraphJob.VERTEX_INDEX_CLASS, - (Class) vertexIndexType, - WritableComparable.class); - conf.setClass(GiraphJob.VERTEX_VALUE_CLASS, - (Class) vertexValueType, - Writable.class); - conf.setClass(GiraphJob.EDGE_VALUE_CLASS, - (Class) edgeValueType, - Writable.class); - conf.setClass(GiraphJob.MESSAGE_VALUE_CLASS, - (Class) messageValueType, - Writable.class); + ", vertex output format - " + classList.get(1)); + } + if (classList.get(2) == null) { + LOG.warn("Output format edge value type is not known"); + } else if (!edgeValueType.equals(classList.get(2))) { + throw new IllegalArgumentException( + "checkClassTypes: Edge value types don't match, " + + "vertex - " + vertexIndexType + + ", vertex output format - " + classList.get(2)); + } } - - /** - * Figure out what functions this mapper should do. Basic logic is as - * follows: - * 1) If not split master, everyone does the everything and/or running - * ZooKeeper. - * 2) If split master/worker, masters also run ZooKeeper (if it's not - * given to us). - * - * @param conf Configuration to use - * @return Functions that this mapper should do. - */ - private static MapFunctions determineMapFunctions( - Configuration conf, - ZooKeeperManager zkManager) { - boolean splitMasterWorker = - conf.getBoolean(GiraphJob.SPLIT_MASTER_WORKER, - GiraphJob.SPLIT_MASTER_WORKER_DEFAULT); - int taskPartition = conf.getInt("mapred.task.partition", -1); - boolean zkAlreadyProvided = - conf.get(GiraphJob.ZOOKEEPER_LIST) != null; - MapFunctions functions = MapFunctions.UNKNOWN; - // What functions should this mapper do? - if (!splitMasterWorker) { - if ((zkManager != null) && zkManager.runsZooKeeper()) { - functions = MapFunctions.ALL; - } else { - functions = MapFunctions.ALL_EXCEPT_ZOOKEEPER; - } + // Vertex resolver might never select the types + Class> + vertexResolverClass = + BspUtils.getVertexResolverClass(conf); + classList = ReflectionUtils.getTypeArguments( + VertexResolver.class, vertexResolverClass); + if (classList.get(0) != null && + !vertexIndexType.equals(classList.get(0))) { + throw new IllegalArgumentException( + "checkClassTypes: Vertex index types don't match, " + + "vertex - " + vertexIndexType + + ", vertex resolver - " + classList.get(0)); + } + if (classList.get(1) != null && + !vertexValueType.equals(classList.get(1))) { + throw new IllegalArgumentException( + "checkClassTypes: Vertex value types don't match, " + + "vertex - " + vertexValueType + + ", vertex resolver - " + classList.get(1)); + } + if (classList.get(2) != null && + !edgeValueType.equals(classList.get(2))) { + throw new IllegalArgumentException( + "checkClassTypes: Edge value types don't match, " + + "vertex - " + edgeValueType + + ", vertex resolver - " + classList.get(2)); + } + if (classList.get(3) != null && + !messageValueType.equals(classList.get(3))) { + throw new IllegalArgumentException( + "checkClassTypes: Message value types don't match, " + + "vertex - " + edgeValueType + + ", vertex resolver - " + classList.get(3)); + } + conf.setClass(GiraphJob.VERTEX_INDEX_CLASS, + (Class) vertexIndexType, + WritableComparable.class); + conf.setClass(GiraphJob.VERTEX_VALUE_CLASS, + (Class) vertexValueType, + Writable.class); + conf.setClass(GiraphJob.EDGE_VALUE_CLASS, + (Class) edgeValueType, + Writable.class); + conf.setClass(GiraphJob.MESSAGE_VALUE_CLASS, + (Class) messageValueType, + Writable.class); + } + + /** + * Figure out what functions this mapper should do. Basic logic is as + * follows: + * 1) If not split master, everyone does the everything and/or running + * ZooKeeper. + * 2) If split master/worker, masters also run ZooKeeper (if it's not + * given to us). + * + * @param conf Configuration to use + * @param zkManager ZooKeeper manager to help determine whether to run + * ZooKeeper + * @return Functions that this mapper should do. + */ + private static MapFunctions determineMapFunctions( + Configuration conf, + ZooKeeperManager zkManager) { + boolean splitMasterWorker = + conf.getBoolean(GiraphJob.SPLIT_MASTER_WORKER, + GiraphJob.SPLIT_MASTER_WORKER_DEFAULT); + int taskPartition = conf.getInt("mapred.task.partition", -1); + boolean zkAlreadyProvided = + conf.get(GiraphJob.ZOOKEEPER_LIST) != null; + MapFunctions functions = MapFunctions.UNKNOWN; + // What functions should this mapper do? + if (!splitMasterWorker) { + if ((zkManager != null) && zkManager.runsZooKeeper()) { + functions = MapFunctions.ALL; + } else { + functions = MapFunctions.ALL_EXCEPT_ZOOKEEPER; + } + } else { + if (zkAlreadyProvided) { + int masterCount = + conf.getInt(GiraphJob.ZOOKEEPER_SERVER_COUNT, + GiraphJob.ZOOKEEPER_SERVER_COUNT_DEFAULT); + if (taskPartition < masterCount) { + functions = MapFunctions.MASTER_ONLY; + } else { + functions = MapFunctions.WORKER_ONLY; + } + } else { + if ((zkManager != null) && zkManager.runsZooKeeper()) { + functions = MapFunctions.MASTER_ZOOKEEPER_ONLY; } else { - if (zkAlreadyProvided) { - int masterCount = - conf.getInt(GiraphJob.ZOOKEEPER_SERVER_COUNT, - GiraphJob.ZOOKEEPER_SERVER_COUNT_DEFAULT); - if (taskPartition < masterCount) { - functions = MapFunctions.MASTER_ONLY; - } else { - functions = MapFunctions.WORKER_ONLY; - } - } else { - if ((zkManager != null) && zkManager.runsZooKeeper()) { - functions = MapFunctions.MASTER_ZOOKEEPER_ONLY; - } else { - functions = MapFunctions.WORKER_ONLY; - } - } + functions = MapFunctions.WORKER_ONLY; } - return functions; + } } - - @Override - public void setup(Context context) - throws IOException, InterruptedException { - context.setStatus("setup: Beginning mapper setup."); - graphState.setContext(context); - // Setting the default handler for uncaught exceptions. - Thread.setDefaultUncaughtExceptionHandler( - new OverrideExceptionHandler()); - conf = context.getConfiguration(); - // Hadoop security needs this property to be set - if (System.getenv("HADOOP_TOKEN_FILE_LOCATION") != null) { - conf.set("mapreduce.job.credentials.binary", - System.getenv("HADOOP_TOKEN_FILE_LOCATION")); + return functions; + } + + @Override + public void setup(Context context) + throws IOException, InterruptedException { + context.setStatus("setup: Beginning mapper setup."); + graphState.setContext(context); + // Setting the default handler for uncaught exceptions. + Thread.setDefaultUncaughtExceptionHandler( + new OverrideExceptionHandler()); + conf = context.getConfiguration(); + // Hadoop security needs this property to be set + if (System.getenv("HADOOP_TOKEN_FILE_LOCATION") != null) { + conf.set("mapreduce.job.credentials.binary", + System.getenv("HADOOP_TOKEN_FILE_LOCATION")); + } + // Ensure the user classes have matching types and figure them out + determineClassTypes(conf); + + // Do some initial setup (possibly starting up a Zookeeper service) + context.setStatus("setup: Initializing Zookeeper services."); + if (!conf.getBoolean(GiraphJob.LOCAL_TEST_MODE, + GiraphJob.LOCAL_TEST_MODE_DEFAULT)) { + Path[] fileClassPaths = DistributedCache.getLocalCacheArchives(conf); + String zkClasspath = null; + if (fileClassPaths == null) { + if (LOG.isInfoEnabled()) { + LOG.info("Distributed cache is empty. Assuming fatjar."); } - // Ensure the user classes have matching types and figure them out - determineClassTypes(conf); - - // Do some initial setup (possibly starting up a Zookeeper service) - context.setStatus("setup: Initializing Zookeeper services."); - if (!conf.getBoolean(GiraphJob.LOCAL_TEST_MODE, - GiraphJob.LOCAL_TEST_MODE_DEFAULT)) { - Path[] fileClassPaths = DistributedCache.getLocalCacheArchives(conf); - String zkClasspath = null; - if(fileClassPaths == null) { - if(LOG.isInfoEnabled()) { - LOG.info("Distributed cache is empty. Assuming fatjar."); - } - String jarFile = context.getJar(); - if (jarFile == null) { - jarFile = findContainingJar(getClass()); - } - zkClasspath = jarFile.replaceFirst("file:", ""); - } else { - StringBuilder sb = new StringBuilder(); - sb.append(fileClassPaths[0]); - - for (int i = 1; i < fileClassPaths.length; i++) { - sb.append(":"); - sb.append(fileClassPaths[i]); - } - zkClasspath = sb.toString(); - } - - if (LOG.isInfoEnabled()) { - LOG.info("setup: classpath @ " + zkClasspath); - } - conf.set(GiraphJob.ZOOKEEPER_JAR, zkClasspath); + String jarFile = context.getJar(); + if (jarFile == null) { + jarFile = findContainingJar(getClass()); } - String serverPortList = - conf.get(GiraphJob.ZOOKEEPER_LIST, ""); - if (serverPortList == "") { - zkManager = new ZooKeeperManager(context); - context.setStatus("setup: Setting up Zookeeper manager."); - zkManager.setup(); - if (zkManager.computationDone()) { - done = true; - return; - } - zkManager.onlineZooKeeperServers(); - serverPortList = zkManager.getZooKeeperServerPortString(); + zkClasspath = jarFile.replaceFirst("file:", ""); + } else { + StringBuilder sb = new StringBuilder(); + sb.append(fileClassPaths[0]); + + for (int i = 1; i < fileClassPaths.length; i++) { + sb.append(":"); + sb.append(fileClassPaths[i]); } - context.setStatus("setup: Connected to Zookeeper service " + - serverPortList); - this.mapFunctions = determineMapFunctions(conf, zkManager); - - // Sometimes it takes a while to get multiple ZooKeeper servers up - if (conf.getInt(GiraphJob.ZOOKEEPER_SERVER_COUNT, - GiraphJob.ZOOKEEPER_SERVER_COUNT_DEFAULT) > 1) { - Thread.sleep(GiraphJob.DEFAULT_ZOOKEEPER_INIT_LIMIT * - GiraphJob.DEFAULT_ZOOKEEPER_TICK_TIME); + zkClasspath = sb.toString(); + } + + if (LOG.isInfoEnabled()) { + LOG.info("setup: classpath @ " + zkClasspath); + } + conf.set(GiraphJob.ZOOKEEPER_JAR, zkClasspath); + } + String serverPortList = + conf.get(GiraphJob.ZOOKEEPER_LIST, ""); + if (serverPortList.isEmpty()) { + zkManager = new ZooKeeperManager(context); + context.setStatus("setup: Setting up Zookeeper manager."); + zkManager.setup(); + if (zkManager.computationDone()) { + done = true; + return; + } + zkManager.onlineZooKeeperServers(); + serverPortList = zkManager.getZooKeeperServerPortString(); + } + context.setStatus("setup: Connected to Zookeeper service " + + serverPortList); + this.mapFunctions = determineMapFunctions(conf, zkManager); + + // Sometimes it takes a while to get multiple ZooKeeper servers up + if (conf.getInt(GiraphJob.ZOOKEEPER_SERVER_COUNT, + GiraphJob.ZOOKEEPER_SERVER_COUNT_DEFAULT) > 1) { + Thread.sleep(GiraphJob.DEFAULT_ZOOKEEPER_INIT_LIMIT * + GiraphJob.DEFAULT_ZOOKEEPER_TICK_TIME); + } + int sessionMsecTimeout = + conf.getInt(GiraphJob.ZOOKEEPER_SESSION_TIMEOUT, + GiraphJob.ZOOKEEPER_SESSION_TIMEOUT_DEFAULT); + try { + if ((mapFunctions == MapFunctions.MASTER_ZOOKEEPER_ONLY) || + (mapFunctions == MapFunctions.MASTER_ONLY) || + (mapFunctions == MapFunctions.ALL) || + (mapFunctions == MapFunctions.ALL_EXCEPT_ZOOKEEPER)) { + if (LOG.isInfoEnabled()) { + LOG.info("setup: Starting up BspServiceMaster " + + "(master thread)..."); } - int sessionMsecTimeout = - conf.getInt(GiraphJob.ZOOKEEPER_SESSION_TIMEOUT, - GiraphJob.ZOOKEEPER_SESSION_TIMEOUT_DEFAULT); - try { - if ((mapFunctions == MapFunctions.MASTER_ZOOKEEPER_ONLY) || - (mapFunctions == MapFunctions.MASTER_ONLY) || - (mapFunctions == MapFunctions.ALL) || - (mapFunctions == MapFunctions.ALL_EXCEPT_ZOOKEEPER)) { - if (LOG.isInfoEnabled()) { - LOG.info("setup: Starting up BspServiceMaster " + - "(master thread)..."); - } - masterThread = - new MasterThread( - new BspServiceMaster(serverPortList, - sessionMsecTimeout, - context, - this), - context); - masterThread.start(); - } - if ((mapFunctions == MapFunctions.WORKER_ONLY) || - (mapFunctions == MapFunctions.ALL) || - (mapFunctions == MapFunctions.ALL_EXCEPT_ZOOKEEPER)) { - if (LOG.isInfoEnabled()) { - LOG.info("setup: Starting up BspServiceWorker..."); - } - serviceWorker = new BspServiceWorker( - serverPortList, + masterThread = + new MasterThread( + new BspServiceMaster(serverPortList, sessionMsecTimeout, context, - this, - graphState); - if (LOG.isInfoEnabled()) { - LOG.info("setup: Registering health of this worker..."); - } - serviceWorker.setup(); - } - } catch (Exception e) { - LOG.error("setup: Caught exception just before end of setup", e); - if (zkManager != null ) { - zkManager.offlineZooKeeperServers( - ZooKeeperManager.State.FAILED); - } - throw new RuntimeException( - "setup: Offlining servers due to exception...", e); - } - context.setStatus(getMapFunctions().toString() + " starting..."); - } - - @Override - public void map(Object key, Object value, Context context) - throws IOException, InterruptedException { - // map() only does computation - // 1) Run checkpoint per frequency policy. - // 2) For every vertex on this mapper, run the compute() function - // 3) Wait until all messaging is done. - // 4) Check if all vertices are done. If not goto 2). - // 5) Dump output. - if (done == true) { - return; + this), + context); + masterThread.start(); + } + if ((mapFunctions == MapFunctions.WORKER_ONLY) || + (mapFunctions == MapFunctions.ALL) || + (mapFunctions == MapFunctions.ALL_EXCEPT_ZOOKEEPER)) { + if (LOG.isInfoEnabled()) { + LOG.info("setup: Starting up BspServiceWorker..."); } - if ((serviceWorker != null) && (graphState.getNumVertices() == 0)) { - return; + serviceWorker = new BspServiceWorker( + serverPortList, + sessionMsecTimeout, + context, + this, + graphState); + if (LOG.isInfoEnabled()) { + LOG.info("setup: Registering health of this worker..."); } + serviceWorker.setup(); + } + } catch (IOException e) { + LOG.error("setup: Caught exception just before end of setup", e); + if (zkManager != null) { + zkManager.offlineZooKeeperServers( + ZooKeeperManager.State.FAILED); + } + throw new RuntimeException( + "setup: Offlining servers due to exception...", e); + } + context.setStatus(getMapFunctions().toString() + " starting..."); + } + + @Override + public void map(Object key, Object value, Context context) + throws IOException, InterruptedException { + // map() only does computation + // 1) Run checkpoint per frequency policy. + // 2) For every vertex on this mapper, run the compute() function + // 3) Wait until all messaging is done. + // 4) Check if all vertices are done. If not goto 2). + // 5) Dump output. + if (done) { + return; + } + if ((serviceWorker != null) && (graphState.getNumVertices() == 0)) { + return; + } - if ((mapFunctions == MapFunctions.MASTER_ZOOKEEPER_ONLY) || - (mapFunctions == MapFunctions.MASTER_ONLY)) { - if (LOG.isInfoEnabled()) { - LOG.info("map: No need to do anything when not a worker"); - } - return; - } + if ((mapFunctions == MapFunctions.MASTER_ZOOKEEPER_ONLY) || + (mapFunctions == MapFunctions.MASTER_ONLY)) { + if (LOG.isInfoEnabled()) { + LOG.info("map: No need to do anything when not a worker"); + } + return; + } - if (mapAlreadyRun) { - throw new RuntimeException("In BSP, map should have only been" + - " run exactly once, (already run)"); - } - mapAlreadyRun = true; - - graphState.setSuperstep(serviceWorker.getSuperstep()). - setContext(context).setGraphMapper(this); - - try { - serviceWorker.getWorkerContext().preApplication(); - } catch (InstantiationException e) { - LOG.fatal("map: preApplication failed in instantiation", e); - throw new RuntimeException( - "map: preApplication failed in instantiation", e); - } catch (IllegalAccessException e) { - LOG.fatal("map: preApplication failed in access", e); - throw new RuntimeException( - "map: preApplication failed in access",e ); - } - context.progress(); - - List partitionStatsList = - new ArrayList(); - do { - long superstep = serviceWorker.getSuperstep(); - - graphState.setSuperstep(superstep); - - Collection masterAssignedPartitionOwners = - serviceWorker.startSuperstep(); - if (zkManager != null && zkManager.runsZooKeeper()) { - if (LOG.isInfoEnabled()) { - LOG.info("map: Chosen to run ZooKeeper..."); - } - context.setStatus("map: Running Zookeeper Server"); - } - - if (LOG.isDebugEnabled()) { - LOG.debug("map: " + MemoryUtils.getRuntimeMemoryStats()); - } - context.progress(); + if (mapAlreadyRun) { + throw new RuntimeException("In BSP, map should have only been" + + " run exactly once, (already run)"); + } + mapAlreadyRun = true; + + graphState.setSuperstep(serviceWorker.getSuperstep()). + setContext(context).setGraphMapper(this); + + try { + serviceWorker.getWorkerContext().preApplication(); + } catch (InstantiationException e) { + LOG.fatal("map: preApplication failed in instantiation", e); + throw new RuntimeException( + "map: preApplication failed in instantiation", e); + } catch (IllegalAccessException e) { + LOG.fatal("map: preApplication failed in access", e); + throw new RuntimeException( + "map: preApplication failed in access", e); + } + context.progress(); - serviceWorker.exchangeVertexPartitions( - masterAssignedPartitionOwners); - context.progress(); + List partitionStatsList = + new ArrayList(); + do { + long superstep = serviceWorker.getSuperstep(); - // Might need to restart from another superstep - // (manually or automatic), or store a checkpoint - if (serviceWorker.getRestartedSuperstep() == superstep) { - if (LOG.isInfoEnabled()) { - LOG.info("map: Loading from checkpoint " + superstep); - } - serviceWorker.loadCheckpoint( - serviceWorker.getRestartedSuperstep()); - } else if (serviceWorker.checkpointFrequencyMet(superstep)) { - serviceWorker.storeCheckpoint(); - } - - serviceWorker.getWorkerContext().setGraphState(graphState); - serviceWorker.getWorkerContext().preSuperstep(); - context.progress(); + graphState.setSuperstep(superstep); - partitionStatsList.clear(); - for (Partition partition : - serviceWorker.getPartitionMap().values()) { - PartitionStats partitionStats = - new PartitionStats(partition.getPartitionId(), 0, 0, 0); - for (BasicVertex basicVertex : - partition.getVertices()) { - // Make sure every vertex has the current - // graphState before computing - basicVertex.setGraphState(graphState); - if (basicVertex.isHalted() - && !Iterables.isEmpty(basicVertex.getMessages())) { - basicVertex.halt = false; - } - if (!basicVertex.isHalted()) { - Iterator vertexMsgIt = - basicVertex.getMessages().iterator(); - context.progress(); - basicVertex.compute(vertexMsgIt); - basicVertex.releaseResources(); - } - if (basicVertex.isHalted()) { - partitionStats.incrFinishedVertexCount(); - } - partitionStats.incrVertexCount(); - partitionStats.addEdgeCount(basicVertex.getNumOutEdges()); - } - partitionStatsList.add(partitionStats); - } - } while (!serviceWorker.finishSuperstep(partitionStatsList)); + Collection masterAssignedPartitionOwners = + serviceWorker.startSuperstep(); + if (zkManager != null && zkManager.runsZooKeeper()) { if (LOG.isInfoEnabled()) { - LOG.info("map: BSP application done " + - "(global vertices marked done)"); + LOG.info("map: Chosen to run ZooKeeper..."); } + context.setStatus("map: Running Zookeeper Server"); + } - serviceWorker.getWorkerContext().postApplication(); - context.progress(); - } + if (LOG.isDebugEnabled()) { + LOG.debug("map: " + MemoryUtils.getRuntimeMemoryStats()); + } + context.progress(); - @Override - public void cleanup(Context context) - throws IOException, InterruptedException { + serviceWorker.exchangeVertexPartitions( + masterAssignedPartitionOwners); + context.progress(); + + // Might need to restart from another superstep + // (manually or automatic), or store a checkpoint + if (serviceWorker.getRestartedSuperstep() == superstep) { if (LOG.isInfoEnabled()) { - LOG.info("cleanup: Starting for " + getMapFunctions()); + LOG.info("map: Loading from checkpoint " + superstep); } - if (done) { - return; + serviceWorker.loadCheckpoint( + serviceWorker.getRestartedSuperstep()); + } else if (serviceWorker.checkpointFrequencyMet(superstep)) { + serviceWorker.storeCheckpoint(); + } + + serviceWorker.getWorkerContext().setGraphState(graphState); + serviceWorker.getWorkerContext().preSuperstep(); + context.progress(); + + partitionStatsList.clear(); + for (Partition partition : + serviceWorker.getPartitionMap().values()) { + PartitionStats partitionStats = + new PartitionStats(partition.getPartitionId(), 0, 0, 0); + for (BasicVertex basicVertex : + partition.getVertices()) { + // Make sure every vertex has the current + // graphState before computing + basicVertex.setGraphState(graphState); + if (basicVertex.isHalted() & + !Iterables.isEmpty(basicVertex.getMessages())) { + basicVertex.halt = false; + } + if (!basicVertex.isHalted()) { + Iterator vertexMsgIt = + basicVertex.getMessages().iterator(); + context.progress(); + basicVertex.compute(vertexMsgIt); + basicVertex.releaseResources(); + } + if (basicVertex.isHalted()) { + partitionStats.incrFinishedVertexCount(); + } + partitionStats.incrVertexCount(); + partitionStats.addEdgeCount(basicVertex.getNumOutEdges()); } + partitionStatsList.add(partitionStats); + } + } while (!serviceWorker.finishSuperstep(partitionStatsList)); + if (LOG.isInfoEnabled()) { + LOG.info("map: BSP application done " + + "(global vertices marked done)"); + } - if (serviceWorker != null) { - serviceWorker.cleanup(); - } - try { - if (masterThread != null) { - masterThread.join(); - } - } catch (InterruptedException e) { - // cleanup phase -- just log the error - LOG.error("cleanup: Master thread couldn't join"); - } - if (zkManager != null) { - zkManager.offlineZooKeeperServers( - ZooKeeperManager.State.FINISHED); - } + serviceWorker.getWorkerContext().postApplication(); + context.progress(); + } + + @Override + public void cleanup(Context context) + throws IOException, InterruptedException { + if (LOG.isInfoEnabled()) { + LOG.info("cleanup: Starting for " + getMapFunctions()); + } + if (done) { + return; } - @Override - public void run(Context context) throws IOException, InterruptedException { - // Notify the master quicker if there is worker failure rather than - // waiting for ZooKeeper to timeout and delete the ephemeral znodes - try { - setup(context); - while (context.nextKeyValue()) { - map(context.getCurrentKey(), - context.getCurrentValue(), - context); - } - cleanup(context); - } catch (Exception e) { - if (mapFunctions == MapFunctions.WORKER_ONLY) { - serviceWorker.failureCleanup(); - } - throw new IllegalStateException( - "run: Caught an unrecoverable exception " + e.getMessage(), e); - } + if (serviceWorker != null) { + serviceWorker.cleanup(); + } + try { + if (masterThread != null) { + masterThread.join(); + } + } catch (InterruptedException e) { + // cleanup phase -- just log the error + LOG.error("cleanup: Master thread couldn't join"); + } + if (zkManager != null) { + zkManager.offlineZooKeeperServers( + ZooKeeperManager.State.FINISHED); + } + } + + @Override + public void run(Context context) throws IOException, InterruptedException { + // Notify the master quicker if there is worker failure rather than + // waiting for ZooKeeper to timeout and delete the ephemeral znodes + try { + setup(context); + while (context.nextKeyValue()) { + map(context.getCurrentKey(), + context.getCurrentValue(), + context); + } + cleanup(context); + } catch (IOException e) { + if (mapFunctions == MapFunctions.WORKER_ONLY) { + serviceWorker.failureCleanup(); + } + throw new IllegalStateException( + "run: Caught an unrecoverable exception " + e.getMessage(), e); } + } } diff --git a/src/main/java/org/apache/giraph/graph/GraphState.java b/src/main/java/org/apache/giraph/graph/GraphState.java index d1474a94e..a9965dd74 100644 --- a/src/main/java/org/apache/giraph/graph/GraphState.java +++ b/src/main/java/org/apache/giraph/graph/GraphState.java @@ -22,84 +22,120 @@ import org.apache.hadoop.io.WritableComparable; import org.apache.hadoop.mapreduce.Mapper; -/* +/** * Global state of the graph. Should be treated as a singleton (but is kept * as a regular bean to facilitate ease of unit testing) * - * @param vertex id - * @param vertex data - * @param edge data - * @param message data + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data */ @SuppressWarnings("rawtypes") public class GraphState { - /** Graph-wide superstep */ - private long superstep = 0; - /** Graph-wide number of vertices */ - private long numVertices = -1; - /** Graph-wide number of edges */ - private long numEdges = -1; - /** Graph-wide map context */ - private Mapper.Context context; - /** Graph-wide BSP Mapper for this Vertex */ - private GraphMapper graphMapper; - /** Graph-wide worker communications */ - private WorkerCommunications workerCommunications; +E extends Writable, M extends Writable> { + /** Graph-wide superstep */ + private long superstep = 0; + /** Graph-wide number of vertices */ + private long numVertices = -1; + /** Graph-wide number of edges */ + private long numEdges = -1; + /** Graph-wide map context */ + private Mapper.Context context; + /** Graph-wide BSP Mapper for this Vertex */ + private GraphMapper graphMapper; + /** Graph-wide worker communications */ + private WorkerCommunications workerCommunications; - public long getSuperstep() { - return superstep; - } + public long getSuperstep() { + return superstep; + } - public GraphState setSuperstep(long superstep) { - this.superstep = superstep; - return this; - } + /** + * Set the current superstep. + * + * @param superstep Current superstep to use. + * @return Returns this object. + */ + public GraphState setSuperstep(long superstep) { + this.superstep = superstep; + return this; + } - public long getNumVertices() { - return numVertices; - } + public long getNumVertices() { + return numVertices; + } - public GraphState setNumVertices(long numVertices) { - this.numVertices = numVertices; - return this; - } + /** + * Set the current number of vertices. + * + * @param numVertices Current number of vertices. + * @return Returns this object. + */ + public GraphState setNumVertices(long numVertices) { + this.numVertices = numVertices; + return this; + } - public long getNumEdges() { - return numEdges; - } + public long getNumEdges() { + return numEdges; + } - public GraphState setNumEdges(long numEdges) { - this.numEdges = numEdges; - return this; - } + /** + * Set the current number of edges. + * + * @param numEdges Current number of edges. + * @return Returns this object. + */ + public GraphState setNumEdges(long numEdges) { + this.numEdges = numEdges; + return this; + } - public Mapper.Context getContext() { - return context; - } + public Mapper.Context getContext() { + return context; + } - public GraphState setContext(Mapper.Context context) { - this.context = context; - return this; - } + /** + * Set the current context. + * + * @param context Current context. + * @return Returns this object. + */ + public GraphState setContext(Mapper.Context context) { + this.context = context; + return this; + } - public GraphMapper getGraphMapper() { - return graphMapper; - } + public GraphMapper getGraphMapper() { + return graphMapper; + } - public GraphState setGraphMapper( - GraphMapper graphMapper) { - this.graphMapper = graphMapper; - return this; - } + /** + * Set the current graph mapper. + * + * @param graphMapper Current graph mapper. + * @return Returns this object. + */ + public GraphState setGraphMapper( + GraphMapper graphMapper) { + this.graphMapper = graphMapper; + return this; + } - public GraphState setWorkerCommunications( - WorkerCommunications workerCommunications) { - this.workerCommunications = workerCommunications; - return this; - } + /** + * Set the current worker communications. + * + * @param workerCommunications Current worker communications. + * @return Returns this object. + */ + public GraphState setWorkerCommunications( + WorkerCommunications workerCommunications) { + this.workerCommunications = workerCommunications; + return this; + } - public WorkerCommunications getWorkerCommunications() { - return workerCommunications; - } + public WorkerCommunications getWorkerCommunications() { + return workerCommunications; + } } diff --git a/src/main/java/org/apache/giraph/graph/HashMapVertex.java b/src/main/java/org/apache/giraph/graph/HashMapVertex.java index d2f86cd29..50d49484c 100644 --- a/src/main/java/org/apache/giraph/graph/HashMapVertex.java +++ b/src/main/java/org/apache/giraph/graph/HashMapVertex.java @@ -48,196 +48,196 @@ */ @SuppressWarnings("rawtypes") public abstract class HashMapVertex - extends MutableVertex { - /** Class logger */ - private static final Logger LOG = Logger.getLogger(HashMapVertex.class); - /** Vertex id */ - private I vertexId = null; - /** Vertex value */ - private V vertexValue = null; - /** Map of destination vertices and their edge values */ - protected final Map> destEdgeMap = - new HashMap>(); - /** List of incoming messages from the previous superstep */ - private final List msgList = Lists.newArrayList(); - - @Override - public void initialize( - I vertexId, V vertexValue, Map edges, Iterable messages) { - if (vertexId != null) { - setVertexId(vertexId); - } - if (vertexValue != null) { - setVertexValue(vertexValue); - } - if (edges != null && !edges.isEmpty()) { - for (Map.Entry entry : edges.entrySet()) { - destEdgeMap.put( - entry.getKey(), - new Edge(entry.getKey(), entry.getValue())); - } - } - if (messages != null) { - Iterables.addAll(msgList, messages); - } - } - - @Override - public final boolean addEdge(I targetVertexId, E edgeValue) { - if (destEdgeMap.put( - targetVertexId, - new Edge(targetVertexId, edgeValue)) != null) { - if (LOG.isDebugEnabled()) { - LOG.debug("addEdge: Vertex=" + vertexId + - ": already added an edge value for dest vertex id " + - targetVertexId); - } - return false; - } else { - return true; - } - } - - @Override - public long getSuperstep() { - return getGraphState().getSuperstep(); - } - - @Override - public final void setVertexId(I vertexId) { - this.vertexId = vertexId; - } - - @Override - public final I getVertexId() { - return vertexId; - } - - @Override - public final V getVertexValue() { - return vertexValue; - } - - @Override - public final void setVertexValue(V vertexValue) { - this.vertexValue = vertexValue; - } - - @Override - public E getEdgeValue(I targetVertexId) { - Edge edge = destEdgeMap.get(targetVertexId); - return edge != null ? edge.getEdgeValue() : null; - } - - @Override - public boolean hasEdge(I targetVertexId) { - return destEdgeMap.containsKey(targetVertexId); - } - - /** - * Get an iterator to the edges on this vertex. - * - * @return A sorted iterator, as defined by the sort-order - * of the vertex ids - */ - @Override - public Iterator iterator() { - return destEdgeMap.keySet().iterator(); - } - - @Override - public int getNumOutEdges() { - return destEdgeMap.size(); - } - - @Override - public E removeEdge(I targetVertexId) { - Edge edge = destEdgeMap.remove(targetVertexId); - if (edge != null) { - return edge.getEdgeValue(); - } else { - return null; - } - } - - @Override - public final void sendMsgToAllEdges(M msg) { - if (msg == null) { - throw new IllegalArgumentException( - "sendMsgToAllEdges: Cannot send null message to all edges"); - } - for (Edge edge : destEdgeMap.values()) { - sendMsg(edge.getDestVertexId(), msg); - } - } - - @Override - final public void readFields(DataInput in) throws IOException { - vertexId = BspUtils.createVertexIndex(getConf()); - vertexId.readFields(in); - boolean hasVertexValue = in.readBoolean(); - if (hasVertexValue) { - vertexValue = BspUtils.createVertexValue(getConf()); - vertexValue.readFields(in); - } - long edgeMapSize = in.readLong(); - for (long i = 0; i < edgeMapSize; ++i) { - Edge edge = new Edge(); - edge.setConf(getConf()); - edge.readFields(in); - addEdge(edge.getDestVertexId(), edge.getEdgeValue()); - } - long msgListSize = in.readLong(); - for (long i = 0; i < msgListSize; ++i) { - M msg = BspUtils.createMessageValue(getConf()); - msg.readFields(in); - msgList.add(msg); - } - halt = in.readBoolean(); - } - - @Override - final public void write(DataOutput out) throws IOException { - vertexId.write(out); - out.writeBoolean(vertexValue != null); - if (vertexValue != null) { - vertexValue.write(out); - } - out.writeLong(destEdgeMap.size()); - for (Edge edge : destEdgeMap.values()) { - edge.write(out); - } - out.writeLong(msgList.size()); - for (M msg : msgList) { - msg.write(out); - } - out.writeBoolean(halt); - } - - @Override - void putMessages(Iterable messages) { - msgList.clear(); - for (M message : messages) { - msgList.add(message); - } - } - - @Override - public Iterable getMessages() { - return Iterables.unmodifiableIterable(msgList); - } - - @Override - void releaseResources() { - // Hint to GC to free the messages - msgList.clear(); - } - - @Override - public String toString() { - return "Vertex(id=" + getVertexId() + ",value=" + getVertexValue() + - ",#edges=" + destEdgeMap.size() + ")"; - } + V extends Writable, E extends Writable, M extends Writable> + extends MutableVertex { + /** Class logger */ + private static final Logger LOG = Logger.getLogger(HashMapVertex.class); + /** Map of destination vertices and their edge values */ + protected final Map> destEdgeMap = + new HashMap>(); + /** Vertex id */ + private I vertexId = null; + /** Vertex value */ + private V vertexValue = null; + /** List of incoming messages from the previous superstep */ + private final List msgList = Lists.newArrayList(); + + @Override + public void initialize( + I vertexId, V vertexValue, Map edges, Iterable messages) { + if (vertexId != null) { + setVertexId(vertexId); + } + if (vertexValue != null) { + setVertexValue(vertexValue); + } + if (edges != null && !edges.isEmpty()) { + for (Map.Entry entry : edges.entrySet()) { + destEdgeMap.put( + entry.getKey(), + new Edge(entry.getKey(), entry.getValue())); + } + } + if (messages != null) { + Iterables.addAll(msgList, messages); + } + } + + @Override + public final boolean addEdge(I targetVertexId, E edgeValue) { + if (destEdgeMap.put( + targetVertexId, + new Edge(targetVertexId, edgeValue)) != null) { + if (LOG.isDebugEnabled()) { + LOG.debug("addEdge: Vertex=" + vertexId + + ": already added an edge value for dest vertex id " + + targetVertexId); + } + return false; + } else { + return true; + } + } + + @Override + public long getSuperstep() { + return getGraphState().getSuperstep(); + } + + @Override + public final void setVertexId(I vertexId) { + this.vertexId = vertexId; + } + + @Override + public final I getVertexId() { + return vertexId; + } + + @Override + public final V getVertexValue() { + return vertexValue; + } + + @Override + public final void setVertexValue(V vertexValue) { + this.vertexValue = vertexValue; + } + + @Override + public E getEdgeValue(I targetVertexId) { + Edge edge = destEdgeMap.get(targetVertexId); + return edge != null ? edge.getEdgeValue() : null; + } + + @Override + public boolean hasEdge(I targetVertexId) { + return destEdgeMap.containsKey(targetVertexId); + } + + /** + * Get an iterator to the edges on this vertex. + * + * @return A sorted iterator, as defined by the sort-order + * of the vertex ids + */ + @Override + public Iterator iterator() { + return destEdgeMap.keySet().iterator(); + } + + @Override + public int getNumOutEdges() { + return destEdgeMap.size(); + } + + @Override + public E removeEdge(I targetVertexId) { + Edge edge = destEdgeMap.remove(targetVertexId); + if (edge != null) { + return edge.getEdgeValue(); + } else { + return null; + } + } + + @Override + public final void sendMsgToAllEdges(M msg) { + if (msg == null) { + throw new IllegalArgumentException( + "sendMsgToAllEdges: Cannot send null message to all edges"); + } + for (Edge edge : destEdgeMap.values()) { + sendMsg(edge.getDestVertexId(), msg); + } + } + + @Override + public final void readFields(DataInput in) throws IOException { + vertexId = BspUtils.createVertexIndex(getConf()); + vertexId.readFields(in); + boolean hasVertexValue = in.readBoolean(); + if (hasVertexValue) { + vertexValue = BspUtils.createVertexValue(getConf()); + vertexValue.readFields(in); + } + long edgeMapSize = in.readLong(); + for (long i = 0; i < edgeMapSize; ++i) { + Edge edge = new Edge(); + edge.setConf(getConf()); + edge.readFields(in); + addEdge(edge.getDestVertexId(), edge.getEdgeValue()); + } + long msgListSize = in.readLong(); + for (long i = 0; i < msgListSize; ++i) { + M msg = BspUtils.createMessageValue(getConf()); + msg.readFields(in); + msgList.add(msg); + } + halt = in.readBoolean(); + } + + @Override + public final void write(DataOutput out) throws IOException { + vertexId.write(out); + out.writeBoolean(vertexValue != null); + if (vertexValue != null) { + vertexValue.write(out); + } + out.writeLong(destEdgeMap.size()); + for (Edge edge : destEdgeMap.values()) { + edge.write(out); + } + out.writeLong(msgList.size()); + for (M msg : msgList) { + msg.write(out); + } + out.writeBoolean(halt); + } + + @Override + void putMessages(Iterable messages) { + msgList.clear(); + for (M message : messages) { + msgList.add(message); + } + } + + @Override + public Iterable getMessages() { + return Iterables.unmodifiableIterable(msgList); + } + + @Override + void releaseResources() { + // Hint to GC to free the messages + msgList.clear(); + } + + @Override + public String toString() { + return "Vertex(id=" + getVertexId() + ",value=" + getVertexValue() + + ",#edges=" + destEdgeMap.size() + ")"; + } } diff --git a/src/main/java/org/apache/giraph/graph/IntIntNullIntVertex.java b/src/main/java/org/apache/giraph/graph/IntIntNullIntVertex.java index 2d6b3c5ff..2b0d00c52 100644 --- a/src/main/java/org/apache/giraph/graph/IntIntNullIntVertex.java +++ b/src/main/java/org/apache/giraph/graph/IntIntNullIntVertex.java @@ -30,135 +30,138 @@ import java.util.Map; /** - * Simple implementation of {@link BasicVertex} using an int as id, value and message. - * Edges are immutable and unweighted. This class aims to be as memory efficient as possible. + * Simple implementation of {@link BasicVertex} using an int as id, value and + * message. Edges are immutable and unweighted. This class aims to be as + * memory efficient as possible. */ public abstract class IntIntNullIntVertex extends - BasicVertex { - - private int id; - private int value; - - private int[] neighbors; - private int[] messages; - - @Override - public void initialize(IntWritable vertexId, IntWritable vertexValue, - Map edges, - Iterable messages) { - id = vertexId.get(); - value = vertexValue.get(); - this.neighbors = new int[edges.size()]; - int n = 0; - for (IntWritable neighbor : edges.keySet()) { - this.neighbors[n++] = neighbor.get(); - } - this.messages = new int[Iterables.size(messages)]; - n = 0; - for (IntWritable message : messages) { - this.messages[n++] = message.get(); - } + BasicVertex { + /** Int represented vertex id */ + private int id; + /** Int represented vertex value */ + private int value; + /** Int array of neighbor vertex ids */ + private int[] neighbors; + /** Int array of messages */ + private int[] messages; + + @Override + public void initialize(IntWritable vertexId, IntWritable vertexValue, + Map edges, + Iterable messages) { + id = vertexId.get(); + value = vertexValue.get(); + this.neighbors = new int[edges.size()]; + int n = 0; + for (IntWritable neighbor : edges.keySet()) { + this.neighbors[n++] = neighbor.get(); } - - @Override - public IntWritable getVertexId() { - return new IntWritable(id); - } - - @Override - public IntWritable getVertexValue() { - return new IntWritable(value); - } - - @Override - public void setVertexValue(IntWritable vertexValue) { - value = vertexValue.get(); - } - - @Override - public Iterator iterator() { - return new UnmodifiableIntArrayIterator(neighbors); - } - - @Override - public NullWritable getEdgeValue(IntWritable targetVertexId) { - return NullWritable.get(); + this.messages = new int[Iterables.size(messages)]; + n = 0; + for (IntWritable message : messages) { + this.messages[n++] = message.get(); } - - @Override - public boolean hasEdge(IntWritable targetVertexId) { - for (int neighbor : neighbors) { - if (neighbor == targetVertexId.get()) { - return true; - } - } - return false; + } + + @Override + public IntWritable getVertexId() { + return new IntWritable(id); + } + + @Override + public IntWritable getVertexValue() { + return new IntWritable(value); + } + + @Override + public void setVertexValue(IntWritable vertexValue) { + value = vertexValue.get(); + } + + @Override + public Iterator iterator() { + return new UnmodifiableIntArrayIterator(neighbors); + } + + @Override + public NullWritable getEdgeValue(IntWritable targetVertexId) { + return NullWritable.get(); + } + + @Override + public boolean hasEdge(IntWritable targetVertexId) { + for (int neighbor : neighbors) { + if (neighbor == targetVertexId.get()) { + return true; + } } - - @Override - public int getNumOutEdges() { - return neighbors.length; + return false; + } + + @Override + public int getNumOutEdges() { + return neighbors.length; + } + + @Override + public void sendMsgToAllEdges(final IntWritable message) { + for (int neighbor : neighbors) { + sendMsg(new IntWritable(neighbor), message); } - - @Override - public void sendMsgToAllEdges(final IntWritable message) { - for (int neighbor : neighbors) { - sendMsg(new IntWritable(neighbor), message); - } + } + + @Override + public Iterable getMessages() { + return new Iterable() { + @Override + public Iterator iterator() { + return new UnmodifiableIntArrayIterator(messages); + } + }; + } + + @Override + public void putMessages(Iterable newMessages) { + messages = new int[Iterables.size(newMessages)]; + int n = 0; + for (IntWritable message : newMessages) { + messages[n++] = message.get(); } - - @Override - public Iterable getMessages() { - return new Iterable() { - @Override - public Iterator iterator() { - return new UnmodifiableIntArrayIterator(messages); - } - }; + } + + @Override + void releaseResources() { + messages = new int[0]; + } + + @Override + public void write(final DataOutput out) throws IOException { + out.writeInt(id); + out.writeInt(value); + out.writeInt(neighbors.length); + for (int n = 0; n < neighbors.length; n++) { + out.writeInt(neighbors[n]); } - - @Override - public void putMessages(Iterable newMessages) { - messages = new int[Iterables.size(newMessages)]; - int n = 0; - for (IntWritable message : newMessages) { - messages[n++] = message.get(); - } + out.writeInt(messages.length); + for (int n = 0; n < messages.length; n++) { + out.writeInt(messages[n]); } - - @Override - void releaseResources() { - messages = new int[0]; - } - - @Override - public void write(final DataOutput out) throws IOException { - out.writeInt(id); - out.writeInt(value); - out.writeInt(neighbors.length); - for (int n = 0; n < neighbors.length; n++) { - out.writeInt(neighbors[n]); - } - out.writeInt(messages.length); - for (int n = 0; n < messages.length; n++) { - out.writeInt(messages[n]); - } + } + + @Override + public void readFields(DataInput in) throws IOException { + id = in.readInt(); + value = in.readInt(); + int numEdges = in.readInt(); + neighbors = new int[numEdges]; + for (int n = 0; n < numEdges; n++) { + neighbors[n] = in.readInt(); } - - @Override - public void readFields(DataInput in) throws IOException { - id = in.readInt(); - value = in.readInt(); - int numEdges = in.readInt(); - neighbors = new int[numEdges]; - for (int n = 0; n < numEdges; n++) { - neighbors[n] = in.readInt(); - } - int numMessages = in.readInt(); - messages = new int[numMessages]; - for (int n = 0; n < numMessages; n++) { - messages[n] = in.readInt(); - } + int numMessages = in.readInt(); + messages = new int[numMessages]; + for (int n = 0; n < numMessages; n++) { + messages[n] = in.readInt(); } + } } diff --git a/src/main/java/org/apache/giraph/graph/LongDoubleFloatDoubleVertex.java b/src/main/java/org/apache/giraph/graph/LongDoubleFloatDoubleVertex.java index b87573623..5de14e2da 100644 --- a/src/main/java/org/apache/giraph/graph/LongDoubleFloatDoubleVertex.java +++ b/src/main/java/org/apache/giraph/graph/LongDoubleFloatDoubleVertex.java @@ -34,279 +34,304 @@ import java.util.Iterator; import java.util.Map; +/** + * Optimized vertex implementation for + * + */ public abstract class LongDoubleFloatDoubleVertex extends - MutableVertex { - /** Class logger */ - private static final Logger LOG = - Logger.getLogger(LongDoubleFloatDoubleVertex.class); - - private long vertexId; - private double vertexValue; - private OpenLongFloatHashMap verticesWithEdgeValues = - new OpenLongFloatHashMap(); - private DoubleArrayList messageList = new DoubleArrayList(); - - @Override - public void initialize(LongWritable vertexIdW, DoubleWritable vertexValueW, - Map edgesW, - Iterable messagesW) { - if (vertexIdW != null ) { - vertexId = vertexIdW.get(); - } - if (vertexValueW != null) { - vertexValue = vertexValueW.get(); - } - if (edgesW != null) { - for (Map.Entry entry : - edgesW.entrySet()) { - verticesWithEdgeValues.put(entry.getKey().get(), - entry.getValue().get()); - } - } - if (messagesW != null) { - for(DoubleWritable m : messagesW) { - messageList.add(m.get()); - } - } + MutableVertex { + /** Class logger */ + private static final Logger LOG = + Logger.getLogger(LongDoubleFloatDoubleVertex.class); + /** Long vertex id */ + private long vertexId; + /** Double vertex value */ + private double vertexValue; + /** Stores the edges */ + private OpenLongFloatHashMap verticesWithEdgeValues = + new OpenLongFloatHashMap(); + /** Message list storage */ + private DoubleArrayList messageList = new DoubleArrayList(); + + @Override + public void initialize(LongWritable vertexIdW, DoubleWritable vertexValueW, + Map edgesW, + Iterable messagesW) { + if (vertexIdW != null) { + vertexId = vertexIdW.get(); } - - @Override - public final boolean addEdge(LongWritable targetId, - FloatWritable edgeValue) { - if (verticesWithEdgeValues.put(targetId.get(), edgeValue.get())) { - if (LOG.isDebugEnabled()) { - LOG.debug("addEdge: Vertex=" + vertexId + - ": already added an edge value for dest vertex id " + - targetId.get()); - } - return false; - } else { - return true; - } + if (vertexValueW != null) { + vertexValue = vertexValueW.get(); } - - @Override - public FloatWritable removeEdge(LongWritable targetVertexId) { - long target = targetVertexId.get(); - if (verticesWithEdgeValues.containsKey(target)) { - float value = verticesWithEdgeValues.get(target); - verticesWithEdgeValues.removeKey(target); - return new FloatWritable(value); - } else { - return null; - } + if (edgesW != null) { + for (Map.Entry entry : + edgesW.entrySet()) { + verticesWithEdgeValues.put(entry.getKey().get(), + entry.getValue().get()); + } } - - @Override - public final void setVertexId(LongWritable vertexId) { - this.vertexId = vertexId.get(); + if (messagesW != null) { + for (DoubleWritable m : messagesW) { + messageList.add(m.get()); + } } - - @Override - public final LongWritable getVertexId() { - // TODO: possibly not make new objects every time? - return new LongWritable(vertexId); + } + + @Override + public final boolean addEdge(LongWritable targetId, + FloatWritable edgeValue) { + if (verticesWithEdgeValues.put(targetId.get(), edgeValue.get())) { + if (LOG.isDebugEnabled()) { + LOG.debug("addEdge: Vertex=" + vertexId + + ": already added an edge value for dest vertex id " + + targetId.get()); + } + return false; + } else { + return true; } - - @Override - public final DoubleWritable getVertexValue() { - return new DoubleWritable(vertexValue); + } + + @Override + public FloatWritable removeEdge(LongWritable targetVertexId) { + long target = targetVertexId.get(); + if (verticesWithEdgeValues.containsKey(target)) { + float value = verticesWithEdgeValues.get(target); + verticesWithEdgeValues.removeKey(target); + return new FloatWritable(value); + } else { + return null; } - - @Override - public final void setVertexValue(DoubleWritable vertexValue) { - this.vertexValue = vertexValue.get(); + } + + @Override + public final void setVertexId(LongWritable vertexId) { + this.vertexId = vertexId.get(); + } + + @Override + public final LongWritable getVertexId() { + // TODO: possibly not make new objects every time? + return new LongWritable(vertexId); + } + + @Override + public final DoubleWritable getVertexValue() { + return new DoubleWritable(vertexValue); + } + + @Override + public final void setVertexValue(DoubleWritable vertexValue) { + this.vertexValue = vertexValue.get(); + } + + @Override + public final void sendMsg(LongWritable id, DoubleWritable msg) { + if (msg == null) { + throw new IllegalArgumentException( + "sendMsg: Cannot send null message to " + id); } - - @Override - public final void sendMsg(LongWritable id, DoubleWritable msg) { - if (msg == null) { - throw new IllegalArgumentException( - "sendMsg: Cannot send null message to " + id); - } - getGraphState().getWorkerCommunications().sendMessageReq(id, msg); + getGraphState().getWorkerCommunications().sendMessageReq(id, msg); + } + + @Override + public final void sendMsgToAllEdges(final DoubleWritable msg) { + if (msg == null) { + throw new IllegalArgumentException( + "sendMsgToAllEdges: Cannot send null message to all edges"); } - - @Override - public final void sendMsgToAllEdges(final DoubleWritable msg) { - if (msg == null) { - throw new IllegalArgumentException( - "sendMsgToAllEdges: Cannot send null message to all edges"); - } - final MutableVertex vertex = this; - verticesWithEdgeValues.forEachKey(new LongProcedure() { - @Override - public boolean apply(long destVertexId) { - vertex.sendMsg(new LongWritable(destVertexId), msg); - return true; - } - }); + final MutableVertex vertex = this; + verticesWithEdgeValues.forEachKey(new LongProcedure() { + @Override + public boolean apply(long destVertexId) { + vertex.sendMsg(new LongWritable(destVertexId), msg); + return true; + } + }); + } + + @Override + public long getNumVertices() { + return getGraphState().getNumVertices(); + } + + @Override + public long getNumEdges() { + return getGraphState().getNumEdges(); + } + + @Override + public Iterator iterator() { + final long[] destVertices = verticesWithEdgeValues.keys().elements(); + final int destVerticesSize = verticesWithEdgeValues.size(); + return new Iterator() { + private int offset = 0; + @Override public boolean hasNext() { + return offset < destVerticesSize; + } + + @Override public LongWritable next() { + return new LongWritable(destVertices[offset++]); + } + + @Override public void remove() { + throw new UnsupportedOperationException( + "Mutation disallowed for edge list via iterator"); + } + }; + } + + @Override + public FloatWritable getEdgeValue(LongWritable targetVertexId) { + return new FloatWritable( + verticesWithEdgeValues.get(targetVertexId.get())); + } + + @Override + public boolean hasEdge(LongWritable targetVertexId) { + return verticesWithEdgeValues.containsKey(targetVertexId.get()); + } + + @Override + public int getNumOutEdges() { + return verticesWithEdgeValues.size(); + } + + @Override + public long getSuperstep() { + return getGraphState().getSuperstep(); + } + + @Override + public final void readFields(DataInput in) throws IOException { + vertexId = in.readLong(); + vertexValue = in.readDouble(); + long edgeMapSize = in.readLong(); + for (long i = 0; i < edgeMapSize; ++i) { + long destVertexId = in.readLong(); + float edgeValue = in.readFloat(); + verticesWithEdgeValues.put(destVertexId, edgeValue); } - - @Override - public long getNumVertices() { - return getGraphState().getNumVertices(); - } - - @Override - public long getNumEdges() { - return getGraphState().getNumEdges(); - } - - @Override - public Iterator iterator() { - final long[] destVertices = verticesWithEdgeValues.keys().elements(); - final int destVerticesSize = verticesWithEdgeValues.size(); - return new Iterator() { - int offset = 0; - @Override public boolean hasNext() { - return offset < destVerticesSize; - } - - @Override public LongWritable next() { - return new LongWritable(destVertices[offset++]); - } - - @Override public void remove() { - throw new UnsupportedOperationException( - "Mutation disallowed for edge list via iterator"); - } - }; + long msgListSize = in.readLong(); + for (long i = 0; i < msgListSize; ++i) { + messageList.add(in.readDouble()); } - - @Override - public FloatWritable getEdgeValue(LongWritable targetVertexId) { - return new FloatWritable( - verticesWithEdgeValues.get(targetVertexId.get())); - } - - @Override - public boolean hasEdge(LongWritable targetVertexId) { - return verticesWithEdgeValues.containsKey(targetVertexId.get()); - } - - @Override - public int getNumOutEdges() { - return verticesWithEdgeValues.size(); - } - - @Override - public long getSuperstep() { - return getGraphState().getSuperstep(); - } - - @Override - final public void readFields(DataInput in) throws IOException { - vertexId = in.readLong(); - vertexValue = in.readDouble(); - long edgeMapSize = in.readLong(); - for (long i = 0; i < edgeMapSize; ++i) { - long destVertexId = in.readLong(); - float edgeValue = in.readFloat(); - verticesWithEdgeValues.put(destVertexId, edgeValue); + halt = in.readBoolean(); + } + + @Override + public final void write(final DataOutput out) throws IOException { + out.writeLong(vertexId); + out.writeDouble(vertexValue); + out.writeLong(verticesWithEdgeValues.size()); + verticesWithEdgeValues.forEachPair(new LongFloatProcedure() { + @Override + public boolean apply(long destVertexId, float edgeValue) { + try { + out.writeLong(destVertexId); + out.writeFloat(edgeValue); + } catch (IOException e) { + throw new IllegalStateException( + "apply: IOException when not allowed", e); } - long msgListSize = in.readLong(); - for (long i = 0; i < msgListSize; ++i) { - messageList.add(in.readDouble()); + return true; + } + }); + out.writeLong(messageList.size()); + messageList.forEach(new DoubleProcedure() { + @Override + public boolean apply(double message) { + try { + out.writeDouble(message); + } catch (IOException e) { + throw new IllegalStateException( + "apply: IOException when not allowed", e); } - halt = in.readBoolean(); + return true; + } + }); + out.writeBoolean(halt); + } + + @Override + void putMessages(Iterable messages) { + messageList.clear(); + for (DoubleWritable message : messages) { + messageList.add(message.get()); } - - @Override - public final void write(final DataOutput out) throws IOException { - out.writeLong(vertexId); - out.writeDouble(vertexValue); - out.writeLong(verticesWithEdgeValues.size()); - verticesWithEdgeValues.forEachPair(new LongFloatProcedure() { - @Override - public boolean apply(long destVertexId, float edgeValue) { - try { - out.writeLong(destVertexId); - out.writeFloat(edgeValue); - } catch (IOException e) { - throw new IllegalStateException( - "apply: IOException when not allowed", e); - } - return true; - } - }); - out.writeLong(messageList.size()); - messageList.forEach(new DoubleProcedure() { - @Override - public boolean apply(double message) { - try { - out.writeDouble(message); - } catch (IOException e) { - throw new IllegalStateException( - "apply: IOException when not allowed", e); - } - return true; - } - }); - out.writeBoolean(halt); + } + + @Override + void releaseResources() { + // Hint to GC to free the messages + messageList.clear(); + } + + @Override + public Iterable getMessages() { + return new UnmodifiableDoubleWritableIterable(messageList); + } + + @Override + public String toString() { + return "Vertex(id=" + getVertexId() + ",value=" + getVertexValue() + + ",#edges=" + getNumOutEdges() + ")"; + } + + /** + * Helper iterable over the messages. + */ + private class UnmodifiableDoubleWritableIterable + implements Iterable { + /** Backing store of messages */ + private final DoubleArrayList elementList; + + /** + * Constructor. + * + * @param elementList Backing store of element list. + */ + public UnmodifiableDoubleWritableIterable( + DoubleArrayList elementList) { + this.elementList = elementList; } @Override - void putMessages(Iterable messages) { - messageList.clear(); - for (DoubleWritable message : messages) { - messageList.add(message.get()); - } + public Iterator iterator() { + return new UnmodifiableDoubleWritableIterator( + elementList); } - - @Override - void releaseResources() { - // Hint to GC to free the messages - messageList.clear(); + } + + /** + * Iterator over the messages. + */ + private class UnmodifiableDoubleWritableIterator + extends UnmodifiableIterator { + /** Double backing list */ + private final DoubleArrayList elementList; + /** Offset into the backing list */ + private int offset = 0; + + /** + * Constructor. + * + * @param elementList Backing store of element list. + */ + UnmodifiableDoubleWritableIterator(DoubleArrayList elementList) { + this.elementList = elementList; } @Override - public Iterable getMessages() { - return new UnmodifiableDoubleWritableIterable(messageList); + public boolean hasNext() { + return offset < elementList.size(); } @Override - public String toString() { - return "Vertex(id=" + getVertexId() + ",value=" + getVertexValue() + - ",#edges=" + getNumOutEdges() + ")"; - } - - private class UnmodifiableDoubleWritableIterable - implements Iterable { - - private final DoubleArrayList elementList; - - public UnmodifiableDoubleWritableIterable( - DoubleArrayList elementList) { - this.elementList = elementList; - } - - @Override - public Iterator iterator() { - return new UnmodifiableDoubleWritableIterator( - elementList); - } - } - - private class UnmodifiableDoubleWritableIterator - extends UnmodifiableIterator { - private final DoubleArrayList elementList; - private int offset = 0; - - UnmodifiableDoubleWritableIterator(DoubleArrayList elementList) { - this.elementList = elementList; - } - - @Override - public boolean hasNext() { - return offset < elementList.size(); - } - - @Override - public DoubleWritable next() { - return new DoubleWritable(elementList.get(offset++)); - } + public DoubleWritable next() { + return new DoubleWritable(elementList.get(offset++)); } + } } diff --git a/src/main/java/org/apache/giraph/graph/MasterThread.java b/src/main/java/org/apache/giraph/graph/MasterThread.java index 2bd2d964b..3e5df2852 100644 --- a/src/main/java/org/apache/giraph/graph/MasterThread.java +++ b/src/main/java/org/apache/giraph/graph/MasterThread.java @@ -18,6 +18,7 @@ package org.apache.giraph.graph; +import java.io.IOException; import java.util.Map; import java.util.Map.Entry; import java.util.TreeMap; @@ -29,152 +30,166 @@ import org.apache.hadoop.io.WritableComparable; import org.apache.hadoop.mapreduce.Mapper.Context; import org.apache.log4j.Logger; +import org.apache.zookeeper.KeeperException; /** * Master thread that will coordinate the activities of the tasks. It runs * on all task processes, however, will only execute its algorithm if it knows * it is the "leader" from ZooKeeper. + * + * @param Vertex id + * @param Vertex value + * @param Edge value + * @param Message data */ @SuppressWarnings("rawtypes") -public class MasterThread extends Thread { - /** Class logger */ - private static final Logger LOG = Logger.getLogger(MasterThread.class); - /** Reference to shared BspService */ - private CentralizedServiceMaster bspServiceMaster = null; - /** Context (for counters) */ - private final Context context; - /** Use superstep counters? */ - private final boolean superstepCounterOn; - /** Setup seconds */ - private double setupSecs = 0d; - /** Superstep timer (in seconds) map */ - private final Map superstepSecsMap = - new TreeMap(); - - /** Counter group name for the Giraph timers */ - public String GIRAPH_TIMERS_COUNTER_GROUP_NAME = "Giraph Timers"; +public class MasterThread extends Thread { + /** Counter group name for the Giraph timers */ + public static final String GIRAPH_TIMERS_COUNTER_GROUP_NAME = "Giraph Timers"; + /** Class logger */ + private static final Logger LOG = Logger.getLogger(MasterThread.class); + /** Reference to shared BspService */ + private CentralizedServiceMaster bspServiceMaster = null; + /** Context (for counters) */ + private final Context context; + /** Use superstep counters? */ + private final boolean superstepCounterOn; + /** Setup seconds */ + private double setupSecs = 0d; + /** Superstep timer (in seconds) map */ + private final Map superstepSecsMap = + new TreeMap(); - /** - * Constructor. - * - * @param bspServiceMaster Master that already exists and setup() has - * been called. - */ - MasterThread(BspServiceMaster bspServiceMaster, - Context context) { - super(MasterThread.class.getName()); - this.bspServiceMaster = bspServiceMaster; - this.context = context; - superstepCounterOn = context.getConfiguration().getBoolean( - GiraphJob.USE_SUPERSTEP_COUNTERS, - GiraphJob.USE_SUPERSTEP_COUNTERS_DEFAULT); - } + /** + * Constructor. + * + * @param bspServiceMaster Master that already exists and setup() has + * been called. + * @param context Context from the Mapper. + */ + MasterThread(BspServiceMaster bspServiceMaster, + Context context) { + super(MasterThread.class.getName()); + this.bspServiceMaster = bspServiceMaster; + this.context = context; + superstepCounterOn = context.getConfiguration().getBoolean( + GiraphJob.USE_SUPERSTEP_COUNTERS, + GiraphJob.USE_SUPERSTEP_COUNTERS_DEFAULT); + } - /** - * The master algorithm. The algorithm should be able to withstand - * failures and resume as necessary since the master may switch during a - * job. - */ - @Override - public void run() { - // Algorithm: - // 1. Become the master - // 2. If desired, restart from a manual checkpoint - // 3. Run all supersteps until complete - try { - long startMillis = System.currentTimeMillis(); - long endMillis = 0; - bspServiceMaster.setup(); - if (bspServiceMaster.becomeMaster() == true) { - // Attempt to create InputSplits if necessary. Bail out if that fails. - if (bspServiceMaster.getRestartedSuperstep() != BspService.UNSET_SUPERSTEP - || bspServiceMaster.createInputSplits() != -1) { - long setupMillis = (System.currentTimeMillis() - startMillis); - context.getCounter(GIRAPH_TIMERS_COUNTER_GROUP_NAME, - "Setup (milliseconds)"). - increment(setupMillis); - setupSecs = setupMillis / 1000.0d; - SuperstepState superstepState = SuperstepState.INITIAL; - long cachedSuperstep = BspService.UNSET_SUPERSTEP; - while (superstepState != SuperstepState.ALL_SUPERSTEPS_DONE) { - long startSuperstepMillis = System.currentTimeMillis(); - cachedSuperstep = bspServiceMaster.getSuperstep(); - superstepState = bspServiceMaster.coordinateSuperstep(); - long superstepMillis = System.currentTimeMillis() - - startSuperstepMillis; - superstepSecsMap.put(new Long(cachedSuperstep), - superstepMillis / 1000.0d); - if (LOG.isInfoEnabled()) { - LOG.info("masterThread: Coordination of superstep " + - cachedSuperstep + " took " + - superstepMillis / 1000.0d + - " seconds ended with state " + superstepState + - " and is now on superstep " + - bspServiceMaster.getSuperstep()); - } - if (superstepCounterOn) { - String counterPrefix; - if (cachedSuperstep == -1) { - counterPrefix = "Vertex input superstep"; - } else { - counterPrefix = "Superstep " + cachedSuperstep; - } - context.getCounter(GIRAPH_TIMERS_COUNTER_GROUP_NAME, - counterPrefix + - " (milliseconds)"). - increment(superstepMillis); - } + /** + * The master algorithm. The algorithm should be able to withstand + * failures and resume as necessary since the master may switch during a + * job. + */ + @Override + public void run() { + // Algorithm: + // 1. Become the master + // 2. If desired, restart from a manual checkpoint + // 3. Run all supersteps until complete + try { + long startMillis = System.currentTimeMillis(); + long endMillis = 0; + bspServiceMaster.setup(); + if (bspServiceMaster.becomeMaster()) { + // Attempt to create InputSplits if necessary. Bail out if that fails. + if (bspServiceMaster.getRestartedSuperstep() != + BspService.UNSET_SUPERSTEP || + bspServiceMaster.createInputSplits() != -1) { + long setupMillis = System.currentTimeMillis() - startMillis; + context.getCounter(GIRAPH_TIMERS_COUNTER_GROUP_NAME, + "Setup (milliseconds)"). + increment(setupMillis); + setupSecs = setupMillis / 1000.0d; + SuperstepState superstepState = SuperstepState.INITIAL; + long cachedSuperstep = BspService.UNSET_SUPERSTEP; + while (superstepState != SuperstepState.ALL_SUPERSTEPS_DONE) { + long startSuperstepMillis = System.currentTimeMillis(); + cachedSuperstep = bspServiceMaster.getSuperstep(); + superstepState = bspServiceMaster.coordinateSuperstep(); + long superstepMillis = System.currentTimeMillis() - + startSuperstepMillis; + superstepSecsMap.put(new Long(cachedSuperstep), + superstepMillis / 1000.0d); + if (LOG.isInfoEnabled()) { + LOG.info("masterThread: Coordination of superstep " + + cachedSuperstep + " took " + + superstepMillis / 1000.0d + + " seconds ended with state " + superstepState + + " and is now on superstep " + + bspServiceMaster.getSuperstep()); + } + if (superstepCounterOn) { + String counterPrefix; + if (cachedSuperstep == -1) { + counterPrefix = "Vertex input superstep"; + } else { + counterPrefix = "Superstep " + cachedSuperstep; + } + context.getCounter(GIRAPH_TIMERS_COUNTER_GROUP_NAME, + counterPrefix + + " (milliseconds)"). + increment(superstepMillis); + } - // If a worker failed, restart from a known good superstep - if (superstepState == SuperstepState.WORKER_FAILURE) { - bspServiceMaster.restartFromCheckpoint( - bspServiceMaster.getLastGoodCheckpoint()); - } - endMillis = System.currentTimeMillis(); - } - bspServiceMaster.setJobState(ApplicationState.FINISHED, -1, -1); - } + // If a worker failed, restart from a known good superstep + if (superstepState == SuperstepState.WORKER_FAILURE) { + bspServiceMaster.restartFromCheckpoint( + bspServiceMaster.getLastGoodCheckpoint()); } - bspServiceMaster.cleanup(); - if (!superstepSecsMap.isEmpty()) { - context.getCounter( - GIRAPH_TIMERS_COUNTER_GROUP_NAME, - "Shutdown (milliseconds)"). - increment(System.currentTimeMillis() - endMillis); - if (LOG.isInfoEnabled()) { - LOG.info("setup: Took " + setupSecs + " seconds."); - } - for (Entry entry : superstepSecsMap.entrySet()) { - if (LOG.isInfoEnabled()) { - if (entry.getKey().longValue() == - BspService.INPUT_SUPERSTEP) { - LOG.info("vertex input superstep: Took " + - entry.getValue() + " seconds."); - } else { - LOG.info("superstep " + entry.getKey() + ": Took " + - entry.getValue() + " seconds."); - } - } - } - if (LOG.isInfoEnabled()) { - LOG.info("shutdown: Took " + - (System.currentTimeMillis() - endMillis) / - 1000.0d + " seconds."); - LOG.info("total: Took " + - ((System.currentTimeMillis() / 1000.0d) - - setupSecs) + " seconds."); - } - context.getCounter( - GIRAPH_TIMERS_COUNTER_GROUP_NAME, - "Total (milliseconds)"). - increment(System.currentTimeMillis() - startMillis); + endMillis = System.currentTimeMillis(); + } + bspServiceMaster.setJobState(ApplicationState.FINISHED, -1, -1); + } + } + bspServiceMaster.cleanup(); + if (!superstepSecsMap.isEmpty()) { + context.getCounter( + GIRAPH_TIMERS_COUNTER_GROUP_NAME, + "Shutdown (milliseconds)"). + increment(System.currentTimeMillis() - endMillis); + if (LOG.isInfoEnabled()) { + LOG.info("setup: Took " + setupSecs + " seconds."); + } + for (Entry entry : superstepSecsMap.entrySet()) { + if (LOG.isInfoEnabled()) { + if (entry.getKey().longValue() == + BspService.INPUT_SUPERSTEP) { + LOG.info("vertex input superstep: Took " + + entry.getValue() + " seconds."); + } else { + LOG.info("superstep " + entry.getKey() + ": Took " + + entry.getValue() + " seconds."); } - } catch (Exception e) { - LOG.error("masterThread: Master algorithm failed: ", e); - throw new RuntimeException(e); + } + } + if (LOG.isInfoEnabled()) { + LOG.info("shutdown: Took " + + (System.currentTimeMillis() - endMillis) / + 1000.0d + " seconds."); + LOG.info("total: Took " + + ((System.currentTimeMillis() / 1000.0d) - + setupSecs) + " seconds."); } + context.getCounter( + GIRAPH_TIMERS_COUNTER_GROUP_NAME, + "Total (milliseconds)"). + increment(System.currentTimeMillis() - startMillis); + } + } catch (IOException e) { + LOG.error("masterThread: Master algorithm failed with " + + "IOException ", e); + throw new IllegalStateException(e); + } catch (InterruptedException e) { + LOG.error("masterThread: Master algorithm failed with " + + "InterruptedException", e); + throw new IllegalStateException(e); + } catch (KeeperException e) { + LOG.error("masterThread: Master algorithm failed with " + + "KeeperException", e); + throw new IllegalStateException(e); } + } } diff --git a/src/main/java/org/apache/giraph/graph/MutableVertex.java b/src/main/java/org/apache/giraph/graph/MutableVertex.java index dcfd9ae4d..42f2be9c9 100644 --- a/src/main/java/org/apache/giraph/graph/MutableVertex.java +++ b/src/main/java/org/apache/giraph/graph/MutableVertex.java @@ -27,98 +27,107 @@ /** * Interface used by VertexReader to set the properties of a new vertex * or mutate the graph. + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data */ @SuppressWarnings("rawtypes") public abstract class MutableVertex - extends BasicVertex { - /** - * Set the vertex id - * - * @param id Vertex id is set to this (instantiated by the user) - */ - public abstract void setVertexId(I id); + V extends Writable, E extends Writable, M extends Writable> + extends BasicVertex { + /** + * Set the vertex id + * + * @param id Vertex id is set to this (instantiated by the user) + */ + public abstract void setVertexId(I id); - /** - * Add an edge for this vertex (happens immediately) - * - * @param targetVertexId target vertex - * @param edgeValue value of the edge - * @return Return true if succeeded, false otherwise - */ - public abstract boolean addEdge(I targetVertexId, E edgeValue); + /** + * Add an edge for this vertex (happens immediately) + * + * @param targetVertexId target vertex + * @param edgeValue value of the edge + * @return Return true if succeeded, false otherwise + */ + public abstract boolean addEdge(I targetVertexId, E edgeValue); - /** - * Removes an edge for this vertex (happens immediately). - * - * @param targetVertexId the target vertex id of the edge to be removed. - * @return the value of the edge which was removed (or null if no - * edge existed to targetVertexId) - */ - public abstract E removeEdge(I targetVertexId); + /** + * Removes an edge for this vertex (happens immediately). + * + * @param targetVertexId the target vertex id of the edge to be removed. + * @return the value of the edge which was removed (or null if no + * edge existed to targetVertexId) + */ + public abstract E removeEdge(I targetVertexId); - /** - * Create a vertex to add to the graph. Calls initialize() for the vertex - * as well. - * - * @return A new vertex for adding to the graph - */ - public BasicVertex instantiateVertex( - I vertexId, V vertexValue, Map edges, Iterable messages) { - MutableVertex mutableVertex = - (MutableVertex) BspUtils - .createVertex(getContext().getConfiguration()); - mutableVertex.setGraphState(getGraphState()); - mutableVertex.initialize(vertexId, vertexValue, edges, messages); - return mutableVertex; - } + /** + * Create a vertex to add to the graph. Calls initialize() for the vertex + * as well. + * + * @param vertexId Id of the new vertex. + * @param vertexValue Value of the new vertex. + * @param edges Map of edges to be added to this vertex. + * @param messages Messages to be added to the vertex (typically empty) + * @return A new vertex for adding to the graph + */ + public BasicVertex instantiateVertex( + I vertexId, V vertexValue, Map edges, Iterable messages) { + MutableVertex mutableVertex = + (MutableVertex) BspUtils + .createVertex(getContext().getConfiguration()); + mutableVertex.setGraphState(getGraphState()); + mutableVertex.initialize(vertexId, vertexValue, edges, messages); + return mutableVertex; + } - /** - * Sends a request to create a vertex that will be available during the - * next superstep. Use instantiateVertex() to do the instantiation. - * - * @param vertex User created vertex - */ - public void addVertexRequest(BasicVertex vertex) - throws IOException { - getGraphState().getWorkerCommunications(). - addVertexReq(vertex); - } + /** + * Sends a request to create a vertex that will be available during the + * next superstep. Use instantiateVertex() to do the instantiation. + * + * @param vertex User created vertex + */ + public void addVertexRequest(BasicVertex vertex) + throws IOException { + getGraphState().getWorkerCommunications(). + addVertexReq(vertex); + } - /** - * Request to remove a vertex from the graph - * (applied just prior to the next superstep). - * - * @param vertexId Id of the vertex to be removed. - */ - public void removeVertexRequest(I vertexId) throws IOException { - getGraphState().getWorkerCommunications(). - removeVertexReq(vertexId); - } + /** + * Request to remove a vertex from the graph + * (applied just prior to the next superstep). + * + * @param vertexId Id of the vertex to be removed. + */ + public void removeVertexRequest(I vertexId) throws IOException { + getGraphState().getWorkerCommunications(). + removeVertexReq(vertexId); + } - /** - * Request to add an edge of a vertex in the graph - * (processed just prior to the next superstep) - * - * @param sourceVertexId Source vertex id of edge - * @param edge Edge to add - */ - public void addEdgeRequest(I sourceVertexId, Edge edge) - throws IOException { - getGraphState().getWorkerCommunications(). - addEdgeReq(sourceVertexId, edge); - } + /** + * Request to add an edge of a vertex in the graph + * (processed just prior to the next superstep) + * + * @param sourceVertexId Source vertex id of edge + * @param edge Edge to add + */ + public void addEdgeRequest(I sourceVertexId, Edge edge) + throws IOException { + getGraphState().getWorkerCommunications(). + addEdgeReq(sourceVertexId, edge); + } - /** - * Request to remove an edge of a vertex from the graph - * (processed just prior to the next superstep). - * - * @param sourceVertexId Source vertex id of edge - * @param destVertexId Destination vertex id of edge - */ - public void removeEdgeRequest(I sourceVertexId, I destVertexId) - throws IOException { - getGraphState().getWorkerCommunications(). - removeEdgeReq(sourceVertexId, destVertexId); - } + /** + * Request to remove an edge of a vertex from the graph + * (processed just prior to the next superstep). + * + * @param sourceVertexId Source vertex id of edge + * @param destVertexId Destination vertex id of edge + */ + public void removeEdgeRequest(I sourceVertexId, I destVertexId) + throws IOException { + getGraphState().getWorkerCommunications(). + removeEdgeReq(sourceVertexId, destVertexId); + } } diff --git a/src/main/java/org/apache/giraph/graph/TextAggregatorWriter.java b/src/main/java/org/apache/giraph/graph/TextAggregatorWriter.java index 113ba2e11..8012e5731 100644 --- a/src/main/java/org/apache/giraph/graph/TextAggregatorWriter.java +++ b/src/main/java/org/apache/giraph/graph/TextAggregatorWriter.java @@ -30,92 +30,98 @@ import org.apache.hadoop.mapreduce.Mapper.Context; /** - * Default implementation of {@link AggregatorWriter}. Each line consists of - * text and contains the aggregator name, the aggregator value and the + * Default implementation of {@link AggregatorWriter}. Each line consists of + * text and contains the aggregator name, the aggregator value and the * aggregator class. */ -public class TextAggregatorWriter - implements AggregatorWriter { - /** The filename of the outputfile */ - public static final String FILENAME = - "giraph.textAggregatorWriter.filename"; - /** The frequency of writing: - * - NEVER: never write, files aren't created at all - * - AT_THE_END: aggregators are written only when the computation is over - * - int: i.e. 1 is every superstep, 2 every two supersteps and so on - */ - public static final String FREQUENCY = - "giraph.textAggregatorWriter.frequency"; - private static final String DEFAULT_FILENAME = "aggregatorValues"; - /** Signal for "never write" frequency */ - public static final int NEVER = 0; - /** Signal for "write only the final values" frequency */ - public static final int AT_THE_END = -1; - /** Handle to the outputfile */ - protected FSDataOutputStream output; - private int frequency; - - @Override - @SuppressWarnings("rawtypes") - public void initialize(Context context, long attempt) throws IOException { - Configuration conf = context.getConfiguration(); - frequency = conf.getInt(FREQUENCY, NEVER); - String filename = conf.get(FILENAME, DEFAULT_FILENAME); - if (frequency != NEVER) { - Path p = new Path(filename+"_"+attempt); - FileSystem fs = FileSystem.get(conf); - if (fs.exists(p)) { - throw new RuntimeException("aggregatorWriter file already" + - " exists: " + p.getName()); - } - output = fs.create(p); - } - } +public class TextAggregatorWriter implements AggregatorWriter { + /** The filename of the outputfile */ + public static final String FILENAME = + "giraph.textAggregatorWriter.filename"; + /** Signal for "never write" frequency */ + public static final int NEVER = 0; + /** Signal for "write only the final values" frequency */ + public static final int AT_THE_END = -1; + /** The frequency of writing: + * - NEVER: never write, files aren't created at all + * - AT_THE_END: aggregators are written only when the computation is over + * - int: i.e. 1 is every superstep, 2 every two supersteps and so on + */ + public static final String FREQUENCY = + "giraph.textAggregatorWriter.frequency"; + /** Default filename for dumping aggregator values */ + private static final String DEFAULT_FILENAME = "aggregatorValues"; + /** Handle to the outputfile */ + protected FSDataOutputStream output; + /** Write every "frequency" supersteps */ + private int frequency; - @Override - final public void writeAggregator( - Map> aggregators, - long superstep) throws IOException { - - if (shouldWrite(superstep)) { - for (Entry> a: - aggregators.entrySet()) { - output.writeUTF(aggregatorToString(a.getKey(), - a.getValue(), - superstep)); - } - output.flush(); - } + @Override + @SuppressWarnings("rawtypes") + public void initialize(Context context, long attempt) throws IOException { + Configuration conf = context.getConfiguration(); + frequency = conf.getInt(FREQUENCY, NEVER); + String filename = conf.get(FILENAME, DEFAULT_FILENAME); + if (frequency != NEVER) { + Path p = new Path(filename + "_" + attempt); + FileSystem fs = FileSystem.get(conf); + if (fs.exists(p)) { + throw new RuntimeException("aggregatorWriter file already" + + " exists: " + p.getName()); + } + output = fs.create(p); } - - /** - * Implements the way an aggregator is converted into a String. - * Override this if you want to implement your own text format. - * - * @param aggregatorName Name of the aggregator - * @param a Aggregator - * @param superstep Current superstep - * @return The String representation for the aggregator - */ - protected String aggregatorToString(String aggregatorName, - Aggregator a, - long superstep) { + } - return new StringBuilder("superstep=").append(superstep).append("\t") - .append(aggregatorName).append("=").append(a.getAggregatedValue()) - .append("\t").append(a.getClass().getCanonicalName()).append("\n") - .toString(); + @Override + public final void writeAggregator( + Map> aggregators, + long superstep) throws IOException { + if (shouldWrite(superstep)) { + for (Entry> a: + aggregators.entrySet()) { + output.writeUTF(aggregatorToString(a.getKey(), + a.getValue(), + superstep)); + } + output.flush(); } + } - private boolean shouldWrite(long superstep) { - return ((frequency == AT_THE_END && superstep == LAST_SUPERSTEP) || - (frequency != NEVER && superstep % frequency == 0)); - } + /** + * Implements the way an aggregator is converted into a String. + * Override this if you want to implement your own text format. + * + * @param aggregatorName Name of the aggregator + * @param a Aggregator + * @param superstep Current superstep + * @return The String representation for the aggregator + */ + protected String aggregatorToString(String aggregatorName, + Aggregator a, + long superstep) { + + return new StringBuilder("superstep=").append(superstep).append("\t") + .append(aggregatorName).append("=").append(a.getAggregatedValue()) + .append("\t").append(a.getClass().getCanonicalName()).append("\n") + .toString(); + } + + /** + * Should write this superstep? + * + * @param superstep Superstep to check + * @return True if should write, false otherwise + */ + private boolean shouldWrite(long superstep) { + return (frequency == AT_THE_END && superstep == LAST_SUPERSTEP) || + (frequency != NEVER && superstep % frequency == 0); + } - @Override - public void close() throws IOException { - if (output != null) { - output.close(); - } + @Override + public void close() throws IOException { + if (output != null) { + output.close(); } + } } diff --git a/src/main/java/org/apache/giraph/graph/VertexChanges.java b/src/main/java/org/apache/giraph/graph/VertexChanges.java index 28dbdde86..d081dc3d9 100644 --- a/src/main/java/org/apache/giraph/graph/VertexChanges.java +++ b/src/main/java/org/apache/giraph/graph/VertexChanges.java @@ -33,40 +33,36 @@ * @param Message value */ @SuppressWarnings("rawtypes") -public interface VertexChanges< - I extends WritableComparable, - V extends Writable, - E extends Writable, - M extends Writable> { +public interface VertexChanges { + /** + * Get the added vertices for this particular vertex index from the previous + * superstep. + * + * @return List of vertices for this vertex index. + */ + List> getAddedVertexList(); - /** - * Get the added vertices for this particular vertex index from the previous - * superstep. - * - * @return List of vertices for this vertex index. - */ - List> getAddedVertexList(); + /** + * Get the number of times this vertex was removed in the previous + * superstep. + * + * @return Count of time this vertex was removed in the previous superstep + */ + int getRemovedVertexCount(); - /** - * Get the number of times this vertex was removed in the previous - * superstep. - * - * @return Count of time this vertex was removed in the previous superstep - */ - int getRemovedVertexCount(); + /** + * Get the added edges for this particular vertex index from the previous + * superstep + * + * @return List of added edges for this vertex index + */ + List> getAddedEdgeList(); - /** - * Get the added edges for this particular vertex index from the previous - * superstep - * - * @return List of added edges for this vertex index - */ - List> getAddedEdgeList(); - - /** - * Get the removed edges by their destination vertex index. - * - * @return List of destination edges for removal from this vertex index - */ - List getRemovedEdgeList(); + /** + * Get the removed edges by their destination vertex index. + * + * @return List of destination edges for removal from this vertex index + */ + List getRemovedEdgeList(); } diff --git a/src/main/java/org/apache/giraph/graph/VertexCombiner.java b/src/main/java/org/apache/giraph/graph/VertexCombiner.java index bbf03443c..e739998b7 100644 --- a/src/main/java/org/apache/giraph/graph/VertexCombiner.java +++ b/src/main/java/org/apache/giraph/graph/VertexCombiner.java @@ -26,23 +26,22 @@ /** * Abstract class to extend for combining of messages sent to the same vertex. * - * @param index - * @param message data + * @param Vertex id + * @param Message data */ @SuppressWarnings("rawtypes") public abstract class VertexCombiner { - - /** - * Combines message values for a particular vertex index. - * - * @param vertexIndex Index of the vertex getting these messages - * @param messages Iterable of the messages to be combined - * @return Iterable of the combined messages. The returned value cannot - * be null and its size is required to be smaller or equal to - * the size of the messages list. - * @throws IOException - */ - public abstract Iterable combine(I vertexIndex, - Iterable messages) throws IOException; + M extends Writable> { + /** + * Combines message values for a particular vertex index. + * + * @param vertexIndex Index of the vertex getting these messages + * @param messages Iterable of the messages to be combined + * @return Iterable of the combined messages. The returned value cannot + * be null and its size is required to be smaller or equal to + * the size of the messages list. + * @throws IOException + */ + public abstract Iterable combine(I vertexIndex, + Iterable messages) throws IOException; } diff --git a/src/main/java/org/apache/giraph/graph/VertexEdgeCount.java b/src/main/java/org/apache/giraph/graph/VertexEdgeCount.java index a8bece30f..c2d13cc6d 100644 --- a/src/main/java/org/apache/giraph/graph/VertexEdgeCount.java +++ b/src/main/java/org/apache/giraph/graph/VertexEdgeCount.java @@ -22,45 +22,67 @@ * Simple immutable structure for storing a final vertex and edge count. */ public class VertexEdgeCount { - /** Immutable vertices */ - private final long vertexCount; - /** Immutable edges */ - private final long edgeCount; + /** Immutable vertices */ + private final long vertexCount; + /** Immutable edges */ + private final long edgeCount; - public VertexEdgeCount() { - vertexCount = 0; - edgeCount = 0; - } + /** + * Default constructor. + */ + public VertexEdgeCount() { + vertexCount = 0; + edgeCount = 0; + } - public VertexEdgeCount(long vertexCount, long edgeCount) { - this.vertexCount = vertexCount; - this.edgeCount = edgeCount; - } + /** + * Constructor with initial values. + * + * @param vertexCount Final number of vertices. + * @param edgeCount Final number of edges. + */ + public VertexEdgeCount(long vertexCount, long edgeCount) { + this.vertexCount = vertexCount; + this.edgeCount = edgeCount; + } - public long getVertexCount() { - return vertexCount; - } + public long getVertexCount() { + return vertexCount; + } - public long getEdgeCount() { - return edgeCount; - } + public long getEdgeCount() { + return edgeCount; + } - public VertexEdgeCount incrVertexEdgeCount( - VertexEdgeCount vertexEdgeCount) { - return new VertexEdgeCount( - vertexCount + vertexEdgeCount.getVertexCount(), - edgeCount + vertexEdgeCount.getEdgeCount()); - } + /** + * Increment the both the vertex edge count with a {@link VertexEdgeCount}. + * + * @param vertexEdgeCount add both the vertices and edges of this object. + * @return New immutable object with the new vertex and edge counts. + */ + public VertexEdgeCount incrVertexEdgeCount( + VertexEdgeCount vertexEdgeCount) { + return new VertexEdgeCount( + vertexCount + vertexEdgeCount.getVertexCount(), + edgeCount + vertexEdgeCount.getEdgeCount()); + } - public VertexEdgeCount incrVertexEdgeCount( - long vertexCount, long edgeCount) { - return new VertexEdgeCount( - this.vertexCount + vertexCount, - this.edgeCount + edgeCount); - } + /** + * Increment the both the vertex edge count with primitives. + * + * @param vertexCount Add this many vertices. + * @param edgeCount Add this many edges. + * @return New immutable object with the new vertex and edge counts. + */ + public VertexEdgeCount incrVertexEdgeCount( + long vertexCount, long edgeCount) { + return new VertexEdgeCount( + this.vertexCount + vertexCount, + this.edgeCount + edgeCount); + } - @Override - public String toString() { - return "(v=" + getVertexCount() + ", e=" + getEdgeCount() + ")"; - } + @Override + public String toString() { + return "(v=" + getVertexCount() + ", e=" + getEdgeCount() + ")"; + } } diff --git a/src/main/java/org/apache/giraph/graph/VertexInputFormat.java b/src/main/java/org/apache/giraph/graph/VertexInputFormat.java index 0b1a86f5d..c115cc46d 100644 --- a/src/main/java/org/apache/giraph/graph/VertexInputFormat.java +++ b/src/main/java/org/apache/giraph/graph/VertexInputFormat.java @@ -36,45 +36,45 @@ * @param Vertex id * @param Vertex value * @param Edge value + * @param Message data */ @SuppressWarnings("rawtypes") public abstract class VertexInputFormat { + V extends Writable, E extends Writable, M extends Writable> { + /** + * Logically split the vertices for a graph processing application. + * + * Each {@link InputSplit} is then assigned to a worker for processing. + * + *

Note: The split is a logical split of the inputs and the + * input files are not physically split into chunks. For e.g. a split could + * be <input-file-path, start, offset> tuple. The InputFormat + * also creates the {@link VertexReader} to read the {@link InputSplit}. + * + * Also, the number of workers is a hint given to the developer to try to + * intelligently determine how many splits to create (if this is + * adjustable) at runtime. + * + * @param context Context of the job + * @param numWorkers Number of workers used for this job + * @return an array of {@link InputSplit}s for the job. + */ + public abstract List getSplits( + JobContext context, int numWorkers) + throws IOException, InterruptedException; - /** - * Logically split the vertices for a graph processing application. - * - * Each {@link InputSplit} is then assigned to a worker for processing. - * - *

Note: The split is a logical split of the inputs and the - * input files are not physically split into chunks. For e.g. a split could - * be <input-file-path, start, offset> tuple. The InputFormat - * also creates the {@link VertexReader} to read the {@link InputSplit}. - * - * Also, the number of workers is a hint given to the developer to try to - * intelligently determine how many splits to create (if this is - * adjustable) at runtime. - * - * @param context Context of the job - * @param numWorkers Number of workers used for this job - * @return an array of {@link InputSplit}s for the job. - */ - public abstract List getSplits( - JobContext context, int numWorkers) - throws IOException, InterruptedException; - - /** - * Create a vertex reader for a given split. The framework will call - * {@link VertexReader#initialize(InputSplit, TaskAttemptContext)} before - * the split is used. - * - * @param split the split to be read - * @param context the information about the task - * @return a new record reader - * @throws IOException - * @throws InterruptedException - */ - public abstract VertexReader createVertexReader( - InputSplit split, - TaskAttemptContext context) throws IOException; + /** + * Create a vertex reader for a given split. The framework will call + * {@link VertexReader#initialize(InputSplit, TaskAttemptContext)} before + * the split is used. + * + * @param split the split to be read + * @param context the information about the task + * @return a new record reader + * @throws IOException + * @throws InterruptedException + */ + public abstract VertexReader createVertexReader( + InputSplit split, + TaskAttemptContext context) throws IOException; } diff --git a/src/main/java/org/apache/giraph/graph/VertexMutations.java b/src/main/java/org/apache/giraph/graph/VertexMutations.java index f201a2e19..85cf983e7 100644 --- a/src/main/java/org/apache/giraph/graph/VertexMutations.java +++ b/src/main/java/org/apache/giraph/graph/VertexMutations.java @@ -36,87 +36,85 @@ * @param Message value */ @SuppressWarnings("rawtypes") -public class VertexMutations< - I extends WritableComparable, - V extends Writable, - E extends Writable, - M extends Writable> implements VertexChanges { - /** List of added vertices during the last superstep */ - private final List> addedVertexList = - new ArrayList>(); - /** Count of remove vertex requests */ - private int removedVertexCount = 0; - /** List of added edges */ - private final List> addedEdgeList = new ArrayList>(); - /** List of removed edges */ - private final List removedEdgeList = new ArrayList(); +public class VertexMutations implements VertexChanges { + /** List of added vertices during the last superstep */ + private final List> addedVertexList = + new ArrayList>(); + /** Count of remove vertex requests */ + private int removedVertexCount = 0; + /** List of added edges */ + private final List> addedEdgeList = new ArrayList>(); + /** List of removed edges */ + private final List removedEdgeList = new ArrayList(); - @Override - public List> getAddedVertexList() { - return addedVertexList; - } + @Override + public List> getAddedVertexList() { + return addedVertexList; + } - /** - * Add a vertex mutation - * - * @param vertex Vertex to be added - */ - public void addVertex(BasicVertex vertex) { - addedVertexList.add(vertex); - } + /** + * Add a vertex mutation + * + * @param vertex Vertex to be added + */ + public void addVertex(BasicVertex vertex) { + addedVertexList.add(vertex); + } - @Override - public int getRemovedVertexCount() { - return removedVertexCount; - } + @Override + public int getRemovedVertexCount() { + return removedVertexCount; + } - /** - * Removed a vertex mutation (increments a count) - */ - public void removeVertex() { - ++removedVertexCount; - } + /** + * Removed a vertex mutation (increments a count) + */ + public void removeVertex() { + ++removedVertexCount; + } - @Override - public List> getAddedEdgeList() { - return addedEdgeList; - } + @Override + public List> getAddedEdgeList() { + return addedEdgeList; + } - /** - * Add an edge to this vertex - * - * @param edge Edge to be added - */ - public void addEdge(Edge edge) { - addedEdgeList.add(edge); - } + /** + * Add an edge to this vertex + * + * @param edge Edge to be added + */ + public void addEdge(Edge edge) { + addedEdgeList.add(edge); + } - @Override - public List getRemovedEdgeList() { - return removedEdgeList; - } + @Override + public List getRemovedEdgeList() { + return removedEdgeList; + } - /** - * Remove an edge on this vertex - * - * @param destinationVertexId Vertex index of the destination of the edge - */ - public void removeEdge(I destinationVertexId) { - removedEdgeList.add(destinationVertexId); - } + /** + * Remove an edge on this vertex + * + * @param destinationVertexId Vertex index of the destination of the edge + */ + public void removeEdge(I destinationVertexId) { + removedEdgeList.add(destinationVertexId); + } - @Override - public String toString() { - JSONObject jsonObject = new JSONObject(); - try { - jsonObject.put("added vertices", getAddedVertexList().toString()); - jsonObject.put("added edges", getAddedEdgeList().toString()); - jsonObject.put("removed vertex count", getRemovedVertexCount()); - jsonObject.put("removed edges", getRemovedEdgeList().toString()); - return jsonObject.toString(); - } catch (JSONException e) { - throw new IllegalStateException("toString: Got a JSON exception", - e); - } + @Override + public String toString() { + JSONObject jsonObject = new JSONObject(); + try { + jsonObject.put("added vertices", getAddedVertexList().toString()); + jsonObject.put("added edges", getAddedEdgeList().toString()); + jsonObject.put("removed vertex count", getRemovedVertexCount()); + jsonObject.put("removed edges", getRemovedEdgeList().toString()); + return jsonObject.toString(); + } catch (JSONException e) { + throw new IllegalStateException("toString: Got a JSON exception", + e); } + } } diff --git a/src/main/java/org/apache/giraph/graph/VertexOutputFormat.java b/src/main/java/org/apache/giraph/graph/VertexOutputFormat.java index 28078ad51..d4c8e1cdc 100644 --- a/src/main/java/org/apache/giraph/graph/VertexOutputFormat.java +++ b/src/main/java/org/apache/giraph/graph/VertexOutputFormat.java @@ -25,7 +25,6 @@ import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableComparable; -import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.TaskAttemptContext; /** @@ -38,45 +37,45 @@ */ @SuppressWarnings("rawtypes") public abstract class VertexOutputFormat< - I extends WritableComparable, V extends Writable, E extends Writable> { - /** - * Create a vertex writer for a given split. The framework will call - * {@link VertexReader#initialize(InputSplit, TaskAttemptContext)} before - * the split is used. - * - * @param context the information about the task - * @return a new vertex writer - * @throws IOException - * @throws InterruptedException - */ - public abstract VertexWriter createVertexWriter( - TaskAttemptContext context) throws IOException, InterruptedException; + I extends WritableComparable, V extends Writable, E extends Writable> { + /** + * Create a vertex writer for a given split. The framework will call + * {@link VertexReader#initialize(InputSplit, TaskAttemptContext)} before + * the split is used. + * + * @param context the information about the task + * @return a new vertex writer + * @throws IOException + * @throws InterruptedException + */ + public abstract VertexWriter createVertexWriter( + TaskAttemptContext context) throws IOException, InterruptedException; - /** - * Check for validity of the output-specification for the job. - * (Copied from Hadoop OutputFormat) - * - *

This is to validate the output specification for the job when it is - * a job is submitted. Typically checks that it does not already exist, - * throwing an exception when it already exists, so that output is not - * overwritten.

- * - * @param context information about the job - * @throws IOException when output should not be attempted - */ - public abstract void checkOutputSpecs(JobContext context) - throws IOException, InterruptedException; + /** + * Check for validity of the output-specification for the job. + * (Copied from Hadoop OutputFormat) + * + *

This is to validate the output specification for the job when it is + * a job is submitted. Typically checks that it does not already exist, + * throwing an exception when it already exists, so that output is not + * overwritten.

+ * + * @param context information about the job + * @throws IOException when output should not be attempted + */ + public abstract void checkOutputSpecs(JobContext context) + throws IOException, InterruptedException; - /** - * Get the output committer for this output format. This is responsible - * for ensuring the output is committed correctly. - * (Copied from Hadoop OutputFormat) - * - * @param context the task context - * @return an output committer - * @throws IOException - * @throws InterruptedException - */ - public abstract OutputCommitter getOutputCommitter( - TaskAttemptContext context) throws IOException, InterruptedException; + /** + * Get the output committer for this output format. This is responsible + * for ensuring the output is committed correctly. + * (Copied from Hadoop OutputFormat) + * + * @param context the task context + * @return an output committer + * @throws IOException + * @throws InterruptedException + */ + public abstract OutputCommitter getOutputCommitter( + TaskAttemptContext context) throws IOException, InterruptedException; } diff --git a/src/main/java/org/apache/giraph/graph/VertexReader.java b/src/main/java/org/apache/giraph/graph/VertexReader.java index d5a00e6aa..e940bec41 100644 --- a/src/main/java/org/apache/giraph/graph/VertexReader.java +++ b/src/main/java/org/apache/giraph/graph/VertexReader.java @@ -25,54 +25,63 @@ import java.io.IOException; +/** + * Analogous to {@link RecordReader} for vertices. Will read the vertices + * from an input split. + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data + */ @SuppressWarnings("rawtypes") -public interface VertexReader< - I extends WritableComparable, - V extends Writable, - E extends Writable, - M extends Writable> { - /** - * Use the input split and context to setup reading the vertices. - * Guaranteed to be called prior to any other function. - * - * @param inputSplit - * @param context - * @throws IOException - * @throws InterruptedException - */ - void initialize(InputSplit inputSplit, TaskAttemptContext context) - throws IOException, InterruptedException; +public interface VertexReader { + /** + * Use the input split and context to setup reading the vertices. + * Guaranteed to be called prior to any other function. + * + * @param inputSplit Input split to be used for reading vertices. + * @param context Context from the task. + * @throws IOException + * @throws InterruptedException + */ + void initialize(InputSplit inputSplit, TaskAttemptContext context) + throws IOException, InterruptedException; - /** - * - * @return false iff there are no more vertices - * @throws IOException - * @throws InterruptedException - */ - boolean nextVertex() throws IOException, InterruptedException; + /** + * + * @return false iff there are no more vertices + * @throws IOException + * @throws InterruptedException + */ + boolean nextVertex() throws IOException, InterruptedException; - /** - * - * @return the current vertex which has been read. nextVertex() should be called first. - * @throws IOException - * @throws InterruptedException - */ - BasicVertex getCurrentVertex() throws IOException, InterruptedException; + /** + * Get the current vertex. + * + * @return the current vertex which has been read. + * nextVertex() should be called first. + * @throws IOException + * @throws InterruptedException + */ + BasicVertex getCurrentVertex() + throws IOException, InterruptedException; - /** - * Close this {@link VertexReader} to future operations. - * - * @throws IOException - */ - void close() throws IOException; + /** + * Close this {@link VertexReader} to future operations. + * + * @throws IOException + */ + void close() throws IOException; - /** - * How much of the input has the {@link VertexReader} consumed i.e. - * has been processed by? - * - * @return Progress from 0.0 to 1.0. - * @throws IOException - * @throws InterruptedException - */ - float getProgress() throws IOException, InterruptedException; + /** + * How much of the input has the {@link VertexReader} consumed i.e. + * has been processed by? + * + * @return Progress from 0.0 to 1.0. + * @throws IOException + * @throws InterruptedException + */ + float getProgress() throws IOException, InterruptedException; } diff --git a/src/main/java/org/apache/giraph/graph/VertexResolver.java b/src/main/java/org/apache/giraph/graph/VertexResolver.java index b971df21c..5e938392b 100644 --- a/src/main/java/org/apache/giraph/graph/VertexResolver.java +++ b/src/main/java/org/apache/giraph/graph/VertexResolver.java @@ -31,110 +31,113 @@ * Default implementation of how to resolve vertex creation/removal, messages * to nonexistent vertices, etc. * - * @param - * @param - * @param - * @param + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data */ @SuppressWarnings("rawtypes") public class VertexResolver - implements BasicVertexResolver, Configurable { - /** Configuration */ - private Configuration conf = null; + E extends Writable, M extends Writable> + implements BasicVertexResolver, Configurable { + /** Class logger */ + private static final Logger LOG = Logger.getLogger(VertexResolver.class); + /** Configuration */ + private Configuration conf = null; + /** Stored graph state */ + private GraphState graphState; - private GraphState graphState; - - /** Class logger */ - private static final Logger LOG = Logger.getLogger(VertexResolver.class); - - @Override - public BasicVertex resolve( - I vertexId, - BasicVertex vertex, - VertexChanges vertexChanges, - Iterable messages) { - // Default algorithm: - // 1. If the vertex exists, first prune the edges - // 2. If vertex removal desired, remove the vertex. - // 3. If creation of vertex desired, pick first vertex - // 4. If vertex doesn't exist, but got messages, create - // 5. If edge addition, add the edges - if (vertex != null) { - if (vertexChanges != null) { - List removedEdgeList = vertexChanges.getRemovedEdgeList(); - for (I removedDestVertex : removedEdgeList) { - E removeEdge = - ((MutableVertex) vertex).removeEdge( - removedDestVertex); - if (removeEdge == null) { - LOG.warn("resolve: Failed to remove edge with " + - "destination " + removedDestVertex + "on " + - vertex + " since it doesn't exist."); - } - } - if (vertexChanges.getRemovedVertexCount() > 0) { - vertex = null; - } - } + @Override + public BasicVertex resolve( + I vertexId, + BasicVertex vertex, + VertexChanges vertexChanges, + Iterable messages) { + // Default algorithm: + // 1. If the vertex exists, first prune the edges + // 2. If vertex removal desired, remove the vertex. + // 3. If creation of vertex desired, pick first vertex + // 4. If vertex doesn't exist, but got messages, create + // 5. If edge addition, add the edges + if (vertex != null) { + if (vertexChanges != null) { + List removedEdgeList = vertexChanges.getRemovedEdgeList(); + for (I removedDestVertex : removedEdgeList) { + E removeEdge = + ((MutableVertex) vertex).removeEdge( + removedDestVertex); + if (removeEdge == null) { + LOG.warn("resolve: Failed to remove edge with " + + "destination " + removedDestVertex + "on " + + vertex + " since it doesn't exist."); + } } - - if (vertex == null) { - if (vertexChanges != null) { - if (!vertexChanges.getAddedVertexList().isEmpty()) { - vertex = vertexChanges.getAddedVertexList().get(0); - } - } - if (vertex == null && messages != null - && !Iterables.isEmpty(messages)) { - vertex = instantiateVertex(); - vertex.initialize(vertexId, - BspUtils.createVertexValue(getConf()), - null, - messages); - } - } else { - if ((vertexChanges != null) && - (!vertexChanges.getAddedVertexList().isEmpty())) { - LOG.warn("resolve: Tried to add a vertex with id = " + - vertex.getVertexId() + " when one already " + - "exists. Ignoring the add vertex request."); - } + if (vertexChanges.getRemovedVertexCount() > 0) { + vertex = null; } + } + } - if (vertexChanges != null && - !vertexChanges.getAddedEdgeList().isEmpty()) { - MutableVertex mutableVertex = - (MutableVertex) vertex; - for (Edge edge : vertexChanges.getAddedEdgeList()) { - edge.setConf(getConf()); - mutableVertex.addEdge(edge.getDestVertexId(), - edge.getEdgeValue()); - } + if (vertex == null) { + if (vertexChanges != null) { + if (!vertexChanges.getAddedVertexList().isEmpty()) { + vertex = vertexChanges.getAddedVertexList().get(0); } - - return vertex; + } + if (vertex == null && messages != null && !Iterables.isEmpty(messages)) { + vertex = instantiateVertex(); + vertex.initialize(vertexId, + BspUtils.createVertexValue(getConf()), + null, + messages); + } + } else { + if ((vertexChanges != null) && + (!vertexChanges.getAddedVertexList().isEmpty())) { + LOG.warn("resolve: Tried to add a vertex with id = " + + vertex.getVertexId() + " when one already " + + "exists. Ignoring the add vertex request."); + } } - @Override - public BasicVertex instantiateVertex() { - BasicVertex vertex = - BspUtils.createVertex(getConf()); - vertex.setGraphState(graphState); - return vertex; + if (vertexChanges != null && + !vertexChanges.getAddedEdgeList().isEmpty()) { + MutableVertex mutableVertex = + (MutableVertex) vertex; + for (Edge edge : vertexChanges.getAddedEdgeList()) { + edge.setConf(getConf()); + mutableVertex.addEdge(edge.getDestVertexId(), + edge.getEdgeValue()); + } } - @Override - public Configuration getConf() { - return conf; - } + return vertex; + } - @Override - public void setConf(Configuration conf) { - this.conf = conf; - } + @Override + public BasicVertex instantiateVertex() { + BasicVertex vertex = + BspUtils.createVertex(getConf()); + vertex.setGraphState(graphState); + return vertex; + } - public void setGraphState(GraphState graphState) { - this.graphState = graphState; - } + @Override + public Configuration getConf() { + return conf; + } + + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } + + /** + * Set the graph state. + * + * @param graphState Graph state saved. + */ + public void setGraphState(GraphState graphState) { + this.graphState = graphState; + } } diff --git a/src/main/java/org/apache/giraph/graph/VertexWriter.java b/src/main/java/org/apache/giraph/graph/VertexWriter.java index 8c300397e..037ec3d3e 100644 --- a/src/main/java/org/apache/giraph/graph/VertexWriter.java +++ b/src/main/java/org/apache/giraph/graph/VertexWriter.java @@ -32,36 +32,34 @@ * @param Edge value */ @SuppressWarnings("rawtypes") -public interface VertexWriter< - I extends WritableComparable, - V extends Writable, - E extends Writable> { - /** - * Use the context to setup writing the vertices. - * Guaranteed to be called prior to any other function. - * - * @param context - * @throws IOException - */ - void initialize(TaskAttemptContext context) throws IOException; +public interface VertexWriter { + /** + * Use the context to setup writing the vertices. + * Guaranteed to be called prior to any other function. + * + * @param context Context used to write the vertices. + * @throws IOException + */ + void initialize(TaskAttemptContext context) throws IOException; - /** - * Writes the next vertex and associated data - * - * @param vertex set the properties of this vertex - * @throws IOException - * @throws InterruptedException - */ - void writeVertex(BasicVertex vertex) - throws IOException, InterruptedException; + /** + * Writes the next vertex and associated data + * + * @param vertex set the properties of this vertex + * @throws IOException + * @throws InterruptedException + */ + void writeVertex(BasicVertex vertex) + throws IOException, InterruptedException; - /** - * Close this {@link VertexWriter} to future operations. - * - * @param context the context of the task - * @throws IOException - * @throws InterruptedException - */ - void close(TaskAttemptContext context) - throws IOException, InterruptedException; + /** + * Close this {@link VertexWriter} to future operations. + * + * @param context the context of the task + * @throws IOException + * @throws InterruptedException + */ + void close(TaskAttemptContext context) + throws IOException, InterruptedException; } diff --git a/src/main/java/org/apache/giraph/graph/WorkerContext.java b/src/main/java/org/apache/giraph/graph/WorkerContext.java index 707c17bf9..5a6329590 100644 --- a/src/main/java/org/apache/giraph/graph/WorkerContext.java +++ b/src/main/java/org/apache/giraph/graph/WorkerContext.java @@ -27,101 +27,106 @@ */ @SuppressWarnings("rawtypes") public abstract class WorkerContext implements AggregatorUsage { - /** Global graph state */ - private GraphState graphState; + /** Global graph state */ + private GraphState graphState; - public void setGraphState(GraphState graphState) { - this.graphState = graphState; - } + /** + * Set the graph state. + * + * @param graphState Used to set the graph state. + */ + public void setGraphState(GraphState graphState) { + this.graphState = graphState; + } - /** - * Initialize the WorkerContext. - * This method is executed once on each Worker before the first - * superstep starts. - * - * @throws IllegalAccessException - * @throws InstantiationException - */ - public abstract void preApplication() throws InstantiationException, - IllegalAccessException; + /** + * Initialize the WorkerContext. + * This method is executed once on each Worker before the first + * superstep starts. + * + * @throws IllegalAccessException Thrown for getting the class + * @throws InstantiationException Expected instantiation in this method. + */ + public abstract void preApplication() throws InstantiationException, + IllegalAccessException; - /** - * Finalize the WorkerContext. - * This method is executed once on each Worker after the last - * superstep ends. - */ - public abstract void postApplication(); + /** + * Finalize the WorkerContext. + * This method is executed once on each Worker after the last + * superstep ends. + */ + public abstract void postApplication(); - /** - * Execute user code. - * This method is executed once on each Worker before each - * superstep starts. - */ - public abstract void preSuperstep(); + /** + * Execute user code. + * This method is executed once on each Worker before each + * superstep starts. + */ + public abstract void preSuperstep(); - /** - * Execute user code. - * This method is executed once on each Worker after each - * superstep ends. - */ - public abstract void postSuperstep(); + /** + * Execute user code. + * This method is executed once on each Worker after each + * superstep ends. + */ + public abstract void postSuperstep(); - /** - * Retrieves the current superstep. - * - * @return Current superstep - */ - public long getSuperstep() { - return graphState.getSuperstep(); - } + /** + * Retrieves the current superstep. + * + * @return Current superstep + */ + public long getSuperstep() { + return graphState.getSuperstep(); + } - /** - * Get the total (all workers) number of vertices that - * existed in the previous superstep. - * - * @return Total number of vertices (-1 if first superstep) - */ - public long getNumVertices() { - return graphState.getNumVertices(); - } + /** + * Get the total (all workers) number of vertices that + * existed in the previous superstep. + * + * @return Total number of vertices (-1 if first superstep) + */ + public long getNumVertices() { + return graphState.getNumVertices(); + } - /** - * Get the total (all workers) number of edges that - * existed in the previous superstep. - * - * @return Total number of edges (-1 if first superstep) - */ - public long getNumEdges() { - return graphState.getNumEdges(); - } + /** + * Get the total (all workers) number of edges that + * existed in the previous superstep. + * + * @return Total number of edges (-1 if first superstep) + */ + public long getNumEdges() { + return graphState.getNumEdges(); + } - /** - * Get the mapper context - * - * @return Mapper context - */ - public Mapper.Context getContext() { - return graphState.getContext(); - } + /** + * Get the mapper context + * + * @return Mapper context + */ + public Mapper.Context getContext() { + return graphState.getContext(); + } - @Override - public final
Aggregator registerAggregator( - String name, - Class> aggregatorClass) - throws InstantiationException, IllegalAccessException { - return graphState.getGraphMapper().getAggregatorUsage(). - registerAggregator(name, aggregatorClass); - } + @Override + public final Aggregator registerAggregator( + String name, + Class> aggregatorClass) + throws InstantiationException, IllegalAccessException { + return graphState.getGraphMapper().getAggregatorUsage(). + registerAggregator(name, aggregatorClass); + } - @Override - public final Aggregator getAggregator(String name) { - return graphState.getGraphMapper().getAggregatorUsage(). - getAggregator(name); - } + @Override + public final Aggregator getAggregator(String name) { + return graphState.getGraphMapper().getAggregatorUsage(). + getAggregator(name); + } - @Override - public final boolean useAggregator(String name) { - return graphState.getGraphMapper().getAggregatorUsage(). - useAggregator(name); - } -} \ No newline at end of file + @Override + public final boolean useAggregator(String name) { + return graphState.getGraphMapper().getAggregatorUsage(). + useAggregator(name); + } +} diff --git a/src/main/java/org/apache/giraph/graph/WorkerInfo.java b/src/main/java/org/apache/giraph/graph/WorkerInfo.java index 51f93134d..d6646a828 100644 --- a/src/main/java/org/apache/giraph/graph/WorkerInfo.java +++ b/src/main/java/org/apache/giraph/graph/WorkerInfo.java @@ -28,84 +28,91 @@ * Information about a worker that is sent to the master and other workers. */ public class WorkerInfo implements Writable { - /** Worker hostname */ - private String hostname; - /** Partition id of this worker */ - private int partitionId = -1; - /** Port that the RPC server is using */ - private int port = -1; - /** Hostname + "_" + id for easier debugging */ - private String hostnameId; + /** Worker hostname */ + private String hostname; + /** Partition id of this worker */ + private int partitionId = -1; + /** Port that the RPC server is using */ + private int port = -1; + /** Hostname + "_" + id for easier debugging */ + private String hostnameId; - /** - * Constructor for reflection - */ - public WorkerInfo() { - } + /** + * Constructor for reflection + */ + public WorkerInfo() { + } - public WorkerInfo(String hostname, int partitionId, int port) { - this.hostname = hostname; - this.partitionId = partitionId; - this.port = port; - this.hostnameId = hostname + "_" + partitionId; - } + /** + * Constructor with paramters. + * + * @param hostname Hostname of this worker. + * @param partitionId partition id of this particular object. + * @param port Port of the service. + */ + public WorkerInfo(String hostname, int partitionId, int port) { + this.hostname = hostname; + this.partitionId = partitionId; + this.port = port; + this.hostnameId = hostname + "_" + partitionId; + } - public String getHostname() { - return hostname; - } + public String getHostname() { + return hostname; + } - public int getPartitionId() { - return partitionId; - } + public int getPartitionId() { + return partitionId; + } - public String getHostnameId() { - return hostnameId; - } + public String getHostnameId() { + return hostnameId; + } - public int getPort() { - return port; - } + public int getPort() { + return port; + } - @Override - public boolean equals(Object other) { - if (other instanceof WorkerInfo) { - WorkerInfo workerInfo = (WorkerInfo) other; - if (hostname.equals(workerInfo.getHostname()) && - (partitionId == workerInfo.getPartitionId()) && - (port == workerInfo.getPort())) { - return true; - } - } - return false; + @Override + public boolean equals(Object other) { + if (other instanceof WorkerInfo) { + WorkerInfo workerInfo = (WorkerInfo) other; + if (hostname.equals(workerInfo.getHostname()) && + (partitionId == workerInfo.getPartitionId()) && + (port == workerInfo.getPort())) { + return true; + } } + return false; + } - @Override - public int hashCode() { - int result = 17; - result = 37 * result + port; - result = 37 * result + hostname.hashCode(); - result = 37 * result + partitionId; - return result; - } + @Override + public int hashCode() { + int result = 17; + result = 37 * result + port; + result = 37 * result + hostname.hashCode(); + result = 37 * result + partitionId; + return result; + } - @Override - public String toString() { - return "Worker(hostname=" + hostname + ", MRpartition=" + - partitionId + ", port=" + port + ")"; - } + @Override + public String toString() { + return "Worker(hostname=" + hostname + ", MRpartition=" + + partitionId + ", port=" + port + ")"; + } - @Override - public void readFields(DataInput input) throws IOException { - hostname = input.readUTF(); - partitionId = input.readInt(); - port = input.readInt(); - hostnameId = hostname + "_" + partitionId; - } + @Override + public void readFields(DataInput input) throws IOException { + hostname = input.readUTF(); + partitionId = input.readInt(); + port = input.readInt(); + hostnameId = hostname + "_" + partitionId; + } - @Override - public void write(DataOutput output) throws IOException { - output.writeUTF(hostname); - output.writeInt(partitionId); - output.writeInt(port); - } + @Override + public void write(DataOutput output) throws IOException { + output.writeUTF(hostname); + output.writeInt(partitionId); + output.writeInt(port); + } } diff --git a/src/main/java/org/apache/giraph/graph/package-info.java b/src/main/java/org/apache/giraph/graph/package-info.java new file mode 100644 index 000000000..26cd46aa3 --- /dev/null +++ b/src/main/java/org/apache/giraph/graph/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/** + * Package of all the graph related objects, built on the + * org.apache.bsp package. + */ +package org.apache.giraph.graph; diff --git a/src/main/java/org/apache/giraph/graph/partition/BasicPartitionOwner.java b/src/main/java/org/apache/giraph/graph/partition/BasicPartitionOwner.java index e5e858898..6e27dca8a 100644 --- a/src/main/java/org/apache/giraph/graph/partition/BasicPartitionOwner.java +++ b/src/main/java/org/apache/giraph/graph/partition/BasicPartitionOwner.java @@ -31,116 +31,132 @@ * owner implementations. */ public class BasicPartitionOwner implements PartitionOwner, Configurable { - /** Configuration */ - private Configuration conf; - /** Partition id */ - private int partitionId = -1; - /** Owning worker information */ - private WorkerInfo workerInfo; - /** Previous (if any) worker info */ - private WorkerInfo previousWorkerInfo; - /** Checkpoint files prefix for this partition */ - private String checkpointFilesPrefix; - - public BasicPartitionOwner() { + /** Configuration */ + private Configuration conf; + /** Partition id */ + private int partitionId = -1; + /** Owning worker information */ + private WorkerInfo workerInfo; + /** Previous (if any) worker info */ + private WorkerInfo previousWorkerInfo; + /** Checkpoint files prefix for this partition */ + private String checkpointFilesPrefix; + + /** + * Default constructor. + */ + public BasicPartitionOwner() { } + + /** + * Constructor with partition id and worker info. + * + * @param partitionId Partition id of this partition. + * @param workerInfo Owner of the partition. + */ + public BasicPartitionOwner(int partitionId, WorkerInfo workerInfo) { + this(partitionId, workerInfo, null, null); + } + + /** + * Constructor with partition id and worker info. + * + * @param partitionId Partition id of this partition. + * @param workerInfo Owner of the partition. + * @param previousWorkerInfo Previous owner of this partition. + * @param checkpointFilesPrefix Prefix of the checkpoint files. + */ + public BasicPartitionOwner(int partitionId, + WorkerInfo workerInfo, + WorkerInfo previousWorkerInfo, + String checkpointFilesPrefix) { + this.partitionId = partitionId; + this.workerInfo = workerInfo; + this.previousWorkerInfo = previousWorkerInfo; + this.checkpointFilesPrefix = checkpointFilesPrefix; + } + + @Override + public int getPartitionId() { + return partitionId; + } + + @Override + public WorkerInfo getWorkerInfo() { + return workerInfo; + } + + @Override + public void setWorkerInfo(WorkerInfo workerInfo) { + this.workerInfo = workerInfo; + } + + @Override + public WorkerInfo getPreviousWorkerInfo() { + return previousWorkerInfo; + } + + @Override + public void setPreviousWorkerInfo(WorkerInfo workerInfo) { + this.previousWorkerInfo = workerInfo; + } + + @Override + public String getCheckpointFilesPrefix() { + return checkpointFilesPrefix; + } + + @Override + public void setCheckpointFilesPrefix(String checkpointFilesPrefix) { + this.checkpointFilesPrefix = checkpointFilesPrefix; + } + + @Override + public void readFields(DataInput input) throws IOException { + partitionId = input.readInt(); + workerInfo = new WorkerInfo(); + workerInfo.readFields(input); + boolean hasPreviousWorkerInfo = input.readBoolean(); + if (hasPreviousWorkerInfo) { + previousWorkerInfo = new WorkerInfo(); + previousWorkerInfo.readFields(input); } - - public BasicPartitionOwner(int partitionId, WorkerInfo workerInfo) { - this(partitionId, workerInfo, null, null); - } - - public BasicPartitionOwner(int partitionId, - WorkerInfo workerInfo, - WorkerInfo previousWorkerInfo, - String checkpointFilesPrefix) { - this.partitionId = partitionId; - this.workerInfo = workerInfo; - this.previousWorkerInfo = previousWorkerInfo; - this.checkpointFilesPrefix = checkpointFilesPrefix; + boolean hasCheckpointFilePrefix = input.readBoolean(); + if (hasCheckpointFilePrefix) { + checkpointFilesPrefix = input.readUTF(); } - - @Override - public int getPartitionId() { - return partitionId; + } + + @Override + public void write(DataOutput output) throws IOException { + output.writeInt(partitionId); + workerInfo.write(output); + if (previousWorkerInfo != null) { + output.writeBoolean(true); + previousWorkerInfo.write(output); + } else { + output.writeBoolean(false); } - - @Override - public WorkerInfo getWorkerInfo() { - return workerInfo; - } - - @Override - public void setWorkerInfo(WorkerInfo workerInfo) { - this.workerInfo = workerInfo; - } - - @Override - public WorkerInfo getPreviousWorkerInfo() { - return previousWorkerInfo; - } - - @Override - public void setPreviousWorkerInfo(WorkerInfo workerInfo) { - this.previousWorkerInfo = workerInfo; - } - - @Override - public String getCheckpointFilesPrefix() { - return checkpointFilesPrefix; - } - - @Override - public void setCheckpointFilesPrefix(String checkpointFilesPrefix) { - this.checkpointFilesPrefix = checkpointFilesPrefix; - } - - @Override - public void readFields(DataInput input) throws IOException { - partitionId = input.readInt(); - workerInfo = new WorkerInfo(); - workerInfo.readFields(input); - boolean hasPreviousWorkerInfo = input.readBoolean(); - if (hasPreviousWorkerInfo) { - previousWorkerInfo = new WorkerInfo(); - previousWorkerInfo.readFields(input); - } - boolean hasCheckpointFilePrefix = input.readBoolean(); - if (hasCheckpointFilePrefix) { - checkpointFilesPrefix = input.readUTF(); - } - } - - @Override - public void write(DataOutput output) throws IOException { - output.writeInt(partitionId); - workerInfo.write(output); - if (previousWorkerInfo != null) { - output.writeBoolean(true); - previousWorkerInfo.write(output); - } else { - output.writeBoolean(false); - } - if (checkpointFilesPrefix != null) { - output.writeBoolean(true); - output.writeUTF(checkpointFilesPrefix); - } else { - output.writeBoolean(false); - } - } - - @Override - public Configuration getConf() { - return conf; - } - - @Override - public void setConf(Configuration conf) { - this.conf = conf; - } - - @Override - public String toString() { - return "(id=" + partitionId + ",cur=" + workerInfo + ",prev=" + - previousWorkerInfo + ",ckpt_file=" + checkpointFilesPrefix + ")"; + if (checkpointFilesPrefix != null) { + output.writeBoolean(true); + output.writeUTF(checkpointFilesPrefix); + } else { + output.writeBoolean(false); } + } + + @Override + public Configuration getConf() { + return conf; + } + + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } + + @Override + public String toString() { + return "(id=" + partitionId + ",cur=" + workerInfo + ",prev=" + + previousWorkerInfo + ",ckpt_file=" + checkpointFilesPrefix + ")"; + } } diff --git a/src/main/java/org/apache/giraph/graph/partition/GraphPartitionerFactory.java b/src/main/java/org/apache/giraph/graph/partition/GraphPartitionerFactory.java index 0e98bedd0..e2fed0de8 100644 --- a/src/main/java/org/apache/giraph/graph/partition/GraphPartitionerFactory.java +++ b/src/main/java/org/apache/giraph/graph/partition/GraphPartitionerFactory.java @@ -31,20 +31,20 @@ */ @SuppressWarnings("rawtypes") public interface GraphPartitionerFactory { - /** - * Create the {@link MasterGraphPartitioner} used by the master. - * Instantiated once by the master and reused. - * - * @return Instantiated master graph partitioner - */ - MasterGraphPartitioner createMasterGraphPartitioner(); + V extends Writable, E extends Writable, M extends Writable> { + /** + * Create the {@link MasterGraphPartitioner} used by the master. + * Instantiated once by the master and reused. + * + * @return Instantiated master graph partitioner + */ + MasterGraphPartitioner createMasterGraphPartitioner(); - /** - * Create the {@link WorkerGraphPartitioner} used by the worker. - * Instantiated once by every worker and reused. - * - * @return Instantiated worker graph partitioner - */ - WorkerGraphPartitioner createWorkerGraphPartitioner(); + /** + * Create the {@link WorkerGraphPartitioner} used by the worker. + * Instantiated once by every worker and reused. + * + * @return Instantiated worker graph partitioner + */ + WorkerGraphPartitioner createWorkerGraphPartitioner(); } diff --git a/src/main/java/org/apache/giraph/graph/partition/HashMasterPartitioner.java b/src/main/java/org/apache/giraph/graph/partition/HashMasterPartitioner.java index 6e940f1e8..b106f8d85 100644 --- a/src/main/java/org/apache/giraph/graph/partition/HashMasterPartitioner.java +++ b/src/main/java/org/apache/giraph/graph/partition/HashMasterPartitioner.java @@ -39,119 +39,119 @@ */ @SuppressWarnings("rawtypes") public class HashMasterPartitioner implements - MasterGraphPartitioner { - /** Provided configuration */ - private Configuration conf; - /** Specified partition count (overrides calculation) */ - private final int userPartitionCount; - /** Partition count (calculated in createInitialPartitionOwners) */ - private int partitionCount = -1; - /** Save the last generated partition owner list */ - private List partitionOwnerList; - /** Class logger */ - private static Logger LOG = Logger.getLogger(HashMasterPartitioner.class); - - /** - * ZooKeeper has a limit of the data in a single znode of 1 MB and - * each entry can go be on the average somewhat more than 300 bytes - */ - private static final int MAX_PARTTIONS = 1024 * 1024 / 350; - - /** - * Multiplier for the current workers squared - */ - public static final String PARTITION_COUNT_MULTIPLIER = - "hash.masterPartitionCountMultipler"; - public static final float DEFAULT_PARTITION_COUNT_MULTIPLIER = 1.0f; - - /** Overrides default partition count calculation if not -1 */ - public static final String USER_PARTITION_COUNT = - "hash.userPartitionCount"; - public static final int DEFAULT_USER_PARTITION_COUNT = -1; - - public HashMasterPartitioner(Configuration conf) { - this.conf = conf; - userPartitionCount = conf.getInt(USER_PARTITION_COUNT, - DEFAULT_USER_PARTITION_COUNT); - } - - @Override - public Collection createInitialPartitionOwners( - Collection availableWorkerInfos, int maxWorkers) { - if (availableWorkerInfos.isEmpty()) { - throw new IllegalArgumentException( - "createInitialPartitionOwners: No available workers"); - } - List ownerList = new ArrayList(); - Iterator workerIt = availableWorkerInfos.iterator(); - if (userPartitionCount == DEFAULT_USER_PARTITION_COUNT) { - float multiplier = conf.getFloat( - PARTITION_COUNT_MULTIPLIER, - DEFAULT_PARTITION_COUNT_MULTIPLIER); - partitionCount = - Math.max((int) (multiplier * availableWorkerInfos.size() * - availableWorkerInfos.size()), - 1); - } else { - partitionCount = userPartitionCount; - } - if (LOG.isInfoEnabled()) { - LOG.info("createInitialPartitionOwners: Creating " + - partitionCount + ", default would have been " + - (availableWorkerInfos.size() * - availableWorkerInfos.size()) + " partitions."); - } - if (partitionCount > MAX_PARTTIONS) { - LOG.warn("createInitialPartitionOwners: " + - "Reducing the partitionCount to " + MAX_PARTTIONS + - " from " + partitionCount); - partitionCount = MAX_PARTTIONS; - } - - for (int i = 0; i < partitionCount; ++i) { - PartitionOwner owner = new BasicPartitionOwner(i, workerIt.next()); - if (!workerIt.hasNext()) { - workerIt = availableWorkerInfos.iterator(); - } - ownerList.add(owner); - } - this.partitionOwnerList = ownerList; - return ownerList; + V extends Writable, E extends Writable, M extends Writable> implements + MasterGraphPartitioner { + /** Multiplier for the current workers squared */ + public static final String PARTITION_COUNT_MULTIPLIER = + "hash.masterPartitionCountMultipler"; + /** Default mulitplier for current workers squared */ + public static final float DEFAULT_PARTITION_COUNT_MULTIPLIER = 1.0f; + /** Overrides default partition count calculation if not -1 */ + public static final String USER_PARTITION_COUNT = + "hash.userPartitionCount"; + /** Default user partition count */ + public static final int DEFAULT_USER_PARTITION_COUNT = -1; + /** Class logger */ + private static Logger LOG = Logger.getLogger(HashMasterPartitioner.class); + /** + * ZooKeeper has a limit of the data in a single znode of 1 MB and + * each entry can go be on the average somewhat more than 300 bytes + */ + private static final int MAX_PARTTIONS = 1024 * 1024 / 350; + /** Provided configuration */ + private Configuration conf; + /** Specified partition count (overrides calculation) */ + private final int userPartitionCount; + /** Partition count (calculated in createInitialPartitionOwners) */ + private int partitionCount = -1; + /** Save the last generated partition owner list */ + private List partitionOwnerList; + + /** + * Constructor. + * + *@param conf Configuration used. + */ + public HashMasterPartitioner(Configuration conf) { + this.conf = conf; + userPartitionCount = conf.getInt(USER_PARTITION_COUNT, + DEFAULT_USER_PARTITION_COUNT); + } + + @Override + public Collection createInitialPartitionOwners( + Collection availableWorkerInfos, int maxWorkers) { + if (availableWorkerInfos.isEmpty()) { + throw new IllegalArgumentException( + "createInitialPartitionOwners: No available workers"); } - - - @Override - public Collection getCurrentPartitionOwners() { - return partitionOwnerList; + List ownerList = new ArrayList(); + Iterator workerIt = availableWorkerInfos.iterator(); + if (userPartitionCount == DEFAULT_USER_PARTITION_COUNT) { + float multiplier = conf.getFloat( + PARTITION_COUNT_MULTIPLIER, + DEFAULT_PARTITION_COUNT_MULTIPLIER); + partitionCount = + Math.max((int) (multiplier * availableWorkerInfos.size() * + availableWorkerInfos.size()), + 1); + } else { + partitionCount = userPartitionCount; } - - /** - * Subclasses can set the partition owner list. - * - * @param partitionOwnerList New partition owner list. - */ - protected void setPartitionOwnerList(List - partitionOwnerList) { - this.partitionOwnerList = partitionOwnerList; + if (LOG.isInfoEnabled()) { + LOG.info("createInitialPartitionOwners: Creating " + + partitionCount + ", default would have been " + + (availableWorkerInfos.size() * + availableWorkerInfos.size()) + " partitions."); } - - @Override - public Collection generateChangedPartitionOwners( - Collection allPartitionStatsList, - Collection availableWorkerInfos, - int maxWorkers, - long superstep) { - return PartitionBalancer.balancePartitionsAcrossWorkers( - conf, - partitionOwnerList, - allPartitionStatsList, - availableWorkerInfos); + if (partitionCount > MAX_PARTTIONS) { + LOG.warn("createInitialPartitionOwners: " + + "Reducing the partitionCount to " + MAX_PARTTIONS + + " from " + partitionCount); + partitionCount = MAX_PARTTIONS; } - @Override - public PartitionStats createPartitionStats() { - return new PartitionStats(); + for (int i = 0; i < partitionCount; ++i) { + PartitionOwner owner = new BasicPartitionOwner(i, workerIt.next()); + if (!workerIt.hasNext()) { + workerIt = availableWorkerInfos.iterator(); + } + ownerList.add(owner); } - + this.partitionOwnerList = ownerList; + return ownerList; + } + + @Override + public Collection getCurrentPartitionOwners() { + return partitionOwnerList; + } + + /** + * Subclasses can set the partition owner list. + * + * @param partitionOwnerList New partition owner list. + */ + protected void setPartitionOwnerList(List + partitionOwnerList) { + this.partitionOwnerList = partitionOwnerList; + } + + @Override + public Collection generateChangedPartitionOwners( + Collection allPartitionStatsList, + Collection availableWorkerInfos, + int maxWorkers, + long superstep) { + return PartitionBalancer.balancePartitionsAcrossWorkers( + conf, + partitionOwnerList, + allPartitionStatsList, + availableWorkerInfos); + } + + @Override + public PartitionStats createPartitionStats() { + return new PartitionStats(); + } } diff --git a/src/main/java/org/apache/giraph/graph/partition/HashPartitionerFactory.java b/src/main/java/org/apache/giraph/graph/partition/HashPartitionerFactory.java index 87cbb67f5..d6d6139d7 100644 --- a/src/main/java/org/apache/giraph/graph/partition/HashPartitionerFactory.java +++ b/src/main/java/org/apache/giraph/graph/partition/HashPartitionerFactory.java @@ -34,28 +34,28 @@ */ @SuppressWarnings("rawtypes") public class HashPartitionerFactory - implements Configurable, - GraphPartitionerFactory { - private Configuration conf; + V extends Writable, E extends Writable, M extends Writable> + implements Configurable, GraphPartitionerFactory { + /** Saved configuration */ + private Configuration conf; - @Override - public MasterGraphPartitioner createMasterGraphPartitioner() { - return new HashMasterPartitioner(getConf()); - } + @Override + public MasterGraphPartitioner createMasterGraphPartitioner() { + return new HashMasterPartitioner(getConf()); + } - @Override - public WorkerGraphPartitioner createWorkerGraphPartitioner() { - return new HashWorkerPartitioner(); - } + @Override + public WorkerGraphPartitioner createWorkerGraphPartitioner() { + return new HashWorkerPartitioner(); + } - @Override - public Configuration getConf() { - return conf; - } + @Override + public Configuration getConf() { + return conf; + } - @Override - public void setConf(Configuration conf) { - this.conf = conf; - } + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } } diff --git a/src/main/java/org/apache/giraph/graph/partition/HashRangePartitionerFactory.java b/src/main/java/org/apache/giraph/graph/partition/HashRangePartitionerFactory.java index 2646100ba..09d34ba22 100644 --- a/src/main/java/org/apache/giraph/graph/partition/HashRangePartitionerFactory.java +++ b/src/main/java/org/apache/giraph/graph/partition/HashRangePartitionerFactory.java @@ -34,27 +34,28 @@ */ @SuppressWarnings("rawtypes") public class HashRangePartitionerFactory - implements Configurable, GraphPartitionerFactory { - private Configuration conf; + V extends Writable, E extends Writable, M extends Writable> + implements Configurable, GraphPartitionerFactory { + /** Saved configuration */ + private Configuration conf; - @Override - public MasterGraphPartitioner createMasterGraphPartitioner() { - return new HashMasterPartitioner(getConf()); - } + @Override + public MasterGraphPartitioner createMasterGraphPartitioner() { + return new HashMasterPartitioner(getConf()); + } - @Override - public WorkerGraphPartitioner createWorkerGraphPartitioner() { - return new HashRangeWorkerPartitioner(); - } + @Override + public WorkerGraphPartitioner createWorkerGraphPartitioner() { + return new HashRangeWorkerPartitioner(); + } - @Override - public Configuration getConf() { - return conf; - } + @Override + public Configuration getConf() { + return conf; + } - @Override - public void setConf(Configuration conf) { - this.conf = conf; - } + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } } diff --git a/src/main/java/org/apache/giraph/graph/partition/HashRangeWorkerPartitioner.java b/src/main/java/org/apache/giraph/graph/partition/HashRangeWorkerPartitioner.java index e64d79334..01d8a8c40 100644 --- a/src/main/java/org/apache/giraph/graph/partition/HashRangeWorkerPartitioner.java +++ b/src/main/java/org/apache/giraph/graph/partition/HashRangeWorkerPartitioner.java @@ -31,12 +31,12 @@ */ @SuppressWarnings("rawtypes") public class HashRangeWorkerPartitioner - extends HashWorkerPartitioner { - @Override - public PartitionOwner getPartitionOwner(I vertexId) { - int rangeSize = Integer.MAX_VALUE / getPartitionOwners().size(); - int index = Math.abs(vertexId.hashCode()) / rangeSize; - return partitionOwnerList.get(index); - } + V extends Writable, E extends Writable, M extends Writable> + extends HashWorkerPartitioner { + @Override + public PartitionOwner getPartitionOwner(I vertexId) { + int rangeSize = Integer.MAX_VALUE / getPartitionOwners().size(); + int index = Math.abs(vertexId.hashCode()) / rangeSize; + return partitionOwnerList.get(index); + } } diff --git a/src/main/java/org/apache/giraph/graph/partition/HashWorkerPartitioner.java b/src/main/java/org/apache/giraph/graph/partition/HashWorkerPartitioner.java index f518ae573..c824283c8 100644 --- a/src/main/java/org/apache/giraph/graph/partition/HashWorkerPartitioner.java +++ b/src/main/java/org/apache/giraph/graph/partition/HashWorkerPartitioner.java @@ -40,77 +40,77 @@ */ @SuppressWarnings("rawtypes") public class HashWorkerPartitioner - implements WorkerGraphPartitioner { - /** Mapping of the vertex ids to {@link PartitionOwner} */ - protected List partitionOwnerList = - new ArrayList(); + V extends Writable, E extends Writable, M extends Writable> + implements WorkerGraphPartitioner { + /** Mapping of the vertex ids to {@link PartitionOwner} */ + protected List partitionOwnerList = + new ArrayList(); - @Override - public PartitionOwner createPartitionOwner() { - return new BasicPartitionOwner(); - } + @Override + public PartitionOwner createPartitionOwner() { + return new BasicPartitionOwner(); + } - @Override - public PartitionOwner getPartitionOwner(I vertexId) { - return partitionOwnerList.get(Math.abs(vertexId.hashCode()) - % partitionOwnerList.size()); - } + @Override + public PartitionOwner getPartitionOwner(I vertexId) { + return partitionOwnerList.get(Math.abs(vertexId.hashCode()) % + partitionOwnerList.size()); + } - @Override - public Collection finalizePartitionStats( - Collection workerPartitionStats, - Map> partitionMap) { - // No modification necessary - return workerPartitionStats; - } + @Override + public Collection finalizePartitionStats( + Collection workerPartitionStats, + Map> partitionMap) { + // No modification necessary + return workerPartitionStats; + } - @Override - public PartitionExchange updatePartitionOwners( - WorkerInfo myWorkerInfo, - Collection masterSetPartitionOwners, - Map> partitionMap) { - partitionOwnerList.clear(); - partitionOwnerList.addAll(masterSetPartitionOwners); + @Override + public PartitionExchange updatePartitionOwners( + WorkerInfo myWorkerInfo, + Collection masterSetPartitionOwners, + Map> partitionMap) { + partitionOwnerList.clear(); + partitionOwnerList.addAll(masterSetPartitionOwners); - Set dependentWorkerSet = new HashSet(); - Map> workerPartitionOwnerMap = - new HashMap>(); - for (PartitionOwner partitionOwner : masterSetPartitionOwners) { - if (partitionOwner.getPreviousWorkerInfo() == null) { - continue; - } else if (partitionOwner.getWorkerInfo().equals( - myWorkerInfo) && - partitionOwner.getPreviousWorkerInfo().equals( - myWorkerInfo)) { - throw new IllegalStateException( - "updatePartitionOwners: Impossible to have the same " + - "previous and current worker info " + partitionOwner + - " as me " + myWorkerInfo); - } else if (partitionOwner.getWorkerInfo().equals(myWorkerInfo)) { - dependentWorkerSet.add(partitionOwner.getPreviousWorkerInfo()); - } else if (partitionOwner.getPreviousWorkerInfo().equals( - myWorkerInfo)) { - if (workerPartitionOwnerMap.containsKey( - partitionOwner.getWorkerInfo())) { - workerPartitionOwnerMap.get( - partitionOwner.getWorkerInfo()).add( - partitionOwner.getPartitionId()); - } else { - List partitionOwnerList = new ArrayList(); - partitionOwnerList.add(partitionOwner.getPartitionId()); - workerPartitionOwnerMap.put(partitionOwner.getWorkerInfo(), - partitionOwnerList); - } - } + Set dependentWorkerSet = new HashSet(); + Map> workerPartitionOwnerMap = + new HashMap>(); + for (PartitionOwner partitionOwner : masterSetPartitionOwners) { + if (partitionOwner.getPreviousWorkerInfo() == null) { + continue; + } else if (partitionOwner.getWorkerInfo().equals( + myWorkerInfo) && + partitionOwner.getPreviousWorkerInfo().equals( + myWorkerInfo)) { + throw new IllegalStateException( + "updatePartitionOwners: Impossible to have the same " + + "previous and current worker info " + partitionOwner + + " as me " + myWorkerInfo); + } else if (partitionOwner.getWorkerInfo().equals(myWorkerInfo)) { + dependentWorkerSet.add(partitionOwner.getPreviousWorkerInfo()); + } else if (partitionOwner.getPreviousWorkerInfo().equals( + myWorkerInfo)) { + if (workerPartitionOwnerMap.containsKey( + partitionOwner.getWorkerInfo())) { + workerPartitionOwnerMap.get( + partitionOwner.getWorkerInfo()).add( + partitionOwner.getPartitionId()); + } else { + List tmpPartitionOwnerList = new ArrayList(); + tmpPartitionOwnerList.add(partitionOwner.getPartitionId()); + workerPartitionOwnerMap.put(partitionOwner.getWorkerInfo(), + tmpPartitionOwnerList); } - - return new PartitionExchange(dependentWorkerSet, - workerPartitionOwnerMap); + } } - @Override - public Collection getPartitionOwners() { - return partitionOwnerList; - } + return new PartitionExchange(dependentWorkerSet, + workerPartitionOwnerMap); + } + + @Override + public Collection getPartitionOwners() { + return partitionOwnerList; + } } diff --git a/src/main/java/org/apache/giraph/graph/partition/MasterGraphPartitioner.java b/src/main/java/org/apache/giraph/graph/partition/MasterGraphPartitioner.java index 98d2c32fc..e911303cd 100644 --- a/src/main/java/org/apache/giraph/graph/partition/MasterGraphPartitioner.java +++ b/src/main/java/org/apache/giraph/graph/partition/MasterGraphPartitioner.java @@ -35,48 +35,49 @@ */ @SuppressWarnings("rawtypes") public interface MasterGraphPartitioner { - /** - * Set some initial partition owners for the graph. Guaranteed to be called - * prior to the graph being loaded (initial or restart). - * - * @param availableWorkerInfos Workers available for partition assignment - * @param maxWorkers Maximum number of workers - */ - Collection createInitialPartitionOwners( - Collection availableWorkerInfos, int maxWorkers); + V extends Writable, E extends Writable, M extends Writable> { + /** + * Set some initial partition owners for the graph. Guaranteed to be called + * prior to the graph being loaded (initial or restart). + * + * @param availableWorkerInfos Workers available for partition assignment + * @param maxWorkers Maximum number of workers + * @return Collection of generated partition owners. + */ + Collection createInitialPartitionOwners( + Collection availableWorkerInfos, int maxWorkers); - /** - * After the worker stats have been merged to a single list, the master can - * use this information to send commands to the workers for any - * {@link Partition} changes. This protocol is specific to the - * {@link MasterGraphPartitioner} implementation. - * - * @param allPartitionStatsList All partition stats from all workers. - * @param availableWorkers Workers available for partition assignment - * @param maxWorkers Maximum number of workers - * @param superstep Partition owners will be set for this superstep - * @return Collection of {@link PartitionOwner} objects that changed from - * the previous superstep, empty list if no change. - */ - Collection generateChangedPartitionOwners( - Collection allPartitionStatsList, - Collection availableWorkers, - int maxWorkers, - long superstep); + /** + * After the worker stats have been merged to a single list, the master can + * use this information to send commands to the workers for any + * {@link Partition} changes. This protocol is specific to the + * {@link MasterGraphPartitioner} implementation. + * + * @param allPartitionStatsList All partition stats from all workers. + * @param availableWorkers Workers available for partition assignment + * @param maxWorkers Maximum number of workers + * @param superstep Partition owners will be set for this superstep + * @return Collection of {@link PartitionOwner} objects that changed from + * the previous superstep, empty list if no change. + */ + Collection generateChangedPartitionOwners( + Collection allPartitionStatsList, + Collection availableWorkers, + int maxWorkers, + long superstep); - /** - * Get current partition owners at this time. - * - * @return Collection of current {@link PartitionOwner} objects - */ - Collection getCurrentPartitionOwners(); + /** + * Get current partition owners at this time. + * + * @return Collection of current {@link PartitionOwner} objects + */ + Collection getCurrentPartitionOwners(); - /** - * Instantiate the {@link PartitionStats} implementation used to read the - * worker stats - * - * @return Instantiated {@link PartitionStats} object - */ - PartitionStats createPartitionStats(); + /** + * Instantiate the {@link PartitionStats} implementation used to read the + * worker stats + * + * @return Instantiated {@link PartitionStats} object + */ + PartitionStats createPartitionStats(); } diff --git a/src/main/java/org/apache/giraph/graph/partition/Partition.java b/src/main/java/org/apache/giraph/graph/partition/Partition.java index 5173cf34f..c0099862c 100644 --- a/src/main/java/org/apache/giraph/graph/partition/Partition.java +++ b/src/main/java/org/apache/giraph/graph/partition/Partition.java @@ -31,115 +31,126 @@ import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableComparable; - /** * A generic container that stores vertices. Vertex ids will map to exactly * one partition. + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data */ @SuppressWarnings("rawtypes") public class Partition - implements Writable { - /** Configuration from the worker */ - private final Configuration conf; - /** Partition id */ - private final int partitionId; - /** Vertex map for this range (keyed by index) */ - private final Map> vertexMap = - new HashMap>(); + V extends Writable, E extends Writable, M extends Writable> + implements Writable { + /** Configuration from the worker */ + private final Configuration conf; + /** Partition id */ + private final int partitionId; + /** Vertex map for this range (keyed by index) */ + private final Map> vertexMap = + new HashMap>(); - public Partition(Configuration conf, int partitionId) { - this.conf = conf; - this.partitionId = partitionId; - } + /** + * Constructor. + * + * @param conf Configuration. + * @param partitionId Partition id. + */ + public Partition(Configuration conf, int partitionId) { + this.conf = conf; + this.partitionId = partitionId; + } - /** - * Get the vertex for this vertex index. - * - * @param vertexIndex Vertex index to search for - * @return Vertex if it exists, null otherwise - */ - public BasicVertex getVertex(I vertexIndex) { - return vertexMap.get(vertexIndex); - } + /** + * Get the vertex for this vertex index. + * + * @param vertexIndex Vertex index to search for + * @return Vertex if it exists, null otherwise + */ + public BasicVertex getVertex(I vertexIndex) { + return vertexMap.get(vertexIndex); + } - /** - * Put a vertex into the Partition - * - * @param vertex Vertex to put in the Partition - * @return old vertex value (i.e. null if none existed prior) - */ - public BasicVertex putVertex(BasicVertex vertex) { - return vertexMap.put(vertex.getVertexId(), vertex); - } + /** + * Put a vertex into the Partition + * + * @param vertex Vertex to put in the Partition + * @return old vertex value (i.e. null if none existed prior) + */ + public BasicVertex putVertex(BasicVertex vertex) { + return vertexMap.put(vertex.getVertexId(), vertex); + } - /** - * Remove a vertex from the Partition - * - * @param vertexIndex Vertex index to remove - */ - public BasicVertex removeVertex(I vertexIndex) { - return vertexMap.remove(vertexIndex); - } + /** + * Remove a vertex from the Partition + * + * @param vertexIndex Vertex index to remove + * @return The removed vertex. + */ + public BasicVertex removeVertex(I vertexIndex) { + return vertexMap.remove(vertexIndex); + } - /** - * Get a collection of the vertices. - * - * @return Collection of the vertices - */ - public Collection> getVertices() { - return vertexMap.values(); - } + /** + * Get a collection of the vertices. + * + * @return Collection of the vertices + */ + public Collection> getVertices() { + return vertexMap.values(); + } - /** - * Get the number of edges in this partition. Computed on the fly. - * - * @return Number of edges. - */ - public long getEdgeCount() { - long edges = 0; - for (BasicVertex vertex : vertexMap.values()) { - edges += vertex.getNumOutEdges(); - } - return edges; + /** + * Get the number of edges in this partition. Computed on the fly. + * + * @return Number of edges. + */ + public long getEdgeCount() { + long edges = 0; + for (BasicVertex vertex : vertexMap.values()) { + edges += vertex.getNumOutEdges(); } + return edges; + } - /** - * Get the partition id. - * - * @return Partition id of this partition. - */ - public int getPartitionId() { - return partitionId; - } + /** + * Get the partition id. + * + * @return Partition id of this partition. + */ + public int getPartitionId() { + return partitionId; + } - @Override - public String toString() { - return "(id=" + getPartitionId() + ",V=" + vertexMap.size() + - ",E=" + getEdgeCount() + ")"; - } + @Override + public String toString() { + return "(id=" + getPartitionId() + ",V=" + vertexMap.size() + + ",E=" + getEdgeCount() + ")"; + } - @Override - public void readFields(DataInput input) throws IOException { - int vertices = input.readInt(); - for (int i = 0; i < vertices; ++i) { - BasicVertex vertex = - BspUtils.createVertex(conf); - vertex.readFields(input); - if (vertexMap.put(vertex.getVertexId(), - (BasicVertex) vertex) != null) { - throw new IllegalStateException( - "readFields: " + this + - " already has same id " + vertex); - } - } + @Override + public void readFields(DataInput input) throws IOException { + int vertices = input.readInt(); + for (int i = 0; i < vertices; ++i) { + BasicVertex vertex = + BspUtils.createVertex(conf); + vertex.readFields(input); + if (vertexMap.put(vertex.getVertexId(), + (BasicVertex) vertex) != null) { + throw new IllegalStateException( + "readFields: " + this + + " already has same id " + vertex); + } } + } - @Override - public void write(DataOutput output) throws IOException { - output.writeInt(vertexMap.size()); - for (BasicVertex vertex : vertexMap.values()) { - vertex.write(output); - } + @Override + public void write(DataOutput output) throws IOException { + output.writeInt(vertexMap.size()); + for (BasicVertex vertex : vertexMap.values()) { + vertex.write(output); } + } } diff --git a/src/main/java/org/apache/giraph/graph/partition/PartitionBalancer.java b/src/main/java/org/apache/giraph/graph/partition/PartitionBalancer.java index 3de3f1cee..2d1c2a2d5 100644 --- a/src/main/java/org/apache/giraph/graph/partition/PartitionBalancer.java +++ b/src/main/java/org/apache/giraph/graph/partition/PartitionBalancer.java @@ -35,234 +35,254 @@ * Helper class for balancing partitions across a set of workers. */ public class PartitionBalancer { - /** Partition balancing algorithm */ - public static final String PARTITION_BALANCE_ALGORITHM = - "hash.partitionBalanceAlgorithm"; - public static final String STATIC_BALANCE_ALGORITHM = - "static"; - public static final String EGDE_BALANCE_ALGORITHM = - "edges"; - public static final String VERTICES_BALANCE_ALGORITHM = - "vertices"; - /** Class logger */ - private static Logger LOG = Logger.getLogger(PartitionBalancer.class); + /** Partition balancing algorithm */ + public static final String PARTITION_BALANCE_ALGORITHM = + "hash.partitionBalanceAlgorithm"; + /** No rebalancing during the supersteps */ + public static final String STATIC_BALANCE_ALGORITHM = + "static"; + /** Rebalance across supersteps by edges */ + public static final String EGDE_BALANCE_ALGORITHM = + "edges"; + /** Rebalance across supersteps by vertices */ + public static final String VERTICES_BALANCE_ALGORITHM = + "vertices"; + /** Class logger */ + private static Logger LOG = Logger.getLogger(PartitionBalancer.class); - /** - * What value to balance partitions with? Edges, vertices? - */ - private enum BalanceValue { - UNSET, - EDGES, - VERTICES + /** + * What value to balance partitions with? Edges, vertices? + */ + private enum BalanceValue { + /** Not chosen */ + UNSET, + /** Balance with edges */ + EDGES, + /** Balance with vertices */ + VERTICES + } + + /** + * Do not construct this class. + */ + private PartitionBalancer() { } + + /** + * Get the value used to balance. + * + * @param partitionStat Stats of this partition. + * @param balanceValue Type of the value to balance. + * @return Balance value. + */ + private static long getBalanceValue(PartitionStats partitionStat, + BalanceValue balanceValue) { + switch (balanceValue) { + case EDGES: + return partitionStat.getEdgeCount(); + case VERTICES: + return partitionStat.getVertexCount(); + default: + throw new IllegalArgumentException( + "getBalanceValue: Illegal balance value " + balanceValue); } + } + + /** + * Used to sort the partition owners from lowest value to highest value + */ + private static class PartitionOwnerComparator implements + Comparator { + /** Map of owner to stats */ + private final Map ownerStatMap; + /** Value type to compare on */ + private final BalanceValue balanceValue; + /** - * Get the value used to balance. + * Only constructor. * - * @param partitionStat - * @param balanceValue - * @return + * @param ownerStatMap Map of owners to stats. + * @param balanceValue Value to balance with. */ - private static long getBalanceValue(PartitionStats partitionStat, - BalanceValue balanceValue) { - switch (balanceValue) { - case EDGES: - return partitionStat.getEdgeCount(); - case VERTICES: - return partitionStat.getVertexCount(); - default: - throw new IllegalArgumentException( - "getBalanceValue: Illegal balance value " + balanceValue); - } + public PartitionOwnerComparator( + Map ownerStatMap, + BalanceValue balanceValue) { + this.ownerStatMap = ownerStatMap; + this.balanceValue = balanceValue; } - /** - * Used to sort the partition owners from lowest value to highest value - */ - private static class PartitionOwnerComparator implements - Comparator { - /** Map of owner to stats */ - private final Map ownerStatMap; - /** Value type to compare on */ - private final BalanceValue balanceValue; - + @Override + public int compare(PartitionOwner owner1, PartitionOwner owner2) { + return (int) + (getBalanceValue(ownerStatMap.get(owner1), balanceValue) - + getBalanceValue(ownerStatMap.get(owner2), balanceValue)); + } + } - /** - * Only constructor. - * - * @param comparatorValue What to compare with? - */ - public PartitionOwnerComparator( - Map ownerStatMap, - BalanceValue balanceValue) { - this.ownerStatMap = ownerStatMap; - this.balanceValue = balanceValue; - } + /** + * Structure to keep track of how much value a {@link WorkerInfo} has + * been assigned. + */ + private static class WorkerInfoAssignments implements + Comparable { + /** Worker info associated */ + private final WorkerInfo workerInfo; + /** Balance value */ + private final BalanceValue balanceValue; + /** Map of owner to stats */ + private final Map ownerStatsMap; + /** Current value of this object */ + private long value = 0; - @Override - public int compare(PartitionOwner owner1, PartitionOwner owner2) { - return (int) - (getBalanceValue(ownerStatMap.get(owner1), balanceValue) - - getBalanceValue(ownerStatMap.get(owner2), balanceValue)); - } + /** + * Constructor with final values. + * + * @param workerInfo Worker info for assignment. + * @param balanceValue Value used to balance. + * @param ownerStatsMap Map of owner to stats. + */ + public WorkerInfoAssignments( + WorkerInfo workerInfo, + BalanceValue balanceValue, + Map ownerStatsMap) { + this.workerInfo = workerInfo; + this.balanceValue = balanceValue; + this.ownerStatsMap = ownerStatsMap; } /** - * Structure to keep track of how much value a {@link WorkerInfo} has - * been assigned. + * Get the total value of all partitions assigned to this worker. + * + * @return Total value of all partition assignments. */ - private static class WorkerInfoAssignments implements - Comparable { - /** Worker info associated */ - private final WorkerInfo workerInfo; - /** Balance value */ - private final BalanceValue balanceValue; - /** Map of owner to stats */ - private final Map ownerStatsMap; - /** Current value of this object */ - private long value = 0; - - public WorkerInfoAssignments( - WorkerInfo workerInfo, - BalanceValue balanceValue, - Map ownerStatsMap) { - this.workerInfo = workerInfo; - this.balanceValue = balanceValue; - this.ownerStatsMap = ownerStatsMap; - } - - /** - * Get the total value of all partitions assigned to this worker. - * - * @return Total value of all partition assignments. - */ - public long getValue() { - return value; - } - - /** - * Assign a {@link PartitionOwner} to this {@link WorkerInfo}. - * - * @param partitionOwner PartitionOwner to assign. - */ - public void assignPartitionOwner( - PartitionOwner partitionOwner) { - value += getBalanceValue(ownerStatsMap.get(partitionOwner), - balanceValue); - if (!partitionOwner.getWorkerInfo().equals(workerInfo)) { - partitionOwner.setPreviousWorkerInfo( - partitionOwner.getWorkerInfo()); - partitionOwner.setWorkerInfo(workerInfo); - } else { - partitionOwner.setPreviousWorkerInfo(null); - } - } - - @Override - public int compareTo(WorkerInfoAssignments other) { - return (int) - (getValue() - ((WorkerInfoAssignments) other).getValue()); - } + public long getValue() { + return value; } /** - * Balance the partitions with an algorithm based on a value. + * Assign a {@link PartitionOwner} to this {@link WorkerInfo}. * - * @param conf Configuration to find the algorithm - * @param allPartitionStats All the partition stats - * @param availableWorkerInfos All the available workers - * @return Balanced partition owners + * @param partitionOwner PartitionOwner to assign. */ - public static Collection balancePartitionsAcrossWorkers( - Configuration conf, - Collection partitionOwners, - Collection allPartitionStats, - Collection availableWorkerInfos) { + public void assignPartitionOwner( + PartitionOwner partitionOwner) { + value += getBalanceValue(ownerStatsMap.get(partitionOwner), + balanceValue); + if (!partitionOwner.getWorkerInfo().equals(workerInfo)) { + partitionOwner.setPreviousWorkerInfo( + partitionOwner.getWorkerInfo()); + partitionOwner.setWorkerInfo(workerInfo); + } else { + partitionOwner.setPreviousWorkerInfo(null); + } + } + + @Override + public int compareTo(WorkerInfoAssignments other) { + return (int) + (getValue() - ((WorkerInfoAssignments) other).getValue()); + } + } - String balanceAlgorithm = - conf.get(PARTITION_BALANCE_ALGORITHM, STATIC_BALANCE_ALGORITHM); - if (LOG.isInfoEnabled()) { - LOG.info("balancePartitionsAcrossWorkers: Using algorithm " + - balanceAlgorithm); - } - BalanceValue balanceValue = BalanceValue.UNSET; - if (balanceAlgorithm.equals(STATIC_BALANCE_ALGORITHM)) { - return partitionOwners; - } else if (balanceAlgorithm.equals(EGDE_BALANCE_ALGORITHM)) { - balanceValue = BalanceValue.EDGES; - } else if (balanceAlgorithm.equals(VERTICES_BALANCE_ALGORITHM)) { - balanceValue = BalanceValue.VERTICES; - } else { - throw new IllegalArgumentException( - "balancePartitionsAcrossWorkers: Illegal balance " + - "algorithm - " + balanceAlgorithm); - } + /** + * Balance the partitions with an algorithm based on a value. + * + * @param conf Configuration to find the algorithm + * @param partitionOwners All the owners of all partitions + * @param allPartitionStats All the partition stats + * @param availableWorkerInfos All the available workers + * @return Balanced partition owners + */ + public static Collection balancePartitionsAcrossWorkers( + Configuration conf, + Collection partitionOwners, + Collection allPartitionStats, + Collection availableWorkerInfos) { - // Join the partition stats and partition owners by partition id - Map idStatMap = - new HashMap(); - for (PartitionStats partitionStats : allPartitionStats) { - if (idStatMap.put(partitionStats.getPartitionId(), partitionStats) - != null) { - throw new IllegalStateException( - "balancePartitionsAcrossWorkers: Duplicate partition id " + - "for " + partitionStats); - } - } - Map ownerStatsMap = - new HashMap(); - for (PartitionOwner partitionOwner : partitionOwners) { - PartitionStats partitionStats = - idStatMap.get(partitionOwner.getPartitionId()); - if (partitionStats == null) { - throw new IllegalStateException( - "balancePartitionsAcrossWorkers: Missing partition " + - "stats for " + partitionOwner); - } - if (ownerStatsMap.put(partitionOwner, partitionStats) != null) { - throw new IllegalStateException( - "balancePartitionsAcrossWorkers: Duplicate partition " + - "owner " + partitionOwner); - } - } - if (ownerStatsMap.size() != partitionOwners.size()) { - throw new IllegalStateException( - "balancePartitionsAcrossWorkers: ownerStats count = " + - ownerStatsMap.size() + ", partitionOwners count = " + - partitionOwners.size() + " and should match."); - } + String balanceAlgorithm = + conf.get(PARTITION_BALANCE_ALGORITHM, STATIC_BALANCE_ALGORITHM); + if (LOG.isInfoEnabled()) { + LOG.info("balancePartitionsAcrossWorkers: Using algorithm " + + balanceAlgorithm); + } + BalanceValue balanceValue = BalanceValue.UNSET; + if (balanceAlgorithm.equals(STATIC_BALANCE_ALGORITHM)) { + return partitionOwners; + } else if (balanceAlgorithm.equals(EGDE_BALANCE_ALGORITHM)) { + balanceValue = BalanceValue.EDGES; + } else if (balanceAlgorithm.equals(VERTICES_BALANCE_ALGORITHM)) { + balanceValue = BalanceValue.VERTICES; + } else { + throw new IllegalArgumentException( + "balancePartitionsAcrossWorkers: Illegal balance " + + "algorithm - " + balanceAlgorithm); + } - List workerInfoAssignmentsList = - new ArrayList(availableWorkerInfos.size()); - for (WorkerInfo workerInfo : availableWorkerInfos) { - workerInfoAssignmentsList.add( - new WorkerInfoAssignments( - workerInfo, balanceValue, ownerStatsMap)); - } + // Join the partition stats and partition owners by partition id + Map idStatMap = + new HashMap(); + for (PartitionStats partitionStats : allPartitionStats) { + if (idStatMap.put(partitionStats.getPartitionId(), partitionStats) != + null) { + throw new IllegalStateException( + "balancePartitionsAcrossWorkers: Duplicate partition id " + + "for " + partitionStats); + } + } + Map ownerStatsMap = + new HashMap(); + for (PartitionOwner partitionOwner : partitionOwners) { + PartitionStats partitionStats = + idStatMap.get(partitionOwner.getPartitionId()); + if (partitionStats == null) { + throw new IllegalStateException( + "balancePartitionsAcrossWorkers: Missing partition " + + "stats for " + partitionOwner); + } + if (ownerStatsMap.put(partitionOwner, partitionStats) != null) { + throw new IllegalStateException( + "balancePartitionsAcrossWorkers: Duplicate partition " + + "owner " + partitionOwner); + } + } + if (ownerStatsMap.size() != partitionOwners.size()) { + throw new IllegalStateException( + "balancePartitionsAcrossWorkers: ownerStats count = " + + ownerStatsMap.size() + ", partitionOwners count = " + + partitionOwners.size() + " and should match."); + } - // A simple heuristic for balancing the partitions across the workers - // using a value (edges, vertices). An improvement would be to - // take into account the already existing partition worker assignments. - // 1. Sort the partitions by size - // 2. Place the workers in a min heap sorted by their total balance - // value. - // 3. From largest partition to the smallest, take the partition - // worker at the top of the heap, add the partition to it, and - // then put it back in the heap - List partitionOwnerList = - new ArrayList(partitionOwners); - Collections.sort(partitionOwnerList, - Collections.reverseOrder( - new PartitionOwnerComparator(ownerStatsMap, balanceValue))); - PriorityQueue minQueue = - new PriorityQueue(workerInfoAssignmentsList); - for (PartitionOwner partitionOwner : partitionOwnerList) { - WorkerInfoAssignments chosenWorker = minQueue.remove(); - chosenWorker.assignPartitionOwner(partitionOwner); - minQueue.add(chosenWorker); - } + List workerInfoAssignmentsList = + new ArrayList(availableWorkerInfos.size()); + for (WorkerInfo workerInfo : availableWorkerInfos) { + workerInfoAssignmentsList.add( + new WorkerInfoAssignments( + workerInfo, balanceValue, ownerStatsMap)); + } - return partitionOwnerList; + // A simple heuristic for balancing the partitions across the workers + // using a value (edges, vertices). An improvement would be to + // take into account the already existing partition worker assignments. + // 1. Sort the partitions by size + // 2. Place the workers in a min heap sorted by their total balance + // value. + // 3. From largest partition to the smallest, take the partition + // worker at the top of the heap, add the partition to it, and + // then put it back in the heap + List partitionOwnerList = + new ArrayList(partitionOwners); + Collections.sort(partitionOwnerList, + Collections.reverseOrder( + new PartitionOwnerComparator(ownerStatsMap, balanceValue))); + PriorityQueue minQueue = + new PriorityQueue(workerInfoAssignmentsList); + for (PartitionOwner partitionOwner : partitionOwnerList) { + WorkerInfoAssignments chosenWorker = minQueue.remove(); + chosenWorker.assignPartitionOwner(partitionOwner); + minQueue.add(chosenWorker); } + + return partitionOwnerList; + } } diff --git a/src/main/java/org/apache/giraph/graph/partition/PartitionExchange.java b/src/main/java/org/apache/giraph/graph/partition/PartitionExchange.java index 107e3af8c..1b2be9a2b 100644 --- a/src/main/java/org/apache/giraph/graph/partition/PartitionExchange.java +++ b/src/main/java/org/apache/giraph/graph/partition/PartitionExchange.java @@ -29,49 +29,49 @@ * exchange between workers. */ public class PartitionExchange { - /** Workers that I am dependent on before I can continue */ - private final Set myDependencyWorkerSet; - /** Workers that I need to sent partitions to */ - private final Map> sendWorkerPartitionMap; + /** Workers that I am dependent on before I can continue */ + private final Set myDependencyWorkerSet; + /** Workers that I need to sent partitions to */ + private final Map> sendWorkerPartitionMap; - /** - * Only constructor. - * - * @param myDependencyWorkerSet All the workers I must wait for - * @param sendWorkerPartitionMap Partitions I need to send to other workers - */ - public PartitionExchange( - Set myDependencyWorkerSet, - Map> sendWorkerPartitionMap) { - this.myDependencyWorkerSet = myDependencyWorkerSet; - this.sendWorkerPartitionMap = sendWorkerPartitionMap; - } + /** + * Only constructor. + * + * @param myDependencyWorkerSet All the workers I must wait for + * @param sendWorkerPartitionMap Partitions I need to send to other workers + */ + public PartitionExchange( + Set myDependencyWorkerSet, + Map> sendWorkerPartitionMap) { + this.myDependencyWorkerSet = myDependencyWorkerSet; + this.sendWorkerPartitionMap = sendWorkerPartitionMap; + } - /** - * Get the workers that I must wait for - * - * @return Set of workers I must wait for - */ - public Set getMyDependencyWorkerSet() { - return myDependencyWorkerSet; - } + /** + * Get the workers that I must wait for + * + * @return Set of workers I must wait for + */ + public Set getMyDependencyWorkerSet() { + return myDependencyWorkerSet; + } - /** - * Get a mapping of worker to list of partition ids I need to send to. - * - * @return Mapping of worker to partition id list I will send to. - */ - public Map> getSendWorkerPartitionMap() { - return sendWorkerPartitionMap; - } + /** + * Get a mapping of worker to list of partition ids I need to send to. + * + * @return Mapping of worker to partition id list I will send to. + */ + public Map> getSendWorkerPartitionMap() { + return sendWorkerPartitionMap; + } - /** - * Is this worker involved in a partition exchange? Receiving or sending? - * - * @return True if needs to be involved in the exchange, false otherwise. - */ - public boolean doExchange() { - return !myDependencyWorkerSet.isEmpty() || - !sendWorkerPartitionMap.isEmpty(); - } + /** + * Is this worker involved in a partition exchange? Receiving or sending? + * + * @return True if needs to be involved in the exchange, false otherwise. + */ + public boolean doExchange() { + return !myDependencyWorkerSet.isEmpty() || + !sendWorkerPartitionMap.isEmpty(); + } } diff --git a/src/main/java/org/apache/giraph/graph/partition/PartitionOwner.java b/src/main/java/org/apache/giraph/graph/partition/PartitionOwner.java index b7a569ad6..5616a8da2 100644 --- a/src/main/java/org/apache/giraph/graph/partition/PartitionOwner.java +++ b/src/main/java/org/apache/giraph/graph/partition/PartitionOwner.java @@ -25,57 +25,57 @@ * Metadata about ownership of a partition. */ public interface PartitionOwner extends Writable { - /** - * Get the partition id that maps to the relevant {@link Partition} object - * - * @return Partition id - */ - int getPartitionId(); + /** + * Get the partition id that maps to the relevant {@link Partition} object + * + * @return Partition id + */ + int getPartitionId(); - /** - * Get the worker information that is currently responsible for - * the partition id. - * - * @return Owning worker information. - */ - WorkerInfo getWorkerInfo(); + /** + * Get the worker information that is currently responsible for + * the partition id. + * + * @return Owning worker information. + */ + WorkerInfo getWorkerInfo(); - /** - * Set the current worker info. - * - * @param workerInfo Worker info responsible for partition - */ - void setWorkerInfo(WorkerInfo workerInfo); + /** + * Set the current worker info. + * + * @param workerInfo Worker info responsible for partition + */ + void setWorkerInfo(WorkerInfo workerInfo); - /** - * Get the worker information that was previously responsible for the - * partition id. - * - * @return Owning worker information or null if no previous worker info. - */ - WorkerInfo getPreviousWorkerInfo(); + /** + * Get the worker information that was previously responsible for the + * partition id. + * + * @return Owning worker information or null if no previous worker info. + */ + WorkerInfo getPreviousWorkerInfo(); - /** - * Set the previous worker info. - * - * @param workerInfo Worker info that was previously responsible for the - * partition. - */ - void setPreviousWorkerInfo(WorkerInfo workerInfo); + /** + * Set the previous worker info. + * + * @param workerInfo Worker info that was previously responsible for the + * partition. + */ + void setPreviousWorkerInfo(WorkerInfo workerInfo); - /** - * If this is a restarted checkpoint, the worker will use this information - * to determine where the checkpointed partition was stored on HDFS. - * - * @return Prefix of the checkpoint HDFS files for this partition, null if - * this is not a restarted superstep. - */ - String getCheckpointFilesPrefix(); + /** + * If this is a restarted checkpoint, the worker will use this information + * to determine where the checkpointed partition was stored on HDFS. + * + * @return Prefix of the checkpoint HDFS files for this partition, null if + * this is not a restarted superstep. + */ + String getCheckpointFilesPrefix(); - /** - * Set the checkpoint files prefix. Master uses this. - * - * @param checkpointFilesPrefix HDFS checkpoint file prefix - */ - void setCheckpointFilesPrefix(String checkpointFilesPrefix); + /** + * Set the checkpoint files prefix. Master uses this. + * + * @param checkpointFilesPrefix HDFS checkpoint file prefix + */ + void setCheckpointFilesPrefix(String checkpointFilesPrefix); } diff --git a/src/main/java/org/apache/giraph/graph/partition/PartitionStats.java b/src/main/java/org/apache/giraph/graph/partition/PartitionStats.java index 040687c3c..4aa679f20 100644 --- a/src/main/java/org/apache/giraph/graph/partition/PartitionStats.java +++ b/src/main/java/org/apache/giraph/graph/partition/PartitionStats.java @@ -29,74 +29,125 @@ * actual partition data, only the statistics. */ public class PartitionStats implements Writable { - private int partitionId = -1; - private long vertexCount = 0; - private long finishedVertexCount = 0; - private long edgeCount = 0; - - public PartitionStats() {} - - public PartitionStats(int partitionId, - long vertexCount, - long finishedVertexCount, - long edgeCount) { - this.partitionId = partitionId; - this.vertexCount = vertexCount; - this.finishedVertexCount = finishedVertexCount; - this.edgeCount = edgeCount; - } - - public void setPartitionId(int partitionId) { - this.partitionId = partitionId; - } - - public int getPartitionId() { - return partitionId; - } - - public void incrVertexCount() { - ++vertexCount; - } - - public long getVertexCount() { - return vertexCount; - } - - public void incrFinishedVertexCount() { - ++finishedVertexCount; - } - - public long getFinishedVertexCount() { - return finishedVertexCount; - } - - public void addEdgeCount(long edgeCount) { - this.edgeCount += edgeCount; - } - - public long getEdgeCount() { - return edgeCount; - } - - @Override - public void readFields(DataInput input) throws IOException { - partitionId = input.readInt(); - vertexCount = input.readLong(); - finishedVertexCount = input.readLong(); - edgeCount = input.readLong(); - } - - @Override - public void write(DataOutput output) throws IOException { - output.writeInt(partitionId); - output.writeLong(vertexCount); - output.writeLong(finishedVertexCount); - output.writeLong(edgeCount); - } - - @Override - public String toString() { - return "(id=" + partitionId + ",vtx=" + vertexCount + ",finVtx=" + - finishedVertexCount + ",edges=" + edgeCount + ")"; - } + /** Id of partition to keep stats for */ + private int partitionId = -1; + /** Vertices in this partition */ + private long vertexCount = 0; + /** Finished vertices in this partition */ + private long finishedVertexCount = 0; + /** Edges in this partition */ + private long edgeCount = 0; + + /** + * Default constructor for reflection. + */ + public PartitionStats() { } + + /** + * Constructor with the initial stats. + * + * @param partitionId Partition count. + * @param vertexCount Vertex count. + * @param finishedVertexCount Finished vertex count. + * @param edgeCount Edge count. + */ + public PartitionStats(int partitionId, + long vertexCount, + long finishedVertexCount, + long edgeCount) { + this.partitionId = partitionId; + this.vertexCount = vertexCount; + this.finishedVertexCount = finishedVertexCount; + this.edgeCount = edgeCount; + } + + /** + * Set the partition id. + * + * @param partitionId New partition id. + */ + public void setPartitionId(int partitionId) { + this.partitionId = partitionId; + } + + /** + * Get partition id. + * + * @return Partition id. + */ + public int getPartitionId() { + return partitionId; + } + + /** + * Increment the vertex count by one. + */ + public void incrVertexCount() { + ++vertexCount; + } + + /** + * Get the vertex count. + * + * @return Vertex count. + */ + public long getVertexCount() { + return vertexCount; + } + + /** + * Increment the finished vertex count by one. + */ + public void incrFinishedVertexCount() { + ++finishedVertexCount; + } + + /** + * Get the finished vertex count. + * + * @return Finished vertex count. + */ + public long getFinishedVertexCount() { + return finishedVertexCount; + } + + /** + * Add edges to the edge count. + * + * @param edgeCount Number of edges to add. + */ + public void addEdgeCount(long edgeCount) { + this.edgeCount += edgeCount; + } + + /** + * Get the edge count. + * + * @return Edge count. + */ + public long getEdgeCount() { + return edgeCount; + } + + @Override + public void readFields(DataInput input) throws IOException { + partitionId = input.readInt(); + vertexCount = input.readLong(); + finishedVertexCount = input.readLong(); + edgeCount = input.readLong(); + } + + @Override + public void write(DataOutput output) throws IOException { + output.writeInt(partitionId); + output.writeLong(vertexCount); + output.writeLong(finishedVertexCount); + output.writeLong(edgeCount); + } + + @Override + public String toString() { + return "(id=" + partitionId + ",vtx=" + vertexCount + ",finVtx=" + + finishedVertexCount + ",edges=" + edgeCount + ")"; + } } diff --git a/src/main/java/org/apache/giraph/graph/partition/PartitionUtils.java b/src/main/java/org/apache/giraph/graph/partition/PartitionUtils.java index cb3fd4d8a..5600dad67 100644 --- a/src/main/java/org/apache/giraph/graph/partition/PartitionUtils.java +++ b/src/main/java/org/apache/giraph/graph/partition/PartitionUtils.java @@ -37,102 +37,115 @@ * Helper class for {@link Partition} related operations. */ public class PartitionUtils { - /** Class logger */ - private static Logger LOG = Logger.getLogger(PartitionUtils.class); + /** Class logger */ + private static Logger LOG = Logger.getLogger(PartitionUtils.class); - private static class EdgeCountComparator implements - Comparator> { + /** + * Do not construct this object. + */ + private PartitionUtils() { } - @Override - public int compare(Entry worker1, - Entry worker2) { - return (int) (worker1.getValue().getEdgeCount() - - worker2.getValue().getEdgeCount()); - } + /** + * Compare edge counts for Entry objects. + */ + private static class EdgeCountComparator implements + Comparator> { + @Override + public int compare(Entry worker1, + Entry worker2) { + return (int) (worker1.getValue().getEdgeCount() - + worker2.getValue().getEdgeCount()); } + } - private static class VertexCountComparator implements - Comparator> { - - @Override - public int compare(Entry worker1, - Entry worker2) { - return (int) (worker1.getValue().getEdgeCount() - - worker2.getValue().getEdgeCount()); - } + /** + * Compare vertex counts between a {@link WorkerInfo} and + * {@link VertexEdgeCount}. + */ + private static class VertexCountComparator implements + Comparator> { + @Override + public int compare(Entry worker1, + Entry worker2) { + return (int) (worker1.getValue().getVertexCount() - + worker2.getValue().getVertexCount()); } + } - /** - * Check for imbalances on a per worker basis, by calculating the - * mean, high and low workers by edges and vertices. - */ - public static void analyzePartitionStats( - Collection partitionOwnerList, - List allPartitionStats) { - Map idOwnerMap = - new HashMap(); - for (PartitionOwner partitionOwner : partitionOwnerList) { - if (idOwnerMap.put(partitionOwner.getPartitionId(), - partitionOwner) != null) { - throw new IllegalStateException( - "analyzePartitionStats: Duplicate partition " + - partitionOwner); - } - } + /** + * Check for imbalances on a per worker basis, by calculating the + * mean, high and low workers by edges and vertices. + * + * @param partitionOwnerList List of partition owners. + * @param allPartitionStats All the partition stats. + */ + public static void analyzePartitionStats( + Collection partitionOwnerList, + List allPartitionStats) { + Map idOwnerMap = + new HashMap(); + for (PartitionOwner partitionOwner : partitionOwnerList) { + if (idOwnerMap.put(partitionOwner.getPartitionId(), + partitionOwner) != null) { + throw new IllegalStateException( + "analyzePartitionStats: Duplicate partition " + + partitionOwner); + } + } - Map workerStatsMap = Maps.newHashMap(); - VertexEdgeCount totalVertexEdgeCount = new VertexEdgeCount(); - for (PartitionStats partitionStats : allPartitionStats) { - WorkerInfo workerInfo = - idOwnerMap.get(partitionStats.getPartitionId()).getWorkerInfo(); - VertexEdgeCount vertexEdgeCount = - workerStatsMap.get(workerInfo); - if (vertexEdgeCount == null) { - workerStatsMap.put( - workerInfo, - new VertexEdgeCount(partitionStats.getVertexCount(), - partitionStats.getEdgeCount())); - } else { - workerStatsMap.put( - workerInfo, - vertexEdgeCount.incrVertexEdgeCount( - partitionStats.getVertexCount(), - partitionStats.getEdgeCount())); - } - totalVertexEdgeCount = - totalVertexEdgeCount.incrVertexEdgeCount( - partitionStats.getVertexCount(), - partitionStats.getEdgeCount()); - } + Map workerStatsMap = Maps.newHashMap(); + VertexEdgeCount totalVertexEdgeCount = new VertexEdgeCount(); + for (PartitionStats partitionStats : allPartitionStats) { + WorkerInfo workerInfo = + idOwnerMap.get(partitionStats.getPartitionId()).getWorkerInfo(); + VertexEdgeCount vertexEdgeCount = + workerStatsMap.get(workerInfo); + if (vertexEdgeCount == null) { + workerStatsMap.put( + workerInfo, + new VertexEdgeCount(partitionStats.getVertexCount(), + partitionStats.getEdgeCount())); + } else { + workerStatsMap.put( + workerInfo, + vertexEdgeCount.incrVertexEdgeCount( + partitionStats.getVertexCount(), + partitionStats.getEdgeCount())); + } + totalVertexEdgeCount = + totalVertexEdgeCount.incrVertexEdgeCount( + partitionStats.getVertexCount(), + partitionStats.getEdgeCount()); + } - List> workerEntryList = - Lists.newArrayList(workerStatsMap.entrySet()); + List> workerEntryList = + Lists.newArrayList(workerStatsMap.entrySet()); - if (LOG.isInfoEnabled()) { - Collections.sort(workerEntryList, new VertexCountComparator()); - LOG.info("analyzePartitionStats: Vertices - Mean: " + - (totalVertexEdgeCount.getVertexCount() / - workerStatsMap.size()) + - ", Min: " + - workerEntryList.get(0).getKey() + " - " + - workerEntryList.get(0).getValue().getVertexCount() + - ", Max: "+ - workerEntryList.get(workerEntryList.size() - 1).getKey() + - " - " + - workerEntryList.get(workerEntryList.size() - 1). - getValue().getVertexCount()); - Collections.sort(workerEntryList, new EdgeCountComparator()); - LOG.info("analyzePartitionStats: Edges - Mean: " + - (totalVertexEdgeCount.getEdgeCount() / - workerStatsMap.size()) + - ", Min: " + - workerEntryList.get(0).getKey() + " - " + - workerEntryList.get(0).getValue().getEdgeCount() + - ", Max: "+ - workerEntryList.get(workerEntryList.size() - 1).getKey() + - " - " + - workerEntryList.get(workerEntryList.size() - 1). - getValue().getEdgeCount()); - } + if (LOG.isInfoEnabled()) { + Collections.sort(workerEntryList, new VertexCountComparator()); + LOG.info("analyzePartitionStats: Vertices - Mean: " + + (totalVertexEdgeCount.getVertexCount() / + workerStatsMap.size()) + + ", Min: " + + workerEntryList.get(0).getKey() + " - " + + workerEntryList.get(0).getValue().getVertexCount() + + ", Max: " + + workerEntryList.get(workerEntryList.size() - 1).getKey() + + " - " + + workerEntryList.get(workerEntryList.size() - 1). + getValue().getVertexCount()); + Collections.sort(workerEntryList, new EdgeCountComparator()); + LOG.info("analyzePartitionStats: Edges - Mean: " + + (totalVertexEdgeCount.getEdgeCount() / + workerStatsMap.size()) + + ", Min: " + + workerEntryList.get(0).getKey() + " - " + + workerEntryList.get(0).getValue().getEdgeCount() + + ", Max: " + + workerEntryList.get(workerEntryList.size() - 1).getKey() + + " - " + + workerEntryList.get(workerEntryList.size() - 1). + getValue().getEdgeCount()); } + } } diff --git a/src/main/java/org/apache/giraph/graph/partition/RangeMasterPartitioner.java b/src/main/java/org/apache/giraph/graph/partition/RangeMasterPartitioner.java index bf7e5a64e..8e417eceb 100644 --- a/src/main/java/org/apache/giraph/graph/partition/RangeMasterPartitioner.java +++ b/src/main/java/org/apache/giraph/graph/partition/RangeMasterPartitioner.java @@ -34,11 +34,10 @@ */ @SuppressWarnings("rawtypes") public abstract class RangeMasterPartitioner implements - MasterGraphPartitioner { - - @Override - public PartitionStats createPartitionStats() { - return new RangePartitionStats(); - } + V extends Writable, E extends Writable, M extends Writable> implements + MasterGraphPartitioner { + @Override + public PartitionStats createPartitionStats() { + return new RangePartitionStats(); + } } diff --git a/src/main/java/org/apache/giraph/graph/partition/RangePartitionOwner.java b/src/main/java/org/apache/giraph/graph/partition/RangePartitionOwner.java index 3b6f7f99c..3daa03acf 100644 --- a/src/main/java/org/apache/giraph/graph/partition/RangePartitionOwner.java +++ b/src/main/java/org/apache/giraph/graph/partition/RangePartitionOwner.java @@ -33,31 +33,43 @@ */ @SuppressWarnings("rawtypes") public class RangePartitionOwner - extends BasicPartitionOwner { - /** Max index for this partition */ - private I maxIndex; - - public RangePartitionOwner() { - } - - public RangePartitionOwner(I maxIndex) { - this.maxIndex = maxIndex; - } - - public I getMaxIndex() { - return maxIndex; - } - - @Override - public void readFields(DataInput input) throws IOException { - super.readFields(input); - maxIndex = BspUtils.createVertexIndex(getConf()); - maxIndex.readFields(input); - } - - @Override - public void write(DataOutput output) throws IOException { - super.write(output); - maxIndex.write(output); - } + extends BasicPartitionOwner { + /** Max index for this partition */ + private I maxIndex; + + /** + * Default constructor. + */ + public RangePartitionOwner() { } + + /** + * Constructor with the max index. + * + * @param maxIndex Max index of this partition. + */ + public RangePartitionOwner(I maxIndex) { + this.maxIndex = maxIndex; + } + + /** + * Get the maximum index of this partition owner. + * + * @return Maximum index. + */ + public I getMaxIndex() { + return maxIndex; + } + + @Override + public void readFields(DataInput input) throws IOException { + super.readFields(input); + maxIndex = BspUtils.createVertexIndex(getConf()); + maxIndex.readFields(input); + } + + @Override + public void write(DataOutput output) throws IOException { + super.write(output); + maxIndex.write(output); + } } diff --git a/src/main/java/org/apache/giraph/graph/partition/RangePartitionStats.java b/src/main/java/org/apache/giraph/graph/partition/RangePartitionStats.java index 2da2a4d4f..3ab43e82e 100644 --- a/src/main/java/org/apache/giraph/graph/partition/RangePartitionStats.java +++ b/src/main/java/org/apache/giraph/graph/partition/RangePartitionStats.java @@ -32,37 +32,37 @@ */ @SuppressWarnings("rawtypes") public class RangePartitionStats - extends PartitionStats { - /** Can be null if no hint, otherwise a splitting hint */ - private RangeSplitHint hint; + extends PartitionStats { + /** Can be null if no hint, otherwise a splitting hint */ + private RangeSplitHint hint; - /** - * Get the range split hint (if any) - * - * @return Hint of how to split the range if desired, null otherwise - */ - public RangeSplitHint getRangeSplitHint() { - return hint; - } + /** + * Get the range split hint (if any) + * + * @return Hint of how to split the range if desired, null otherwise + */ + public RangeSplitHint getRangeSplitHint() { + return hint; + } - @Override - public void readFields(DataInput input) throws IOException { - super.readFields(input); - boolean hintExists = input.readBoolean(); - if (hintExists) { - hint = new RangeSplitHint(); - hint.readFields(input); - } else { - hint = null; - } + @Override + public void readFields(DataInput input) throws IOException { + super.readFields(input); + boolean hintExists = input.readBoolean(); + if (hintExists) { + hint = new RangeSplitHint(); + hint.readFields(input); + } else { + hint = null; } + } - @Override - public void write(DataOutput output) throws IOException { - super.write(output); - output.writeBoolean(hint != null); - if (hint != null) { - hint.write(output); - } + @Override + public void write(DataOutput output) throws IOException { + super.write(output); + output.writeBoolean(hint != null); + if (hint != null) { + hint.write(output); } + } } diff --git a/src/main/java/org/apache/giraph/graph/partition/RangeSplitHint.java b/src/main/java/org/apache/giraph/graph/partition/RangeSplitHint.java index c2753328a..cb585b49a 100644 --- a/src/main/java/org/apache/giraph/graph/partition/RangeSplitHint.java +++ b/src/main/java/org/apache/giraph/graph/partition/RangeSplitHint.java @@ -36,38 +36,38 @@ */ @SuppressWarnings("rawtypes") public class RangeSplitHint - implements Writable, Configurable { - /** Hinted split index */ - private I splitIndex; - /** Number of vertices in this range before the split */ - private long preSplitVertexCount; - /** Number of vertices in this range after the split */ - private long postSplitVertexCount; - /** Configuration */ - private Configuration conf; + implements Writable, Configurable { + /** Hinted split index */ + private I splitIndex; + /** Number of vertices in this range before the split */ + private long preSplitVertexCount; + /** Number of vertices in this range after the split */ + private long postSplitVertexCount; + /** Configuration */ + private Configuration conf; - @Override - public void readFields(DataInput input) throws IOException { - splitIndex = BspUtils.createVertexIndex(conf); - splitIndex.readFields(input); - preSplitVertexCount = input.readLong(); - postSplitVertexCount = input.readLong(); - } + @Override + public void readFields(DataInput input) throws IOException { + splitIndex = BspUtils.createVertexIndex(conf); + splitIndex.readFields(input); + preSplitVertexCount = input.readLong(); + postSplitVertexCount = input.readLong(); + } - @Override - public void write(DataOutput output) throws IOException { - splitIndex.write(output); - output.writeLong(preSplitVertexCount); - output.writeLong(postSplitVertexCount); - } + @Override + public void write(DataOutput output) throws IOException { + splitIndex.write(output); + output.writeLong(preSplitVertexCount); + output.writeLong(postSplitVertexCount); + } - @Override - public Configuration getConf() { - return conf; - } + @Override + public Configuration getConf() { + return conf; + } - @Override - public void setConf(Configuration conf) { - this.conf = conf; - } + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } } diff --git a/src/main/java/org/apache/giraph/graph/partition/RangeWorkerPartitioner.java b/src/main/java/org/apache/giraph/graph/partition/RangeWorkerPartitioner.java index 6e3f3e09d..b963d86e1 100644 --- a/src/main/java/org/apache/giraph/graph/partition/RangeWorkerPartitioner.java +++ b/src/main/java/org/apache/giraph/graph/partition/RangeWorkerPartitioner.java @@ -43,36 +43,36 @@ */ @SuppressWarnings("rawtypes") public abstract class RangeWorkerPartitioner implements - WorkerGraphPartitioner { - /** Mapping of the vertex ids to the {@link PartitionOwner} */ - protected NavigableMap> vertexRangeMap = - new TreeMap>(); + V extends Writable, E extends Writable, M extends Writable> implements + WorkerGraphPartitioner { + /** Mapping of the vertex ids to the {@link PartitionOwner} */ + protected NavigableMap> vertexRangeMap = + new TreeMap>(); - @Override - public PartitionOwner createPartitionOwner() { - return new RangePartitionOwner(); - } + @Override + public PartitionOwner createPartitionOwner() { + return new RangePartitionOwner(); + } - @Override - public PartitionOwner getPartitionOwner(I vertexId) { - // Find the partition owner based on the maximum partition id. - // If the vertex id exceeds any of the maximum partition ids, give - // it to the last one - if (vertexId == null) { - throw new IllegalArgumentException( - "getPartitionOwner: Illegal null vertex id"); - } - I maxVertexIndex = vertexRangeMap.ceilingKey(vertexId); - if (maxVertexIndex == null) { - return vertexRangeMap.lastEntry().getValue(); - } else { - return vertexRangeMap.get(vertexId); - } + @Override + public PartitionOwner getPartitionOwner(I vertexId) { + // Find the partition owner based on the maximum partition id. + // If the vertex id exceeds any of the maximum partition ids, give + // it to the last one + if (vertexId == null) { + throw new IllegalArgumentException( + "getPartitionOwner: Illegal null vertex id"); } - - @Override - public Collection getPartitionOwners() { - return vertexRangeMap.values(); + I maxVertexIndex = vertexRangeMap.ceilingKey(vertexId); + if (maxVertexIndex == null) { + return vertexRangeMap.lastEntry().getValue(); + } else { + return vertexRangeMap.get(vertexId); } + } + + @Override + public Collection getPartitionOwners() { + return vertexRangeMap.values(); + } } diff --git a/src/main/java/org/apache/giraph/graph/partition/WorkerGraphPartitioner.java b/src/main/java/org/apache/giraph/graph/partition/WorkerGraphPartitioner.java index ce1068056..6bf9a828c 100644 --- a/src/main/java/org/apache/giraph/graph/partition/WorkerGraphPartitioner.java +++ b/src/main/java/org/apache/giraph/graph/partition/WorkerGraphPartitioner.java @@ -29,62 +29,67 @@ * Stores the {@link PartitionOwner} objects from the master and provides the * mapping of vertex to {@link PartitionOwner}. Also generates the partition * owner implementation. + * + * @param Vertex id + * @param Vertex value + * @param Edge value + * @param Message data */ @SuppressWarnings("rawtypes") public interface WorkerGraphPartitioner { - /** - * Instantiate the {@link PartitionOwner} implementation used to read the - * master assignments. - * - * @return Instantiated {@link PartitionOwner} object - */ - PartitionOwner createPartitionOwner(); + V extends Writable, E extends Writable, M extends Writable> { + /** + * Instantiate the {@link PartitionOwner} implementation used to read the + * master assignments. + * + * @return Instantiated {@link PartitionOwner} object + */ + PartitionOwner createPartitionOwner(); - /** - * Figure out the owner of a vertex - * - * @param vertexId Vertex id to get the partition for - * @return Correct partition owner - */ - PartitionOwner getPartitionOwner(I vertexId); + /** + * Figure out the owner of a vertex + * + * @param vertexId Vertex id to get the partition for + * @return Correct partition owner + */ + PartitionOwner getPartitionOwner(I vertexId); - /** - * At the end of a superstep, workers have {@link PartitionStats} generated - * for each of their partitions. This method will allow the user to - * modify or create their own {@link PartitionStats} interfaces to send to - * the master. - * - * @param workerPartitionStats Stats generated by the infrastructure during - * the superstep - * @param partitionMap Map of all the partitions owned by this worker - * (could be used to provide more useful stat information) - * @return Final partition stats - */ - Collection finalizePartitionStats( - Collection workerPartitionStats, - Map> partitionMap); + /** + * At the end of a superstep, workers have {@link PartitionStats} generated + * for each of their partitions. This method will allow the user to + * modify or create their own {@link PartitionStats} interfaces to send to + * the master. + * + * @param workerPartitionStats Stats generated by the infrastructure during + * the superstep + * @param partitionMap Map of all the partitions owned by this worker + * (could be used to provide more useful stat information) + * @return Final partition stats + */ + Collection finalizePartitionStats( + Collection workerPartitionStats, + Map> partitionMap); - /** - * Get the partitions owners and update locally. Returns the partitions - * to send to other workers and other dependencies. - * - * @param myWorkerInfo Worker info. - * @param masterSetPartitionOwners Master set partition owners, received - * prior to beginning the superstep - * @param partitionMap Map of all the partitions owned by this worker - * (can be used to fill the return map of partitions to send) - * @return Information for the partition exchange. - */ - PartitionExchange updatePartitionOwners( - WorkerInfo myWorkerInfo, - Collection masterSetPartitionOwners, - Map> partitionMap); + /** + * Get the partitions owners and update locally. Returns the partitions + * to send to other workers and other dependencies. + * + * @param myWorkerInfo Worker info. + * @param masterSetPartitionOwners Master set partition owners, received + * prior to beginning the superstep + * @param partitionMap Map of all the partitions owned by this worker + * (can be used to fill the return map of partitions to send) + * @return Information for the partition exchange. + */ + PartitionExchange updatePartitionOwners( + WorkerInfo myWorkerInfo, + Collection masterSetPartitionOwners, + Map> partitionMap); - /** - * Get a collection of the {@link PartitionOwner} objects. - * - * @return Collection of owners for every partition. - */ - Collection getPartitionOwners(); + /** + * Get a collection of the {@link PartitionOwner} objects. + * + * @return Collection of owners for every partition. + */ + Collection getPartitionOwners(); } diff --git a/src/main/java/org/apache/giraph/graph/partition/package-info.java b/src/main/java/org/apache/giraph/graph/partition/package-info.java new file mode 100644 index 000000000..4d6f6c335 --- /dev/null +++ b/src/main/java/org/apache/giraph/graph/partition/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/** + * Package of partitioning related objects. + */ +package org.apache.giraph.graph.partition; diff --git a/src/main/java/org/apache/giraph/hadoop/BspPolicyProvider.java b/src/main/java/org/apache/giraph/hadoop/BspPolicyProvider.java index d5d639858..900a7b225 100644 --- a/src/main/java/org/apache/giraph/hadoop/BspPolicyProvider.java +++ b/src/main/java/org/apache/giraph/hadoop/BspPolicyProvider.java @@ -23,18 +23,21 @@ import org.apache.hadoop.security.authorize.Service; /** - * {@link PolicyProvider} for Map-Reduce protocols. - */ + * {@link PolicyProvider} for Map-Reduce protocols. + */ public class BspPolicyProvider extends PolicyProvider { - private static final Service[] bspCommunicationsServices = - new Service[] { - new Service("security.bsp.communications.protocol.acl", - CommunicationsInterface.class), + /** + * Communication services array. + */ + private static final Service[] BSP_COMMUNICATION_SERVICES = + new Service[] { + new Service("security.bsp.communications.protocol.acl", + CommunicationsInterface.class), }; - @Override - public Service[] getServices() { - return bspCommunicationsServices; - } + @Override + public Service[] getServices() { + return BSP_COMMUNICATION_SERVICES; + } } diff --git a/src/main/java/org/apache/giraph/hadoop/BspTokenSelector.java b/src/main/java/org/apache/giraph/hadoop/BspTokenSelector.java index 458c0b94c..5c6ad8188 100644 --- a/src/main/java/org/apache/giraph/hadoop/BspTokenSelector.java +++ b/src/main/java/org/apache/giraph/hadoop/BspTokenSelector.java @@ -27,25 +27,24 @@ import org.apache.hadoop.security.token.TokenSelector; /** - * Look through tokens to find the first job token that matches the service - * and return it. - */ + * Look through tokens to find the first job token that matches the service + * and return it. + */ public class BspTokenSelector implements TokenSelector { - - @SuppressWarnings("unchecked") - @Override - public Token selectToken(Text service, - Collection> tokens) { - if (service == null) { - return null; - } - Text KIND_NAME = new Text("mapreduce.job"); - for (Token token : tokens) { - if (KIND_NAME.equals(token.getKind())) { - return (Token) token; - } - } - return null; + @SuppressWarnings("unchecked") + @Override + public Token selectToken(Text service, + Collection> tokens) { + if (service == null) { + return null; + } + Text kindName = new Text("mapreduce.job"); + for (Token token : tokens) { + if (kindName.equals(token.getKind())) { + return (Token) token; + } } + return null; + } } diff --git a/src/main/java/org/apache/giraph/hadoop/package-info.java b/src/main/java/org/apache/giraph/hadoop/package-info.java new file mode 100644 index 000000000..3eab09381 --- /dev/null +++ b/src/main/java/org/apache/giraph/hadoop/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/** + * Package of policy and token information for Hadoop. + */ +package org.apache.giraph.hadoop; diff --git a/src/main/java/org/apache/giraph/integration/SuperstepHashPartitionerFactory.java b/src/main/java/org/apache/giraph/integration/SuperstepHashPartitionerFactory.java index 02b615767..2ed02335c 100644 --- a/src/main/java/org/apache/giraph/integration/SuperstepHashPartitionerFactory.java +++ b/src/main/java/org/apache/giraph/integration/SuperstepHashPartitionerFactory.java @@ -38,84 +38,90 @@ * Example graph partitioner that builds on {@link HashMasterPartitioner} to * send the partitions to the worker that matches the superstep. It is for * testing only and should never be used in practice. + * + * @param Vertex id + * @param Vertex value + * @param Edge value + * @param Message data */ @SuppressWarnings("rawtypes") -public class SuperstepHashPartitionerFactory< - I extends WritableComparable, - V extends Writable, E extends Writable, M extends Writable> - extends HashPartitionerFactory { +public class SuperstepHashPartitionerFactory + extends HashPartitionerFactory { + /** + * Changes the {@link HashMasterPartitioner} to make ownership of the + * partitions based on a superstep. For testing only as it is totally + * unbalanced. + * + * @param vertex id + * @param vertex data + * @param edge data + * @param message data + */ + private static class SuperstepMasterPartition + extends HashMasterPartitioner { + /** Class logger */ + private static Logger LOG = + Logger.getLogger(SuperstepMasterPartition.class); /** - * Changes the {@link HashMasterPartitioner} to make ownership of the - * partitions based on a superstep. For testing only as it is totally - * unbalanced. + * Construction with configuration. * - * @param vertex id - * @param vertex data - * @param edge data - * @param message data + * @param conf Configuration to be stored. */ - private static class SuperstepMasterPartition< - I extends WritableComparable, - V extends Writable, E extends Writable, M extends Writable> - extends HashMasterPartitioner { - /** Class logger */ - private static Logger LOG = - Logger.getLogger(SuperstepMasterPartition.class); + public SuperstepMasterPartition(Configuration conf) { + super(conf); + } - public SuperstepMasterPartition(Configuration conf) { - super(conf); + @Override + public Collection generateChangedPartitionOwners( + Collection allPartitionStatsList, + Collection availableWorkerInfos, + int maxWorkers, + long superstep) { + // Assign all the partitions to + // superstep mod availableWorkerInfos + // Guaranteed to be different if the workers (and their order) + // do not change + long workerIndex = superstep % availableWorkerInfos.size(); + int i = 0; + WorkerInfo chosenWorkerInfo = null; + for (WorkerInfo workerInfo : availableWorkerInfos) { + if (workerIndex == i) { + chosenWorkerInfo = workerInfo; } + ++i; + } + if (LOG.isInfoEnabled()) { + LOG.info("generateChangedPartitionOwners: Chosen worker " + + "for superstep " + superstep + " is " + + chosenWorkerInfo); + } - @Override - public Collection generateChangedPartitionOwners( - Collection allPartitionStatsList, - Collection availableWorkerInfos, - int maxWorkers, - long superstep) { - // Assign all the partitions to - // superstep mod availableWorkerInfos - // Guaranteed to be different if the workers (and their order) - // do not change - long workerIndex = superstep % availableWorkerInfos.size(); - int i = 0; - WorkerInfo chosenWorkerInfo = null; - for (WorkerInfo workerInfo : availableWorkerInfos) { - if (workerIndex == i) { - chosenWorkerInfo = workerInfo; - } - ++i; - } - if (LOG.isInfoEnabled()) { - LOG.info("generateChangedPartitionOwners: Chosen worker " + - "for superstep " + superstep + " is " + - chosenWorkerInfo); - } - - List partitionOwnerList = - new ArrayList(); - for (PartitionOwner partitionOwner : - getCurrentPartitionOwners()) { - WorkerInfo prevWorkerinfo = - partitionOwner.getWorkerInfo().equals(chosenWorkerInfo) ? - null : partitionOwner.getWorkerInfo(); - PartitionOwner tmpPartitionOwner = - new BasicPartitionOwner(partitionOwner.getPartitionId(), - chosenWorkerInfo, - prevWorkerinfo, - null); - partitionOwnerList.add(tmpPartitionOwner); - LOG.info("partition owner was " + partitionOwner + - ", new " + tmpPartitionOwner); - } - setPartitionOwnerList(partitionOwnerList); - return partitionOwnerList; - } + List partitionOwnerList = new ArrayList(); + for (PartitionOwner partitionOwner : + getCurrentPartitionOwners()) { + WorkerInfo prevWorkerinfo = + partitionOwner.getWorkerInfo().equals(chosenWorkerInfo) ? + null : partitionOwner.getWorkerInfo(); + PartitionOwner tmpPartitionOwner = + new BasicPartitionOwner(partitionOwner.getPartitionId(), + chosenWorkerInfo, + prevWorkerinfo, + null); + partitionOwnerList.add(tmpPartitionOwner); + LOG.info("partition owner was " + partitionOwner + + ", new " + tmpPartitionOwner); + } + setPartitionOwnerList(partitionOwnerList); + return partitionOwnerList; } + } - @Override - public MasterGraphPartitioner - createMasterGraphPartitioner() { - return new SuperstepMasterPartition(getConf()); - } + @Override + public MasterGraphPartitioner + createMasterGraphPartitioner() { + return new SuperstepMasterPartition(getConf()); + } } diff --git a/src/main/java/org/apache/giraph/integration/package-info.java b/src/main/java/org/apache/giraph/integration/package-info.java new file mode 100644 index 000000000..4c6ae30a1 --- /dev/null +++ b/src/main/java/org/apache/giraph/integration/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/** + * Package of all helper integration test objects. + */ +package org.apache.giraph.integration; diff --git a/src/main/java/org/apache/giraph/lib/AdjacencyListTextVertexOutputFormat.java b/src/main/java/org/apache/giraph/lib/AdjacencyListTextVertexOutputFormat.java index d966dae6c..16f8d3acb 100644 --- a/src/main/java/org/apache/giraph/lib/AdjacencyListTextVertexOutputFormat.java +++ b/src/main/java/org/apache/giraph/lib/AdjacencyListTextVertexOutputFormat.java @@ -38,26 +38,41 @@ * @param Edge value */ @SuppressWarnings("rawtypes") -public class AdjacencyListTextVertexOutputFormat extends TextVertexOutputFormat{ +public class AdjacencyListTextVertexOutputFormat + extends TextVertexOutputFormat { + /** + * Vertex writer associated wtih {@link AdjacencyListTextVertexOutputFormat}. + * + * @param Vertex id + * @param Vertex data + * @param Edge data + */ static class AdjacencyListVertexWriter extends TextVertexWriter { + /** Split delimiter */ public static final String LINE_TOKENIZE_VALUE = "output.delimiter"; + /** Default split delimiter */ public static final String LINE_TOKENIZE_VALUE_DEFAULT = "\t"; - + /** Cached split delimeter */ private String delimiter; - public AdjacencyListVertexWriter(RecordWriter recordWriter) { + /** + * Constructor with writer. + * + * @param recordWriter Record writer used for writing. + */ + public AdjacencyListVertexWriter(RecordWriter recordWriter) { super(recordWriter); } @Override public void writeVertex(BasicVertex vertex) throws IOException, - InterruptedException { + InterruptedException { if (delimiter == null) { delimiter = getContext().getConfiguration() - .get(LINE_TOKENIZE_VALUE, LINE_TOKENIZE_VALUE_DEFAULT); + .get(LINE_TOKENIZE_VALUE, LINE_TOKENIZE_VALUE_DEFAULT); } StringBuffer sb = new StringBuffer(vertex.getVertexId().toString()); @@ -75,9 +90,8 @@ public void writeVertex(BasicVertex vertex) throws IOException, @Override public VertexWriter createVertexWriter(TaskAttemptContext context) - throws IOException, InterruptedException { + throws IOException, InterruptedException { return new AdjacencyListVertexWriter - (textOutputFormat.getRecordWriter(context)); + (textOutputFormat.getRecordWriter(context)); } - } diff --git a/src/main/java/org/apache/giraph/lib/AdjacencyListVertexReader.java b/src/main/java/org/apache/giraph/lib/AdjacencyListVertexReader.java index b9ccd6c6e..3b5af7f6e 100644 --- a/src/main/java/org/apache/giraph/lib/AdjacencyListVertexReader.java +++ b/src/main/java/org/apache/giraph/lib/AdjacencyListVertexReader.java @@ -21,6 +21,7 @@ import org.apache.giraph.graph.BasicVertex; import org.apache.giraph.graph.BspUtils; import org.apache.giraph.graph.Edge; +import org.apache.giraph.lib.TextVertexInputFormat.TextVertexReader; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; @@ -42,15 +43,17 @@ * @param Vertex index value * @param Vertex value * @param Edge value + * @param Message data */ @SuppressWarnings("rawtypes") public abstract class AdjacencyListVertexReader extends TextVertexInputFormat.TextVertexReader { - + /** Delimiter for split */ public static final String LINE_TOKENIZE_VALUE = "adj.list.input.delimiter"; + /** Default delimiter for split */ public static final String LINE_TOKENIZE_VALUE_DEFAULT = "\t"; - + /** Cached delimiter used for split */ private String splitValue = null; /** @@ -59,17 +62,37 @@ public abstract class AdjacencyListVertexReader lineRecordReader) { + /** + * Constructor with line record reader. + * + * @param lineRecordReader Reader from {@link TextVertexReader}. + */ + public AdjacencyListVertexReader( + RecordReader lineRecordReader) { super(lineRecordReader); + sanitizer = null; } - public AdjacencyListVertexReader(RecordReader lineRecordReader, + /** + * Constructor with line record reader. + * + * @param lineRecordReader Reader from {@link TextVertexReader}. + * @param sanitizer Sanitizer to be used. + */ + public AdjacencyListVertexReader( + RecordReader lineRecordReader, LineSanitizer sanitizer) { super(lineRecordReader); this.sanitizer = sanitizer; @@ -77,17 +100,18 @@ public AdjacencyListVertexReader(RecordReader lineRecordRead /** * Store the Id for this line in an instance of its correct type. + * * @param s Id of vertex from line * @param id Instance of Id's type, in which to store its value */ - abstract public void decodeId(String s, I id); + public abstract void decodeId(String s, I id); /** * Store the value for this line in an instance of its correct type. * @param s Value from line * @param value Instance of value's type, in which to store its value */ - abstract public void decodeValue(String s, V value); + public abstract void decodeValue(String s, V value); /** * Store an edge from the line into an instance of a correctly typed Edge @@ -95,7 +119,7 @@ public AdjacencyListVertexReader(RecordReader lineRecordRead * @param value The edge's value from the line * @param edge Instance of edge in which to store the id and value */ - abstract public void decodeEdge(String id, String value, Edge edge); + public abstract void decodeEdge(String id, String value, Edge edge); @Override @@ -104,7 +128,8 @@ public boolean nextVertex() throws IOException, InterruptedException { } @Override - public BasicVertex getCurrentVertex() throws IOException, InterruptedException { + public BasicVertex getCurrentVertex() + throws IOException, InterruptedException { Configuration conf = getContext().getConfiguration(); String line = getRecordReader().getCurrentValue().toString(); BasicVertex vertex = BspUtils.createVertex(conf); @@ -120,7 +145,8 @@ public BasicVertex getCurrentVertex() throws IOException, Interrupte String [] values = line.split(splitValue); if ((values.length < 2) || (values.length % 2 != 0)) { - throw new IllegalArgumentException("Line did not split correctly: " + line); + throw new IllegalArgumentException( + "Line did not split correctly: " + line); } I vertexId = BspUtils.createVertexIndex(conf); @@ -132,7 +158,7 @@ public BasicVertex getCurrentVertex() throws IOException, Interrupte int i = 2; Map edges = Maps.newHashMap(); Edge edge = new Edge(); - while(i < values.length) { + while (i < values.length) { decodeEdge(values[i], values[i + 1], edge); edges.put(edge.getDestVertexId(), edge.getEdgeValue()); i += 2; diff --git a/src/main/java/org/apache/giraph/lib/IdWithValueTextOutputFormat.java b/src/main/java/org/apache/giraph/lib/IdWithValueTextOutputFormat.java index 9d2e08836..b39dd8e1f 100644 --- a/src/main/java/org/apache/giraph/lib/IdWithValueTextOutputFormat.java +++ b/src/main/java/org/apache/giraph/lib/IdWithValueTextOutputFormat.java @@ -40,30 +40,45 @@ * @param Edge value */ @SuppressWarnings("rawtypes") -public class IdWithValueTextOutputFormat extends TextVertexOutputFormat{ - +public class IdWithValueTextOutputFormat + extends TextVertexOutputFormat { + + /** + * Vertex writer used with {@link IdWithValueTextOutputFormat}. + * + * @param Vertex id + * @param Vertex data + * @param Edge data + */ static class IdWithValueVertexWriter extends TextVertexWriter { - + /** Specify the output delimiter */ public static final String LINE_TOKENIZE_VALUE = "output.delimiter"; + /** Default output delimiter */ public static final String LINE_TOKENIZE_VALUE_DEFAULT = "\t"; - + /** Reverse id and value order? */ public static final String REVERSE_ID_AND_VALUE = "reverse.id.and.value"; + /** Default is to not reverse id and value order. */ public static final boolean REVERSE_ID_AND_VALUE_DEFAULT = false; - + /** Saved delimiter */ private String delimiter; + /** + * Constructor with record writer. + * + * @param recordWriter Writer from LineRecordWriter. + */ public IdWithValueVertexWriter(RecordWriter recordWriter) { super(recordWriter); } @Override public void writeVertex(BasicVertex vertex) throws IOException, - InterruptedException { + InterruptedException { if (delimiter == null) { delimiter = getContext().getConfiguration() - .get(LINE_TOKENIZE_VALUE, LINE_TOKENIZE_VALUE_DEFAULT); + .get(LINE_TOKENIZE_VALUE, LINE_TOKENIZE_VALUE_DEFAULT); } String first; @@ -87,9 +102,8 @@ public void writeVertex(BasicVertex vertex) throws IOException, @Override public VertexWriter createVertexWriter(TaskAttemptContext context) - throws IOException, InterruptedException { + throws IOException, InterruptedException { return new IdWithValueVertexWriter - (textOutputFormat.getRecordWriter(context)); + (textOutputFormat.getRecordWriter(context)); } - } diff --git a/src/main/java/org/apache/giraph/lib/JsonBase64VertexFormat.java b/src/main/java/org/apache/giraph/lib/JsonBase64VertexFormat.java index c4164d37b..16329ef38 100644 --- a/src/main/java/org/apache/giraph/lib/JsonBase64VertexFormat.java +++ b/src/main/java/org/apache/giraph/lib/JsonBase64VertexFormat.java @@ -21,11 +21,16 @@ /** * Keeps the vertex keys for the input/output vertex format */ -public interface JsonBase64VertexFormat { - /** Vertex id key */ - public static final String VERTEX_ID_KEY = "vertexId"; - /** Vertex value key*/ - public static final String VERTEX_VALUE_KEY = "vertexValue"; - /** Edge value array key (all the edges are stored here) */ - public static final String EDGE_ARRAY_KEY = "edgeArray"; +public class JsonBase64VertexFormat { + /** Vertex id key */ + public static final String VERTEX_ID_KEY = "vertexId"; + /** Vertex value key*/ + public static final String VERTEX_VALUE_KEY = "vertexValue"; + /** Edge value array key (all the edges are stored here) */ + public static final String EDGE_ARRAY_KEY = "edgeArray"; + + /** + * Don't construct. + */ + private JsonBase64VertexFormat() { } } diff --git a/src/main/java/org/apache/giraph/lib/JsonBase64VertexInputFormat.java b/src/main/java/org/apache/giraph/lib/JsonBase64VertexInputFormat.java index 5111a87fb..637da7b17 100644 --- a/src/main/java/org/apache/giraph/lib/JsonBase64VertexInputFormat.java +++ b/src/main/java/org/apache/giraph/lib/JsonBase64VertexInputFormat.java @@ -51,110 +51,110 @@ * @param Vertex index value * @param Vertex value * @param Edge value + * @param Message data */ @SuppressWarnings("rawtypes") -public class JsonBase64VertexInputFormat< - I extends WritableComparable, V extends Writable, E extends Writable, - M extends Writable> - extends TextVertexInputFormat implements - JsonBase64VertexFormat { +public class JsonBase64VertexInputFormat + extends TextVertexInputFormat { + /** + * Simple reader that supports {@link JsonBase64VertexInputFormat} + * + * @param Vertex index value + * @param Vertex value + * @param Edge value + * @param Message data + */ + private static class JsonBase64VertexReader + extends TextVertexReader { /** - * Simple reader that supports {@link JsonBase64VertexInputFormat} + * Only constructor. Requires the LineRecordReader * - * @param Vertex index value - * @param Vertex value - * @param Edge value + * @param lineRecordReader Line record reader to read from */ - private static class JsonBase64VertexReader< - I extends WritableComparable, V extends Writable, - E extends Writable, M extends Writable> extends TextVertexReader { - /** - * Only constructor. Requires the LineRecordReader - * - * @param lineRecordReader Line record reader to read from - */ - public JsonBase64VertexReader(RecordReader lineRecordReader) { - super(lineRecordReader); - } + public JsonBase64VertexReader( + RecordReader lineRecordReader) { + super(lineRecordReader); + } - @Override - public boolean nextVertex() throws IOException, InterruptedException { - return getRecordReader().nextKeyValue(); - } + @Override + public boolean nextVertex() throws IOException, InterruptedException { + return getRecordReader().nextKeyValue(); + } - @Override - public BasicVertex getCurrentVertex() - throws IOException, InterruptedException { - Configuration conf = getContext().getConfiguration(); - BasicVertex vertex = BspUtils.createVertex(conf); + @Override + public BasicVertex getCurrentVertex() + throws IOException, InterruptedException { + Configuration conf = getContext().getConfiguration(); + BasicVertex vertex = BspUtils.createVertex(conf); - Text line = getRecordReader().getCurrentValue(); - JSONObject vertexObject; - try { - vertexObject = new JSONObject(line.toString()); - } catch (JSONException e) { - throw new IllegalArgumentException( - "next: Failed to get the vertex", e); - } - DataInput input = null; - byte[] decodedWritable = null; - I vertexId = null; - try { - decodedWritable = Base64.decode( - vertexObject.getString(VERTEX_ID_KEY)); - input = new DataInputStream( - new ByteArrayInputStream(decodedWritable)); - vertexId = BspUtils.createVertexIndex(conf); - vertexId.readFields(input); - } catch (JSONException e) { - throw new IllegalArgumentException( - "next: Failed to get vertex id", e); - } - V vertexValue = null; - try { - decodedWritable = Base64.decode( - vertexObject.getString(VERTEX_VALUE_KEY)); - input = new DataInputStream( - new ByteArrayInputStream(decodedWritable)); - vertexValue = BspUtils.createVertexValue(conf); - vertexValue.readFields(input); - } catch (JSONException e) { - throw new IllegalArgumentException( - "next: Failed to get vertex value", e); - } - JSONArray edgeArray = null; - try { - edgeArray = vertexObject.getJSONArray(EDGE_ARRAY_KEY); - } catch (JSONException e) { - throw new IllegalArgumentException( - "next: Failed to get edge array", e); - } - Map edgeMap = Maps.newHashMap(); - for (int i = 0; i < edgeArray.length(); ++i) { - try { - decodedWritable = - Base64.decode(edgeArray.getString(i)); - } catch (JSONException e) { - throw new IllegalArgumentException( - "next: Failed to get edge value", e); - } - input = new DataInputStream( - new ByteArrayInputStream(decodedWritable)); - Edge edge = new Edge(); - edge.setConf(getContext().getConfiguration()); - edge.readFields(input); - edgeMap.put(edge.getDestVertexId(), edge.getEdgeValue()); - } - vertex.initialize(vertexId, vertexValue, edgeMap, null); - return vertex; + Text line = getRecordReader().getCurrentValue(); + JSONObject vertexObject; + try { + vertexObject = new JSONObject(line.toString()); + } catch (JSONException e) { + throw new IllegalArgumentException( + "next: Failed to get the vertex", e); + } + DataInput input = null; + byte[] decodedWritable = null; + I vertexId = null; + try { + decodedWritable = Base64.decode( + vertexObject.getString(JsonBase64VertexFormat.VERTEX_ID_KEY)); + input = new DataInputStream( + new ByteArrayInputStream(decodedWritable)); + vertexId = BspUtils.createVertexIndex(conf); + vertexId.readFields(input); + } catch (JSONException e) { + throw new IllegalArgumentException( + "next: Failed to get vertex id", e); + } + V vertexValue = null; + try { + decodedWritable = Base64.decode( + vertexObject.getString(JsonBase64VertexFormat.VERTEX_VALUE_KEY)); + input = new DataInputStream( + new ByteArrayInputStream(decodedWritable)); + vertexValue = BspUtils.createVertexValue(conf); + vertexValue.readFields(input); + } catch (JSONException e) { + throw new IllegalArgumentException( + "next: Failed to get vertex value", e); + } + JSONArray edgeArray = null; + try { + edgeArray = vertexObject.getJSONArray( + JsonBase64VertexFormat.EDGE_ARRAY_KEY); + } catch (JSONException e) { + throw new IllegalArgumentException( + "next: Failed to get edge array", e); + } + Map edgeMap = Maps.newHashMap(); + for (int i = 0; i < edgeArray.length(); ++i) { + try { + decodedWritable = Base64.decode(edgeArray.getString(i)); + } catch (JSONException e) { + throw new IllegalArgumentException( + "next: Failed to get edge value", e); } + input = new DataInputStream( + new ByteArrayInputStream(decodedWritable)); + Edge edge = new Edge(); + edge.setConf(getContext().getConfiguration()); + edge.readFields(input); + edgeMap.put(edge.getDestVertexId(), edge.getEdgeValue()); + } + vertex.initialize(vertexId, vertexValue, edgeMap, null); + return vertex; } + } - @Override - public VertexReader createVertexReader( - InputSplit split, - TaskAttemptContext context) throws IOException { - return new JsonBase64VertexReader(textInputFormat.createRecordReader(split, - context)); - } + @Override + public VertexReader createVertexReader( + InputSplit split, TaskAttemptContext context) throws IOException { + return new JsonBase64VertexReader( + textInputFormat.createRecordReader(split, context)); + } } diff --git a/src/main/java/org/apache/giraph/lib/JsonBase64VertexOutputFormat.java b/src/main/java/org/apache/giraph/lib/JsonBase64VertexOutputFormat.java index fbd5b036a..293289be8 100644 --- a/src/main/java/org/apache/giraph/lib/JsonBase64VertexOutputFormat.java +++ b/src/main/java/org/apache/giraph/lib/JsonBase64VertexOutputFormat.java @@ -47,80 +47,80 @@ * @param Edge value */ @SuppressWarnings("rawtypes") -public class JsonBase64VertexOutputFormat< - I extends WritableComparable, V extends Writable, E extends Writable> - extends TextVertexOutputFormat - implements JsonBase64VertexFormat { +public class JsonBase64VertexOutputFormat extends + TextVertexOutputFormat { + /** + * Simple writer that supports {@link JsonBase64VertexOutputFormat} + * + * @param Vertex index value + * @param Vertex value + * @param Edge value + */ + private static class JsonBase64VertexWriter extends + TextVertexWriter { /** - * Simple writer that supports {@link JsonBase64VertexOutputFormat} + * Only constructor. Requires the LineRecordWriter * - * @param Vertex index value - * @param Vertex value - * @param Edge value + * @param lineRecordWriter Line record writer to write to */ - private static class JsonBase64VertexWriter< - I extends WritableComparable, V extends Writable, - E extends Writable> extends TextVertexWriter { - /** - * Only constructor. Requires the LineRecordWriter - * - * @param lineRecordWriter Line record writer to write to - */ - public JsonBase64VertexWriter( - RecordWriter lineRecordWriter) { - super(lineRecordWriter); - } - - @Override - public void writeVertex(BasicVertex vertex) - throws IOException, InterruptedException { - ByteArrayOutputStream outputStream = - new ByteArrayOutputStream(); - DataOutput output = new DataOutputStream(outputStream); - JSONObject vertexObject = new JSONObject(); - vertex.getVertexId().write(output); - try { - vertexObject.put( - VERTEX_ID_KEY, - Base64.encodeBytes(outputStream.toByteArray())); - } catch (JSONException e) { - throw new IllegalStateException( - "writerVertex: Failed to insert vertex id", e); - } - outputStream.reset(); - vertex.getVertexValue().write(output); - try { - vertexObject.put( - VERTEX_VALUE_KEY, - Base64.encodeBytes(outputStream.toByteArray())); - } catch (JSONException e) { - throw new IllegalStateException( - "writerVertex: Failed to insert vertex value", e); - } - JSONArray edgeArray = new JSONArray(); - for (I targetVertexId : vertex) { - Edge edge = new Edge( - targetVertexId, vertex.getEdgeValue(targetVertexId)); - edge.setConf(getContext().getConfiguration()); - outputStream.reset(); - edge.write(output); - edgeArray.put(Base64.encodeBytes(outputStream.toByteArray())); - } - try { - vertexObject.put(EDGE_ARRAY_KEY, edgeArray); - } catch (JSONException e) { - throw new IllegalStateException( - "writerVertex: Failed to insert edge array", e); - } - getRecordWriter().write(new Text(vertexObject.toString()), null); - } + public JsonBase64VertexWriter( + RecordWriter lineRecordWriter) { + super(lineRecordWriter); } @Override - public VertexWriter createVertexWriter(TaskAttemptContext context) - throws IOException, InterruptedException { - return new JsonBase64VertexWriter( - textOutputFormat.getRecordWriter(context)); + public void writeVertex(BasicVertex vertex) + throws IOException, InterruptedException { + ByteArrayOutputStream outputStream = + new ByteArrayOutputStream(); + DataOutput output = new DataOutputStream(outputStream); + JSONObject vertexObject = new JSONObject(); + vertex.getVertexId().write(output); + try { + vertexObject.put( + JsonBase64VertexFormat.VERTEX_ID_KEY, + Base64.encodeBytes(outputStream.toByteArray())); + } catch (JSONException e) { + throw new IllegalStateException( + "writerVertex: Failed to insert vertex id", e); + } + outputStream.reset(); + vertex.getVertexValue().write(output); + try { + vertexObject.put( + JsonBase64VertexFormat.VERTEX_VALUE_KEY, + Base64.encodeBytes(outputStream.toByteArray())); + } catch (JSONException e) { + throw new IllegalStateException( + "writerVertex: Failed to insert vertex value", e); + } + JSONArray edgeArray = new JSONArray(); + for (I targetVertexId : vertex) { + Edge edge = new Edge( + targetVertexId, vertex.getEdgeValue(targetVertexId)); + edge.setConf(getContext().getConfiguration()); + outputStream.reset(); + edge.write(output); + edgeArray.put(Base64.encodeBytes(outputStream.toByteArray())); + } + try { + vertexObject.put( + JsonBase64VertexFormat.EDGE_ARRAY_KEY, + edgeArray); + } catch (JSONException e) { + throw new IllegalStateException( + "writerVertex: Failed to insert edge array", e); + } + getRecordWriter().write(new Text(vertexObject.toString()), null); } + } + @Override + public VertexWriter createVertexWriter(TaskAttemptContext context) + throws IOException, InterruptedException { + return new JsonBase64VertexWriter( + textOutputFormat.getRecordWriter(context)); + } } diff --git a/src/main/java/org/apache/giraph/lib/LongDoubleDoubleAdjacencyListVertexInputFormat.java b/src/main/java/org/apache/giraph/lib/LongDoubleDoubleAdjacencyListVertexInputFormat.java index c9eb527b4..e7b313124 100644 --- a/src/main/java/org/apache/giraph/lib/LongDoubleDoubleAdjacencyListVertexInputFormat.java +++ b/src/main/java/org/apache/giraph/lib/LongDoubleDoubleAdjacencyListVertexInputFormat.java @@ -35,19 +35,40 @@ * 22 0.1 45 0.3 99 0.44 * to repesent a vertex with id 22, value of 0.1 and edges to nodes 45 and 99, * with values of 0.3 and 0.44, respectively. + * + * @param Message data */ -public class LongDoubleDoubleAdjacencyListVertexInputFormat extends - TextVertexInputFormat { +public class LongDoubleDoubleAdjacencyListVertexInputFormat + extends TextVertexInputFormat { + /** + * VertexReader associated with + * {@link LongDoubleDoubleAdjacencyListVertexInputFormat}. + * + * @param Message data. + */ static class VertexReader extends - AdjacencyListVertexReader { + AdjacencyListVertexReader { + /** + * Constructor with Line record reader. + * + * @param lineRecordReader Reader to internally use. + */ VertexReader(RecordReader lineRecordReader) { super(lineRecordReader); } + /** + * Constructor with Line record reader and sanitizer. + * + * @param lineRecordReader Reader to internally use. + * @param sanitizer Line sanitizer. + */ VertexReader(RecordReader lineRecordReader, - LineSanitizer sanitizer) { + LineSanitizer sanitizer) { super(lineRecordReader, sanitizer); } @@ -62,8 +83,10 @@ public void decodeValue(String s, DoubleWritable value) { } @Override - public void decodeEdge(String s1, String s2, Edge - textIntWritableEdge) { + public void decodeEdge( + String s1, + String s2, + Edge textIntWritableEdge) { textIntWritableEdge.setDestVertexId(new LongWritable(Long.valueOf(s1))); textIntWritableEdge.setEdgeValue(new DoubleWritable(Double.valueOf(s2))); } @@ -71,10 +94,10 @@ public void decodeEdge(String s1, String s2, Edge @Override public org.apache.giraph.graph.VertexReader createVertexReader( + DoubleWritable, DoubleWritable, M> createVertexReader( InputSplit split, TaskAttemptContext context) throws IOException { return new VertexReader(textInputFormat.createRecordReader( - split, context)); + split, context)); } } diff --git a/src/main/java/org/apache/giraph/lib/SequenceFileVertexInputFormat.java b/src/main/java/org/apache/giraph/lib/SequenceFileVertexInputFormat.java index 41a19c24a..cabb3d3c5 100644 --- a/src/main/java/org/apache/giraph/lib/SequenceFileVertexInputFormat.java +++ b/src/main/java/org/apache/giraph/lib/SequenceFileVertexInputFormat.java @@ -31,49 +31,73 @@ import java.io.IOException; import java.util.List; +/** + * Sequence file vertex input format based on {@link SequenceFileInputFormat}. + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data + * @param Value type + */ public class SequenceFileVertexInputFormat, - V extends Writable, - E extends Writable, - M extends Writable, - X extends BasicVertex> + V extends Writable, E extends Writable, M extends Writable, + X extends BasicVertex> extends VertexInputFormat { - protected SequenceFileInputFormat sequenceFileInputFormat - = new SequenceFileInputFormat(); + /** Internal input format */ + protected SequenceFileInputFormat sequenceFileInputFormat = + new SequenceFileInputFormat(); - @Override public List getSplits(JobContext context, int numWorkers) - throws IOException, InterruptedException { + @Override + public List getSplits(JobContext context, int numWorkers) + throws IOException, InterruptedException { return sequenceFileInputFormat.getSplits(context); } @Override public VertexReader createVertexReader(InputSplit split, - TaskAttemptContext context) - throws IOException { + TaskAttemptContext context) throws IOException { return new SequenceFileVertexReader( sequenceFileInputFormat.createRecordReader(split, context)); } + /** + * Vertex reader used with {@link SequenceFileVertexInputFormat}. + * + * @param Vertex id + * @param Vertex data + * @param Edge data + * @param Message data + * @param Value type + */ public static class SequenceFileVertexReader, V extends Writable, E extends Writable, M extends Writable, X extends BasicVertex> implements VertexReader { + /** Internal record reader from {@link SequenceFileInputFormat} */ private final RecordReader recordReader; + /** + * Constructor with record reader. + * + * @param recordReader Reader from {@link SequenceFileInputFormat}. + */ public SequenceFileVertexReader(RecordReader recordReader) { this.recordReader = recordReader; } - @Override public void initialize(InputSplit inputSplit, TaskAttemptContext context) - throws IOException, InterruptedException { + @Override public void initialize(InputSplit inputSplit, + TaskAttemptContext context) throws IOException, InterruptedException { recordReader.initialize(inputSplit, context); } - @Override public boolean nextVertex() throws IOException, InterruptedException { + @Override public boolean nextVertex() throws IOException, + InterruptedException { return recordReader.nextKeyValue(); } @Override public BasicVertex getCurrentVertex() - throws IOException, InterruptedException { + throws IOException, InterruptedException { return recordReader.getCurrentValue(); } @@ -82,7 +106,8 @@ public SequenceFileVertexReader(RecordReader recordReader) { recordReader.close(); } - @Override public float getProgress() throws IOException, InterruptedException { + @Override public float getProgress() throws IOException, + InterruptedException { return recordReader.getProgress(); } } diff --git a/src/main/java/org/apache/giraph/lib/TextDoubleDoubleAdjacencyListVertexInputFormat.java b/src/main/java/org/apache/giraph/lib/TextDoubleDoubleAdjacencyListVertexInputFormat.java index 9b2d69ca8..2cb97a924 100644 --- a/src/main/java/org/apache/giraph/lib/TextDoubleDoubleAdjacencyListVertexInputFormat.java +++ b/src/main/java/org/apache/giraph/lib/TextDoubleDoubleAdjacencyListVertexInputFormat.java @@ -32,19 +32,38 @@ * Class to read graphs stored as adjacency lists with ids represented by * Strings and values as doubles. This is a good inputformat for reading * graphs where the id types do not matter and can be stashed in a String. + * + * @param Message type. */ public class TextDoubleDoubleAdjacencyListVertexInputFormat extends TextVertexInputFormat { - static class VertexReader extends AdjacencyListVertexReader { + /** + * Vertex reader used with + * {@link TextDoubleDoubleAdjacencyListVertexInputFormat} + * + * @param Message type. + */ + static class VertexReader extends + AdjacencyListVertexReader { + /** + * Constructor without sanitzer. + * + * @param lineRecordReader Internal reader. + */ VertexReader(RecordReader lineRecordReader) { super(lineRecordReader); } + /** + * Constructor with {@link LineRecordReader} + * + * @param lineRecordReader Internal reader. + * @param sanitizer Sanitizer of the lines. + */ VertexReader(RecordReader lineRecordReader, - LineSanitizer sanitizer) { + LineSanitizer sanitizer) { super(lineRecordReader, sanitizer); } @@ -59,8 +78,8 @@ public void decodeValue(String s, DoubleWritable value) { } @Override - public void decodeEdge(String s1, String s2, Edge - textIntWritableEdge) { + public void decodeEdge(String s1, String s2, + Edge textIntWritableEdge) { textIntWritableEdge.setDestVertexId(new Text(s1)); textIntWritableEdge.setEdgeValue(new DoubleWritable(Double.valueOf(s2))); } @@ -68,11 +87,9 @@ public void decodeEdge(String s1, String s2, Edge @Override public org.apache.giraph.graph.VertexReader createVertexReader( - InputSplit split, + DoubleWritable, M> createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException { return new VertexReader(textInputFormat.createRecordReader( - split, context)); + split, context)); } - } diff --git a/src/main/java/org/apache/giraph/lib/TextVertexInputFormat.java b/src/main/java/org/apache/giraph/lib/TextVertexInputFormat.java index 62fb43547..0e426dc33 100644 --- a/src/main/java/org/apache/giraph/lib/TextVertexInputFormat.java +++ b/src/main/java/org/apache/giraph/lib/TextVertexInputFormat.java @@ -43,85 +43,80 @@ * @param Message value */ @SuppressWarnings("rawtypes") -public abstract class TextVertexInputFormat< - I extends WritableComparable, - V extends Writable, - E extends Writable, - M extends Writable> - extends VertexInputFormat { - /** Uses the TextInputFormat to do everything */ - protected TextInputFormat textInputFormat = new TextInputFormat(); +public abstract class TextVertexInputFormat + extends VertexInputFormat { + /** Uses the TextInputFormat to do everything */ + protected TextInputFormat textInputFormat = new TextInputFormat(); + + /** + * Abstract class to be implemented by the user based on their specific + * vertex input. Easiest to ignore the key value separator and only use + * key instead. + * + * @param Vertex index value + * @param Vertex value + * @param Edge value + */ + public abstract static class TextVertexReader + implements VertexReader { + /** Internal line record reader */ + private final RecordReader lineRecordReader; + /** Context passed to initialize */ + private TaskAttemptContext context; /** - * Abstract class to be implemented by the user based on their specific - * vertex input. Easiest to ignore the key value separator and only use - * key instead. + * Initialize with the LineRecordReader. * - * @param Vertex index value - * @param Vertex value - * @param Edge value + * @param lineRecordReader Line record reader from TextInputFormat */ - public static abstract class TextVertexReader - implements VertexReader { - /** Internal line record reader */ - private final RecordReader lineRecordReader; - /** Context passed to initialize */ - private TaskAttemptContext context; - - /** - * Initialize with the LineRecordReader. - * - * @param lineRecordReader Line record reader from TextInputFormat - */ - public TextVertexReader( - RecordReader lineRecordReader) { - this.lineRecordReader = lineRecordReader; - } - - @Override - public void initialize(InputSplit inputSplit, - TaskAttemptContext context) - throws IOException, InterruptedException { - lineRecordReader.initialize(inputSplit, context); - this.context = context; - } + public TextVertexReader( + RecordReader lineRecordReader) { + this.lineRecordReader = lineRecordReader; + } - @Override - public void close() throws IOException { - lineRecordReader.close(); - } + @Override + public void initialize(InputSplit inputSplit, TaskAttemptContext context) + throws IOException, InterruptedException { + lineRecordReader.initialize(inputSplit, context); + this.context = context; + } - @Override - public float getProgress() throws IOException, InterruptedException { - return lineRecordReader.getProgress(); - } + @Override + public void close() throws IOException { + lineRecordReader.close(); + } - /** - * Get the line record reader. - * - * @return Record reader to be used for reading. - */ - protected RecordReader getRecordReader() { - return lineRecordReader; - } + @Override + public float getProgress() throws IOException, InterruptedException { + return lineRecordReader.getProgress(); + } - /** - * Get the context. - * - * @return Context passed to initialize. - */ - protected TaskAttemptContext getContext() { - return context; - } + /** + * Get the line record reader. + * + * @return Record reader to be used for reading. + */ + protected RecordReader getRecordReader() { + return lineRecordReader; } - @Override - public List getSplits( - JobContext context, int numWorkers) - throws IOException, InterruptedException { - // Ignore the hint of numWorkers here since we are using TextInputFormat - // to do this for us - return textInputFormat.getSplits(context); + /** + * Get the context. + * + * @return Context passed to initialize. + */ + protected TaskAttemptContext getContext() { + return context; } + } + + @Override + public List getSplits(JobContext context, int numWorkers) + throws IOException, InterruptedException { + // Ignore the hint of numWorkers here since we are using TextInputFormat + // to do this for us + return textInputFormat.getSplits(context); + } } diff --git a/src/main/java/org/apache/giraph/lib/TextVertexOutputFormat.java b/src/main/java/org/apache/giraph/lib/TextVertexOutputFormat.java index a67ca3b4d..ef07b38ec 100644 --- a/src/main/java/org/apache/giraph/lib/TextVertexOutputFormat.java +++ b/src/main/java/org/apache/giraph/lib/TextVertexOutputFormat.java @@ -40,78 +40,77 @@ * @param Edge value */ @SuppressWarnings("rawtypes") -public abstract class TextVertexOutputFormat< - I extends WritableComparable, V extends Writable, E extends Writable> - extends VertexOutputFormat { - /** Uses the TextOutputFormat to do everything */ - protected TextOutputFormat textOutputFormat = - new TextOutputFormat(); +public abstract class TextVertexOutputFormat + extends VertexOutputFormat { + /** Uses the TextOutputFormat to do everything */ + protected TextOutputFormat textOutputFormat = + new TextOutputFormat(); + + /** + * Abstract class to be implemented by the user based on their specific + * vertex output. Easiest to ignore the key value separator and only use + * key instead. + * + * @param Vertex index value + * @param Vertex value + * @param Edge value + */ + public abstract static class TextVertexWriter implements VertexWriter { + /** Context passed to initialize */ + private TaskAttemptContext context; + /** Internal line record writer */ + private final RecordWriter lineRecordWriter; /** - * Abstract class to be implemented by the user based on their specific - * vertex output. Easiest to ignore the key value separator and only use - * key instead. + * Initialize with the LineRecordWriter. * - * @param Vertex index value - * @param Vertex value - * @param Edge value + * @param lineRecordWriter Line record writer from TextOutputFormat */ - public static abstract class TextVertexWriter - implements VertexWriter { - /** Context passed to initialize */ - private TaskAttemptContext context; - /** Internal line record writer */ - private final RecordWriter lineRecordWriter; - - /** - * Initialize with the LineRecordWriter. - * - * @param lineRecordWriter Line record writer from TextOutputFormat - */ - public TextVertexWriter(RecordWriter lineRecordWriter) { - this.lineRecordWriter = lineRecordWriter; - } - - @Override - public void initialize(TaskAttemptContext context) throws IOException { - this.context = context; - } - - @Override - public void close(TaskAttemptContext context) - throws IOException, InterruptedException { - lineRecordWriter.close(context); - } - - /** - * Get the line record writer. - * - * @return Record writer to be used for writing. - */ - public RecordWriter getRecordWriter() { - return lineRecordWriter; - } - - /** - * Get the context. - * - * @return Context passed to initialize. - */ - public TaskAttemptContext getContext() { - return context; - } + public TextVertexWriter(RecordWriter lineRecordWriter) { + this.lineRecordWriter = lineRecordWriter; } @Override - public void checkOutputSpecs(JobContext context) - throws IOException, InterruptedException { - textOutputFormat.checkOutputSpecs(context); + public void initialize(TaskAttemptContext context) throws IOException { + this.context = context; } @Override - public OutputCommitter getOutputCommitter(TaskAttemptContext context) - throws IOException, InterruptedException { - return textOutputFormat.getOutputCommitter(context); + public void close(TaskAttemptContext context) throws IOException, + InterruptedException { + lineRecordWriter.close(context); + } + + /** + * Get the line record writer. + * + * @return Record writer to be used for writing. + */ + public RecordWriter getRecordWriter() { + return lineRecordWriter; } + + /** + * Get the context. + * + * @return Context passed to initialize. + */ + public TaskAttemptContext getContext() { + return context; + } + } + + @Override + public void checkOutputSpecs(JobContext context) + throws IOException, InterruptedException { + textOutputFormat.checkOutputSpecs(context); + } + + @Override + public OutputCommitter getOutputCommitter(TaskAttemptContext context) + throws IOException, InterruptedException { + return textOutputFormat.getOutputCommitter(context); + } } diff --git a/src/main/java/org/apache/giraph/lib/package-info.java b/src/main/java/org/apache/giraph/lib/package-info.java new file mode 100644 index 000000000..f63bb5f00 --- /dev/null +++ b/src/main/java/org/apache/giraph/lib/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/** + * Package of reusable library Giraph objects. + */ +package org.apache.giraph.lib; diff --git a/src/main/java/org/apache/giraph/package-info.java b/src/main/java/org/apache/giraph/package-info.java new file mode 100644 index 000000000..3c5f960aa --- /dev/null +++ b/src/main/java/org/apache/giraph/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/** + * Base giraph package. + */ +package org.apache.giraph; diff --git a/src/main/java/org/apache/giraph/utils/ComparisonUtils.java b/src/main/java/org/apache/giraph/utils/ComparisonUtils.java index c49171f69..bb940eab1 100644 --- a/src/main/java/org/apache/giraph/utils/ComparisonUtils.java +++ b/src/main/java/org/apache/giraph/utils/ComparisonUtils.java @@ -20,29 +20,43 @@ import java.util.Iterator; -/** simple helper class for comparisons and equality checking */ +/** Simple helper class for comparisons and equality checking */ public class ComparisonUtils { - private ComparisonUtils() { - } + /** Do not construct this object */ + private ComparisonUtils() { } - /** compare elements, sort order and length */ - public static boolean equal(Iterable first, Iterable second) { - return equal(first.iterator(), second.iterator()); - } + /** + * Compare elements, sort order and length + * + * @param Type of iterable to compare. + * @param first First iterable to compare. + * @param second Second iterable to compare. + * @return True if equal, false otherwise. + */ + public static boolean equal(Iterable first, Iterable second) { + return equal(first.iterator(), second.iterator()); + } - /** compare elements, sort order and length */ - public static boolean equal(Iterator first, Iterator second) { - while (first.hasNext() && second.hasNext()) { - T message = first.next(); - T otherMessage = second.next(); - /* element-wise equality */ - if (!(message == null ? otherMessage == null : - message.equals(otherMessage))) { - return false; - } - } - /* length must also be equal */ - return !(first.hasNext() || second.hasNext()); + /** + * Compare elements, sort order and length + * + * @param Type of iterable to compare. + * @param first First iterable to compare. + * @param second Second iterable to compare. + * @return True if equal, false otherwise. + */ + public static boolean equal(Iterator first, Iterator second) { + while (first.hasNext() && second.hasNext()) { + T message = first.next(); + T otherMessage = second.next(); + /* element-wise equality */ + if (!(message == null ? otherMessage == null : + message.equals(otherMessage))) { + return false; + } } + /* length must also be equal */ + return !(first.hasNext() || second.hasNext()); + } } diff --git a/src/main/java/org/apache/giraph/utils/EmptyIterable.java b/src/main/java/org/apache/giraph/utils/EmptyIterable.java index 795cace89..5ec7cc140 100644 --- a/src/main/java/org/apache/giraph/utils/EmptyIterable.java +++ b/src/main/java/org/apache/giraph/utils/EmptyIterable.java @@ -20,26 +20,30 @@ import java.util.Iterator; +/** + * Helper empty iterable when there are no messages. + * + * @param Message data + */ public class EmptyIterable implements Iterable, Iterator { - - @Override - public Iterator iterator() { - return this; - } - - @Override - public boolean hasNext() { - return false; - } - - @Override - public M next() { - return null; - } - - @Override - public void remove() { - throw new UnsupportedOperationException(); - } + @Override + public Iterator iterator() { + return this; + } + + @Override + public boolean hasNext() { + return false; + } + + @Override + public M next() { + return null; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } } diff --git a/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java b/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java index 5db84215e..d987d0de1 100644 --- a/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java +++ b/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java @@ -42,208 +42,253 @@ /** * A base class for running internal tests on a vertex * - * Extending classes only have to invoke the run() method to test their vertex. All data - * is written to a local tmp directory that is removed afterwards. A local zookeeper - * instance is started in an extra thread and shutdown at the end. + * Extending classes only have to invoke the run() method to test their vertex. + * All data is written to a local tmp directory that is removed afterwards. + * A local zookeeper instance is started in an extra thread and + * shutdown at the end. * * Heavily inspired from Apache Mahout's MahoutTestCase */ public class InternalVertexRunner { + /** ZooKeeper port to use for tests */ + public static final int LOCAL_ZOOKEEPER_PORT = 22182; - public static final int LOCAL_ZOOKEEPER_PORT = 22182; + /** + * Default constructor. + */ + private InternalVertexRunner() { } - private InternalVertexRunner() { - } + /** + * Attempts to run the vertex internally in the current JVM, reading from + * and writing to a temporary folder on local disk. Will start + * its own ZooKeeper instance. + * + * @param vertexClass the vertex class to instantiate + * @param vertexInputFormatClass the inputformat to use + * @param vertexOutputFormatClass the outputformat to use + * @param params a map of parameters to add to the hadoop configuration + * @param data linewise input data + * @return linewise output data + * @throws Exception + */ + public static Iterable run(Class vertexClass, + Class vertexInputFormatClass, Class vertexOutputFormatClass, + Map params, String... data) throws Exception { + return run(vertexClass, null, vertexInputFormatClass, + vertexOutputFormatClass, params, data); + } - /** - * Attempts to run the vertex internally in the current JVM, reading from and writing to a - * temporary folder on local disk. Will start an own zookeeper instance. - * - * @param vertexClass the vertex class to instantiate - * @param vertexInputFormatClass the inputformat to use - * @param vertexOutputFormatClass the outputformat to use - * @param params a map of parameters to add to the hadoop configuration - * @param data linewise input data - * @return linewise output data - * @throws Exception - */ - public static Iterable run(Class vertexClass, - Class vertexInputFormatClass, Class vertexOutputFormatClass, - Map params, String... data) throws Exception { - return run(vertexClass, null, vertexInputFormatClass, - vertexOutputFormatClass, params, data); - } - - /** - * Attempts to run the vertex internally in the current JVM, reading from and writing to a - * temporary folder on local disk. Will start an own zookeeper instance. - * - * @param vertexClass the vertex class to instantiate - * @param vertexCombinerClass the vertex combiner to use (or null) - * @param vertexInputFormatClass the inputformat to use - * @param vertexOutputFormatClass the outputformat to use - * @param params a map of parameters to add to the hadoop configuration - * @param data linewise input data - * @return linewise output data - * @throws Exception - */ - public static Iterable run(Class vertexClass, - Class vertexCombinerClass, Class vertexInputFormatClass, - Class vertexOutputFormatClass, Map params, - String... data) throws Exception { - - File tmpDir = null; - try { - // prepare input file, output folder and zookeeper folder - tmpDir = createTestDir(vertexClass); - File inputFile = createTempFile(tmpDir, "graph.txt"); - File outputDir = createTempDir(tmpDir, "output"); - File zkDir = createTempDir(tmpDir, "zooKeeper"); - - // write input data to disk - writeLines(inputFile, data); - - // create and configure the job to run the vertex - GiraphJob job = new GiraphJob(vertexClass.getName()); - job.setVertexClass(vertexClass); - job.setVertexInputFormatClass(vertexInputFormatClass); - job.setVertexOutputFormatClass(vertexOutputFormatClass); - - if (vertexCombinerClass != null) { - job.setVertexCombinerClass(vertexCombinerClass); - } - - job.setWorkerConfiguration(1, 1, 100.0f); - Configuration conf = job.getConfiguration(); - conf.setBoolean(GiraphJob.SPLIT_MASTER_WORKER, false); - conf.setBoolean(GiraphJob.LOCAL_TEST_MODE, true); - conf.set(GiraphJob.ZOOKEEPER_LIST, "localhost:" + - String.valueOf(LOCAL_ZOOKEEPER_PORT)); - - for (Map.Entry param : params.entrySet()) { - conf.set(param.getKey(), param.getValue()); - } - - FileInputFormat.addInputPath(job, new Path(inputFile.toString())); - FileOutputFormat.setOutputPath(job, new Path(outputDir.toString())); - - // configure a local zookeeper instance - Properties zkProperties = new Properties(); - zkProperties.setProperty("tickTime", "2000"); - zkProperties.setProperty("dataDir", zkDir.getAbsolutePath()); - zkProperties.setProperty("clientPort", - String.valueOf(LOCAL_ZOOKEEPER_PORT)); - zkProperties.setProperty("maxClientCnxns", "10000"); - zkProperties.setProperty("minSessionTimeout", "10000"); - zkProperties.setProperty("maxSessionTimeout", "100000"); - zkProperties.setProperty("initLimit", "10"); - zkProperties.setProperty("syncLimit", "5"); - zkProperties.setProperty("snapCount", "50000"); - - QuorumPeerConfig qpConfig = new QuorumPeerConfig(); - qpConfig.parseProperties(zkProperties); - - // create and run the zookeeper instance - final InternalZooKeeper zookeeper = new InternalZooKeeper(); - final ServerConfig zkConfig = new ServerConfig(); - zkConfig.readFrom(qpConfig); - - ExecutorService executorService = Executors.newSingleThreadExecutor(); - executorService.execute(new Runnable() { - @Override - public void run() { - try { - zookeeper.runFromConfig(zkConfig); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - }); - try { - job.run(true); - } finally { - executorService.shutdown(); - zookeeper.end(); - } - - return Files.readLines(new File(outputDir, "part-m-00000"), - Charsets.UTF_8); - } finally { - if (tmpDir != null) { - new DeletingVisitor().accept(tmpDir); - } - } - } + /** + * Attempts to run the vertex internally in the current JVM, reading from + * and writing to a temporary folder on local disk. Will start its own + * zookeeper instance. + * + * @param vertexClass the vertex class to instantiate + * @param vertexCombinerClass the vertex combiner to use (or null) + * @param vertexInputFormatClass the inputformat to use + * @param vertexOutputFormatClass the outputformat to use + * @param params a map of parameters to add to the hadoop configuration + * @param data linewise input data + * @return linewise output data + * @throws Exception + */ + public static Iterable run(Class vertexClass, + Class vertexCombinerClass, Class vertexInputFormatClass, + Class vertexOutputFormatClass, Map params, + String... data) throws Exception { - /** - * Create a temporary folder that will be removed after the test - */ - private static final File createTestDir(Class vertexClass) - throws IOException { - String systemTmpDir = System.getProperty("java.io.tmpdir"); - long simpleRandomLong = (long) (Long.MAX_VALUE * Math.random()); - File testTempDir = new File(systemTmpDir, "giraph-" + - vertexClass.getSimpleName() + '-' + simpleRandomLong); - if (!testTempDir.mkdir()) { - throw new IOException("Could not create " + testTempDir); + File tmpDir = null; + try { + // prepare input file, output folder and zookeeper folder + tmpDir = createTestDir(vertexClass); + File inputFile = createTempFile(tmpDir, "graph.txt"); + File outputDir = createTempDir(tmpDir, "output"); + File zkDir = createTempDir(tmpDir, "zooKeeper"); + + // write input data to disk + writeLines(inputFile, data); + + // create and configure the job to run the vertex + GiraphJob job = new GiraphJob(vertexClass.getName()); + job.setVertexClass(vertexClass); + job.setVertexInputFormatClass(vertexInputFormatClass); + job.setVertexOutputFormatClass(vertexOutputFormatClass); + + if (vertexCombinerClass != null) { + job.setVertexCombinerClass(vertexCombinerClass); + } + + job.setWorkerConfiguration(1, 1, 100.0f); + Configuration conf = job.getConfiguration(); + conf.setBoolean(GiraphJob.SPLIT_MASTER_WORKER, false); + conf.setBoolean(GiraphJob.LOCAL_TEST_MODE, true); + conf.set(GiraphJob.ZOOKEEPER_LIST, "localhost:" + + String.valueOf(LOCAL_ZOOKEEPER_PORT)); + + for (Map.Entry param : params.entrySet()) { + conf.set(param.getKey(), param.getValue()); + } + + FileInputFormat.addInputPath(job, new Path(inputFile.toString())); + FileOutputFormat.setOutputPath(job, new Path(outputDir.toString())); + + // configure a local zookeeper instance + Properties zkProperties = new Properties(); + zkProperties.setProperty("tickTime", "2000"); + zkProperties.setProperty("dataDir", zkDir.getAbsolutePath()); + zkProperties.setProperty("clientPort", + String.valueOf(LOCAL_ZOOKEEPER_PORT)); + zkProperties.setProperty("maxClientCnxns", "10000"); + zkProperties.setProperty("minSessionTimeout", "10000"); + zkProperties.setProperty("maxSessionTimeout", "100000"); + zkProperties.setProperty("initLimit", "10"); + zkProperties.setProperty("syncLimit", "5"); + zkProperties.setProperty("snapCount", "50000"); + + QuorumPeerConfig qpConfig = new QuorumPeerConfig(); + qpConfig.parseProperties(zkProperties); + + // create and run the zookeeper instance + final InternalZooKeeper zookeeper = new InternalZooKeeper(); + final ServerConfig zkConfig = new ServerConfig(); + zkConfig.readFrom(qpConfig); + + ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.execute(new Runnable() { + @Override + public void run() { + try { + zookeeper.runFromConfig(zkConfig); + } catch (IOException e) { + throw new RuntimeException(e); + } } - testTempDir.deleteOnExit(); - return testTempDir; - } + }); + try { + job.run(true); + } finally { + executorService.shutdown(); + zookeeper.end(); + } - private static final File createTempFile(File parent, String name) - throws IOException { - return createTestTempFileOrDir(parent, name, false); + return Files.readLines(new File(outputDir, "part-m-00000"), + Charsets.UTF_8); + } finally { + if (tmpDir != null) { + new DeletingVisitor().accept(tmpDir); + } } + } - private static final File createTempDir(File parent, String name) - throws IOException { - File dir = createTestTempFileOrDir(parent, name, true); - dir.delete(); - return dir; + /** + * Create a temporary folder that will be removed after the test. + * + * @param vertexClass Used for generating the folder name. + * @return File object for the directory. + */ + private static File createTestDir(Class vertexClass) + throws IOException { + String systemTmpDir = System.getProperty("java.io.tmpdir"); + long simpleRandomLong = (long) (Long.MAX_VALUE * Math.random()); + File testTempDir = new File(systemTmpDir, "giraph-" + + vertexClass.getSimpleName() + '-' + simpleRandomLong); + if (!testTempDir.mkdir()) { + throw new IOException("Could not create " + testTempDir); } + testTempDir.deleteOnExit(); + return testTempDir; + } - private static File createTestTempFileOrDir(File parent, String name, - boolean dir) throws IOException { - File f = new File(parent, name); - f.deleteOnExit(); - if (dir && !f.mkdirs()) { - throw new IOException("Could not make directory " + f); - } - return f; + /** + * Make a temporary file. + * + * @param parent Parent directory. + * @param name File name. + * @return File object to temporary file. + * @throws IOException + */ + private static File createTempFile(File parent, String name) + throws IOException { + return createTestTempFileOrDir(parent, name, false); + } + + /** + * Make a temporary directory. + * + * @param parent Parent directory. + * @param name Directory name. + * @return File object to temporary file. + * @throws IOException + */ + private static File createTempDir(File parent, String name) + throws IOException { + File dir = createTestTempFileOrDir(parent, name, true); + dir.delete(); + return dir; + } + + /** + * Creae a test temp file or directory. + * + * @param parent Parent directory + * @param name Name of file + * @param dir Is directory? + * @return File object + * @throws IOException + */ + private static File createTestTempFileOrDir(File parent, String name, + boolean dir) throws IOException { + File f = new File(parent, name); + f.deleteOnExit(); + if (dir && !f.mkdirs()) { + throw new IOException("Could not make directory " + f); } + return f; + } - private static void writeLines(File file, String... lines) - throws IOException { - Writer writer = Files.newWriter(file, Charsets.UTF_8); - try { - for (String line : lines) { - writer.write(line); - writer.write('\n'); - } - } finally { - Closeables.closeQuietly(writer); - } + /** + * Write lines to a file. + * + * @param file File to write lines to + * @param lines Strings written to the file + * @throws IOException + */ + private static void writeLines(File file, String... lines) + throws IOException { + Writer writer = Files.newWriter(file, Charsets.UTF_8); + try { + for (String line : lines) { + writer.write(line); + writer.write('\n'); + } + } finally { + Closeables.closeQuietly(writer); } + } - private static class DeletingVisitor implements FileFilter { - @Override - public boolean accept(File f) { - if (!f.isFile()) { - f.listFiles(this); - } - f.delete(); - return false; - } + /** + * Deletes files. + */ + private static class DeletingVisitor implements FileFilter { + @Override + public boolean accept(File f) { + if (!f.isFile()) { + f.listFiles(this); + } + f.delete(); + return false; } + } + /** + * Extension of {@link ZooKeeperServerMain} that allows programmatic shutdown + */ + private static class InternalZooKeeper extends ZooKeeperServerMain { /** - * Extension of {@link ZooKeeperServerMain} that allows programmatic shutdown + * Shutdown the ZooKeeper instance. */ - private static class InternalZooKeeper extends ZooKeeperServerMain { - void end() { - shutdown(); - } + void end() { + shutdown(); } - + } } diff --git a/src/main/java/org/apache/giraph/utils/MemoryUtils.java b/src/main/java/org/apache/giraph/utils/MemoryUtils.java index 31c30f98e..8fc403201 100644 --- a/src/main/java/org/apache/giraph/utils/MemoryUtils.java +++ b/src/main/java/org/apache/giraph/utils/MemoryUtils.java @@ -22,18 +22,20 @@ * Helper static methods for tracking memory usage. */ public class MemoryUtils { - /** - * Get stringified runtime memory stats - * - * @return String of all Runtime stats. - */ - public static String getRuntimeMemoryStats() { - return "totalMem = " + - (Runtime.getRuntime().totalMemory() / 1024f / 1024f) + - "M, maxMem = " + - (Runtime.getRuntime().maxMemory() / 1024f / 1024f) + - "M, freeMem = " + - (Runtime.getRuntime().freeMemory() / 1024f / 1024f) - + "M"; - } + /** Do not instantiate. */ + private MemoryUtils() { } + + /** + * Get stringified runtime memory stats + * + * @return String of all Runtime stats. + */ + public static String getRuntimeMemoryStats() { + return "totalMem = " + + (Runtime.getRuntime().totalMemory() / 1024f / 1024f) + + "M, maxMem = " + + (Runtime.getRuntime().maxMemory() / 1024f / 1024f) + + "M, freeMem = " + + (Runtime.getRuntime().freeMemory() / 1024f / 1024f) + "M"; + } } diff --git a/src/main/java/org/apache/giraph/utils/ReflectionUtils.java b/src/main/java/org/apache/giraph/utils/ReflectionUtils.java index 8652b62a7..cf7d6df2a 100644 --- a/src/main/java/org/apache/giraph/utils/ReflectionUtils.java +++ b/src/main/java/org/apache/giraph/utils/ReflectionUtils.java @@ -35,114 +35,127 @@ * generic classes, not interfaces. */ public class ReflectionUtils { - /** - * Get the underlying class for a type, or null if the type is - * a variable type. - * - * @param type the type - * @return the underlying class - */ - public static Class getClass(Type type) { - if (type instanceof Class) { - return (Class) type; - } - else if (type instanceof ParameterizedType) { - return getClass(((ParameterizedType) type).getRawType()); - } - else if (type instanceof GenericArrayType) { - Type componentType = - ((GenericArrayType) type).getGenericComponentType(); - Class componentClass = getClass(componentType); - if (componentClass != null ) { - return Array.newInstance(componentClass, 0).getClass(); - } - else { - return null; - } - } - else { - return null; - } - } + /** + * Do not instantiate. + */ + private ReflectionUtils() { } - /** - * Get the actual type arguments a child class has used to extend a - * generic base class. - * - * @param baseClass the base class - * @param childClass the child class - * @return a list of the raw classes for the actual type arguments. - */ - public static List> getTypeArguments( - Class baseClass, Class childClass) { - Map resolvedTypes = new HashMap(); - Type type = childClass; - // start walking up the inheritance hierarchy until we hit baseClass - while (! getClass(type).equals(baseClass)) { - if (type instanceof Class) { - // there is no useful information for us in raw types, - // so just keep going. - type = ((Class) type).getGenericSuperclass(); - } - else { - ParameterizedType parameterizedType = (ParameterizedType) type; - Class rawType = (Class) parameterizedType.getRawType(); + /** + * Get the underlying class for a type, or null if the type is + * a variable type. + * + * @param type the type + * @return the underlying class + */ + public static Class getClass(Type type) { + if (type instanceof Class) { + return (Class) type; + } else if (type instanceof ParameterizedType) { + return getClass(((ParameterizedType) type).getRawType()); + } else if (type instanceof GenericArrayType) { + Type componentType = + ((GenericArrayType) type).getGenericComponentType(); + Class componentClass = getClass(componentType); + if (componentClass != null) { + return Array.newInstance(componentClass, 0).getClass(); + } else { + return null; + } + } else { + return null; + } + } - Type[] actualTypeArguments = - parameterizedType.getActualTypeArguments(); - TypeVariable[] typeParameters = rawType.getTypeParameters(); - for (int i = 0; i < actualTypeArguments.length; i++) { - resolvedTypes.put(typeParameters[i], - actualTypeArguments[i]); - } + /** + * Get the actual type arguments a child class has used to extend a + * generic base class. + * + * @param Type to evaluate. + * @param baseClass the base class + * @param childClass the child class + * @return a list of the raw classes for the actual type arguments. + */ + public static List> getTypeArguments( + Class baseClass, Class childClass) { + Map resolvedTypes = new HashMap(); + Type type = childClass; + // start walking up the inheritance hierarchy until we hit baseClass + while (! getClass(type).equals(baseClass)) { + if (type instanceof Class) { + // there is no useful information for us in raw types, + // so just keep going. + type = ((Class) type).getGenericSuperclass(); + } else { + ParameterizedType parameterizedType = (ParameterizedType) type; + Class rawType = (Class) parameterizedType.getRawType(); - if (!rawType.equals(baseClass)) { - type = rawType.getGenericSuperclass(); - } - } + Type[] actualTypeArguments = + parameterizedType.getActualTypeArguments(); + TypeVariable[] typeParameters = rawType.getTypeParameters(); + for (int i = 0; i < actualTypeArguments.length; i++) { + resolvedTypes.put(typeParameters[i], + actualTypeArguments[i]); } - // finally, for each actual type argument provided to baseClass, - // determine (if possible) - // the raw class for that type argument. - Type[] actualTypeArguments; - if (type instanceof Class) { - actualTypeArguments = ((Class) type).getTypeParameters(); - } - else { - actualTypeArguments = - ((ParameterizedType) type).getActualTypeArguments(); - } - List> typeArgumentsAsClasses = new ArrayList>(); - // resolve types by chasing down type variables. - for (Type baseType: actualTypeArguments) { - while (resolvedTypes.containsKey(baseType)) { - baseType = resolvedTypes.get(baseType); - } - typeArgumentsAsClasses.add(getClass(baseType)); + if (!rawType.equals(baseClass)) { + type = rawType.getGenericSuperclass(); } - return typeArgumentsAsClasses; + } } - /** try to directly set a (possibly private) field on an Object */ - public static void setField(Object target, String fieldname, Object value) - throws NoSuchFieldException, IllegalAccessException { - Field field = findDeclaredField(target.getClass(), fieldname); - field.setAccessible(true); - field.set(target, value); + // finally, for each actual type argument provided to baseClass, + // determine (if possible) + // the raw class for that type argument. + Type[] actualTypeArguments; + if (type instanceof Class) { + actualTypeArguments = ((Class) type).getTypeParameters(); + } else { + actualTypeArguments = + ((ParameterizedType) type).getActualTypeArguments(); } + List> typeArgumentsAsClasses = new ArrayList>(); + // resolve types by chasing down type variables. + for (Type baseType: actualTypeArguments) { + while (resolvedTypes.containsKey(baseType)) { + baseType = resolvedTypes.get(baseType); + } + typeArgumentsAsClasses.add(getClass(baseType)); + } + return typeArgumentsAsClasses; + } + + /** + * Try to directly set a (possibly private) field on an Object. + * + * @param target Target to set the field on. + * @param fieldname Name of field. + * @param value Value to set on target. + */ + public static void setField(Object target, String fieldname, Object value) + throws NoSuchFieldException, IllegalAccessException { + Field field = findDeclaredField(target.getClass(), fieldname); + field.setAccessible(true); + field.set(target, value); + } - /** find a declared field in a class or one of its super classes */ - private static Field findDeclaredField(Class inClass, String fieldname) - throws NoSuchFieldException { - while (!Object.class.equals(inClass)) { - for (Field field : inClass.getDeclaredFields()) { - if (field.getName().equalsIgnoreCase(fieldname)) { - return field; - } - } - inClass = inClass.getSuperclass(); + /** + * Find a declared field in a class or one of its super classes + * + * @param inClass Class to search for declared field. + * @param fieldname Field name to search for + * @return Field or will throw. + * @throws NoSuchFieldException When field not found. + */ + private static Field findDeclaredField(Class inClass, String fieldname) + throws NoSuchFieldException { + while (!Object.class.equals(inClass)) { + for (Field field : inClass.getDeclaredFields()) { + if (field.getName().equalsIgnoreCase(fieldname)) { + return field; } - throw new NoSuchFieldException(); + } + inClass = inClass.getSuperclass(); } + throw new NoSuchFieldException(); + } } diff --git a/src/main/java/org/apache/giraph/utils/UnmodifiableIntArrayIterator.java b/src/main/java/org/apache/giraph/utils/UnmodifiableIntArrayIterator.java index e01d242f9..c9ba0efa3 100644 --- a/src/main/java/org/apache/giraph/utils/UnmodifiableIntArrayIterator.java +++ b/src/main/java/org/apache/giraph/utils/UnmodifiableIntArrayIterator.java @@ -24,24 +24,30 @@ /** * {@link UnmodifiableIterator} over a primitive int array */ -public class UnmodifiableIntArrayIterator - extends UnmodifiableIterator { +public class UnmodifiableIntArrayIterator extends + UnmodifiableIterator { + /** Array to iterate over */ + private final int[] intArray; + /** Offset to array */ + private int offset; - private final int[] arr; - private int offset; + /** + * Constructor with array to iterate over. + * + * @param intArray Array to iterate over. + */ + public UnmodifiableIntArrayIterator(int[] intArray) { + this.intArray = intArray; + offset = 0; + } - public UnmodifiableIntArrayIterator(int[] arr) { - this.arr = arr; - offset = 0; - } + @Override + public boolean hasNext() { + return offset < intArray.length; + } - @Override - public boolean hasNext() { - return offset < arr.length; - } - - @Override - public IntWritable next() { - return new IntWritable(arr[offset++]); - } -} \ No newline at end of file + @Override + public IntWritable next() { + return new IntWritable(intArray[offset++]); + } +} diff --git a/src/main/java/org/apache/giraph/utils/WritableUtils.java b/src/main/java/org/apache/giraph/utils/WritableUtils.java index 885dde337..5f7ced453 100644 --- a/src/main/java/org/apache/giraph/utils/WritableUtils.java +++ b/src/main/java/org/apache/giraph/utils/WritableUtils.java @@ -41,147 +41,215 @@ * Helper static methods for working with Writable objects. */ public class WritableUtils { - public static void readFieldsFromByteArray( - byte[] byteArray, Writable writableObject) { - DataInputStream inputStream = - new DataInputStream(new ByteArrayInputStream(byteArray)); - try { - writableObject.readFields(inputStream); - } catch (IOException e) { - throw new IllegalStateException( - "readFieldsFromByteArray: IOException", e); - } + /** + * Don't construct. + */ + private WritableUtils() { } + + /** + * Read fields from byteArray to a Writeable object. + * + * @param byteArray Byte array to find the fields in. + * @param writableObject Object to fill in the fields. + */ + public static void readFieldsFromByteArray( + byte[] byteArray, Writable writableObject) { + DataInputStream inputStream = + new DataInputStream(new ByteArrayInputStream(byteArray)); + try { + writableObject.readFields(inputStream); + } catch (IOException e) { + throw new IllegalStateException( + "readFieldsFromByteArray: IOException", e); } + } - public static void readFieldsFromZnode(ZooKeeperExt zkExt, - String zkPath, - boolean watch, - Stat stat, - Writable writableObject) { - try { - byte[] zkData = zkExt.getData(zkPath, false, stat); - readFieldsFromByteArray(zkData, writableObject); - } catch (KeeperException e) { - throw new IllegalStateException( - "readFieldsFromZnode: KeeperException on " + zkPath, e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "readFieldsFromZnode: InterrruptedStateException on " + zkPath, - e); - } + /** + * Read fields from a ZooKeeper znode. + * + * @param zkExt ZooKeeper instance. + * @param zkPath Path of znode. + * @param watch Add a watch? + * @param stat Stat of znode if desired. + * @param writableObject Object to read into. + */ + public static void readFieldsFromZnode(ZooKeeperExt zkExt, + String zkPath, + boolean watch, + Stat stat, + Writable writableObject) { + try { + byte[] zkData = zkExt.getData(zkPath, false, stat); + readFieldsFromByteArray(zkData, writableObject); + } catch (KeeperException e) { + throw new IllegalStateException( + "readFieldsFromZnode: KeeperException on " + zkPath, e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "readFieldsFromZnode: InterrruptedStateException on " + zkPath, e); } + } - public static byte[] writeToByteArray(Writable writableObject) { - ByteArrayOutputStream outputStream = - new ByteArrayOutputStream(); - DataOutput output = new DataOutputStream(outputStream); - try { - writableObject.write(output); - } catch (IOException e) { - throw new IllegalStateException( - "writeToByteArray: IOStateException", e); - } - return outputStream.toByteArray(); + /** + * Write object to a byte array. + * + * @param writableObject Object to write from. + * @return Byte array with serialized object. + */ + public static byte[] writeToByteArray(Writable writableObject) { + ByteArrayOutputStream outputStream = + new ByteArrayOutputStream(); + DataOutput output = new DataOutputStream(outputStream); + try { + writableObject.write(output); + } catch (IOException e) { + throw new IllegalStateException( + "writeToByteArray: IOStateException", e); } + return outputStream.toByteArray(); + } - public static PathStat writeToZnode(ZooKeeperExt zkExt, - String zkPath, - int version, - Writable writableObject) { - try { - byte[] byteArray = writeToByteArray(writableObject); - return zkExt.createOrSetExt(zkPath, - byteArray, - Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT, - true, - version); - } catch (KeeperException e) { - throw new IllegalStateException( - "writeToZnode: KeeperException on " + zkPath, e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "writeToZnode: InterruptedException on " + zkPath, e); - } + /** + * Write object to a ZooKeeper znode. + * + * @param zkExt ZooKeeper instance. + * @param zkPath Path of znode. + * @param version Version of the write. + * @param writableObject Object to write from. + * @return Path and stat information of the znode. + */ + public static PathStat writeToZnode(ZooKeeperExt zkExt, + String zkPath, + int version, + Writable writableObject) { + try { + byte[] byteArray = writeToByteArray(writableObject); + return zkExt.createOrSetExt(zkPath, + byteArray, + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT, + true, + version); + } catch (KeeperException e) { + throw new IllegalStateException( + "writeToZnode: KeeperException on " + zkPath, e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "writeToZnode: InterruptedException on " + zkPath, e); } + } - public static byte[] writeListToByteArray( - List writableList) { - ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); - DataOutput output = new DataOutputStream(outputStream); - try { - output.writeInt(writableList.size()); - for (Writable writable : writableList) { - writable.write(output); - } - } catch (IOException e) { - throw new IllegalStateException( - "writeListToByteArray: IOException", e); - } - return outputStream.toByteArray(); + /** + * Write list of object to a byte array. + * + * @param writableList List of object to write from. + * @return Byte array with serialized objects. + */ + public static byte[] writeListToByteArray( + List writableList) { + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + DataOutput output = new DataOutputStream(outputStream); + try { + output.writeInt(writableList.size()); + for (Writable writable : writableList) { + writable.write(output); + } + } catch (IOException e) { + throw new IllegalStateException( + "writeListToByteArray: IOException", e); } + return outputStream.toByteArray(); + } - public static PathStat writeListToZnode( - ZooKeeperExt zkExt, - String zkPath, - int version, - List writableList) { - try { - return zkExt.createOrSetExt( - zkPath, - writeListToByteArray(writableList), - Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT, - true, - version); - } catch (KeeperException e) { - throw new IllegalStateException( - "writeListToZnode: KeeperException on " + zkPath, e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "writeListToZnode: InterruptedException on " + zkPath, e); - } + /** + * Write list of objects to a ZooKeeper znode. + * + * @param zkExt ZooKeeper instance. + * @param zkPath Path of znode. + * @param version Version of the write. + * @param writableList List of objects to write from. + * @return Path and stat information of the znode. + */ + public static PathStat writeListToZnode( + ZooKeeperExt zkExt, + String zkPath, + int version, + List writableList) { + try { + return zkExt.createOrSetExt( + zkPath, + writeListToByteArray(writableList), + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT, + true, + version); + } catch (KeeperException e) { + throw new IllegalStateException( + "writeListToZnode: KeeperException on " + zkPath, e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "writeListToZnode: InterruptedException on " + zkPath, e); } + } - public static List readListFieldsFromByteArray( - byte[] byteArray, - Class writableClass, - Configuration conf) { - try { - DataInputStream inputStream = - new DataInputStream(new ByteArrayInputStream(byteArray)); - int size = inputStream.readInt(); - List writableList = new ArrayList(size); - for (int i = 0; i < size; ++i) { - Writable writable = - ReflectionUtils.newInstance(writableClass, conf); - writable.readFields(inputStream); - writableList.add(writable); - } - return writableList; - } catch (IOException e) { - throw new IllegalStateException( - "readListFieldsFromZnode: IOException", e); - } + /** + * Read fields from byteArray to a list of Writeable objects. + * + * @param byteArray Byte array to find the fields in. + * @param writableClass Class of the objects to instantiate. + * @param conf Configuration used for instantiation (i.e Configurable) + * @return List of writable objects. + */ + public static List readListFieldsFromByteArray( + byte[] byteArray, + Class writableClass, + Configuration conf) { + try { + DataInputStream inputStream = + new DataInputStream(new ByteArrayInputStream(byteArray)); + int size = inputStream.readInt(); + List writableList = new ArrayList(size); + for (int i = 0; i < size; ++i) { + Writable writable = + ReflectionUtils.newInstance(writableClass, conf); + writable.readFields(inputStream); + writableList.add(writable); + } + return writableList; + } catch (IOException e) { + throw new IllegalStateException( + "readListFieldsFromZnode: IOException", e); } + } - public static List readListFieldsFromZnode( - ZooKeeperExt zkExt, - String zkPath, - boolean watch, - Stat stat, - Class writableClass, - Configuration conf) { - try { - byte[] zkData = zkExt.getData(zkPath, false, stat); - return readListFieldsFromByteArray(zkData, writableClass, conf); - } catch (KeeperException e) { - throw new IllegalStateException( - "readListFieldsFromZnode: KeeperException on " + zkPath, e); - } catch (InterruptedException e) { - throw new IllegalStateException( - "readListFieldsFromZnode: InterruptedException on " + zkPath, - e); - } + /** + * Read fields from a ZooKeeper znode into a list of objects. + * + * @param zkExt ZooKeeper instance. + * @param zkPath Path of znode. + * @param watch Add a watch? + * @param stat Stat of znode if desired. + * @param writableClass Class of the objects to instantiate. + * @param conf Configuration used for instantiation (i.e Configurable) + * @return List of writable objects. + */ + public static List readListFieldsFromZnode( + ZooKeeperExt zkExt, + String zkPath, + boolean watch, + Stat stat, + Class writableClass, + Configuration conf) { + try { + byte[] zkData = zkExt.getData(zkPath, false, stat); + return readListFieldsFromByteArray(zkData, writableClass, conf); + } catch (KeeperException e) { + throw new IllegalStateException( + "readListFieldsFromZnode: KeeperException on " + zkPath, e); + } catch (InterruptedException e) { + throw new IllegalStateException( + "readListFieldsFromZnode: InterruptedException on " + zkPath, + e); } + } } diff --git a/src/main/java/org/apache/giraph/utils/package-info.java b/src/main/java/org/apache/giraph/utils/package-info.java new file mode 100644 index 000000000..b7dc437e5 --- /dev/null +++ b/src/main/java/org/apache/giraph/utils/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/** + * Package of all generic utility classes. + */ +package org.apache.giraph.utils; diff --git a/src/main/java/org/apache/giraph/zk/BspEvent.java b/src/main/java/org/apache/giraph/zk/BspEvent.java index 787b710aa..7d22f9a22 100644 --- a/src/main/java/org/apache/giraph/zk/BspEvent.java +++ b/src/main/java/org/apache/giraph/zk/BspEvent.java @@ -23,27 +23,27 @@ * event. */ public interface BspEvent { - /** - * Reset the permanent signal. - */ - void reset(); + /** + * Reset the permanent signal. + */ + void reset(); - /** - * The event occurred and the occurrence has been logged for future - * waiters. - */ - void signal(); + /** + * The event occurred and the occurrence has been logged for future + * waiters. + */ + void signal(); - /** - * Wait until the event occurred or waiting timed out. - * @param msecs Milliseconds to wait until the event occurred. 0 indicates - * check immediately. -1 indicates wait forever. - * @return true if event occurred, false if timed out while waiting - */ - boolean waitMsecs(int msecs); + /** + * Wait until the event occurred or waiting timed out. + * @param msecs Milliseconds to wait until the event occurred. 0 indicates + * check immediately. -1 indicates wait forever. + * @return true if event occurred, false if timed out while waiting + */ + boolean waitMsecs(int msecs); - /** - * Wait indefinitely until the event occurs. - */ - void waitForever(); + /** + * Wait indefinitely until the event occurs. + */ + void waitForever(); } diff --git a/src/main/java/org/apache/giraph/zk/ContextLock.java b/src/main/java/org/apache/giraph/zk/ContextLock.java index 265ac03b6..f4fb00a6d 100644 --- a/src/main/java/org/apache/giraph/zk/ContextLock.java +++ b/src/main/java/org/apache/giraph/zk/ContextLock.java @@ -24,29 +24,29 @@ * A lock that will keep the job context updated while waiting. */ public class ContextLock extends PredicateLock { - /** Job context (for progress) */ - @SuppressWarnings("rawtypes") - private final Context context; - /** Msecs to refresh the progress meter */ - private static final int msecPeriod = 10000; + /** Msecs to refresh the progress meter */ + private static final int MSEC_PERIOD = 10000; + /** Job context (for progress) */ + @SuppressWarnings("rawtypes") + private final Context context; - /** - * Constructor. - * - * @param context used to call progress() - */ - ContextLock(@SuppressWarnings("rawtypes") Context context) { - this.context = context; - } + /** + * Constructor. + * + * @param context used to call progress() + */ + ContextLock(@SuppressWarnings("rawtypes") Context context) { + this.context = context; + } - /** - * Specialized version of waitForever() that will keep the job progressing - * while waiting. - */ - @Override - public void waitForever() { - while (waitMsecs(msecPeriod) == false) { - context.progress(); - } + /** + * Specialized version of waitForever() that will keep the job progressing + * while waiting. + */ + @Override + public void waitForever() { + while (!waitMsecs(MSEC_PERIOD)) { + context.progress(); } + } } diff --git a/src/main/java/org/apache/giraph/zk/PredicateLock.java b/src/main/java/org/apache/giraph/zk/PredicateLock.java index f5fe27eea..ed5deb498 100644 --- a/src/main/java/org/apache/giraph/zk/PredicateLock.java +++ b/src/main/java/org/apache/giraph/zk/PredicateLock.java @@ -29,89 +29,88 @@ * A lock with a predicate that was be used to synchronize events. */ public class PredicateLock implements BspEvent { - /** Lock */ - private Lock lock = new ReentrantLock(); - /** Condition associated with lock */ - private Condition cond = lock.newCondition(); - /** Predicate */ - private boolean eventOccurred = false; - /** Class logger */ - private Logger LOG = Logger.getLogger(PredicateLock.class); + /** Class logger */ + private static final Logger LOG = Logger.getLogger(PredicateLock.class); + /** Lock */ + private Lock lock = new ReentrantLock(); + /** Condition associated with lock */ + private Condition cond = lock.newCondition(); + /** Predicate */ + private boolean eventOccurred = false; - @Override - public void reset() { - lock.lock(); - try { - eventOccurred = false; - } finally { - lock.unlock(); - } + @Override + public void reset() { + lock.lock(); + try { + eventOccurred = false; + } finally { + lock.unlock(); } + } - @Override - public void signal() { - lock.lock(); - try { - eventOccurred = true; - cond.signalAll(); - } finally { - lock.unlock(); - } + @Override + public void signal() { + lock.lock(); + try { + eventOccurred = true; + cond.signalAll(); + } finally { + lock.unlock(); } + } - @Override - public boolean waitMsecs(int msecs) { - if (msecs < -1) { - throw new RuntimeException("msecs < -1"); - } + @Override + public boolean waitMsecs(int msecs) { + if (msecs < -1) { + throw new RuntimeException("msecs < -1"); + } - long maxMsecs = System.currentTimeMillis() + msecs; - long curMsecTimeout = 0; - lock.lock(); - try { - while (eventOccurred == false) { - if (msecs == -1) { - try { - cond.await(); - } catch (InterruptedException e) { - throw new IllegalStateException( - "waitMsecs: Caught interrupted " + - "exception on cond.await()", e); - } - } - else { - // Keep the wait non-negative - curMsecTimeout = - Math.max(maxMsecs - System.currentTimeMillis(), 0); - if (LOG.isDebugEnabled()) { - LOG.debug("waitMsecs: Wait for " + curMsecTimeout); - } - try { - boolean signaled = - cond.await(curMsecTimeout, TimeUnit.MILLISECONDS); - if (LOG.isDebugEnabled()) { - LOG.debug("waitMsecs: Got timed signaled of " + - signaled); - } - } catch (InterruptedException e) { - throw new IllegalStateException( - "waitMsecs: Caught interrupted " + - "exception on cond.await() " + - curMsecTimeout, e); - } - if (System.currentTimeMillis() > maxMsecs) { - return false; - } - } + long maxMsecs = System.currentTimeMillis() + msecs; + long curMsecTimeout = 0; + lock.lock(); + try { + while (!eventOccurred) { + if (msecs == -1) { + try { + cond.await(); + } catch (InterruptedException e) { + throw new IllegalStateException( + "waitMsecs: Caught interrupted " + + "exception on cond.await()", e); + } + } else { + // Keep the wait non-negative + curMsecTimeout = + Math.max(maxMsecs - System.currentTimeMillis(), 0); + if (LOG.isDebugEnabled()) { + LOG.debug("waitMsecs: Wait for " + curMsecTimeout); + } + try { + boolean signaled = + cond.await(curMsecTimeout, TimeUnit.MILLISECONDS); + if (LOG.isDebugEnabled()) { + LOG.debug("waitMsecs: Got timed signaled of " + + signaled); } - } finally { - lock.unlock(); + } catch (InterruptedException e) { + throw new IllegalStateException( + "waitMsecs: Caught interrupted " + + "exception on cond.await() " + + curMsecTimeout, e); + } + if (System.currentTimeMillis() > maxMsecs) { + return false; + } } - return true; + } + } finally { + lock.unlock(); } + return true; + } - @Override - public void waitForever() { - waitMsecs(-1); - } + @Override + public void waitForever() { + waitMsecs(-1); + } } diff --git a/src/main/java/org/apache/giraph/zk/ZooKeeperExt.java b/src/main/java/org/apache/giraph/zk/ZooKeeperExt.java index 587108e0e..1e2aed3b8 100644 --- a/src/main/java/org/apache/giraph/zk/ZooKeeperExt.java +++ b/src/main/java/org/apache/giraph/zk/ZooKeeperExt.java @@ -39,270 +39,268 @@ * non-atomic operations that are useful. */ public class ZooKeeperExt extends ZooKeeper { - /** Internal logger */ - private static final Logger LOG = Logger.getLogger(ZooKeeperExt.class); - /** Length of the ZK sequence number */ - private static final int SEQUENCE_NUMBER_LENGTH = 10; + /** Internal logger */ + private static final Logger LOG = Logger.getLogger(ZooKeeperExt.class); + /** Length of the ZK sequence number */ + private static final int SEQUENCE_NUMBER_LENGTH = 10; - /** - * Constructor to connect to ZooKeeper - * - * @param connectString Comma separated host:port pairs, each corresponding - * to a zk server. e.g. - * "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" If the optional - * chroot suffix is used the example would look - * like: "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002/app/a" - * where the client would be rooted at "/app/a" and all paths - * would be relative to this root - ie getting/setting/etc... - * "/foo/bar" would result in operations being run on - * "/app/a/foo/bar" (from the server perspective). - * @param sessionTimeout Session timeout in milliseconds - * @param watcher A watcher object which will be notified of state changes, - * may also be notified for node events - * @throws IOException - */ - public ZooKeeperExt(String connectString, - int sessionTimeout, - Watcher watcher) throws IOException { - super(connectString, sessionTimeout, watcher); - } + /** + * Constructor to connect to ZooKeeper + * + * @param connectString Comma separated host:port pairs, each corresponding + * to a zk server. e.g. + * "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" If the optional + * chroot suffix is used the example would look + * like: "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002/app/a" + * where the client would be rooted at "/app/a" and all paths + * would be relative to this root - ie getting/setting/etc... + * "/foo/bar" would result in operations being run on + * "/app/a/foo/bar" (from the server perspective). + * @param sessionTimeout Session timeout in milliseconds + * @param watcher A watcher object which will be notified of state changes, + * may also be notified for node events + * @throws IOException + */ + public ZooKeeperExt(String connectString, + int sessionTimeout, + Watcher watcher) throws IOException { + super(connectString, sessionTimeout, watcher); + } - /** - * Provides a possibility of a creating a path consisting of more than one - * znode (not atomic). If recursive is false, operates exactly the - * same as create(). - * - * @param path path to create - * @param data data to set on the final znode - * @param acl acls on each znode created - * @param createMode only affects the final znode - * @param recursive if true, creates all ancestors - * @return Actual created path - * @throws KeeperException - * @throws InterruptedException - */ - public String createExt( - final String path, - byte data[], - List acl, - CreateMode createMode, - boolean recursive) throws KeeperException, InterruptedException { - if (LOG.isDebugEnabled()) { - LOG.debug("createExt: Creating path " + path); - } + /** + * Provides a possibility of a creating a path consisting of more than one + * znode (not atomic). If recursive is false, operates exactly the + * same as create(). + * + * @param path path to create + * @param data data to set on the final znode + * @param acl acls on each znode created + * @param createMode only affects the final znode + * @param recursive if true, creates all ancestors + * @return Actual created path + * @throws KeeperException + * @throws InterruptedException + */ + public String createExt( + final String path, + byte[] data, + List acl, + CreateMode createMode, + boolean recursive) throws KeeperException, InterruptedException { + if (LOG.isDebugEnabled()) { + LOG.debug("createExt: Creating path " + path); + } - if (!recursive) { - return create(path, data, acl, createMode); - } + if (!recursive) { + return create(path, data, acl, createMode); + } - try { - return create(path, data, acl, createMode); - } catch (KeeperException.NoNodeException e) { - if (LOG.isDebugEnabled()) { - LOG.debug("createExt: Cannot directly create node " + path); - } - } + try { + return create(path, data, acl, createMode); + } catch (KeeperException.NoNodeException e) { + if (LOG.isDebugEnabled()) { + LOG.debug("createExt: Cannot directly create node " + path); + } + } - int pos = path.indexOf("/", 1); - for (; pos != -1; pos = path.indexOf("/", pos + 1)) { - try { - create( - path.substring(0, pos), null, acl, CreateMode.PERSISTENT); - } catch (KeeperException.NodeExistsException e) { - if (LOG.isDebugEnabled()) { - LOG.debug("createExt: Znode " + path.substring(0, pos) + - " already exists"); - } - } + int pos = path.indexOf("/", 1); + for (; pos != -1; pos = path.indexOf("/", pos + 1)) { + try { + create( + path.substring(0, pos), null, acl, CreateMode.PERSISTENT); + } catch (KeeperException.NodeExistsException e) { + if (LOG.isDebugEnabled()) { + LOG.debug("createExt: Znode " + path.substring(0, pos) + + " already exists"); } - return create(path, data, acl, createMode); + } } + return create(path, data, acl, createMode); + } + + /** + * Data structure for handling the output of createOrSet() + */ + public class PathStat { + /** Path to created znode (if any) */ + private String path; + /** Stat from set znode (if any) */ + private Stat stat; /** - * Data structure for handling the output of createOrSet() + * Put in results from createOrSet() + * + * @param path Path to created znode (or null) + * @param stat Stat from set znode (if set) */ - public class PathStat { - private String path; - private Stat stat; - - /** - * Put in results from createOrSet() - * - * @param path Path to created znode (or null) - * @param stat Stat from set znode (if set) - */ - public PathStat(String path, Stat stat) { - this.path = path; - this.stat = stat; - } - - /** - * Get the path of the created znode if it was created. - * - * @return Path of created znode or null if not created - */ - public String getPath() { - return path; - } - - /** - * Get the stat of the set znode if set - * - * @return Stat of set znode or null if not set - */ - public Stat getStat() { - return stat; - } + public PathStat(String path, Stat stat) { + this.path = path; + this.stat = stat; } /** - * Create a znode. Set the znode if the created znode already exists. + * Get the path of the created znode if it was created. * - * @param path path to create - * @param data data to set on the final znode - * @param acl acls on each znode created - * @param createMode only affects the final znode - * @param recursive if true, creates all ancestors - * @return Path of created znode or Stat of set znode - * @throws InterruptedException - * @throws KeeperException + * @return Path of created znode or null if not created */ - public PathStat createOrSetExt(final String path, - byte data[], - List acl, - CreateMode createMode, - boolean recursive, - int version) - throws KeeperException, InterruptedException { - String createdPath = null; - Stat setStat = null; - try { - createdPath = createExt(path, data, acl, createMode, recursive); - } catch (KeeperException.NodeExistsException e) { - if (LOG.isDebugEnabled()) { - LOG.debug("createOrSet: Node exists on path " + path); - } - setStat = setData(path, data, version); - } - return new PathStat(createdPath, setStat); + public String getPath() { + return path; } /** - * Create a znode if there is no other znode there + * Get the stat of the set znode if set * - * @param path path to create - * @param data data to set on the final znode - * @param acl acls on each znode created - * @param createMode only affects the final znode - * @param recursive if true, creates all ancestors - * @return Path of created znode or Stat of set znode - * @throws InterruptedException - * @throws KeeperException + * @return Stat of set znode or null if not set */ - public PathStat createOnceExt(final String path, - byte data[], - List acl, - CreateMode createMode, - boolean recursive) - throws KeeperException, InterruptedException { - String createdPath = null; - Stat setStat = null; - try { - createdPath = createExt(path, data, acl, createMode, recursive); - } catch (KeeperException.NodeExistsException e) { - if (LOG.isDebugEnabled()) { - LOG.debug("createOnceExt: Node already exists on path " + path); - } - } - return new PathStat(createdPath, setStat); + public Stat getStat() { + return stat; } + } - /** - * Delete a path recursively. When the deletion is recursive, it is a - * non-atomic operation, hence, not part of ZooKeeper. - * @param path path to remove (i.e. /tmp will remove /tmp/1 and /tmp/2) - * @param version expected version (-1 for all) - * @param recursive if true, remove all children, otherwise behave like - * remove() - * @throws InterruptedException - * @throws KeeperException - */ - public void deleteExt(final String path, int version, boolean recursive) - throws InterruptedException, KeeperException { - if (!recursive) { - delete(path, version); - return; - } + /** + * Create a znode. Set the znode if the created znode already exists. + * + * @param path path to create + * @param data data to set on the final znode + * @param acl acls on each znode created + * @param createMode only affects the final znode + * @param recursive if true, creates all ancestors + * @param version Version to set if setting + * @return Path of created znode or Stat of set znode + * @throws InterruptedException + * @throws KeeperException + */ + public PathStat createOrSetExt(final String path, + byte[] data, + List acl, + CreateMode createMode, + boolean recursive, + int version) throws KeeperException, InterruptedException { + String createdPath = null; + Stat setStat = null; + try { + createdPath = createExt(path, data, acl, createMode, recursive); + } catch (KeeperException.NodeExistsException e) { + if (LOG.isDebugEnabled()) { + LOG.debug("createOrSet: Node exists on path " + path); + } + setStat = setData(path, data, version); + } + return new PathStat(createdPath, setStat); + } - try { - delete(path, version); - return; - } catch (KeeperException.NotEmptyException e) { - if (LOG.isDebugEnabled()) { - LOG.debug("deleteExt: Cannot directly remove node " + path); - } - } + /** + * Create a znode if there is no other znode there + * + * @param path path to create + * @param data data to set on the final znode + * @param acl acls on each znode created + * @param createMode only affects the final znode + * @param recursive if true, creates all ancestors + * @return Path of created znode or Stat of set znode + * @throws InterruptedException + * @throws KeeperException + */ + public PathStat createOnceExt(final String path, + byte[] data, + List acl, + CreateMode createMode, + boolean recursive) throws KeeperException, InterruptedException { + String createdPath = null; + Stat setStat = null; + try { + createdPath = createExt(path, data, acl, createMode, recursive); + } catch (KeeperException.NodeExistsException e) { + if (LOG.isDebugEnabled()) { + LOG.debug("createOnceExt: Node already exists on path " + path); + } + } + return new PathStat(createdPath, setStat); + } - List childList = getChildren(path, false); - for (String child : childList) { - deleteExt(path + "/" + child, -1, true); - } + /** + * Delete a path recursively. When the deletion is recursive, it is a + * non-atomic operation, hence, not part of ZooKeeper. + * @param path path to remove (i.e. /tmp will remove /tmp/1 and /tmp/2) + * @param version expected version (-1 for all) + * @param recursive if true, remove all children, otherwise behave like + * remove() + * @throws InterruptedException + * @throws KeeperException + */ + public void deleteExt(final String path, int version, boolean recursive) + throws InterruptedException, KeeperException { + if (!recursive) { + delete(path, version); + return; + } - delete(path, version); + try { + delete(path, version); + return; + } catch (KeeperException.NotEmptyException e) { + if (LOG.isDebugEnabled()) { + LOG.debug("deleteExt: Cannot directly remove node " + path); + } } - /** - * Get the children of the path with extensions. - * Extension 1: Sort the children based on sequence number - * Extension 2: Get the full path instead of relative path - * - * @param path path to znode - * @param watch set the watch? - * @param sequenceSorted sort by the sequence number - * @param fullPath if true, get the fully znode path back - * @return list of children - * @throws InterruptedException - * @throws KeeperException - */ - public List getChildrenExt( - final String path, - boolean watch, - boolean sequenceSorted, - boolean fullPath) - throws KeeperException, InterruptedException { - List childList = getChildren(path, watch); - /* Sort children according to the sequence number, if desired */ - if (sequenceSorted) { - Collections.sort(childList, - new Comparator() { - public int compare(String s1, String s2) { - if ((s1.length() <= SEQUENCE_NUMBER_LENGTH) || - (s2.length() <= SEQUENCE_NUMBER_LENGTH)) { - throw new RuntimeException( - "getChildrenExt: Invalid length for sequence " + - " sorting > " + - SEQUENCE_NUMBER_LENGTH + - " for s1 (" + - s1.length() + ") or s2 (" + s2.length() + ")"); - } - int s1sequenceNumber = Integer.parseInt( - s1.substring(s1.length() - - SEQUENCE_NUMBER_LENGTH)); - int s2sequenceNumber = Integer.parseInt( - s2.substring(s2.length() - - SEQUENCE_NUMBER_LENGTH)); - return s1sequenceNumber - s2sequenceNumber; - } - } - ); - } - if (fullPath) { - List fullChildList = new ArrayList(); - for (String child : childList) { - fullChildList.add(path + "/" + child); - } - return fullChildList; + List childList = getChildren(path, false); + for (String child : childList) { + deleteExt(path + "/" + child, -1, true); + } + + delete(path, version); + } + + /** + * Get the children of the path with extensions. + * Extension 1: Sort the children based on sequence number + * Extension 2: Get the full path instead of relative path + * + * @param path path to znode + * @param watch set the watch? + * @param sequenceSorted sort by the sequence number + * @param fullPath if true, get the fully znode path back + * @return list of children + * @throws InterruptedException + * @throws KeeperException + */ + public List getChildrenExt( + final String path, + boolean watch, + boolean sequenceSorted, + boolean fullPath) throws KeeperException, InterruptedException { + List childList = getChildren(path, watch); + /* Sort children according to the sequence number, if desired */ + if (sequenceSorted) { + Collections.sort(childList, new Comparator() { + public int compare(String s1, String s2) { + if ((s1.length() <= SEQUENCE_NUMBER_LENGTH) || + (s2.length() <= SEQUENCE_NUMBER_LENGTH)) { + throw new RuntimeException( + "getChildrenExt: Invalid length for sequence " + + " sorting > " + + SEQUENCE_NUMBER_LENGTH + + " for s1 (" + + s1.length() + ") or s2 (" + s2.length() + ")"); + } + int s1sequenceNumber = Integer.parseInt( + s1.substring(s1.length() - + SEQUENCE_NUMBER_LENGTH)); + int s2sequenceNumber = Integer.parseInt( + s2.substring(s2.length() - + SEQUENCE_NUMBER_LENGTH)); + return s1sequenceNumber - s2sequenceNumber; } - return childList; + }); + } + if (fullPath) { + List fullChildList = new ArrayList(); + for (String child : childList) { + fullChildList.add(path + "/" + child); + } + return fullChildList; } + return childList; + } } diff --git a/src/main/java/org/apache/giraph/zk/ZooKeeperManager.java b/src/main/java/org/apache/giraph/zk/ZooKeeperManager.java index 27447bc4a..9bdb9f430 100644 --- a/src/main/java/org/apache/giraph/zk/ZooKeeperManager.java +++ b/src/main/java/org/apache/giraph/zk/ZooKeeperManager.java @@ -56,80 +56,127 @@ * etc. */ public class ZooKeeperManager { - /** Job context (mainly for progress) */ - private Mapper.Context context; - /** Hadoop configuration */ - private final Configuration conf; - /** Class logger */ - private static final Logger LOG = Logger.getLogger(ZooKeeperManager.class); - /** Task partition, to ensure uniqueness */ - private final int taskPartition; - /** HDFS base directory for all file-based coordination */ - private final Path baseDirectory; - /** - * HDFS task ZooKeeper candidate/completed - * directory for all file-based coordination - */ - private final Path taskDirectory; - /** - * HDFS ZooKeeper server ready/done directory - * for all file-based coordination - */ - private final Path serverDirectory; - /** HDFS path to whether the task is done */ - private final Path myClosedPath; - /** Polling msecs timeout */ - private final int pollMsecs; - /** Server count */ - private final int serverCount; - /** File system */ - private final FileSystem fs; - /** ZooKeeper process */ - private Process zkProcess = null; - /** Thread that gets the zkProcess output */ - private StreamCollector zkProcessCollector = null; - /** ZooKeeper local file system directory */ - private String zkDir = null; - /** ZooKeeper config file path */ - private String configFilePath = null; - /** ZooKeeper server list */ - private final Map zkServerPortMap = Maps.newTreeMap(); - /** ZooKeeper base port */ - private int zkBasePort = -1; - /** Final ZooKeeper server port list (for clients) */ - private String zkServerPortString; - /** My hostname */ - private String myHostname = null; - /** Job id, to ensure uniqueness */ - private final String jobId; - /** - * Default local ZooKeeper prefix directory to use (where ZooKeeper server - * files will go) - */ - private final String zkDirDefault; - - - /** Separates the hostname and task in the candidate stamp */ - private static final String HOSTNAME_TASK_SEPARATOR = " "; - /** The ZooKeeperString filename prefix */ - private static final String ZOOKEEPER_SERVER_LIST_FILE_PREFIX = - "zkServerList_"; - /** Denotes that the computation is done for a partition */ - private static final String COMPUTATION_DONE_SUFFIX = ".COMPUTATION_DONE"; - /** State of the application */ - public enum State { - FAILED, - FINISHED - } + /** Class logger */ + private static final Logger LOG = Logger.getLogger(ZooKeeperManager.class); + /** Separates the hostname and task in the candidate stamp */ + private static final String HOSTNAME_TASK_SEPARATOR = " "; + /** The ZooKeeperString filename prefix */ + private static final String ZOOKEEPER_SERVER_LIST_FILE_PREFIX = + "zkServerList_"; + /** Denotes that the computation is done for a partition */ + private static final String COMPUTATION_DONE_SUFFIX = ".COMPUTATION_DONE"; + /** Job context (mainly for progress) */ + private Mapper.Context context; + /** Hadoop configuration */ + private final Configuration conf; + /** Task partition, to ensure uniqueness */ + private final int taskPartition; + /** HDFS base directory for all file-based coordination */ + private final Path baseDirectory; + /** + * HDFS task ZooKeeper candidate/completed + * directory for all file-based coordination + */ + private final Path taskDirectory; + /** + * HDFS ZooKeeper server ready/done directory + * for all file-based coordination + */ + private final Path serverDirectory; + /** HDFS path to whether the task is done */ + private final Path myClosedPath; + /** Polling msecs timeout */ + private final int pollMsecs; + /** Server count */ + private final int serverCount; + /** File system */ + private final FileSystem fs; + /** ZooKeeper process */ + private Process zkProcess = null; + /** Thread that gets the zkProcess output */ + private StreamCollector zkProcessCollector = null; + /** ZooKeeper local file system directory */ + private String zkDir = null; + /** ZooKeeper config file path */ + private String configFilePath = null; + /** ZooKeeper server list */ + private final Map zkServerPortMap = Maps.newTreeMap(); + /** ZooKeeper base port */ + private int zkBasePort = -1; + /** Final ZooKeeper server port list (for clients) */ + private String zkServerPortString; + /** My hostname */ + private String myHostname = null; + /** Job id, to ensure uniqueness */ + private final String jobId; + /** + * Default local ZooKeeper prefix directory to use (where ZooKeeper server + * files will go) + */ + private final String zkDirDefault; - /** - * Generate the final ZooKeeper coordination directory on HDFS - * - * @return directory path with job id - */ - final private String getFinalZooKeeperPath() { - return GiraphJob.ZOOKEEPER_MANAGER_DIR_DEFAULT + "/" + jobId; + /** State of the application */ + public enum State { + /** Failure occurred */ + FAILED, + /** Application finished */ + FINISHED + } + + /** + * Constructor with context. + * + * @param context Context to be stord internally + * @throws IOException + */ + public ZooKeeperManager(Mapper.Context context) + throws IOException { + this.context = context; + conf = context.getConfiguration(); + taskPartition = conf.getInt("mapred.task.partition", -1); + jobId = conf.get("mapred.job.id", "Unknown Job"); + baseDirectory = + new Path(conf.get(GiraphJob.ZOOKEEPER_MANAGER_DIRECTORY, + getFinalZooKeeperPath())); + taskDirectory = new Path(baseDirectory, + "_task"); + serverDirectory = new Path(baseDirectory, + "_zkServer"); + myClosedPath = new Path(taskDirectory, + Integer.toString(taskPartition) + + COMPUTATION_DONE_SUFFIX); + pollMsecs = conf.getInt( + GiraphJob.ZOOKEEPER_SERVERLIST_POLL_MSECS, + GiraphJob.ZOOKEEPER_SERVERLIST_POLL_MSECS_DEFAULT); + serverCount = conf.getInt( + GiraphJob.ZOOKEEPER_SERVER_COUNT, + GiraphJob.ZOOKEEPER_SERVER_COUNT_DEFAULT); + String jobLocalDir = conf.get("job.local.dir"); + if (jobLocalDir != null) { // for non-local jobs + zkDirDefault = jobLocalDir + + "/_bspZooKeeper"; + } else { + zkDirDefault = System.getProperty("user.dir") + "/_bspZooKeeper"; } + zkDir = conf.get(GiraphJob.ZOOKEEPER_DIR, zkDirDefault); + configFilePath = zkDir + "/zoo.cfg"; + zkBasePort = conf.getInt( + GiraphJob.ZOOKEEPER_SERVER_PORT, + GiraphJob.ZOOKEEPER_SERVER_PORT_DEFAULT); + + + myHostname = InetAddress.getLocalHost().getCanonicalHostName(); + fs = FileSystem.get(conf); + } + + /** + * Generate the final ZooKeeper coordination directory on HDFS + * + * @return directory path with job id + */ + private String getFinalZooKeeperPath() { + return GiraphJob.ZOOKEEPER_MANAGER_DIR_DEFAULT + "/" + jobId; + } /** * Return the base ZooKeeper ZNode from which all other ZNodes Giraph creates @@ -139,708 +186,668 @@ final private String getFinalZooKeeperPath() { * @param conf Necessary to access user-provided values * @return String of path without trailing slash */ - public static String getBasePath(Configuration conf) { - String result = conf.get(BASE_ZNODE_KEY, ""); - if (!result.equals("") && !result.startsWith("/")) { - throw new IllegalArgumentException("Value for " + - BASE_ZNODE_KEY + " must start with /: " + result); - } - - return result; + public static String getBasePath(Configuration conf) { + String result = conf.get(BASE_ZNODE_KEY, ""); + if (!result.equals("") && !result.startsWith("/")) { + throw new IllegalArgumentException("Value for " + + BASE_ZNODE_KEY + " must start with /: " + result); } + return result; + } + + /** + * Collects the output of a stream and dumps it to the log. + */ + private static class StreamCollector extends Thread { + /** Class logger */ + private static final Logger LOG = Logger.getLogger(StreamCollector.class); + /** Input stream to dump */ + private final InputStream is; + /** - * Collects the output of a stream and dumps it to the log. + * Constructor. + * + * @param is InputStream to dump to LOG.info */ - private static class StreamCollector extends Thread { - /** Input stream to dump */ - private final InputStream is; - /** Class logger */ - private static final Logger LOG = - Logger.getLogger(StreamCollector.class); - - /** - * Constructor. - * - * @param is InputStream to dump to LOG.info - */ - public StreamCollector(final InputStream is) { - super(StreamCollector.class.getName()); - this.is = is; - } + public StreamCollector(final InputStream is) { + super(StreamCollector.class.getName()); + this.is = is; + } - @Override - public void run() { - InputStreamReader streamReader = new InputStreamReader(is); - BufferedReader bufferedReader = new BufferedReader(streamReader); - String line; - try { - while ((line = bufferedReader.readLine()) != null) { - if (LOG.isDebugEnabled()) { - LOG.debug("run: " + line); - } - } - } catch (IOException e) { - LOG.error("run: Ignoring IOException", e); - } + @Override + public void run() { + InputStreamReader streamReader = new InputStreamReader(is); + BufferedReader bufferedReader = new BufferedReader(streamReader); + String line; + try { + while ((line = bufferedReader.readLine()) != null) { + if (LOG.isDebugEnabled()) { + LOG.debug("run: " + line); + } } + } catch (IOException e) { + LOG.error("run: Ignoring IOException", e); + } } + } - public ZooKeeperManager(Mapper.Context context) - throws IOException { - this.context = context; - conf = context.getConfiguration(); - taskPartition = conf.getInt("mapred.task.partition", -1); - jobId = conf.get("mapred.job.id", "Unknown Job"); - baseDirectory = - new Path(conf.get(GiraphJob.ZOOKEEPER_MANAGER_DIRECTORY, - getFinalZooKeeperPath())); - taskDirectory = new Path(baseDirectory, - "_task"); - serverDirectory = new Path(baseDirectory, - "_zkServer"); - myClosedPath = new Path(taskDirectory, - Integer.toString(taskPartition) + - COMPUTATION_DONE_SUFFIX); - pollMsecs = conf.getInt( - GiraphJob.ZOOKEEPER_SERVERLIST_POLL_MSECS, - GiraphJob.ZOOKEEPER_SERVERLIST_POLL_MSECS_DEFAULT); - serverCount = conf.getInt( - GiraphJob.ZOOKEEPER_SERVER_COUNT, - GiraphJob.ZOOKEEPER_SERVER_COUNT_DEFAULT); - String jobLocalDir = conf.get("job.local.dir"); - if (jobLocalDir != null) { // for non-local jobs - zkDirDefault = jobLocalDir + - "/_bspZooKeeper"; - } else { - zkDirDefault = System.getProperty("user.dir") + "/_bspZooKeeper"; - } - zkDir = conf.get(GiraphJob.ZOOKEEPER_DIR, zkDirDefault); - configFilePath = zkDir + "/zoo.cfg"; - zkBasePort = conf.getInt( - GiraphJob.ZOOKEEPER_SERVER_PORT, - GiraphJob.ZOOKEEPER_SERVER_PORT_DEFAULT); + /** + * Create the candidate stamps and decide on the servers to start if + * you are partition 0. + * + * @throws IOException + * @throws InterruptedException + */ + public void setup() throws IOException, InterruptedException { + createCandidateStamp(); + getZooKeeperServerList(); + } + + /** + * Create a HDFS stamp for this task. If another task already + * created it, then this one will fail, which is fine. + */ + public void createCandidateStamp() { + try { + fs.mkdirs(baseDirectory); + LOG.info("createCandidateStamp: Made the directory " + + baseDirectory); + } catch (IOException e) { + LOG.error("createCandidateStamp: Failed to mkdirs " + + baseDirectory); + } + // Check that the base directory exists and is a directory + try { + if (!fs.getFileStatus(baseDirectory).isDir()) { + throw new IllegalArgumentException( + "createCandidateStamp: " + baseDirectory + + " is not a directory, but should be."); + } + } catch (IOException e) { + throw new IllegalArgumentException( + "createCandidateStamp: Couldn't get file status " + + "for base directory " + baseDirectory + ". If there is an " + + "issue with this directory, please set an accesible " + + "base directory with the Hadoop configuration option " + + GiraphJob.ZOOKEEPER_MANAGER_DIRECTORY); + } + Path myCandidacyPath = new Path( + taskDirectory, myHostname + + HOSTNAME_TASK_SEPARATOR + taskPartition); + try { + if (LOG.isInfoEnabled()) { + LOG.info("createCandidateStamp: Creating my filestamp " + + myCandidacyPath); + } + fs.createNewFile(myCandidacyPath); + } catch (IOException e) { + LOG.error("createCandidateStamp: Failed (maybe previous task " + + "failed) to create filestamp " + myCandidacyPath, e); + } + } - myHostname = InetAddress.getLocalHost().getCanonicalHostName(); - fs = FileSystem.get(conf); + /** + * Every task must create a stamp to let the ZooKeeper servers know that + * they can shutdown. This also lets the task know that it was already + * completed. + */ + private void createZooKeeperClosedStamp() { + try { + LOG.info("createZooKeeperClosedStamp: Creating my filestamp " + + myClosedPath); + fs.createNewFile(myClosedPath); + } catch (IOException e) { + LOG.error("createZooKeeperClosedStamp: Failed (maybe previous task " + + "failed) to create filestamp " + myClosedPath); } + } - /** - * Create the candidate stamps and decide on the servers to start if - * you are partition 0. - * - * @throws IOException - * @throws InterruptedException - */ - public void setup() throws IOException, InterruptedException { - createCandidateStamp(); - getZooKeeperServerList(); + /** + * Check if all the computation is done. + * @return true if all computation is done. + */ + public boolean computationDone() { + try { + return fs.exists(myClosedPath); + } catch (IOException e) { + throw new RuntimeException(e); } + } - /** - * Create a HDFS stamp for this task. If another task already - * created it, then this one will fail, which is fine. - */ - public void createCandidateStamp() { - try { - fs.mkdirs(baseDirectory); - LOG.info("createCandidateStamp: Made the directory " + - baseDirectory); - } catch (IOException e) { - LOG.error("createCandidateStamp: Failed to mkdirs " + - baseDirectory); + /** + * Task 0 will call this to create the ZooKeeper server list. The result is + * a file that describes the ZooKeeper servers through the filename. + * + * @throws IOException + * @throws InterruptedException + */ + private void createZooKeeperServerList() throws IOException, + InterruptedException { + int candidateRetrievalAttempt = 0; + Map hostnameTaskMap = Maps.newTreeMap(); + while (true) { + FileStatus [] fileStatusArray = fs.listStatus(taskDirectory); + hostnameTaskMap.clear(); + if (fileStatusArray.length > 0) { + for (FileStatus fileStatus : fileStatusArray) { + String[] hostnameTaskArray = + fileStatus.getPath().getName().split( + HOSTNAME_TASK_SEPARATOR); + if (hostnameTaskArray.length != 2) { + throw new RuntimeException( + "getZooKeeperServerList: Task 0 failed " + + "to parse " + + fileStatus.getPath().getName()); + } + if (!hostnameTaskMap.containsKey(hostnameTaskArray[0])) { + hostnameTaskMap.put(hostnameTaskArray[0], + new Integer(hostnameTaskArray[1])); + } } - // Check that the base directory exists and is a directory - try { - if (!fs.getFileStatus(baseDirectory).isDir()) { - throw new IllegalArgumentException( - "createCandidateStamp: " + baseDirectory + - " is not a directory, but should be."); - } - } catch (IOException e) { - throw new IllegalArgumentException( - "createCandidateStamp: Couldn't get file status " + - "for base directory " + baseDirectory + ". If there is an " + - "issue with this directory, please set an accesible " + - "base directory with the Hadoop configuration option " + - GiraphJob.ZOOKEEPER_MANAGER_DIRECTORY); + if (LOG.isInfoEnabled()) { + LOG.info("getZooKeeperServerList: Got " + + hostnameTaskMap.keySet() + " " + + hostnameTaskMap.size() + " hosts from " + + fileStatusArray.length + " candidates when " + + serverCount + " required (polling period is " + + pollMsecs + ") on attempt " + + candidateRetrievalAttempt); } - Path myCandidacyPath = new Path( - taskDirectory, myHostname + - HOSTNAME_TASK_SEPARATOR + taskPartition); - try { - if (LOG.isInfoEnabled()) { - LOG.info("createCandidateStamp: Creating my filestamp " + - myCandidacyPath); - } - fs.createNewFile(myCandidacyPath); - } catch (IOException e) { - LOG.error("createCandidateStamp: Failed (maybe previous task " + - "failed) to create filestamp " + myCandidacyPath, e); + if (hostnameTaskMap.size() >= serverCount) { + break; } + ++candidateRetrievalAttempt; + Thread.sleep(pollMsecs); + } + } + StringBuffer serverListFile = + new StringBuffer(ZOOKEEPER_SERVER_LIST_FILE_PREFIX); + int numServers = 0; + for (Map.Entry hostnameTask : + hostnameTaskMap.entrySet()) { + serverListFile.append(hostnameTask.getKey() + + HOSTNAME_TASK_SEPARATOR + hostnameTask.getValue() + + HOSTNAME_TASK_SEPARATOR); + if (++numServers == serverCount) { + break; + } } + Path serverListPath = + new Path(baseDirectory, serverListFile.toString()); + if (LOG.isInfoEnabled()) { + LOG.info("createZooKeeperServerList: Creating the final " + + "ZooKeeper file '" + serverListPath + "'"); + } + fs.createNewFile(serverListPath); + } - /** - * Every task must create a stamp to let the ZooKeeper servers know that - * they can shutdown. This also lets the task know that it was already - * completed. - */ - private void createZooKeeperClosedStamp() { - try { - LOG.info("createZooKeeperClosedStamp: Creating my filestamp " + - myClosedPath); - fs.createNewFile(myClosedPath); - } catch (IOException e) { - LOG.error("createZooKeeperClosedStamp: Failed (maybe previous task " + - "failed) to create filestamp " + myClosedPath); - } + /** + * Make an attempt to get the server list file by looking for a file in + * the appropriate directory with the prefix + * ZOOKEEPER_SERVER_LIST_FILE_PREFIX. + * @return null if not found or the filename if found + * @throws IOException + */ + private String getServerListFile() throws IOException { + String serverListFile = null; + FileStatus [] fileStatusArray = fs.listStatus(baseDirectory); + for (FileStatus fileStatus : fileStatusArray) { + if (fileStatus.getPath().getName().startsWith( + ZOOKEEPER_SERVER_LIST_FILE_PREFIX)) { + serverListFile = fileStatus.getPath().getName(); + break; + } } + return serverListFile; + } - /** - * Check if all the computation is done. - * @return true if all computation is done. - */ - public boolean computationDone() { - try { - return fs.exists(myClosedPath); - } catch (IOException e) { - throw new RuntimeException(e); - } + /** + * Task 0 is the designated master and will generate the server list + * (unless it has already done so). Other + * tasks will consume the file after it is created (just the filename). + * @throws IOException + * @throws InterruptedException + */ + private void getZooKeeperServerList() throws IOException, + InterruptedException { + String serverListFile; + + if (taskPartition == 0) { + serverListFile = getServerListFile(); + if (serverListFile == null) { + createZooKeeperServerList(); + } } - /** - * Task 0 will call this to create the ZooKeeper server list. The result is - * a file that describes the ZooKeeper servers through the filename. - * - * @throws IOException - * @throws InterruptedException - */ - private void createZooKeeperServerList() - throws IOException, InterruptedException { - int candidateRetrievalAttempt = 0; - Map hostnameTaskMap = Maps.newTreeMap(); - while (true) { - FileStatus [] fileStatusArray = fs.listStatus(taskDirectory); - hostnameTaskMap.clear(); - if (fileStatusArray.length > 0) { - for (FileStatus fileStatus : fileStatusArray) { - String[] hostnameTaskArray = - fileStatus.getPath().getName().split( - HOSTNAME_TASK_SEPARATOR); - if (hostnameTaskArray.length != 2) { - throw new RuntimeException( - "getZooKeeperServerList: Task 0 failed " + - "to parse " + - fileStatus.getPath().getName()); - } - if (!hostnameTaskMap.containsKey(hostnameTaskArray[0])) { - hostnameTaskMap.put(hostnameTaskArray[0], - new Integer(hostnameTaskArray[1])); - } - } - if (LOG.isInfoEnabled()) { - LOG.info("getZooKeeperServerList: Got " + - hostnameTaskMap.keySet() + " " + - hostnameTaskMap.size() + " hosts from " + - fileStatusArray.length + " candidates when " + - serverCount + " required (polling period is " + - pollMsecs + ") on attempt " + - candidateRetrievalAttempt); - } - - if (hostnameTaskMap.size() >= serverCount) { - break; - } - ++candidateRetrievalAttempt; - Thread.sleep(pollMsecs); - } - } - StringBuffer serverListFile = - new StringBuffer(ZOOKEEPER_SERVER_LIST_FILE_PREFIX); - int numServers = 0; - for (Map.Entry hostnameTask : - hostnameTaskMap.entrySet()) { - serverListFile.append(hostnameTask.getKey() + - HOSTNAME_TASK_SEPARATOR + hostnameTask.getValue() + - HOSTNAME_TASK_SEPARATOR); - if (++numServers == serverCount) { - break; - } - } - Path serverListPath = - new Path(baseDirectory, serverListFile.toString()); - if (LOG.isInfoEnabled()) { - LOG.info("createZooKeeperServerList: Creating the final " + - "ZooKeeper file '" + serverListPath + "'"); - } - fs.createNewFile(serverListPath); + while (true) { + serverListFile = getServerListFile(); + if (LOG.isInfoEnabled()) { + LOG.info("getZooKeeperServerList: For task " + taskPartition + + ", got file '" + serverListFile + + "' (polling period is " + + pollMsecs + ")"); + } + if (serverListFile != null) { + break; + } + try { + Thread.sleep(pollMsecs); + } catch (InterruptedException e) { + LOG.warn("getZooKeeperServerList: Strange interrupted " + + "exception " + e.getMessage()); + } + } - /** - * Make an attempt to get the server list file by looking for a file in - * the appropriate directory with the prefix - * ZOOKEEPER_SERVER_LIST_FILE_PREFIX. - * @return null if not found or the filename if found - * @throws IOException - */ - private String getServerListFile() throws IOException { - String serverListFile = null; - FileStatus [] fileStatusArray = fs.listStatus(baseDirectory); - for (FileStatus fileStatus : fileStatusArray) { - if (fileStatus.getPath().getName().startsWith( - ZOOKEEPER_SERVER_LIST_FILE_PREFIX)) { - serverListFile = fileStatus.getPath().getName(); - break; - } - } - return serverListFile; + List serverHostList = Arrays.asList(serverListFile.substring( + ZOOKEEPER_SERVER_LIST_FILE_PREFIX.length()).split( + HOSTNAME_TASK_SEPARATOR)); + if (LOG.isInfoEnabled()) { + LOG.info("getZooKeeperServerList: Found " + serverHostList + " " + + serverHostList.size() + + " hosts in filename '" + serverListFile + "'"); + } + if (serverHostList.size() != serverCount * 2) { + throw new IllegalStateException( + "getZooKeeperServerList: Impossible " + + " that " + serverHostList.size() + + " != 2 * " + + serverCount + " asked for."); } - /** - * Task 0 is the designated master and will generate the server list - * (unless it has already done so). Other - * tasks will consume the file after it is created (just the filename). - * @throws IOException - * @throws InterruptedException - */ - private void getZooKeeperServerList() - throws IOException, InterruptedException { - String serverListFile; - - if (taskPartition == 0) { - serverListFile = getServerListFile(); - if (serverListFile == null) { - createZooKeeperServerList(); - } - } + for (int i = 0; i < serverHostList.size(); i += 2) { + zkServerPortMap.put(serverHostList.get(i), + Integer.parseInt(serverHostList.get(i + 1))); + } + zkServerPortString = ""; + for (String server : zkServerPortMap.keySet()) { + if (zkServerPortString.length() > 0) { + zkServerPortString += ","; + } + zkServerPortString += server + ":" + zkBasePort; + } + } - while (true) { - serverListFile = getServerListFile(); - if (LOG.isInfoEnabled()) { - LOG.info("getZooKeeperServerList: For task " + taskPartition + - ", got file '" + serverListFile + - "' (polling period is " + - pollMsecs + ")"); - } - if (serverListFile != null) { - break; - } - try { - Thread.sleep(pollMsecs); - } catch (InterruptedException e) { - LOG.warn("getZooKeeperServerList: Strange interrupted " + - "exception " + e.getMessage()); - } + /** + * Users can get the server port string to connect to ZooKeeper + * @return server port string - comma separated + */ + public String getZooKeeperServerPortString() { + return zkServerPortString; + } + /** + * Whoever is elected to be a ZooKeeper server must generate a config file + * locally. + * + * @param serverList List of ZooKeeper servers. + */ + private void generateZooKeeperConfigFile(List serverList) { + if (LOG.isInfoEnabled()) { + LOG.info("generateZooKeeperConfigFile: Creating file " + + configFilePath + " in " + zkDir + " with base port " + + zkBasePort); + } + try { + File zkDirFile = new File(this.zkDir); + boolean mkDirRet = zkDirFile.mkdirs(); + if (LOG.isInfoEnabled()) { + LOG.info("generateZooKeeperConfigFile: Make directory of " + + zkDirFile.getName() + " = " + mkDirRet); + } + File configFile = new File(configFilePath); + boolean deletedRet = configFile.delete(); + if (LOG.isInfoEnabled()) { + LOG.info("generateZooKeeperConfigFile: Delete of " + + configFile.getName() + " = " + deletedRet); + } + if (!configFile.createNewFile()) { + throw new IllegalStateException( + "generateZooKeeperConfigFile: Failed to " + + "create config file " + configFile.getName()); + } + // Make writable by everybody + if (!configFile.setWritable(true, false)) { + throw new IllegalStateException( + "generateZooKeeperConfigFile: Failed to make writable " + + configFile.getName()); + } + + Writer writer = null; + try { + writer = new FileWriter(configFilePath); + writer.write("tickTime=" + + GiraphJob.DEFAULT_ZOOKEEPER_TICK_TIME + "\n"); + writer.write("dataDir=" + this.zkDir + "\n"); + writer.write("clientPort=" + zkBasePort + "\n"); + writer.write("maxClientCnxns=" + + GiraphJob.DEFAULT_ZOOKEEPER_MAX_CLIENT_CNXNS + + "\n"); + writer.write("minSessionTimeout=" + + GiraphJob.DEFAULT_ZOOKEEPER_MIN_SESSION_TIMEOUT + + "\n"); + writer.write("maxSessionTimeout=" + + GiraphJob.DEFAULT_ZOOKEEPER_MAX_SESSION_TIMEOUT + + "\n"); + writer.write("initLimit=" + + GiraphJob.DEFAULT_ZOOKEEPER_INIT_LIMIT + "\n"); + writer.write("syncLimit=" + + GiraphJob.DEFAULT_ZOOKEEPER_SYNC_LIMIT + "\n"); + writer.write("snapCount=" + + GiraphJob.DEFAULT_ZOOKEEPER_SNAP_COUNT + "\n"); + if (serverList.size() != 1) { + writer.write("electionAlg=0\n"); + for (int i = 0; i < serverList.size(); ++i) { + writer.write("server." + i + "=" + serverList.get(i) + + ":" + (zkBasePort + 1) + + ":" + (zkBasePort + 2) + "\n"); + if (myHostname.equals(serverList.get(i))) { + Writer myidWriter = null; + try { + myidWriter = new FileWriter(zkDir + "/myid"); + myidWriter.write(i + "\n"); + } finally { + Closeables.closeQuietly(myidWriter); + } + } + } } + } finally { + Closeables.closeQuietly(writer); + } + } catch (IOException e) { + throw new IllegalStateException( + "generateZooKeeperConfigFile: Failed to write file", e); + } + } - List serverHostList = Arrays.asList(serverListFile.substring( - ZOOKEEPER_SERVER_LIST_FILE_PREFIX.length()).split( - HOSTNAME_TASK_SEPARATOR)); + /** + * If this task has been selected, online a ZooKeeper server. Otherwise, + * wait until this task knows that the ZooKeeper servers have been onlined. + */ + public void onlineZooKeeperServers() { + Integer taskId = zkServerPortMap.get(myHostname); + if ((taskId != null) && (taskId.intValue() == taskPartition)) { + File zkDirFile = new File(this.zkDir); + try { if (LOG.isInfoEnabled()) { - LOG.info("getZooKeeperServerList: Found " + serverHostList + " " + - serverHostList.size() + - " hosts in filename '" + serverListFile + "'"); + LOG.info("onlineZooKeeperServers: Trying to delete old " + + "directory " + this.zkDir); } - if (serverHostList.size() != serverCount * 2) { - throw new IllegalStateException( - "getZooKeeperServerList: Impossible " + - " that " + serverHostList.size() + - " != 2 * " + - serverCount + " asked for."); + FileUtils.deleteDirectory(zkDirFile); + } catch (IOException e) { + LOG.warn("onlineZooKeeperServers: Failed to delete " + + "directory " + this.zkDir, e); + } + generateZooKeeperConfigFile( + new ArrayList(zkServerPortMap.keySet())); + ProcessBuilder processBuilder = new ProcessBuilder(); + List commandList = Lists.newArrayList(); + String javaHome = System.getProperty("java.home"); + if (javaHome == null) { + throw new IllegalArgumentException( + "onlineZooKeeperServers: java.home is not set!"); + } + commandList.add(javaHome + "/bin/java"); + String zkJavaOptsString = + conf.get(GiraphJob.ZOOKEEPER_JAVA_OPTS, + GiraphJob.ZOOKEEPER_JAVA_OPTS_DEFAULT); + String[] zkJavaOptsArray = zkJavaOptsString.split(" "); + if (zkJavaOptsArray != null) { + for (String javaOpt : zkJavaOptsArray) { + commandList.add(javaOpt); } - - for (int i = 0; i < serverHostList.size(); i += 2) { - zkServerPortMap.put(serverHostList.get(i), - Integer.parseInt(serverHostList.get(i+1))); + } + commandList.add("-cp"); + Path fullJarPath = new Path(conf.get(GiraphJob.ZOOKEEPER_JAR)); + commandList.add(fullJarPath.toString()); + commandList.add(QuorumPeerMain.class.getName()); + commandList.add(configFilePath); + processBuilder.command(commandList); + File execDirectory = new File(zkDir); + processBuilder.directory(execDirectory); + processBuilder.redirectErrorStream(true); + if (LOG.isInfoEnabled()) { + LOG.info("onlineZooKeeperServers: Attempting to " + + "start ZooKeeper server with command " + commandList + + " in directory " + execDirectory.toString()); + } + try { + synchronized (this) { + zkProcess = processBuilder.start(); + zkProcessCollector = + new StreamCollector(zkProcess.getInputStream()); + zkProcessCollector.start(); } - zkServerPortString = ""; - for (String server : zkServerPortMap.keySet()) { - if (zkServerPortString.length() > 0) { - zkServerPortString += ","; + Runnable runnable = new Runnable() { + public void run() { + synchronized (this) { + if (zkProcess != null) { + LOG.warn("onlineZooKeeperServers: " + + "Forced a shutdown hook kill of the " + + "ZooKeeper process."); + zkProcess.destroy(); + } } - zkServerPortString += server + ":" + zkBasePort; + } + }; + Runtime.getRuntime().addShutdownHook(new Thread(runnable)); + } catch (IOException e) { + LOG.error("onlineZooKeeperServers: Failed to start " + + "ZooKeeper process", e); + throw new RuntimeException(e); + } + + // Once the server is up and running, notify that this server is up + // and running by dropping a ready stamp. + int connectAttempts = 0; + final int maxConnectAttempts = 10; + while (connectAttempts < maxConnectAttempts) { + try { + if (LOG.isInfoEnabled()) { + LOG.info("onlineZooKeeperServers: Connect attempt " + + connectAttempts + " of " + + maxConnectAttempts + + " max trying to connect to " + + myHostname + ":" + zkBasePort + + " with poll msecs = " + pollMsecs); + } + InetSocketAddress zkServerAddress = + new InetSocketAddress(myHostname, zkBasePort); + Socket testServerSock = new Socket(); + testServerSock.connect(zkServerAddress, 5000); + if (LOG.isInfoEnabled()) { + LOG.info("onlineZooKeeperServers: Connected to " + + zkServerAddress + "!"); + } + break; + } catch (SocketTimeoutException e) { + LOG.warn("onlineZooKeeperServers: Got " + + "SocketTimeoutException", e); + } catch (ConnectException e) { + LOG.warn("onlineZooKeeperServers: Got " + + "ConnectException", e); + } catch (IOException e) { + LOG.warn("onlineZooKeeperServers: Got " + + "IOException", e); } - } - - /** - * Users can get the server port string to connect to ZooKeeper - * @return server port string - comma separated - */ - public String getZooKeeperServerPortString() { - return zkServerPortString; - } - /** - * Whoever is elected to be a ZooKeeper server must generate a config file - * locally. - */ - private void generateZooKeeperConfigFile(List serverList) { + ++connectAttempts; + try { + Thread.sleep(pollMsecs); + } catch (InterruptedException e) { + LOG.warn("onlineZooKeeperServers: Sleep of " + pollMsecs + + " interrupted - " + e.getMessage()); + } + } + if (connectAttempts == maxConnectAttempts) { + throw new IllegalStateException( + "onlineZooKeeperServers: Failed to connect in " + + connectAttempts + " tries!"); + } + Path myReadyPath = new Path( + serverDirectory, myHostname + + HOSTNAME_TASK_SEPARATOR + taskPartition); + try { if (LOG.isInfoEnabled()) { - LOG.info("generateZooKeeperConfigFile: Creating file " + - configFilePath + " in " + zkDir + " with base port " + - zkBasePort); + LOG.info("onlineZooKeeperServers: Creating my filestamp " + + myReadyPath); } + fs.createNewFile(myReadyPath); + } catch (IOException e) { + LOG.error("onlineZooKeeperServers: Failed (maybe previous " + + "task failed) to create filestamp " + myReadyPath, e); + } + } else { + List foundList = new ArrayList(); + int readyRetrievalAttempt = 0; + while (true) { try { - File zkDirFile = new File(this.zkDir); - boolean mkDirRet = zkDirFile.mkdirs(); - if (LOG.isInfoEnabled()) { - LOG.info("generateZooKeeperConfigFile: Make directory of " + - zkDirFile.getName() + " = " + mkDirRet); + FileStatus [] fileStatusArray = + fs.listStatus(serverDirectory); + foundList.clear(); + if ((fileStatusArray != null) && + (fileStatusArray.length > 0)) { + for (int i = 0; i < fileStatusArray.length; ++i) { + String[] hostnameTaskArray = + fileStatusArray[i].getPath().getName().split( + HOSTNAME_TASK_SEPARATOR); + if (hostnameTaskArray.length != 2) { + throw new RuntimeException( + "getZooKeeperServerList: Task 0 failed " + + "to parse " + + fileStatusArray[i].getPath().getName()); + } + foundList.add(hostnameTaskArray[0]); } - File configFile = new File(configFilePath); - boolean deletedRet = configFile.delete(); if (LOG.isInfoEnabled()) { - LOG.info("generateZooKeeperConfigFile: Delete of " + - configFile.getName() + " = " + deletedRet); + LOG.info("onlineZooKeeperServers: Got " + + foundList + " " + + foundList.size() + " hosts from " + + fileStatusArray.length + + " ready servers when " + + serverCount + + " required (polling period is " + + pollMsecs + ") on attempt " + + readyRetrievalAttempt); } - if (!configFile.createNewFile()) { - throw new IllegalStateException( - "generateZooKeeperConfigFile: Failed to " + - "create config file " + configFile.getName()); + if (foundList.containsAll(zkServerPortMap.keySet())) { + break; } - // Make writable by everybody - if (!configFile.setWritable(true, false)) { - throw new IllegalStateException( - "generateZooKeeperConfigFile: Failed to make writable " + - configFile.getName()); - } - - Writer writer = null; - try { - writer = new FileWriter(configFilePath); - writer.write("tickTime=" + - GiraphJob.DEFAULT_ZOOKEEPER_TICK_TIME + "\n"); - writer.write("dataDir=" + this.zkDir + "\n"); - writer.write("clientPort=" + zkBasePort + "\n"); - writer.write("maxClientCnxns=" + - GiraphJob.DEFAULT_ZOOKEEPER_MAX_CLIENT_CNXNS + - "\n"); - writer.write("minSessionTimeout=" + - GiraphJob.DEFAULT_ZOOKEEPER_MIN_SESSION_TIMEOUT + - "\n"); - writer.write("maxSessionTimeout=" + - GiraphJob.DEFAULT_ZOOKEEPER_MAX_SESSION_TIMEOUT + - "\n"); - writer.write("initLimit=" + - GiraphJob.DEFAULT_ZOOKEEPER_INIT_LIMIT + "\n"); - writer.write("syncLimit=" + - GiraphJob.DEFAULT_ZOOKEEPER_SYNC_LIMIT + "\n"); - writer.write("snapCount=" + - GiraphJob.DEFAULT_ZOOKEEPER_SNAP_COUNT + "\n"); - if (serverList.size() != 1) { - writer.write("electionAlg=0\n"); - for (int i = 0; i < serverList.size(); ++i) { - writer.write("server." + i + "=" + serverList.get(i) + - ":" + (zkBasePort + 1) + - ":" + (zkBasePort + 2) + "\n"); - if (myHostname.equals(serverList.get(i))) { - Writer myidWriter = null; - try { - myidWriter = new FileWriter(zkDir + "/myid"); - myidWriter.write(i + "\n"); - } finally { - Closeables.closeQuietly(myidWriter); - } - } - } - } - } finally { - Closeables.closeQuietly(writer); + } else { + if (LOG.isInfoEnabled()) { + LOG.info("onlineZooKeeperSErvers: Empty " + + "directory " + serverDirectory + + ", waiting " + pollMsecs + " msecs."); } + } + Thread.sleep(pollMsecs); + ++readyRetrievalAttempt; } catch (IOException e) { - throw new IllegalStateException( - "generateZooKeeperConfigFile: Failed to write file", e); + throw new RuntimeException(e); + } catch (InterruptedException e) { + LOG.warn("onlineZooKeeperServers: Strange interrupt from " + + e.getMessage(), e); } + } } + } - /** - * If this task has been selected, online a ZooKeeper server. Otherwise, - * wait until this task knows that the ZooKeeper servers have been onlined. - */ - public void onlineZooKeeperServers() { - Integer taskId = zkServerPortMap.get(myHostname); - if ((taskId != null) && (taskId.intValue() == taskPartition)) { - File zkDirFile = new File(this.zkDir); - try { - if (LOG.isInfoEnabled()) { - LOG.info("onlineZooKeeperServers: Trying to delete old " + - "directory " + this.zkDir); - } - FileUtils.deleteDirectory(zkDirFile); - } catch (IOException e) { - LOG.warn("onlineZooKeeperServers: Failed to delete " + - "directory " + this.zkDir, e); - } - generateZooKeeperConfigFile( - new ArrayList(zkServerPortMap.keySet())); - ProcessBuilder processBuilder = new ProcessBuilder(); - List commandList = Lists.newArrayList(); - String javaHome = System.getProperty("java.home"); - if (javaHome == null) { - throw new IllegalArgumentException( - "onlineZooKeeperServers: java.home is not set!"); - } - commandList.add(javaHome + "/bin/java"); - String zkJavaOptsString = - conf.get(GiraphJob.ZOOKEEPER_JAVA_OPTS, - GiraphJob.ZOOKEEPER_JAVA_OPTS_DEFAULT); - String[] zkJavaOptsArray = zkJavaOptsString.split(" "); - if (zkJavaOptsArray != null) { - for (String javaOpt : zkJavaOptsArray) { - commandList.add(javaOpt); - } - } - commandList.add("-cp"); - Path fullJarPath = new Path(conf.get(GiraphJob.ZOOKEEPER_JAR)); - commandList.add(fullJarPath.toString()); - commandList.add(QuorumPeerMain.class.getName()); - commandList.add(configFilePath); - processBuilder.command(commandList); - File execDirectory = new File(zkDir); - processBuilder.directory(execDirectory); - processBuilder.redirectErrorStream(true); - if (LOG.isInfoEnabled()) { - LOG.info("onlineZooKeeperServers: Attempting to " + - "start ZooKeeper server with command " + commandList + - " in directory " + execDirectory.toString()); - } - try { - synchronized (this) { - zkProcess = processBuilder.start(); - zkProcessCollector = - new StreamCollector(zkProcess.getInputStream()); - zkProcessCollector.start(); - } - Runnable runnable = new Runnable() { - public void run() { - synchronized (this) { - if (zkProcess != null) { - LOG.warn("onlineZooKeeperServers: "+ - "Forced a shutdown hook kill of the " + - "ZooKeeper process."); - zkProcess.destroy(); - } - } - } - }; - Runtime.getRuntime().addShutdownHook(new Thread(runnable)); - } catch (IOException e) { - LOG.error("onlineZooKeeperServers: Failed to start " + - "ZooKeeper process", e); - throw new RuntimeException(e); - } - - // Once the server is up and running, notify that this server is up - // and running by dropping a ready stamp. - int connectAttempts = 0; - final int maxConnectAttempts = 10; - while (connectAttempts < maxConnectAttempts) { - try { - if (LOG.isInfoEnabled()) { - LOG.info("onlineZooKeeperServers: Connect attempt " + - connectAttempts + " of " + - maxConnectAttempts + - " max trying to connect to " + - myHostname + ":" + zkBasePort + - " with poll msecs = " + pollMsecs); - } - InetSocketAddress zkServerAddress = - new InetSocketAddress(myHostname, zkBasePort); - Socket testServerSock = new Socket(); - testServerSock.connect(zkServerAddress, 5000); - if (LOG.isInfoEnabled()) { - LOG.info("onlineZooKeeperServers: Connected to " + - zkServerAddress + "!"); - } - break; - } catch (SocketTimeoutException e) { - LOG.warn("onlineZooKeeperServers: Got " + - "SocketTimeoutException", e); - } catch (ConnectException e) { - LOG.warn("onlineZooKeeperServers: Got " + - "ConnectException", e); - } catch (IOException e) { - LOG.warn("onlineZooKeeperServers: Got " + - "IOException", e); - } - - ++connectAttempts; - try { - Thread.sleep(pollMsecs); - } catch (InterruptedException e) { - LOG.warn("onlineZooKeeperServers: Sleep of " + pollMsecs + - " interrupted - " + e.getMessage()); - } - } - if (connectAttempts == maxConnectAttempts) { - throw new IllegalStateException( - "onlineZooKeeperServers: Failed to connect in " + - connectAttempts + " tries!"); - } - Path myReadyPath = new Path( - serverDirectory, myHostname + - HOSTNAME_TASK_SEPARATOR + taskPartition); - try { - if (LOG.isInfoEnabled()) { - LOG.info("onlineZooKeeperServers: Creating my filestamp " + - myReadyPath); - } - fs.createNewFile(myReadyPath); - } catch (IOException e) { - LOG.error("onlineZooKeeperServers: Failed (maybe previous " + - "task failed) to create filestamp " + myReadyPath, e); + /** + * Wait for all map tasks to signal completion. + * + * @param totalMapTasks Number of map tasks to wait for + */ + private void waitUntilAllTasksDone(int totalMapTasks) { + int attempt = 0; + while (true) { + try { + FileStatus [] fileStatusArray = + fs.listStatus(taskDirectory); + int totalDone = 0; + if (fileStatusArray.length > 0) { + for (int i = 0; i < fileStatusArray.length; ++i) { + if (fileStatusArray[i].getPath().getName().endsWith( + COMPUTATION_DONE_SUFFIX)) { + ++totalDone; } + } } - else { - List foundList = new ArrayList(); - int readyRetrievalAttempt = 0; - while (true) { - try { - FileStatus [] fileStatusArray = - fs.listStatus(serverDirectory); - foundList.clear(); - if ((fileStatusArray != null) && - (fileStatusArray.length > 0)) { - for (int i = 0; i < fileStatusArray.length; ++i) { - String[] hostnameTaskArray = - fileStatusArray[i].getPath().getName().split( - HOSTNAME_TASK_SEPARATOR); - if (hostnameTaskArray.length != 2) { - throw new RuntimeException( - "getZooKeeperServerList: Task 0 failed " + - "to parse " + - fileStatusArray[i].getPath().getName()); - } - foundList.add(hostnameTaskArray[0]); - } - if (LOG.isInfoEnabled()) { - LOG.info("onlineZooKeeperServers: Got " + - foundList + " " + - foundList.size() + " hosts from " + - fileStatusArray.length + - " ready servers when " + - serverCount + - " required (polling period is " + - pollMsecs + ") on attempt " + - readyRetrievalAttempt); - } - if (foundList.containsAll(zkServerPortMap.keySet())) { - break; - } - } else { - if (LOG.isInfoEnabled()) { - LOG.info("onlineZooKeeperSErvers: Empty " + - "directory " + serverDirectory + - ", waiting " + pollMsecs + " msecs."); - } - } - Thread.sleep(pollMsecs); - ++readyRetrievalAttempt; - } catch (IOException e) { - throw new RuntimeException(e); - } catch (InterruptedException e) { - LOG.warn("onlineZooKeeperServers: Strange interrupt from " + - e.getMessage(), e); - } - } + if (LOG.isInfoEnabled()) { + LOG.info("waitUntilAllTasksDone: Got " + totalDone + + " and " + totalMapTasks + + " desired (polling period is " + + pollMsecs + ") on attempt " + + attempt); } - } - - /** - * Wait for all map tasks to signal completion. - * - * @param totalMapTasks Number of map tasks to wait for - */ - private void waitUntilAllTasksDone(int totalMapTasks) { - int attempt = 0; - while (true) { - try { - FileStatus [] fileStatusArray = - fs.listStatus(taskDirectory); - int totalDone = 0; - if (fileStatusArray.length > 0) { - for (int i = 0; i < fileStatusArray.length; ++i) { - if (fileStatusArray[i].getPath().getName().endsWith( - COMPUTATION_DONE_SUFFIX)) { - ++totalDone; - } - } - } - if (LOG.isInfoEnabled()) { - LOG.info("waitUntilAllTasksDone: Got " + totalDone + - " and " + totalMapTasks + - " desired (polling period is " + - pollMsecs + ") on attempt " + - attempt); - } - if (totalDone >= totalMapTasks) { - break; - } - ++attempt; - Thread.sleep(pollMsecs); - context.progress(); - } catch (IOException e) { - LOG.warn("waitUntilAllTasksDone: Got IOException.", e); - } catch (InterruptedException e) { - LOG.warn("waitUntilAllTasksDone: Got InterruptedException", e); - } + if (totalDone >= totalMapTasks) { + break; } + ++attempt; + Thread.sleep(pollMsecs); + context.progress(); + } catch (IOException e) { + LOG.warn("waitUntilAllTasksDone: Got IOException.", e); + } catch (InterruptedException e) { + LOG.warn("waitUntilAllTasksDone: Got InterruptedException", e); + } } + } - /** - * Notify the ZooKeeper servers that this partition is done with all - * ZooKeeper communication. If this task is running a ZooKeeper server, - * kill it when all partitions are done and wait for - * completion. Clean up the ZooKeeper local directory as well. - * - * @param state State of the application - */ - public void offlineZooKeeperServers(State state) { - if (state == State.FINISHED) { - createZooKeeperClosedStamp(); + /** + * Notify the ZooKeeper servers that this partition is done with all + * ZooKeeper communication. If this task is running a ZooKeeper server, + * kill it when all partitions are done and wait for + * completion. Clean up the ZooKeeper local directory as well. + * + * @param state State of the application + */ + public void offlineZooKeeperServers(State state) { + if (state == State.FINISHED) { + createZooKeeperClosedStamp(); + } + synchronized (this) { + if (zkProcess != null) { + int totalMapTasks = conf.getInt("mapred.map.tasks", -1); + waitUntilAllTasksDone(totalMapTasks); + zkProcess.destroy(); + int exitValue = -1; + File zkDirFile; + try { + zkProcessCollector.join(); + exitValue = zkProcess.waitFor(); + zkDirFile = new File(zkDir); + FileUtils.deleteDirectory(zkDirFile); + } catch (InterruptedException e) { + LOG.warn("offlineZooKeeperServers: " + + "InterruptedException, but continuing ", + e); + } catch (IOException e) { + LOG.warn("offlineZooKeeperSevers: " + + "IOException, but continuing", + e); } - synchronized (this) { - if (zkProcess != null) { - int totalMapTasks = conf.getInt("mapred.map.tasks", -1); - waitUntilAllTasksDone(totalMapTasks); - zkProcess.destroy(); - int exitValue = -1; - File zkDirFile; - try { - zkProcessCollector.join(); - exitValue = zkProcess.waitFor(); - zkDirFile = new File(zkDir); - FileUtils.deleteDirectory(zkDirFile); - } catch (InterruptedException e) { - LOG.warn("offlineZooKeeperServers: " + - "InterruptedException, but continuing ", - e); - } catch (IOException e) { - LOG.warn("offlineZooKeeperSevers: " + - "IOException, but continuing", - e); - } - if (LOG.isInfoEnabled()) { - LOG.info("offlineZooKeeperServers: waitFor returned " + - exitValue + " and deleted directory " + zkDir); - } - zkProcess = null; - } + if (LOG.isInfoEnabled()) { + LOG.info("offlineZooKeeperServers: waitFor returned " + + exitValue + " and deleted directory " + zkDir); } + zkProcess = null; + } } + } - /** - * Is this task running a ZooKeeper server? Only could be true if called - * after onlineZooKeeperServers(). - * - * @return true if running a ZooKeeper server, false otherwise - */ - public boolean runsZooKeeper() { - synchronized (this) { - return zkProcess != null; - } + /** + * Is this task running a ZooKeeper server? Only could be true if called + * after onlineZooKeeperServers(). + * + * @return true if running a ZooKeeper server, false otherwise + */ + public boolean runsZooKeeper() { + synchronized (this) { + return zkProcess != null; } + } } diff --git a/src/main/java/org/apache/giraph/zk/package-info.java b/src/main/java/org/apache/giraph/zk/package-info.java new file mode 100644 index 000000000..e1dcf5250 --- /dev/null +++ b/src/main/java/org/apache/giraph/zk/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/** + * Package of {@link org.apache.zookeeper.ZooKeeper} related objects. + */ +package org.apache.giraph.zk; diff --git a/src/test/java/org/apache/giraph/TestBspBasic.java b/src/test/java/org/apache/giraph/TestBspBasic.java index 96e364914..42ce452ff 100644 --- a/src/test/java/org/apache/giraph/TestBspBasic.java +++ b/src/test/java/org/apache/giraph/TestBspBasic.java @@ -63,340 +63,340 @@ * Unit test for many simple BSP applications. */ public class TestBspBasic extends BspCase { - /** - * Create the test case - * - * @param testName name of the test case - */ - public TestBspBasic(String testName) { - super(testName); - } + /** + * Create the test case + * + * @param testName name of the test case + */ + public TestBspBasic(String testName) { + super(testName); + } - /** - * @return the suite of tests being tested - */ - public static Test suite() { - return new TestSuite(TestBspBasic.class); - } + /** + * @return the suite of tests being tested + */ + public static Test suite() { + return new TestSuite(TestBspBasic.class); + } - /** - * Just instantiate the vertex (all functions are implemented) and the - * VertexInputFormat using reflection. - * - * @throws IllegalAccessException - * @throws InstantiationException - * @throws InterruptedException - * @throws IOException - * @throws InvocationTargetException - * @throws IllegalArgumentException - * @throws NoSuchMethodException - * @throws SecurityException - */ - public void testInstantiateVertex() - throws InstantiationException, IllegalAccessException, - IOException, InterruptedException, IllegalArgumentException, - InvocationTargetException, SecurityException, NoSuchMethodException { - System.out.println("testInstantiateVertex: java.class.path=" + - System.getProperty("java.class.path")); - GiraphJob job = new GiraphJob(getCallingMethodName()); - job.setVertexClass(SimpleSuperstepVertex.class); - job.setVertexInputFormatClass( - SimpleSuperstepVertex.SimpleSuperstepVertexInputFormat.class); - GraphState gs = - new GraphState(); - BasicVertex vertex = - BspUtils.createVertex(job.getConfiguration()); - vertex.initialize(null, null, null, null); - System.out.println("testInstantiateVertex: Got vertex " + vertex + - ", graphState" + gs); - VertexInputFormat - inputFormat = BspUtils.createVertexInputFormat(job.getConfiguration()); - List splitArray = - inputFormat.getSplits( - new JobContext(new Configuration(), new JobID()), 1); - ByteArrayOutputStream byteArrayOutputStream = - new ByteArrayOutputStream(); - DataOutputStream outputStream = - new DataOutputStream(byteArrayOutputStream); - ((Writable) splitArray.get(0)).write(outputStream); - System.out.println("testInstantiateVertex: Example output split = " + - byteArrayOutputStream.toString()); - } - - /** - * Do some checks for local job runner. - * - * @throws IOException - * @throws ClassNotFoundException - * @throws InterruptedException - */ - public void testLocalJobRunnerConfig() - throws IOException, InterruptedException, ClassNotFoundException { - if (getJobTracker() != null) { - System.out.println("testLocalJobRunnerConfig: Skipping for " + - "non-local"); - return; - } - GiraphJob job = new GiraphJob(getCallingMethodName()); - setupConfiguration(job); - job.setWorkerConfiguration(5, 5, 100.0f); - job.getConfiguration().setBoolean(GiraphJob.SPLIT_MASTER_WORKER, true); - job.setVertexClass(SimpleSuperstepVertex.class); - job.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class); - try { - job.run(true); - assertTrue(false); - } catch (IllegalArgumentException e) { - } + /** + * Just instantiate the vertex (all functions are implemented) and the + * VertexInputFormat using reflection. + * + * @throws IllegalAccessException + * @throws InstantiationException + * @throws InterruptedException + * @throws IOException + * @throws InvocationTargetException + * @throws IllegalArgumentException + * @throws NoSuchMethodException + * @throws SecurityException + */ + public void testInstantiateVertex() + throws InstantiationException, IllegalAccessException, + IOException, InterruptedException, IllegalArgumentException, + InvocationTargetException, SecurityException, NoSuchMethodException { + System.out.println("testInstantiateVertex: java.class.path=" + + System.getProperty("java.class.path")); + GiraphJob job = new GiraphJob(getCallingMethodName()); + job.setVertexClass(SimpleSuperstepVertex.class); + job.setVertexInputFormatClass( + SimpleSuperstepVertex.SimpleSuperstepVertexInputFormat.class); + GraphState gs = + new GraphState(); + BasicVertex vertex = + BspUtils.createVertex(job.getConfiguration()); + vertex.initialize(null, null, null, null); + System.out.println("testInstantiateVertex: Got vertex " + vertex + + ", graphState" + gs); + VertexInputFormat + inputFormat = BspUtils.createVertexInputFormat(job.getConfiguration()); + List splitArray = + inputFormat.getSplits( + new JobContext(new Configuration(), new JobID()), 1); + ByteArrayOutputStream byteArrayOutputStream = + new ByteArrayOutputStream(); + DataOutputStream outputStream = + new DataOutputStream(byteArrayOutputStream); + ((Writable) splitArray.get(0)).write(outputStream); + System.out.println("testInstantiateVertex: Example output split = " + + byteArrayOutputStream.toString()); + } - job.getConfiguration().setBoolean(GiraphJob.SPLIT_MASTER_WORKER, false); - try { - job.run(true); - assertTrue(false); - } catch (IllegalArgumentException e) { - } - job.setWorkerConfiguration(1, 1, 100.0f); - job.run(true); + /** + * Do some checks for local job runner. + * + * @throws IOException + * @throws ClassNotFoundException + * @throws InterruptedException + */ + public void testLocalJobRunnerConfig() + throws IOException, InterruptedException, ClassNotFoundException { + if (getJobTracker() != null) { + System.out.println("testLocalJobRunnerConfig: Skipping for " + + "non-local"); + return; } - - /** - * Run a sample BSP job in JobTracker, kill a task, and make sure - * the job fails (not enough attempts to restart) - * - * @throws IOException - * @throws ClassNotFoundException - * @throws InterruptedException - */ - public void testBspFail() - throws IOException, InterruptedException, ClassNotFoundException { - // Allow this test only to be run on a real Hadoop setup - if (getJobTracker() == null) { - System.out.println("testBspFail: not executed for local setup."); - return; - } - - GiraphJob job = new GiraphJob(getCallingMethodName()); - setupConfiguration(job); - job.getConfiguration().setInt("mapred.map.max.attempts", 1); - job.setVertexClass(SimpleFailVertex.class); - job.setVertexInputFormatClass(SimplePageRankVertexInputFormat.class); - Path outputPath = new Path("/tmp/" + getCallingMethodName()); - removeAndSetOutput(job, outputPath); - assertTrue(!job.run(true)); + GiraphJob job = new GiraphJob(getCallingMethodName()); + setupConfiguration(job); + job.setWorkerConfiguration(5, 5, 100.0f); + job.getConfiguration().setBoolean(GiraphJob.SPLIT_MASTER_WORKER, true); + job.setVertexClass(SimpleSuperstepVertex.class); + job.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class); + try { + job.run(true); + assertTrue(false); + } catch (IllegalArgumentException e) { } - /** - * Run a sample BSP job locally and test supersteps. - * - * @throws IOException - * @throws ClassNotFoundException - * @throws InterruptedException - */ - public void testBspSuperStep() - throws IOException, InterruptedException, ClassNotFoundException { - GiraphJob job = new GiraphJob(getCallingMethodName()); - setupConfiguration(job); - job.getConfiguration().setFloat(GiraphJob.TOTAL_INPUT_SPLIT_MULTIPLIER, - 2.0f); - // GeneratedInputSplit will generate 10 vertices - job.getConfiguration().setLong(GeneratedVertexReader.READER_VERTICES, - 10); - job.setVertexClass(SimpleSuperstepVertex.class); - job.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class); - job.setVertexOutputFormatClass(SimpleSuperstepVertexOutputFormat.class); - Path outputPath = new Path("/tmp/" + getCallingMethodName()); - removeAndSetOutput(job, outputPath); - assertTrue(job.run(true)); - if (getJobTracker() == null) { - FileStatus fileStatus = getSinglePartFileStatus(job, outputPath); - assertTrue(fileStatus.getLen() == 49); - } + job.getConfiguration().setBoolean(GiraphJob.SPLIT_MASTER_WORKER, false); + try { + job.run(true); + assertTrue(false); + } catch (IllegalArgumentException e) { } + job.setWorkerConfiguration(1, 1, 100.0f); + job.run(true); + } - /** - * Run a sample BSP job locally and test messages. - * - * @throws IOException - * @throws ClassNotFoundException - * @throws InterruptedException - */ - public void testBspMsg() - throws IOException, InterruptedException, ClassNotFoundException { - GiraphJob job = new GiraphJob(getCallingMethodName()); - setupConfiguration(job); - job.setVertexClass(SimpleMsgVertex.class); - job.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class); - assertTrue(job.run(true)); + /** + * Run a sample BSP job in JobTracker, kill a task, and make sure + * the job fails (not enough attempts to restart) + * + * @throws IOException + * @throws ClassNotFoundException + * @throws InterruptedException + */ + public void testBspFail() + throws IOException, InterruptedException, ClassNotFoundException { + // Allow this test only to be run on a real Hadoop setup + if (getJobTracker() == null) { + System.out.println("testBspFail: not executed for local setup."); + return; } + GiraphJob job = new GiraphJob(getCallingMethodName()); + setupConfiguration(job); + job.getConfiguration().setInt("mapred.map.max.attempts", 1); + job.setVertexClass(SimpleFailVertex.class); + job.setVertexInputFormatClass(SimplePageRankVertexInputFormat.class); + Path outputPath = new Path("/tmp/" + getCallingMethodName()); + removeAndSetOutput(job, outputPath); + assertTrue(!job.run(true)); + } - /** - * Run a sample BSP job locally with no vertices and make sure - * it completes. - * - * @throws IOException - * @throws ClassNotFoundException - * @throws InterruptedException - */ - public void testEmptyVertexInputFormat() - throws IOException, InterruptedException, ClassNotFoundException { - GiraphJob job = new GiraphJob(getCallingMethodName()); - setupConfiguration(job); - job.getConfiguration().setLong(GeneratedVertexReader.READER_VERTICES, - 0); - job.setVertexClass(SimpleMsgVertex.class); - job.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class); - assertTrue(job.run(true)); + /** + * Run a sample BSP job locally and test supersteps. + * + * @throws IOException + * @throws ClassNotFoundException + * @throws InterruptedException + */ + public void testBspSuperStep() + throws IOException, InterruptedException, ClassNotFoundException { + GiraphJob job = new GiraphJob(getCallingMethodName()); + setupConfiguration(job); + job.getConfiguration().setFloat(GiraphJob.TOTAL_INPUT_SPLIT_MULTIPLIER, + 2.0f); + // GeneratedInputSplit will generate 10 vertices + job.getConfiguration().setLong(GeneratedVertexReader.READER_VERTICES, + 10); + job.setVertexClass(SimpleSuperstepVertex.class); + job.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class); + job.setVertexOutputFormatClass(SimpleSuperstepVertexOutputFormat.class); + Path outputPath = new Path("/tmp/" + getCallingMethodName()); + removeAndSetOutput(job, outputPath); + assertTrue(job.run(true)); + if (getJobTracker() == null) { + FileStatus fileStatus = getSinglePartFileStatus(job, outputPath); + assertTrue(fileStatus.getLen() == 49); } + } - /** - * Run a sample BSP job locally with combiner and checkout output value. - * - * @throws IOException - * @throws ClassNotFoundException - * @throws InterruptedException - */ - public void testBspCombiner() - throws IOException, InterruptedException, ClassNotFoundException { - GiraphJob job = new GiraphJob(getCallingMethodName()); - setupConfiguration(job); - job.setVertexClass(SimpleCombinerVertex.class); - job.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class); - job.setVertexCombinerClass(SimpleSumCombiner.class); - assertTrue(job.run(true)); - } + /** + * Run a sample BSP job locally and test messages. + * + * @throws IOException + * @throws ClassNotFoundException + * @throws InterruptedException + */ + public void testBspMsg() + throws IOException, InterruptedException, ClassNotFoundException { + GiraphJob job = new GiraphJob(getCallingMethodName()); + setupConfiguration(job); + job.setVertexClass(SimpleMsgVertex.class); + job.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class); + assertTrue(job.run(true)); + } - /** - * Run a sample BSP job locally and test PageRank. - * - * @throws IOException - * @throws ClassNotFoundException - * @throws InterruptedException - */ - public void testBspPageRank() - throws IOException, InterruptedException, ClassNotFoundException { - GiraphJob job = new GiraphJob(getCallingMethodName()); - setupConfiguration(job); - job.setVertexClass(SimplePageRankVertex.class); - job.setWorkerContextClass( - SimplePageRankVertex.SimplePageRankVertexWorkerContext.class); - job.setVertexInputFormatClass(SimplePageRankVertexInputFormat.class); - assertTrue(job.run(true)); - if (getJobTracker() == null) { - double maxPageRank = - SimplePageRankVertex.SimplePageRankVertexWorkerContext.finalMax; - double minPageRank = - SimplePageRankVertex.SimplePageRankVertexWorkerContext.finalMin; - long numVertices = - SimplePageRankVertex.SimplePageRankVertexWorkerContext.finalSum; - System.out.println("testBspPageRank: maxPageRank=" + maxPageRank + - " minPageRank=" + minPageRank + - " numVertices=" + numVertices); - assertTrue("34.030 !< " + maxPageRank + " !< " + " 34.0301", - maxPageRank > 34.030 && maxPageRank < 34.0301); - assertTrue("0.03 !< " + minPageRank + " !< " + "0.03001", - minPageRank > 0.03 && minPageRank < 0.03001); - assertTrue("numVertices = " + numVertices + " != 5", numVertices == 5); - } + + /** + * Run a sample BSP job locally with no vertices and make sure + * it completes. + * + * @throws IOException + * @throws ClassNotFoundException + * @throws InterruptedException + */ + public void testEmptyVertexInputFormat() + throws IOException, InterruptedException, ClassNotFoundException { + GiraphJob job = new GiraphJob(getCallingMethodName()); + setupConfiguration(job); + job.getConfiguration().setLong(GeneratedVertexReader.READER_VERTICES, + 0); + job.setVertexClass(SimpleMsgVertex.class); + job.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class); + assertTrue(job.run(true)); + } + + /** + * Run a sample BSP job locally with combiner and checkout output value. + * + * @throws IOException + * @throws ClassNotFoundException + * @throws InterruptedException + */ + public void testBspCombiner() + throws IOException, InterruptedException, ClassNotFoundException { + GiraphJob job = new GiraphJob(getCallingMethodName()); + setupConfiguration(job); + job.setVertexClass(SimpleCombinerVertex.class); + job.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class); + job.setVertexCombinerClass(SimpleSumCombiner.class); + assertTrue(job.run(true)); + } + + /** + * Run a sample BSP job locally and test PageRank. + * + * @throws IOException + * @throws ClassNotFoundException + * @throws InterruptedException + */ + public void testBspPageRank() + throws IOException, InterruptedException, ClassNotFoundException { + GiraphJob job = new GiraphJob(getCallingMethodName()); + setupConfiguration(job); + job.setVertexClass(SimplePageRankVertex.class); + job.setWorkerContextClass( + SimplePageRankVertex.SimplePageRankVertexWorkerContext.class); + job.setVertexInputFormatClass(SimplePageRankVertexInputFormat.class); + assertTrue(job.run(true)); + if (getJobTracker() == null) { + double maxPageRank = + SimplePageRankVertex.SimplePageRankVertexWorkerContext.getFinalMax(); + double minPageRank = + SimplePageRankVertex.SimplePageRankVertexWorkerContext.getFinalMin(); + long numVertices = + SimplePageRankVertex.SimplePageRankVertexWorkerContext.getFinalSum(); + System.out.println("testBspPageRank: maxPageRank=" + maxPageRank + + " minPageRank=" + minPageRank + + " numVertices=" + numVertices); + assertTrue("34.030 !< " + maxPageRank + " !< " + " 34.0301", + maxPageRank > 34.030 && maxPageRank < 34.0301); + assertTrue("0.03 !< " + minPageRank + " !< " + "0.03001", + minPageRank > 0.03 && minPageRank < 0.03001); + assertTrue("numVertices = " + numVertices + " != 5", numVertices == 5); } + } - /** - * Run a sample BSP job locally and test shortest paths. - * - * @throws IOException - * @throws ClassNotFoundException - * @throws InterruptedException - */ - public void testBspShortestPaths() - throws IOException, InterruptedException, ClassNotFoundException { - GiraphJob job = new GiraphJob(getCallingMethodName()); - setupConfiguration(job); - job.setVertexClass(SimpleShortestPathsVertex.class); - job.setVertexInputFormatClass(SimplePageRankVertexInputFormat.class); - job.setVertexOutputFormatClass( - SimpleShortestPathsVertexOutputFormat.class); - job.getConfiguration().setLong(SimpleShortestPathsVertex.SOURCE_ID, 0); - Path outputPath = new Path("/tmp/" + getCallingMethodName()); - removeAndSetOutput(job, outputPath); - assertTrue(job.run(true)); + /** + * Run a sample BSP job locally and test shortest paths. + * + * @throws IOException + * @throws ClassNotFoundException + * @throws InterruptedException + */ + public void testBspShortestPaths() + throws IOException, InterruptedException, ClassNotFoundException { + GiraphJob job = new GiraphJob(getCallingMethodName()); + setupConfiguration(job); + job.setVertexClass(SimpleShortestPathsVertex.class); + job.setVertexInputFormatClass(SimplePageRankVertexInputFormat.class); + job.setVertexOutputFormatClass( + SimpleShortestPathsVertexOutputFormat.class); + job.getConfiguration().setLong(SimpleShortestPathsVertex.SOURCE_ID, 0); + Path outputPath = new Path("/tmp/" + getCallingMethodName()); + removeAndSetOutput(job, outputPath); + assertTrue(job.run(true)); - job = new GiraphJob(getCallingMethodName()); - setupConfiguration(job); - job.setVertexClass(SimpleShortestPathsVertex.class); - job.setVertexInputFormatClass(SimplePageRankVertexInputFormat.class); - job.setVertexOutputFormatClass( - SimpleShortestPathsVertexOutputFormat.class); - job.getConfiguration().setLong(SimpleShortestPathsVertex.SOURCE_ID, 0); - Path outputPath2 = new Path("/tmp/" + getCallingMethodName() + "2"); - removeAndSetOutput(job, outputPath2); - assertTrue(job.run(true)); - if (getJobTracker() == null) { - FileStatus fileStatus = getSinglePartFileStatus(job, outputPath); - FileStatus fileStatus2 = getSinglePartFileStatus(job, outputPath2); - assertTrue(fileStatus.getLen() == fileStatus2.getLen()); - } + job = new GiraphJob(getCallingMethodName()); + setupConfiguration(job); + job.setVertexClass(SimpleShortestPathsVertex.class); + job.setVertexInputFormatClass(SimplePageRankVertexInputFormat.class); + job.setVertexOutputFormatClass( + SimpleShortestPathsVertexOutputFormat.class); + job.getConfiguration().setLong(SimpleShortestPathsVertex.SOURCE_ID, 0); + Path outputPath2 = new Path("/tmp/" + getCallingMethodName() + "2"); + removeAndSetOutput(job, outputPath2); + assertTrue(job.run(true)); + if (getJobTracker() == null) { + FileStatus fileStatus = getSinglePartFileStatus(job, outputPath); + FileStatus fileStatus2 = getSinglePartFileStatus(job, outputPath2); + assertTrue(fileStatus.getLen() == fileStatus2.getLen()); } + } - /** - * Run a sample BSP job locally and test PageRank with AggregatorWriter. - * - * @throws IOException - * @throws ClassNotFoundException - * @throws InterruptedException - */ - public void testBspPageRankWithAggregatorWriter() - throws IOException, InterruptedException, ClassNotFoundException { - GiraphJob job = new GiraphJob(getCallingMethodName()); - setupConfiguration(job); - job.setVertexClass(SimplePageRankVertex.class); - job.setWorkerContextClass( - SimplePageRankVertex.SimplePageRankVertexWorkerContext.class); - job.setVertexInputFormatClass(SimplePageRankVertexInputFormat.class); - job.setAggregatorWriterClass(SimpleAggregatorWriter.class); - Path outputPath = new Path("/tmp/" + getCallingMethodName()); - removeAndSetOutput(job, outputPath); - assertTrue(job.run(true)); - if (getJobTracker() == null) { - double maxPageRank = - SimplePageRankVertex.SimplePageRankVertexWorkerContext.finalMax; - double minPageRank = - SimplePageRankVertex.SimplePageRankVertexWorkerContext.finalMin; - long numVertices = - SimplePageRankVertex.SimplePageRankVertexWorkerContext.finalSum; - System.out.println("testBspPageRank: maxPageRank=" + maxPageRank + - " minPageRank=" + minPageRank + - " numVertices=" + numVertices); - FileSystem fs = FileSystem.get(new Configuration()); - FSDataInputStream input = - fs.open(new Path(SimpleAggregatorWriter.filename)); - int i, all; - for (i = 0; ; i++) { - all = 0; - try { - DoubleWritable max = new DoubleWritable(); - max.readFields(input); - all++; - DoubleWritable min = new DoubleWritable(); - min.readFields(input); - all++; - LongWritable sum = new LongWritable(); - sum.readFields(input); - all++; - if (i > 0) { - assertTrue(max.get() == maxPageRank); - assertTrue(min.get() == minPageRank); - assertTrue(sum.get() == numVertices); - } - } catch (IOException e) { - break; - } - } - input.close(); - // contained all supersteps - assertTrue(i == SimplePageRankVertex.MAX_SUPERSTEPS+1 && all == 0); - remove(new Configuration(), - new Path(SimpleAggregatorWriter.filename)); + /** + * Run a sample BSP job locally and test PageRank with AggregatorWriter. + * + * @throws IOException + * @throws ClassNotFoundException + * @throws InterruptedException + */ + public void testBspPageRankWithAggregatorWriter() + throws IOException, InterruptedException, ClassNotFoundException { + GiraphJob job = new GiraphJob(getCallingMethodName()); + setupConfiguration(job); + job.setVertexClass(SimplePageRankVertex.class); + job.setWorkerContextClass( + SimplePageRankVertex.SimplePageRankVertexWorkerContext.class); + job.setVertexInputFormatClass(SimplePageRankVertexInputFormat.class); + job.setAggregatorWriterClass(SimpleAggregatorWriter.class); + Path outputPath = new Path("/tmp/" + getCallingMethodName()); + removeAndSetOutput(job, outputPath); + assertTrue(job.run(true)); + if (getJobTracker() == null) { + double maxPageRank = + SimplePageRankVertex.SimplePageRankVertexWorkerContext.getFinalMax(); + double minPageRank = + SimplePageRankVertex.SimplePageRankVertexWorkerContext.getFinalMin(); + long numVertices = + SimplePageRankVertex.SimplePageRankVertexWorkerContext.getFinalSum(); + System.out.println("testBspPageRank: maxPageRank=" + maxPageRank + + " minPageRank=" + minPageRank + + " numVertices=" + numVertices); + FileSystem fs = FileSystem.get(new Configuration()); + FSDataInputStream input = + fs.open(new Path(SimpleAggregatorWriter.getFilename())); + int i, all; + for (i = 0; ; i++) { + all = 0; + try { + DoubleWritable max = new DoubleWritable(); + max.readFields(input); + all++; + DoubleWritable min = new DoubleWritable(); + min.readFields(input); + all++; + LongWritable sum = new LongWritable(); + sum.readFields(input); + all++; + if (i > 0) { + assertTrue(max.get() == maxPageRank); + assertTrue(min.get() == minPageRank); + assertTrue(sum.get() == numVertices); + } + } catch (IOException e) { + break; } + } + input.close(); + // contained all supersteps + assertTrue(i == SimplePageRankVertex.MAX_SUPERSTEPS+1 && all == 0); + remove(new Configuration(), + new Path(SimpleAggregatorWriter.getFilename())); } + } } diff --git a/src/test/java/org/apache/giraph/TestJsonBase64Format.java b/src/test/java/org/apache/giraph/TestJsonBase64Format.java index 38dd2da0e..e65707b08 100644 --- a/src/test/java/org/apache/giraph/TestJsonBase64Format.java +++ b/src/test/java/org/apache/giraph/TestJsonBase64Format.java @@ -21,6 +21,7 @@ import java.io.IOException; import org.apache.giraph.benchmark.PageRankBenchmark; +import org.apache.giraph.benchmark.PageRankComputation; import org.apache.giraph.benchmark.PseudoRandomVertexInputFormat; import org.apache.giraph.graph.GiraphJob; import org.apache.giraph.lib.JsonBase64VertexInputFormat; @@ -36,80 +37,80 @@ * Test out the JsonBase64 format. */ public class TestJsonBase64Format extends BspCase { - /** - * Create the test case - * - * @param testName name of the test case - */ - public TestJsonBase64Format(String testName) { - super(testName); - } + /** + * Create the test case + * + * @param testName name of the test case + */ + public TestJsonBase64Format(String testName) { + super(testName); + } - /** - * @return the suite of tests being tested - */ - public static Test suite() { - return new TestSuite(TestJsonBase64Format.class); - } + /** + * @return the suite of tests being tested + */ + public static Test suite() { + return new TestSuite(TestJsonBase64Format.class); + } - /** - * Start a job and finish after i supersteps, then begin a new job and - * continue on more j supersteps. Check the results against a single job - * with i + j supersteps. - * - * @throws IOException - * @throws ClassNotFoundException - * @throws InterruptedException - */ - public void testContinue() - throws IOException, InterruptedException, ClassNotFoundException { - GiraphJob job = new GiraphJob(getCallingMethodName()); - setupConfiguration(job); - job.setVertexClass(PageRankBenchmark.class); - job.setVertexInputFormatClass(PseudoRandomVertexInputFormat.class); - job.setVertexOutputFormatClass(JsonBase64VertexOutputFormat.class); - job.getConfiguration().setLong( - PseudoRandomVertexInputFormat.AGGREGATE_VERTICES, 101); - job.getConfiguration().setLong( - PseudoRandomVertexInputFormat.EDGES_PER_VERTEX, 2); - job.getConfiguration().setInt(PageRankBenchmark.SUPERSTEP_COUNT, 2); - Path outputPath = new Path("/tmp/" + getCallingMethodName()); - removeAndSetOutput(job, outputPath); - assertTrue(job.run(true)); + /** + * Start a job and finish after i supersteps, then begin a new job and + * continue on more j supersteps. Check the results against a single job + * with i + j supersteps. + * + * @throws IOException + * @throws ClassNotFoundException + * @throws InterruptedException + */ + public void testContinue() + throws IOException, InterruptedException, ClassNotFoundException { + GiraphJob job = new GiraphJob(getCallingMethodName()); + setupConfiguration(job); + job.setVertexClass(PageRankBenchmark.class); + job.setVertexInputFormatClass(PseudoRandomVertexInputFormat.class); + job.setVertexOutputFormatClass(JsonBase64VertexOutputFormat.class); + job.getConfiguration().setLong( + PseudoRandomVertexInputFormat.AGGREGATE_VERTICES, 101); + job.getConfiguration().setLong( + PseudoRandomVertexInputFormat.EDGES_PER_VERTEX, 2); + job.getConfiguration().setInt(PageRankComputation.SUPERSTEP_COUNT, 2); + Path outputPath = new Path("/tmp/" + getCallingMethodName()); + removeAndSetOutput(job, outputPath); + assertTrue(job.run(true)); - job = new GiraphJob(getCallingMethodName()); - setupConfiguration(job); - job.setVertexClass(PageRankBenchmark.class); - job.setVertexInputFormatClass(JsonBase64VertexInputFormat.class); - job.setVertexOutputFormatClass(JsonBase64VertexOutputFormat.class); - job.getConfiguration().setInt(PageRankBenchmark.SUPERSTEP_COUNT, 3); - FileInputFormat.setInputPaths(job, outputPath); - Path outputPath2 = new Path("/tmp/" + getCallingMethodName() + "2"); - removeAndSetOutput(job, outputPath2); - assertTrue(job.run(true)); + job = new GiraphJob(getCallingMethodName()); + setupConfiguration(job); + job.setVertexClass(PageRankBenchmark.class); + job.setVertexInputFormatClass(JsonBase64VertexInputFormat.class); + job.setVertexOutputFormatClass(JsonBase64VertexOutputFormat.class); + job.getConfiguration().setInt(PageRankComputation.SUPERSTEP_COUNT, 3); + FileInputFormat.setInputPaths(job, outputPath); + Path outputPath2 = new Path("/tmp/" + getCallingMethodName() + "2"); + removeAndSetOutput(job, outputPath2); + assertTrue(job.run(true)); - FileStatus twoJobsFile = null; - if (getJobTracker() == null) { - twoJobsFile = getSinglePartFileStatus(job, outputPath); - } + FileStatus twoJobsFile = null; + if (getJobTracker() == null) { + twoJobsFile = getSinglePartFileStatus(job, outputPath); + } - job = new GiraphJob(getCallingMethodName()); - setupConfiguration(job); - job.setVertexClass(PageRankBenchmark.class); - job.setVertexInputFormatClass(PseudoRandomVertexInputFormat.class); - job.setVertexOutputFormatClass(JsonBase64VertexOutputFormat.class); - job.getConfiguration().setLong( - PseudoRandomVertexInputFormat.AGGREGATE_VERTICES, 101); - job.getConfiguration().setLong( - PseudoRandomVertexInputFormat.EDGES_PER_VERTEX, 2); - job.getConfiguration().setInt(PageRankBenchmark.SUPERSTEP_COUNT, 5); - Path outputPath3 = new Path("/tmp/" + getCallingMethodName() + "3"); - removeAndSetOutput(job, outputPath3); - assertTrue(job.run(true)); + job = new GiraphJob(getCallingMethodName()); + setupConfiguration(job); + job.setVertexClass(PageRankBenchmark.class); + job.setVertexInputFormatClass(PseudoRandomVertexInputFormat.class); + job.setVertexOutputFormatClass(JsonBase64VertexOutputFormat.class); + job.getConfiguration().setLong( + PseudoRandomVertexInputFormat.AGGREGATE_VERTICES, 101); + job.getConfiguration().setLong( + PseudoRandomVertexInputFormat.EDGES_PER_VERTEX, 2); + job.getConfiguration().setInt(PageRankComputation.SUPERSTEP_COUNT, 5); + Path outputPath3 = new Path("/tmp/" + getCallingMethodName() + "3"); + removeAndSetOutput(job, outputPath3); + assertTrue(job.run(true)); - if (getJobTracker() == null) { - FileStatus oneJobFile = getSinglePartFileStatus(job, outputPath3); - assertTrue(twoJobsFile.getLen() == oneJobFile.getLen()); - } + if (getJobTracker() == null) { + FileStatus oneJobFile = getSinglePartFileStatus(job, outputPath3); + assertTrue(twoJobsFile.getLen() == oneJobFile.getLen()); } + } } diff --git a/src/test/java/org/apache/giraph/TestManualCheckpoint.java b/src/test/java/org/apache/giraph/TestManualCheckpoint.java index d4252c81c..2aeb0e3f7 100644 --- a/src/test/java/org/apache/giraph/TestManualCheckpoint.java +++ b/src/test/java/org/apache/giraph/TestManualCheckpoint.java @@ -33,88 +33,88 @@ * Unit test for manual checkpoint restarting */ public class TestManualCheckpoint extends BspCase { - /** Where the checkpoints will be stored and restarted */ - private final String HDFS_CHECKPOINT_DIR = - "/tmp/testBspCheckpoints"; + /** Where the checkpoints will be stored and restarted */ + private final String HDFS_CHECKPOINT_DIR = + "/tmp/testBspCheckpoints"; - /** - * Create the test case - * - * @param testName name of the test case - */ - public TestManualCheckpoint(String testName) { - super(testName); - } + /** + * Create the test case + * + * @param testName name of the test case + */ + public TestManualCheckpoint(String testName) { + super(testName); + } - /** - * @return the suite of tests being tested - */ - public static Test suite() { - return new TestSuite(TestManualCheckpoint.class); - } + /** + * @return the suite of tests being tested + */ + public static Test suite() { + return new TestSuite(TestManualCheckpoint.class); + } - /** - * Run a sample BSP job locally and test checkpointing. - * @throws IOException - * @throws ClassNotFoundException - * @throws InterruptedException - */ - public void testBspCheckpoint() - throws IOException, InterruptedException, ClassNotFoundException { - GiraphJob job = new GiraphJob(getCallingMethodName()); - setupConfiguration(job); - job.getConfiguration().set(GiraphJob.CHECKPOINT_DIRECTORY, - HDFS_CHECKPOINT_DIR); - job.getConfiguration().setBoolean( - GiraphJob.CLEANUP_CHECKPOINTS_AFTER_SUCCESS, false); - job.setVertexClass(SimpleCheckpointVertex.class); - job.setWorkerContextClass( - SimpleCheckpointVertex.SimpleCheckpointVertexWorkerContext.class); - job.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class); - job.setVertexOutputFormatClass(SimpleSuperstepVertexOutputFormat.class); - Path outputPath = new Path("/tmp/" + getCallingMethodName()); - removeAndSetOutput(job, outputPath); - assertTrue(job.run(true)); - long fileLen = 0; - long idSum = 0; - if (getJobTracker() == null) { - FileStatus fileStatus = getSinglePartFileStatus(job, outputPath); - fileLen = fileStatus.getLen(); - idSum = - SimpleCheckpointVertex.SimpleCheckpointVertexWorkerContext.finalSum; - System.out.println("testBspCheckpoint: idSum = " + idSum + - " fileLen = " + fileLen); - } + /** + * Run a sample BSP job locally and test checkpointing. + * @throws IOException + * @throws ClassNotFoundException + * @throws InterruptedException + */ + public void testBspCheckpoint() + throws IOException, InterruptedException, ClassNotFoundException { + GiraphJob job = new GiraphJob(getCallingMethodName()); + setupConfiguration(job); + job.getConfiguration().set(GiraphJob.CHECKPOINT_DIRECTORY, + HDFS_CHECKPOINT_DIR); + job.getConfiguration().setBoolean( + GiraphJob.CLEANUP_CHECKPOINTS_AFTER_SUCCESS, false); + job.setVertexClass(SimpleCheckpointVertex.class); + job.setWorkerContextClass( + SimpleCheckpointVertex.SimpleCheckpointVertexWorkerContext.class); + job.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class); + job.setVertexOutputFormatClass(SimpleSuperstepVertexOutputFormat.class); + Path outputPath = new Path("/tmp/" + getCallingMethodName()); + removeAndSetOutput(job, outputPath); + assertTrue(job.run(true)); + long fileLen = 0; + long idSum = 0; + if (getJobTracker() == null) { + FileStatus fileStatus = getSinglePartFileStatus(job, outputPath); + fileLen = fileStatus.getLen(); + idSum = + SimpleCheckpointVertex.SimpleCheckpointVertexWorkerContext.getFinalSum(); + System.out.println("testBspCheckpoint: idSum = " + idSum + + " fileLen = " + fileLen); + } - // Restart the test from superstep 2 - System.out.println( - "testBspCheckpoint: Restarting from superstep 2" + + // Restart the test from superstep 2 + System.out.println( + "testBspCheckpoint: Restarting from superstep 2" + " with checkpoint path = " + HDFS_CHECKPOINT_DIR); - GiraphJob restartedJob = new GiraphJob(getCallingMethodName() + - "Restarted"); - setupConfiguration(restartedJob); - restartedJob.getConfiguration().set(GiraphJob.CHECKPOINT_DIRECTORY, - HDFS_CHECKPOINT_DIR); - restartedJob.getConfiguration().setLong(GiraphJob.RESTART_SUPERSTEP, 2); - restartedJob.setVertexClass(SimpleCheckpointVertex.class); - restartedJob.setWorkerContextClass( - SimpleCheckpointVertex.SimpleCheckpointVertexWorkerContext.class); - restartedJob.setVertexInputFormatClass( - SimpleSuperstepVertexInputFormat.class); - restartedJob.setVertexOutputFormatClass( - SimpleSuperstepVertexOutputFormat.class); - outputPath = new Path("/tmp/" + getCallingMethodName() + "Restarted"); - removeAndSetOutput(restartedJob, outputPath); - assertTrue(restartedJob.run(true)); - if (getJobTracker() == null) { - FileStatus fileStatus = getSinglePartFileStatus(job, outputPath); - fileLen = fileStatus.getLen(); - assertTrue(fileStatus.getLen() == fileLen); - long idSumRestarted = - SimpleCheckpointVertex.SimpleCheckpointVertexWorkerContext.finalSum; - System.out.println("testBspCheckpoint: idSumRestarted = " + - idSumRestarted); - assertTrue(idSum == idSumRestarted); - } + GiraphJob restartedJob = new GiraphJob(getCallingMethodName() + + "Restarted"); + setupConfiguration(restartedJob); + restartedJob.getConfiguration().set(GiraphJob.CHECKPOINT_DIRECTORY, + HDFS_CHECKPOINT_DIR); + restartedJob.getConfiguration().setLong(GiraphJob.RESTART_SUPERSTEP, 2); + restartedJob.setVertexClass(SimpleCheckpointVertex.class); + restartedJob.setWorkerContextClass( + SimpleCheckpointVertex.SimpleCheckpointVertexWorkerContext.class); + restartedJob.setVertexInputFormatClass( + SimpleSuperstepVertexInputFormat.class); + restartedJob.setVertexOutputFormatClass( + SimpleSuperstepVertexOutputFormat.class); + outputPath = new Path("/tmp/" + getCallingMethodName() + "Restarted"); + removeAndSetOutput(restartedJob, outputPath); + assertTrue(restartedJob.run(true)); + if (getJobTracker() == null) { + FileStatus fileStatus = getSinglePartFileStatus(job, outputPath); + fileLen = fileStatus.getLen(); + assertTrue(fileStatus.getLen() == fileLen); + long idSumRestarted = + SimpleCheckpointVertex.SimpleCheckpointVertexWorkerContext.getFinalSum(); + System.out.println("testBspCheckpoint: idSumRestarted = " + + idSumRestarted); + assertTrue(idSum == idSumRestarted); } + } } diff --git a/src/test/java/org/apache/giraph/examples/SimpleShortestPathVertexTest.java b/src/test/java/org/apache/giraph/examples/SimpleShortestPathVertexTest.java index c1d861714..10f9fca54 100644 --- a/src/test/java/org/apache/giraph/examples/SimpleShortestPathVertexTest.java +++ b/src/test/java/org/apache/giraph/examples/SimpleShortestPathVertexTest.java @@ -38,110 +38,111 @@ */ public class SimpleShortestPathVertexTest extends TestCase { - /** - * Test the behavior when a shorter path to a vertex has been found - */ - public void testOnShorterPathFound() throws Exception { - - SimpleShortestPathsVertex vertex = new SimpleShortestPathsVertex(); - vertex.initialize(null, null, null, null); - vertex.addEdge(new LongWritable(10L), new FloatWritable(2.5f)); - vertex.addEdge(new LongWritable(20L), new FloatWritable(0.5f)); - - MockUtils.MockedEnvironment env = MockUtils.prepareVertex(vertex, 1L, - new LongWritable(7L), new DoubleWritable(Double.MAX_VALUE), - false); - - Mockito.when(env.getConfiguration().getLong( - SimpleShortestPathsVertex.SOURCE_ID, - SimpleShortestPathsVertex.SOURCE_ID_DEFAULT)).thenReturn(2L); - - vertex.compute(Lists.newArrayList(new DoubleWritable(2), - new DoubleWritable(1.5)).iterator()); - - assertTrue(vertex.isHalted()); - assertEquals(1.5, vertex.getVertexValue().get()); - - env.verifyMessageSent(new LongWritable(10L), new DoubleWritable(4)); - env.verifyMessageSent(new LongWritable(20L), new DoubleWritable(2)); - } - - /** - * Test the behavior when a new, but not shorter path to a vertex has been found - */ - public void testOnNoShorterPathFound() throws Exception { - - SimpleShortestPathsVertex vertex = new SimpleShortestPathsVertex(); - vertex.initialize(null, null, null, null); - vertex.addEdge(new LongWritable(10L), new FloatWritable(2.5f)); - vertex.addEdge(new LongWritable(20L), new FloatWritable(0.5f)); - - MockUtils.MockedEnvironment env = MockUtils.prepareVertex(vertex, 1L, - new LongWritable(7L), new DoubleWritable(0.5), false); - - Mockito.when(env.getConfiguration().getLong( - SimpleShortestPathsVertex.SOURCE_ID, - SimpleShortestPathsVertex.SOURCE_ID_DEFAULT)).thenReturn(2L); - - vertex.compute(Lists.newArrayList(new DoubleWritable(2), - new DoubleWritable(1.5)).iterator()); - - assertTrue(vertex.isHalted()); - assertEquals(0.5, vertex.getVertexValue().get()); - - env.verifyNoMessageSent(); - } - - /** - * A local integration test on toy data - */ - public void testToyData() throws Exception { - - // a small four vertex graph - String[] graph = new String[] { - "[1,0,[[2,1],[3,3]]]", - "[2,0,[[3,1],[4,10]]]", - "[3,0,[[4,2]]]", - "[4,0,[]]" }; - - // start from vertex 1 - Map params = Maps.newHashMap(); - params.put(SimpleShortestPathsVertex.SOURCE_ID, "1"); - - // run internally - Iterable results = InternalVertexRunner.run( - SimpleShortestPathsVertex.class, - SimpleShortestPathsVertex. - SimpleShortestPathsVertexInputFormat.class, - SimpleShortestPathsVertex. - SimpleShortestPathsVertexOutputFormat.class, - params, graph); - - Map distances = parseDistances(results); - - // verify results - assertNotNull(distances); - assertEquals(4, distances.size()); - assertEquals(0.0, distances.get(1L)); - assertEquals(1.0, distances.get(2L)); - assertEquals(2.0, distances.get(3L)); - assertEquals(4.0, distances.get(4L)); - } - - private Map parseDistances(Iterable results) { - Map distances = - Maps.newHashMapWithExpectedSize(Iterables.size(results)); - for (String line : results) { - try { - JSONArray jsonVertex = new JSONArray(line); - distances.put(jsonVertex.getLong(0), jsonVertex.getDouble(1)); - } catch (JSONException e) { - throw new IllegalArgumentException( - "Couldn't get vertex from line " + line, e); - } - } - return distances; + /** + * Test the behavior when a shorter path to a vertex has been found + */ + public void testOnShorterPathFound() throws Exception { + + SimpleShortestPathsVertex vertex = new SimpleShortestPathsVertex(); + vertex.initialize(null, null, null, null); + vertex.addEdge(new LongWritable(10L), new FloatWritable(2.5f)); + vertex.addEdge(new LongWritable(20L), new FloatWritable(0.5f)); + + MockUtils.MockedEnvironment env = MockUtils.prepareVertex(vertex, 1L, + new LongWritable(7L), new DoubleWritable(Double.MAX_VALUE), + false); + + Mockito.when(env.getConfiguration().getLong( + SimpleShortestPathsVertex.SOURCE_ID, + SimpleShortestPathsVertex.SOURCE_ID_DEFAULT)).thenReturn(2L); + + vertex.compute(Lists.newArrayList(new DoubleWritable(2), + new DoubleWritable(1.5)).iterator()); + + assertTrue(vertex.isHalted()); + assertEquals(1.5, vertex.getVertexValue().get()); + + env.verifyMessageSent(new LongWritable(10L), new DoubleWritable(4)); + env.verifyMessageSent(new LongWritable(20L), new DoubleWritable(2)); + } + + /** + * Test the behavior when a new, but not shorter path to a vertex has been + * found. + */ + public void testOnNoShorterPathFound() throws Exception { + + SimpleShortestPathsVertex vertex = new SimpleShortestPathsVertex(); + vertex.initialize(null, null, null, null); + vertex.addEdge(new LongWritable(10L), new FloatWritable(2.5f)); + vertex.addEdge(new LongWritable(20L), new FloatWritable(0.5f)); + + MockUtils.MockedEnvironment env = MockUtils.prepareVertex(vertex, 1L, + new LongWritable(7L), new DoubleWritable(0.5), false); + + Mockito.when(env.getConfiguration().getLong( + SimpleShortestPathsVertex.SOURCE_ID, + SimpleShortestPathsVertex.SOURCE_ID_DEFAULT)).thenReturn(2L); + + vertex.compute(Lists.newArrayList(new DoubleWritable(2), + new DoubleWritable(1.5)).iterator()); + + assertTrue(vertex.isHalted()); + assertEquals(0.5, vertex.getVertexValue().get()); + + env.verifyNoMessageSent(); + } + + /** + * A local integration test on toy data + */ + public void testToyData() throws Exception { + + // a small four vertex graph + String[] graph = new String[] { + "[1,0,[[2,1],[3,3]]]", + "[2,0,[[3,1],[4,10]]]", + "[3,0,[[4,2]]]", + "[4,0,[]]" }; + + // start from vertex 1 + Map params = Maps.newHashMap(); + params.put(SimpleShortestPathsVertex.SOURCE_ID, "1"); + + // run internally + Iterable results = InternalVertexRunner.run( + SimpleShortestPathsVertex.class, + SimpleShortestPathsVertex. + SimpleShortestPathsVertexInputFormat.class, + SimpleShortestPathsVertex. + SimpleShortestPathsVertexOutputFormat.class, + params, graph); + + Map distances = parseDistances(results); + + // verify results + assertNotNull(distances); + assertEquals(4, distances.size()); + assertEquals(0.0, distances.get(1L)); + assertEquals(1.0, distances.get(2L)); + assertEquals(2.0, distances.get(3L)); + assertEquals(4.0, distances.get(4L)); + } + + private Map parseDistances(Iterable results) { + Map distances = + Maps.newHashMapWithExpectedSize(Iterables.size(results)); + for (String line : results) { + try { + JSONArray jsonVertex = new JSONArray(line); + distances.put(jsonVertex.getLong(0), jsonVertex.getDouble(1)); + } catch (JSONException e) { + throw new IllegalArgumentException( + "Couldn't get vertex from line " + line, e); + } } + return distances; + } } diff --git a/src/test/java/org/apache/giraph/graph/TestEdgeListVertex.java b/src/test/java/org/apache/giraph/graph/TestEdgeListVertex.java index d1b009434..0927820ae 100644 --- a/src/test/java/org/apache/giraph/graph/TestEdgeListVertex.java +++ b/src/test/java/org/apache/giraph/graph/TestEdgeListVertex.java @@ -41,136 +41,134 @@ * Tests {@link EdgeListVertex}. */ public class TestEdgeListVertex extends TestCase { - /** Instantiated vertex filled in from setup() */ - private IFDLEdgeListVertex vertex; - /** Job filled in by setup() */ - private GiraphJob job; - - /** - * Simple instantiable class that extends {@link EdgeArrayVertex}. - */ - private static class IFDLEdgeListVertex extends - EdgeListVertex { - - @Override - public void compute(Iterator msgIterator) - throws IOException { - } - } + /** Instantiated vertex filled in from setup() */ + private IFDLEdgeListVertex vertex; + /** Job filled in by setup() */ + private GiraphJob job; + /** + * Simple instantiable class that extends {@link EdgeArrayVertex}. + */ + private static class IFDLEdgeListVertex extends + EdgeListVertex { @Override - public void setUp() { - try { - job = new GiraphJob("TestEdgeArrayVertex"); - } catch (IOException e) { - throw new RuntimeException("setUp: Failed", e); - } - job.setVertexClass(IFDLEdgeListVertex.class); - job.getConfiguration().setClass(GiraphJob.VERTEX_INDEX_CLASS, - IntWritable.class, WritableComparable.class); - job.getConfiguration().setClass(GiraphJob.VERTEX_VALUE_CLASS, - FloatWritable.class, Writable.class); - job.getConfiguration().setClass(GiraphJob.EDGE_VALUE_CLASS, - DoubleWritable.class, Writable.class); - job.getConfiguration().setClass(GiraphJob.MESSAGE_VALUE_CLASS, - LongWritable.class, Writable.class); - vertex = (IFDLEdgeListVertex) - BspUtils. - createVertex(job.getConfiguration()); + public void compute(Iterator msgIterator) + throws IOException { } + } - public void testInstantiate() throws IOException { - assertNotNull(vertex); + @Override + public void setUp() { + try { + job = new GiraphJob("TestEdgeArrayVertex"); + } catch (IOException e) { + throw new RuntimeException("setUp: Failed", e); } + job.setVertexClass(IFDLEdgeListVertex.class); + job.getConfiguration().setClass(GiraphJob.VERTEX_INDEX_CLASS, + IntWritable.class, WritableComparable.class); + job.getConfiguration().setClass(GiraphJob.VERTEX_VALUE_CLASS, + FloatWritable.class, Writable.class); + job.getConfiguration().setClass(GiraphJob.EDGE_VALUE_CLASS, + DoubleWritable.class, Writable.class); + job.getConfiguration().setClass(GiraphJob.MESSAGE_VALUE_CLASS, + LongWritable.class, Writable.class); + vertex = (IFDLEdgeListVertex) + BspUtils. + createVertex(job.getConfiguration()); + } - public void testEdges() { - Map edgeMap = Maps.newHashMap(); - for (int i = 1000; i > 0; --i) { - edgeMap.put(new IntWritable(i), new DoubleWritable(i * 2.0)); - } - vertex.initialize(null, null, edgeMap, null); - assertEquals(vertex.getNumOutEdges(), 1000); - int expectedIndex = 1; - for (IntWritable index : vertex) { - assertEquals(index.get(), expectedIndex); - assertEquals(vertex.getEdgeValue(index).get(), - expectedIndex * 2.0d); - ++expectedIndex; - } - assertEquals(vertex.removeEdge(new IntWritable(500)), - new DoubleWritable(1000)); - assertEquals(vertex.getNumOutEdges(), 999); - } + public void testInstantiate() throws IOException { + assertNotNull(vertex); + } - public void testGetEdges() { - Map edgeMap = Maps.newHashMap(); - for (int i = 1000; i > 0; --i) { - edgeMap.put(new IntWritable(i), new DoubleWritable(i * 3.0)); - } - vertex.initialize(null, null, edgeMap, null); - assertEquals(vertex.getNumOutEdges(), 1000); - assertEquals(vertex.getEdgeValue(new IntWritable(600)), - new DoubleWritable(600 * 3.0)); - assertEquals(vertex.removeEdge(new IntWritable(600)), - new DoubleWritable(600 * 3.0)); - assertEquals(vertex.getNumOutEdges(), 999); - assertEquals(vertex.getEdgeValue(new IntWritable(500)), - new DoubleWritable(500 * 3.0)); - assertEquals(vertex.getEdgeValue(new IntWritable(700)), - new DoubleWritable(700 * 3.0)); + public void testEdges() { + Map edgeMap = Maps.newHashMap(); + for (int i = 1000; i > 0; --i) { + edgeMap.put(new IntWritable(i), new DoubleWritable(i * 2.0)); + } + vertex.initialize(null, null, edgeMap, null); + assertEquals(vertex.getNumOutEdges(), 1000); + int expectedIndex = 1; + for (IntWritable index : vertex) { + assertEquals(index.get(), expectedIndex); + assertEquals(vertex.getEdgeValue(index).get(), + expectedIndex * 2.0d); + ++expectedIndex; } + assertEquals(vertex.removeEdge(new IntWritable(500)), + new DoubleWritable(1000)); + assertEquals(vertex.getNumOutEdges(), 999); + } - public void testAddRemoveEdges() { - Map edgeMap = Maps.newHashMap(); - vertex.initialize(null, null, edgeMap, null); - assertEquals(vertex.getNumOutEdges(), 0); - assertTrue(vertex.addEdge(new IntWritable(2), - new DoubleWritable(2.0))); - assertEquals(vertex.getNumOutEdges(), 1); - assertEquals(vertex.getEdgeValue(new IntWritable(2)), - new DoubleWritable(2.0)); - assertTrue(vertex.addEdge(new IntWritable(4), - new DoubleWritable(4.0))); - assertTrue(vertex.addEdge(new IntWritable(3), - new DoubleWritable(3.0))); - assertTrue(vertex.addEdge(new IntWritable(1), - new DoubleWritable(1.0))); - assertEquals(vertex.getNumOutEdges(), 4); - assertNull(vertex.getEdgeValue(new IntWritable(5))); - assertNull(vertex.getEdgeValue(new IntWritable(0))); - int i = 1; - for (IntWritable edgeDestId : vertex) { - assertEquals(i, edgeDestId.get()); - assertEquals(i * 1.0d, vertex.getEdgeValue(edgeDestId).get()); - ++i; - } - assertNotNull(vertex.removeEdge(new IntWritable(1))); - assertEquals(vertex.getNumOutEdges(), 3); - assertNotNull(vertex.removeEdge(new IntWritable(3))); - assertEquals(vertex.getNumOutEdges(), 2); - assertNotNull(vertex.removeEdge(new IntWritable(2))); - assertEquals(vertex.getNumOutEdges(), 1); - assertNotNull(vertex.removeEdge(new IntWritable(4))); - assertEquals(vertex.getNumOutEdges(), 0); + public void testGetEdges() { + Map edgeMap = Maps.newHashMap(); + for (int i = 1000; i > 0; --i) { + edgeMap.put(new IntWritable(i), new DoubleWritable(i * 3.0)); } + vertex.initialize(null, null, edgeMap, null); + assertEquals(vertex.getNumOutEdges(), 1000); + assertEquals(vertex.getEdgeValue(new IntWritable(600)), + new DoubleWritable(600 * 3.0)); + assertEquals(vertex.removeEdge(new IntWritable(600)), + new DoubleWritable(600 * 3.0)); + assertEquals(vertex.getNumOutEdges(), 999); + assertEquals(vertex.getEdgeValue(new IntWritable(500)), + new DoubleWritable(500 * 3.0)); + assertEquals(vertex.getEdgeValue(new IntWritable(700)), + new DoubleWritable(700 * 3.0)); + } + public void testAddRemoveEdges() { + Map edgeMap = Maps.newHashMap(); + vertex.initialize(null, null, edgeMap, null); + assertEquals(vertex.getNumOutEdges(), 0); + assertTrue(vertex.addEdge(new IntWritable(2), + new DoubleWritable(2.0))); + assertEquals(vertex.getNumOutEdges(), 1); + assertEquals(vertex.getEdgeValue(new IntWritable(2)), + new DoubleWritable(2.0)); + assertTrue(vertex.addEdge(new IntWritable(4), + new DoubleWritable(4.0))); + assertTrue(vertex.addEdge(new IntWritable(3), + new DoubleWritable(3.0))); + assertTrue(vertex.addEdge(new IntWritable(1), + new DoubleWritable(1.0))); + assertEquals(vertex.getNumOutEdges(), 4); + assertNull(vertex.getEdgeValue(new IntWritable(5))); + assertNull(vertex.getEdgeValue(new IntWritable(0))); + int i = 1; + for (IntWritable edgeDestId : vertex) { + assertEquals(i, edgeDestId.get()); + assertEquals(i * 1.0d, vertex.getEdgeValue(edgeDestId).get()); + ++i; + } + assertNotNull(vertex.removeEdge(new IntWritable(1))); + assertEquals(vertex.getNumOutEdges(), 3); + assertNotNull(vertex.removeEdge(new IntWritable(3))); + assertEquals(vertex.getNumOutEdges(), 2); + assertNotNull(vertex.removeEdge(new IntWritable(2))); + assertEquals(vertex.getNumOutEdges(), 1); + assertNotNull(vertex.removeEdge(new IntWritable(4))); + assertEquals(vertex.getNumOutEdges(), 0); + } - public void testSerialize() { - Map edgeMap = Maps.newHashMap(); - for (int i = 1000; i > 0; --i) { - edgeMap.put(new IntWritable(i), new DoubleWritable(i * 2.0)); - } - List messageList = Lists.newArrayList(); - messageList.add(new LongWritable(4)); - messageList.add(new LongWritable(5)); - vertex.initialize( - new IntWritable(2), new FloatWritable(3.0f), edgeMap, messageList); - byte[] byteArray = WritableUtils.writeToByteArray(vertex); - IFDLEdgeListVertex readVertex = (IFDLEdgeListVertex) - BspUtils. - createVertex(job.getConfiguration()); - WritableUtils.readFieldsFromByteArray(byteArray, readVertex); - assertEquals(vertex, readVertex); + public void testSerialize() { + Map edgeMap = Maps.newHashMap(); + for (int i = 1000; i > 0; --i) { + edgeMap.put(new IntWritable(i), new DoubleWritable(i * 2.0)); } + List messageList = Lists.newArrayList(); + messageList.add(new LongWritable(4)); + messageList.add(new LongWritable(5)); + vertex.initialize( + new IntWritable(2), new FloatWritable(3.0f), edgeMap, messageList); + byte[] byteArray = WritableUtils.writeToByteArray(vertex); + IFDLEdgeListVertex readVertex = (IFDLEdgeListVertex) + BspUtils. + createVertex(job.getConfiguration()); + WritableUtils.readFieldsFromByteArray(byteArray, readVertex); + assertEquals(vertex, readVertex); + } } diff --git a/src/test/java/org/apache/giraph/utils/MockUtils.java b/src/test/java/org/apache/giraph/utils/MockUtils.java index 93418abcb..6ea0aec72 100644 --- a/src/test/java/org/apache/giraph/utils/MockUtils.java +++ b/src/test/java/org/apache/giraph/utils/MockUtils.java @@ -44,7 +44,7 @@ private MockUtils() { public static class MockedEnvironment { - private final GraphState graphState; + private final GraphState graphState; private final Mapper.Context context; private final Configuration conf; private final WorkerCommunications communications; diff --git a/src/test/java/zk/TestZooKeeperManager.java b/src/test/java/zk/TestZooKeeperManager.java index 7b95603c6..899fd1e6e 100644 --- a/src/test/java/zk/TestZooKeeperManager.java +++ b/src/test/java/zk/TestZooKeeperManager.java @@ -31,7 +31,8 @@ public void testGetBasePath() { "", ZooKeeperManager.getBasePath(conf)); conf.set(GiraphJob.BASE_ZNODE_KEY, "/howdy"); - assertEquals("Base path should reflect value of " + GiraphJob.BASE_ZNODE_KEY, + assertEquals("Base path should reflect value of " + + GiraphJob.BASE_ZNODE_KEY, "/howdy", ZooKeeperManager.getBasePath(conf)); conf.set(GiraphJob.BASE_ZNODE_KEY, "no_slash");