Index: examples/src/main/java/org/apache/hama/examples/PageRank.java =================================================================== --- examples/src/main/java/org/apache/hama/examples/PageRank.java (revision 1214052) +++ examples/src/main/java/org/apache/hama/examples/PageRank.java (working copy) @@ -42,18 +42,18 @@ import org.apache.hama.bsp.SequenceFileInputFormat; import org.apache.hama.bsp.SequenceFileOutputFormat; import org.apache.hama.bsp.sync.SyncException; -import org.apache.hama.graph.Vertex; +import org.apache.hama.graph.VertexWritable; import org.apache.hama.util.KeyValuePair; public class PageRank extends - BSP { + BSP { public static final Log LOG = LogFactory.getLog(PageRank.class); - private final HashMap adjacencyList = new HashMap(); - private final HashMap vertexLookupMap = new HashMap(); - private final HashMap tentativePagerank = new HashMap(); + private final HashMap adjacencyList = new HashMap(); + private final HashMap vertexLookupMap = new HashMap(); + private final HashMap tentativePagerank = new HashMap(); // backup of the last pagerank to determine the error - private final HashMap lastTentativePagerank = new HashMap(); + private final HashMap lastTentativePagerank = new HashMap(); protected static int MAX_ITERATIONS = 30; protected static String masterTaskName; @@ -64,11 +64,11 @@ @Override public void setup( - BSPPeer peer) + BSPPeer peer) throws IOException { // map our stuff into ram - KeyValuePair next = null; + KeyValuePair next = null; while ((next = peer.readNext()) != null) { adjacencyList.put(next.getKey(), (ShortestPathVertex[]) next.getValue() .toArray()); @@ -86,7 +86,7 @@ @Override public void bsp( - BSPPeer peer) + BSPPeer peer) throws IOException, SyncException, InterruptedException { // while the error not converges against epsilon do the pagerank stuff @@ -102,10 +102,10 @@ // copy the old pagerank to the backup copyTentativePageRankToBackup(); // sum up all incoming messages for a vertex - HashMap sumMap = new HashMap(); + HashMap sumMap = new HashMap(); DoubleMessage msg = null; while ((msg = (DoubleMessage) peer.getCurrentMessage()) != null) { - Vertex k = vertexLookupMap.get(msg.getTag()); + VertexWritable k = vertexLookupMap.get(msg.getTag()); if (k == null) { LOG.fatal("If you see this, partitioning has totally failed."); } @@ -118,7 +118,7 @@ // pregel formula: // ALPHA = 0.15 / NumVertices() // P(i) = ALPHA + 0.85 * sum - for (Entry entry : sumMap.entrySet()) { + for (Entry entry : sumMap.entrySet()) { tentativePagerank.put(entry.getKey(), ALPHA + (entry.getValue() * DAMPING_FACTOR)); } @@ -129,7 +129,7 @@ } // in every step send the tentative pagerank of a vertex to its // adjacent vertices - for (Vertex vertex : adjacencyList.keySet()) { + for (VertexWritable vertex : adjacencyList.keySet()) { sendMessageToNeighbors(peer, vertex); } @@ -142,9 +142,9 @@ @Override public void cleanup( - BSPPeer peer) { + BSPPeer peer) { try { - for (Entry row : tentativePagerank.entrySet()) { + for (Entry row : tentativePagerank.entrySet()) { peer.write(new Text(row.getKey().getName()), new DoubleWritable(row .getValue())); } @@ -154,7 +154,7 @@ } private double broadcastError( - BSPPeer peer, + BSPPeer peer, double error) throws IOException, SyncException, InterruptedException { peer.send(masterTaskName, new DoubleMessage("", error)); peer.sync(); @@ -180,7 +180,7 @@ private double determineError() { double error = 0.0; - for (Entry entry : tentativePagerank.entrySet()) { + for (Entry entry : tentativePagerank.entrySet()) { error += Math.abs(lastTentativePagerank.get(entry.getKey()) - entry.getValue()); } @@ -188,16 +188,16 @@ } private void copyTentativePageRankToBackup() { - for (Entry entry : tentativePagerank.entrySet()) { + for (Entry entry : tentativePagerank.entrySet()) { lastTentativePagerank.put(entry.getKey(), entry.getValue()); } } private void sendMessageToNeighbors( - BSPPeer peer, - Vertex v) throws IOException { - Vertex[] outgoingEdges = adjacencyList.get(v); - for (Vertex adjacent : outgoingEdges) { + BSPPeer peer, + VertexWritable v) throws IOException { + VertexWritable[] outgoingEdges = adjacencyList.get(v); + for (VertexWritable adjacent : outgoingEdges) { int mod = Math.abs(adjacent.hashCode() % peer.getNumPeers()); // send a message of the tentative pagerank divided by the size of // the outgoing edges to all adjacents Index: examples/src/main/java/org/apache/hama/examples/ShortestPathVertex.java =================================================================== --- examples/src/main/java/org/apache/hama/examples/ShortestPathVertex.java (revision 1214052) +++ examples/src/main/java/org/apache/hama/examples/ShortestPathVertex.java (working copy) @@ -22,9 +22,9 @@ import java.io.IOException; import org.apache.hadoop.io.WritableComparable; -import org.apache.hama.graph.Vertex; +import org.apache.hama.graph.VertexWritable; -public final class ShortestPathVertex extends Vertex implements +public final class ShortestPathVertex extends VertexWritable implements WritableComparable { private int weight; Index: graph/src/main/java/org/apache/hama/graph/Vertex.java =================================================================== --- graph/src/main/java/org/apache/hama/graph/Vertex.java (revision 1214053) +++ graph/src/main/java/org/apache/hama/graph/Vertex.java (working copy) @@ -1,72 +0,0 @@ -/** - * 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.hama.graph; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; - -import org.apache.hadoop.io.Writable; - -public class Vertex implements Writable { - - protected String name; - - public Vertex() { - super(); - } - - public Vertex(String name) { - super(); - this.name = name; - } - - @Override - public void readFields(DataInput in) throws IOException { - this.name = in.readUTF(); - } - - @Override - public void write(DataOutput out) throws IOException { - out.writeUTF(name); - } - - @Override - public int hashCode() { - return name.hashCode(); - } - - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - Vertex other = (Vertex) obj; - if (!name.equals(other.name)) - return false; - return true; - } - - public String getName() { - return name; - } - -} Index: graph/src/main/java/org/apache/hama/graph/VertexArrayWritable.java =================================================================== --- graph/src/main/java/org/apache/hama/graph/VertexArrayWritable.java (revision 1214053) +++ graph/src/main/java/org/apache/hama/graph/VertexArrayWritable.java (working copy) @@ -22,7 +22,7 @@ public class VertexArrayWritable extends ArrayWritable { public VertexArrayWritable() { - super(Vertex.class); + super(VertexWritable.class); } } Index: graph/src/main/java/org/apache/hama/graph/VertexWritable.java =================================================================== --- graph/src/main/java/org/apache/hama/graph/VertexWritable.java (revision 0) +++ graph/src/main/java/org/apache/hama/graph/VertexWritable.java (revision 0) @@ -0,0 +1,72 @@ +/** + * 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.hama.graph; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import org.apache.hadoop.io.Writable; + +public class VertexWritable implements Writable { + + protected String name; + + public VertexWritable() { + super(); + } + + public VertexWritable(String name) { + super(); + this.name = name; + } + + @Override + public void readFields(DataInput in) throws IOException { + this.name = in.readUTF(); + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeUTF(name); + } + + @Override + public int hashCode() { + return name.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + VertexWritable other = (VertexWritable) obj; + if (!name.equals(other.name)) + return false; + return true; + } + + public String getName() { + return name; + } + +} \ No newline at end of file