cross merge fullstack_release_candidate into trunk
git-svn-id: https://hyracks.googlecode.com/svn/trunk/fullstack@3208 123451ca-8445-de46-9d55-352943316053
diff --git a/pregelix/pom.xml b/pregelix/pom.xml
index 4b6a968..7d08fb7 100644
--- a/pregelix/pom.xml
+++ b/pregelix/pom.xml
@@ -44,6 +44,7 @@
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
+ <version>2.13</version>
<configuration>
<forkMode>pertest</forkMode>
<argLine>-enableassertions -Djava.util.logging.config.file=${user.home}/logging.properties -Xdebug -Xrunjdwp:transport=dt_socket,server=y,address=8000,suspend=n ${jvm.extraargs}</argLine>
@@ -57,6 +58,7 @@
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-changelog-plugin</artifactId>
+ <version>2.2</version>
</plugin>
</plugins>
</reporting>
diff --git a/pregelix/pregelix-api/pom.xml b/pregelix/pregelix-api/pom.xml
index 66a0186..8212e1c 100644
--- a/pregelix/pregelix-api/pom.xml
+++ b/pregelix/pregelix-api/pom.xml
@@ -21,8 +21,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
<plugin>
@@ -41,6 +42,7 @@
</plugin>
<plugin>
<artifactId>maven-clean-plugin</artifactId>
+ <version>2.5</version>
<configuration>
<filesets>
<fileset>
@@ -76,11 +78,11 @@
<scope>test</scope>
</dependency>
<dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-core</artifactId>
- <version>0.20.2</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-hdfs-core</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
</dependencies>
</project>
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Edge.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Edge.java
index 4af35fe..e5f42fe 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Edge.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Edge.java
@@ -42,7 +42,7 @@
private E edgeValue = null;
/** Configuration - Used to instantiate classes */
private Configuration conf = null;
- /** Whether the edgeValue field is not null*/
+ /** Whether the edgeValue field is not null */
private boolean hasEdgeValue = false;
/**
@@ -115,8 +115,9 @@
destVertexId.readFields(input);
hasEdgeValue = input.readBoolean();
if (hasEdgeValue) {
- if (edgeValue == null)
+ if (edgeValue == null) {
edgeValue = (E) BspUtils.createEdgeValue(getConf());
+ }
edgeValue.readFields(input);
}
}
@@ -128,8 +129,9 @@
}
destVertexId.write(output);
output.writeBoolean(hasEdgeValue);
- if (hasEdgeValue)
+ if (hasEdgeValue) {
edgeValue.write(output);
+ }
}
@Override
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/MsgList.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/MsgList.java
index 734b1af..8d3d4c6 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/MsgList.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/MsgList.java
@@ -29,7 +29,7 @@
*/
public class MsgList<M extends Writable> extends ArrayListWritable<M> {
/** Defining a layout version for a serializable class. */
- private static final long serialVersionUID = 100L;
+ private static final long serialVersionUID = 1L;
/**
* Default constructor.s
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java
index 6856e9a..cd49184 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java
@@ -26,7 +26,7 @@
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -49,384 +49,525 @@
*/
@SuppressWarnings("rawtypes")
public abstract class Vertex<I extends WritableComparable, V extends Writable, E extends Writable, M extends Writable>
- implements Writable {
- private static long superstep = 0;
- /** Class-wide number of vertices */
- private static long numVertices = -1;
- /** Class-wide number of edges */
- private static long numEdges = -1;
- /** Vertex id */
- private I vertexId = null;
- /** Vertex value */
- private V vertexValue = null;
- /** Map of destination vertices and their edge values */
- private final List<Edge<I, E>> destEdgeList = new ArrayList<Edge<I, E>>();
- /** If true, do not do anymore computation on this vertex. */
- boolean halt = false;
- /** List of incoming messages from the previous superstep */
- private final List<M> msgList = new ArrayList<M>();
- /** map context */
- private static Mapper.Context context = null;
- /** a delegate for hyracks stuff */
- private VertexDelegate<I, V, E, M> delegate = new VertexDelegate<I, V, E, M>(this);
- /** this vertex is updated or not */
- private boolean updated = false;
- /** has outgoing messages */
- private boolean hasMessage = false;
+ implements Writable {
+ private static long superstep = 0;
+ /** Class-wide number of vertices */
+ private static long numVertices = -1;
+ /** Class-wide number of edges */
+ private static long numEdges = -1;
+ /** Vertex id */
+ private I vertexId = null;
+ /** Vertex value */
+ private V vertexValue = null;
+ /** Map of destination vertices and their edge values */
+ private final List<Edge<I, E>> destEdgeList = new ArrayList<Edge<I, E>>();
+ /** If true, do not do anymore computation on this vertex. */
+ boolean halt = false;
+ /** List of incoming messages from the previous superstep */
+ private final List<M> msgList = new ArrayList<M>();
+ /** map context */
+ private static TaskAttemptContext context = null;
+ /** a delegate for hyracks stuff */
+ private VertexDelegate<I, V, E, M> delegate = new VertexDelegate<I, V, E, M>(
+ this);
+ /** this vertex is updated or not */
+ private boolean updated = false;
+ /** has outgoing messages */
+ private boolean hasMessage = false;
+ /** created new vertex */
+ private boolean createdNewLiveVertex = false;
- /**
- * use object pool for re-using objects
- */
- private List<Edge<I, E>> edgePool = new ArrayList<Edge<I, E>>();
- private List<M> msgPool = new ArrayList<M>();
- private List<V> valuePool = new ArrayList<V>();
- private int usedEdge = 0;
- private int usedMessage = 0;
- private int usedValue = 0;
+ /**
+ * use object pool for re-using objects
+ */
+ private List<Edge<I, E>> edgePool = new ArrayList<Edge<I, E>>();
+ private List<M> msgPool = new ArrayList<M>();
+ private List<V> valuePool = new ArrayList<V>();
+ private int usedEdge = 0;
+ private int usedMessage = 0;
+ private int usedValue = 0;
- /**
- * The key method that users need to implement
- *
- * @param msgIterator
- * an iterator of incoming messages
- */
- public abstract void compute(Iterator<M> msgIterator);
+ /**
+ * The key method that users need to implement
+ *
+ * @param msgIterator
+ * an iterator of incoming messages
+ */
+ public abstract void compute(Iterator<M> msgIterator);
- /**
- * Add an edge for the vertex.
- *
- * @param targetVertexId
- * @param edgeValue
- * @return successful or not
- */
- public final boolean addEdge(I targetVertexId, E edgeValue) {
- Edge<I, E> edge = this.allocateEdge();
- edge.setDestVertexId(targetVertexId);
- edge.setEdgeValue(edgeValue);
- destEdgeList.add(edge);
- return true;
- }
+ /**
+ * Add an edge for the vertex.
+ *
+ * @param targetVertexId
+ * @param edgeValue
+ * @return successful or not
+ */
+ public final boolean addEdge(I targetVertexId, E edgeValue) {
+ Edge<I, E> edge = this.allocateEdge();
+ edge.setDestVertexId(targetVertexId);
+ edge.setEdgeValue(edgeValue);
+ destEdgeList.add(edge);
+ updated = true;
+ return true;
+ }
- /**
- * Initialize a new vertex
- *
- * @param vertexId
- * @param vertexValue
- * @param edges
- * @param messages
- */
- public void initialize(I vertexId, V vertexValue, Map<I, E> edges, List<M> messages) {
- if (vertexId != null) {
- setVertexId(vertexId);
- }
- if (vertexValue != null) {
- setVertexValue(vertexValue);
- }
- destEdgeList.clear();
- if (edges != null && !edges.isEmpty()) {
- for (Map.Entry<I, E> entry : edges.entrySet()) {
- destEdgeList.add(new Edge<I, E>(entry.getKey(), entry.getValue()));
- }
- }
- if (messages != null && !messages.isEmpty()) {
- msgList.addAll(messages);
- }
- }
+ /**
+ * Initialize a new vertex
+ *
+ * @param vertexId
+ * @param vertexValue
+ * @param edges
+ * @param messages
+ */
+ public void initialize(I vertexId, V vertexValue, Map<I, E> edges,
+ List<M> messages) {
+ if (vertexId != null) {
+ setVertexId(vertexId);
+ }
+ if (vertexValue != null) {
+ setVertexValue(vertexValue);
+ }
+ destEdgeList.clear();
+ if (edges != null && !edges.isEmpty()) {
+ for (Map.Entry<I, E> entry : edges.entrySet()) {
+ destEdgeList.add(new Edge<I, E>(entry.getKey(), entry
+ .getValue()));
+ }
+ }
+ if (messages != null && !messages.isEmpty()) {
+ msgList.addAll(messages);
+ }
+ }
- /**
- * reset a vertex object: clear its internal states
- */
- public void reset() {
- usedEdge = 0;
- usedMessage = 0;
- usedValue = 0;
- }
+ /**
+ * reset a vertex object: clear its internal states
+ */
+ public void reset() {
+ usedEdge = 0;
+ usedMessage = 0;
+ usedValue = 0;
+ updated = false;
+ }
- private Edge<I, E> allocateEdge() {
- Edge<I, E> edge;
- if (usedEdge < edgePool.size()) {
- edge = edgePool.get(usedEdge);
- usedEdge++;
- } else {
- edge = new Edge<I, E>();
- edgePool.add(edge);
- usedEdge++;
- }
- return edge;
- }
+ /**
+ * Set the vertex id
+ *
+ * @param vertexId
+ */
+ public final void setVertexId(I vertexId) {
+ this.vertexId = vertexId;
+ delegate.setVertexId(vertexId);
+ }
- private M allocateMessage() {
- M message;
- if (usedMessage < msgPool.size()) {
- message = msgPool.get(usedEdge);
- usedMessage++;
- } else {
- message = BspUtils.<M> createMessageValue(getContext().getConfiguration());
- msgPool.add(message);
- usedMessage++;
- }
- return message;
- }
+ /**
+ * Get the vertex id
+ *
+ * @return vertex id
+ */
+ public final I getVertexId() {
+ return vertexId;
+ }
- private V allocateValue() {
- V value;
- if (usedValue < valuePool.size()) {
- value = valuePool.get(usedEdge);
- usedValue++;
- } else {
- value = BspUtils.<V> createVertexValue(getContext().getConfiguration());
- valuePool.add(value);
- usedValue++;
- }
- return value;
- }
+ /**
+ * Get the vertex value
+ *
+ * @return the vertex value
+ */
+ public final V getVertexValue() {
+ return vertexValue;
+ }
- /**
- * Set the vertex id
- *
- * @param vertexId
- */
- public final void setVertexId(I vertexId) {
- this.vertexId = vertexId;
- delegate.setVertexId(vertexId);
- }
+ /**
+ * Set the vertex value
+ *
+ * @param vertexValue
+ */
+ public final void setVertexValue(V vertexValue) {
+ this.vertexValue = vertexValue;
+ this.updated = true;
+ }
- /**
- * Get the vertex id
- *
- * @return vertex id
- */
- public final I getVertexId() {
- return vertexId;
- }
+ /***
+ * Send a message to a specific vertex
+ *
+ * @param id
+ * the receiver vertex id
+ * @param msg
+ * the message
+ */
+ public final void sendMsg(I id, M msg) {
+ if (msg == null) {
+ throw new IllegalArgumentException(
+ "sendMsg: Cannot send null message to " + id);
+ }
+ delegate.sendMsg(id, msg);
+ this.hasMessage = true;
+ }
- /**
- * Set the global superstep for all the vertices (internal use)
- *
- * @param superstep
- * New superstep
- */
- public static void setSuperstep(long superstep) {
- Vertex.superstep = superstep;
- }
+ /**
+ * Send a message to all direct outgoing neighbors
+ *
+ * @param msg
+ * the message
+ */
+ public final void sendMsgToAllEdges(M msg) {
+ if (msg == null) {
+ throw new IllegalArgumentException(
+ "sendMsgToAllEdges: Cannot send null message to all edges");
+ }
+ for (Edge<I, E> edge : destEdgeList) {
+ sendMsg(edge.getDestVertexId(), msg);
+ }
+ }
- public static long getCurrentSuperstep() {
- return superstep;
- }
+ /**
+ * Vote to halt. Once all vertex vote to halt and no more messages, a
+ * Pregelix job will terminate.
+ */
+ public final void voteToHalt() {
+ halt = true;
+ updated = true;
+ }
- public final long getSuperstep() {
- return superstep;
- }
+ /**
+ * @return the vertex is halted (true) or not (false)
+ */
+ public final boolean isHalted() {
+ return halt;
+ }
- public final V getVertexValue() {
- return vertexValue;
- }
+ @Override
+ final public void readFields(DataInput in) throws IOException {
+ reset();
+ if (vertexId == null)
+ vertexId = BspUtils.<I> createVertexIndex(getContext()
+ .getConfiguration());
+ vertexId.readFields(in);
+ delegate.setVertexId(vertexId);
+ boolean hasVertexValue = in.readBoolean();
- public final void setVertexValue(V vertexValue) {
- this.vertexValue = vertexValue;
- this.updated = true;
- }
+ if (hasVertexValue) {
+ vertexValue = allocateValue();
+ vertexValue.readFields(in);
+ delegate.setVertex(this);
+ }
+ destEdgeList.clear();
+ long edgeMapSize = SerDeUtils.readVLong(in);
+ for (long i = 0; i < edgeMapSize; ++i) {
+ Edge<I, E> edge = allocateEdge();
+ edge.setConf(getContext().getConfiguration());
+ edge.readFields(in);
+ addEdge(edge);
+ }
+ msgList.clear();
+ long msgListSize = SerDeUtils.readVLong(in);
+ for (long i = 0; i < msgListSize; ++i) {
+ M msg = allocateMessage();
+ msg.readFields(in);
+ msgList.add(msg);
+ }
+ halt = in.readBoolean();
+ updated = false;
+ hasMessage = false;
+ createdNewLiveVertex = false;
+ }
- /**
- * Set the total number of vertices from the last superstep.
- *
- * @param numVertices
- * Aggregate vertices in the last superstep
- */
- public static void setNumVertices(long numVertices) {
- Vertex.numVertices = numVertices;
- }
+ @Override
+ public void write(DataOutput out) throws IOException {
+ vertexId.write(out);
+ out.writeBoolean(vertexValue != null);
+ if (vertexValue != null) {
+ vertexValue.write(out);
+ }
+ SerDeUtils.writeVLong(out, destEdgeList.size());
+ for (Edge<I, E> edge : destEdgeList) {
+ edge.write(out);
+ }
+ SerDeUtils.writeVLong(out, msgList.size());
+ for (M msg : msgList) {
+ msg.write(out);
+ }
+ out.writeBoolean(halt);
+ }
- public final long getNumVertices() {
- return numVertices;
- }
+ /**
+ * Get the list of incoming messages
+ *
+ * @return the list of messages
+ */
+ public List<M> getMsgList() {
+ return msgList;
+ }
- /**
- * Set the total number of edges from the last superstep.
- *
- * @param numEdges
- * Aggregate edges in the last superstep
- */
- public static void setNumEdges(long numEdges) {
- Vertex.numEdges = numEdges;
- }
+ /**
+ * Get outgoing edge list
+ *
+ * @return a list of outgoing edges
+ */
+ public List<Edge<I, E>> getEdges() {
+ return this.destEdgeList;
+ }
- public final long getNumEdges() {
- return numEdges;
- }
+ @Override
+ @SuppressWarnings("unchecked")
+ public String toString() {
+ Collections.sort(destEdgeList);
+ StringBuffer edgeBuffer = new StringBuffer();
+ edgeBuffer.append("(");
+ for (Edge<I, E> edge : destEdgeList) {
+ edgeBuffer.append(edge.getDestVertexId()).append(",");
+ }
+ edgeBuffer.append(")");
+ return "Vertex(id=" + getVertexId() + ",value=" + getVertexValue()
+ + ", edges=" + edgeBuffer + ")";
+ }
- /***
- * Send a message to a specific vertex
- *
- * @param id
- * the receiver vertex id
- * @param msg
- * the message
- */
- public final void sendMsg(I id, M msg) {
- if (msg == null) {
- throw new IllegalArgumentException("sendMsg: Cannot send null message to " + id);
- }
- delegate.sendMsg(id, msg);
- this.hasMessage = true;
- }
+ /**
+ * Get the number of outgoing edges
+ *
+ * @return the number of outging edges
+ */
+ public int getNumOutEdges() {
+ return destEdgeList.size();
+ }
- /**
- * Send a message to all direct outgoing neighbors
- *
- * @param msg
- * the message
- */
- public final void sendMsgToAllEdges(M msg) {
- if (msg == null) {
- throw new IllegalArgumentException("sendMsgToAllEdges: Cannot send null message to all edges");
- }
- for (Edge<I, E> edge : destEdgeList) {
- sendMsg(edge.getDestVertexId(), msg);
- }
- }
+ /**
+ * Pregelix internal use only
+ *
+ * @param writers
+ */
+ public void setOutputWriters(List<IFrameWriter> writers) {
+ delegate.setOutputWriters(writers);
+ }
- /**
- * Vote to halt. Once all vertex vote to halt and no more messages, a
- * Pregelix job will terminate.
- */
- public final void voteToHalt() {
- halt = true;
- }
+ /**
+ * Pregelix internal use only
+ *
+ * @param writers
+ */
+ public void setOutputAppenders(List<FrameTupleAppender> appenders) {
+ delegate.setOutputAppenders(appenders);
+ }
- /**
- * @return the vertex is halted (true) or not (false)
- */
- public final boolean isHalted() {
- return halt;
- }
+ /**
+ * Pregelix internal use only
+ *
+ * @param writers
+ */
+ public void setOutputTupleBuilders(List<ArrayTupleBuilder> tbs) {
+ delegate.setOutputTupleBuilders(tbs);
+ }
- @Override
- final public void readFields(DataInput in) throws IOException {
- reset();
- if (vertexId == null)
- vertexId = BspUtils.<I> createVertexIndex(getContext().getConfiguration());
- vertexId.readFields(in);
- delegate.setVertexId(vertexId);
- boolean hasVertexValue = in.readBoolean();
- if (hasVertexValue) {
- vertexValue = allocateValue();
- vertexValue.readFields(in);
- delegate.setVertex(this);
- }
- destEdgeList.clear();
- long edgeMapSize = SerDeUtils.readVLong(in);
- for (long i = 0; i < edgeMapSize; ++i) {
- Edge<I, E> edge = allocateEdge();
- edge.setConf(getContext().getConfiguration());
- edge.readFields(in);
- addEdge(edge);
- }
- msgList.clear();
- long msgListSize = SerDeUtils.readVLong(in);
- for (long i = 0; i < msgListSize; ++i) {
- M msg = allocateMessage();
- msg.readFields(in);
- msgList.add(msg);
- }
- halt = in.readBoolean();
- updated = false;
- hasMessage = false;
- }
+ /**
+ * Pregelix internal use only
+ *
+ * @param writers
+ */
+ public void finishCompute() throws IOException {
+ delegate.finishCompute();
+ }
- @Override
- public void write(DataOutput out) throws IOException {
- vertexId.write(out);
- out.writeBoolean(vertexValue != null);
- if (vertexValue != null) {
- vertexValue.write(out);
- }
- SerDeUtils.writeVLong(out, destEdgeList.size());
- for (Edge<I, E> edge : destEdgeList) {
- edge.write(out);
- }
- SerDeUtils.writeVLong(out, msgList.size());
- for (M msg : msgList) {
- msg.write(out);
- }
- out.writeBoolean(halt);
- }
+ /**
+ * Pregelix internal use only
+ */
+ public boolean hasUpdate() {
+ return this.updated;
+ }
- private boolean addEdge(Edge<I, E> edge) {
- edge.setConf(getContext().getConfiguration());
- destEdgeList.add(edge);
- return true;
- }
+ /**
+ * Pregelix internal use only
+ */
+ public boolean hasMessage() {
+ return this.hasMessage;
+ }
- /**
- * Get the list of incoming messages
- *
- * @return the list of messages
- */
- public List<M> getMsgList() {
- return msgList;
- }
+ /**
+ * Pregelix internal use only
+ */
+ public boolean createdNewLiveVertex() {
+ return this.createdNewLiveVertex;
+ }
- /**
- * Get outgoing edge list
- *
- * @return a list of outgoing edges
- */
- public List<Edge<I, E>> getEdges() {
- return this.destEdgeList;
- }
+ /**
+ * sort the edges
+ */
+ @SuppressWarnings("unchecked")
+ public void sortEdges() {
+ updated = true;
+ Collections.sort(destEdgeList);
+ }
- public final Mapper<?, ?, ?, ?>.Context getContext() {
- return context;
- }
+ /**
+ * Allocate a new edge from the edge pool
+ */
+ private Edge<I, E> allocateEdge() {
+ Edge<I, E> edge;
+ if (usedEdge < edgePool.size()) {
+ edge = edgePool.get(usedEdge);
+ usedEdge++;
+ } else {
+ edge = new Edge<I, E>();
+ edgePool.add(edge);
+ usedEdge++;
+ }
+ return edge;
+ }
- public final static void setContext(Mapper<?, ?, ?, ?>.Context context) {
- Vertex.context = context;
- }
+ /**
+ * Allocate a new message from the message pool
+ */
+ private M allocateMessage() {
+ M message;
+ if (usedMessage < msgPool.size()) {
+ message = msgPool.get(usedEdge);
+ usedMessage++;
+ } else {
+ message = BspUtils.<M> createMessageValue(getContext()
+ .getConfiguration());
+ msgPool.add(message);
+ usedMessage++;
+ }
+ return message;
+ }
- @SuppressWarnings("unchecked")
- public String toString() {
- Collections.sort(destEdgeList);
- StringBuffer edgeBuffer = new StringBuffer();
- edgeBuffer.append("(");
- for (Edge<I, E> edge : destEdgeList) {
- edgeBuffer.append(edge.getDestVertexId()).append(",");
- }
- edgeBuffer.append(")");
- return "Vertex(id=" + getVertexId() + ",value=" + getVertexValue() + ", edges=" + edgeBuffer + ")";
- }
+ /**
+ * Set the global superstep for all the vertices (internal use)
+ *
+ * @param superstep
+ * New superstep
+ */
+ public static final void setSuperstep(long superstep) {
+ Vertex.superstep = superstep;
+ }
- public void setOutputWriters(List<IFrameWriter> writers) {
- delegate.setOutputWriters(writers);
- }
+ /**
+ * Add an outgoing edge into the vertex
+ *
+ * @param edge
+ * the edge to be added
+ * @return true if the edge list changed as a result of this call
+ */
+ public boolean addEdge(Edge<I, E> edge) {
+ edge.setConf(getContext().getConfiguration());
+ updated = true;
+ return destEdgeList.add(edge);
+ }
- public void setOutputAppenders(List<FrameTupleAppender> appenders) {
- delegate.setOutputAppenders(appenders);
- }
+ /**
+ * remove an outgoing edge in the graph
+ *
+ * @param edge
+ * the edge to be removed
+ * @return true if the edge is in the edge list of the vertex
+ */
+ public boolean removeEdge(Edge<I, E> edge) {
+ updated = true;
+ return destEdgeList.remove(edge);
+ }
- public void setOutputTupleBuilders(List<ArrayTupleBuilder> tbs) {
- delegate.setOutputTupleBuilders(tbs);
- }
+ /**
+ * Add a new vertex into the graph
+ *
+ * @param vertexId
+ * the vertex id
+ * @param vertex
+ * the vertex
+ */
+ public final void addVertex(I vertexId, Vertex vertex) {
+ createdNewLiveVertex |= !vertex.isHalted();
+ delegate.addVertex(vertexId, vertex);
+ }
- public void finishCompute() throws IOException {
- delegate.finishCompute();
- }
+ /**
+ * Delete a vertex from id
+ *
+ * @param vertexId
+ * the vertex id
+ */
+ public final void deleteVertex(I vertexId) {
+ delegate.deleteVertex(vertexId);
+ }
- public boolean hasUpdate() {
- return this.updated;
- }
+ /**
+ * Allocate a vertex value from the object pool
+ *
+ * @return a vertex value instance
+ */
+ private V allocateValue() {
+ V value;
+ if (usedValue < valuePool.size()) {
+ value = valuePool.get(usedValue);
+ usedValue++;
+ } else {
+ value = BspUtils.<V> createVertexValue(getContext()
+ .getConfiguration());
+ valuePool.add(value);
+ usedValue++;
+ }
+ return value;
+ }
- public boolean hasMessage() {
- return this.hasMessage;
- }
+ /**
+ * Get the current global superstep number
+ *
+ * @return the current superstep number
+ */
+ public static final long getSuperstep() {
+ return superstep;
+ }
- public int getNumOutEdges() {
- return destEdgeList.size();
- }
+ /**
+ * Set the total number of vertices from the last superstep.
+ *
+ * @param numVertices
+ * Aggregate vertices in the last superstep
+ */
+ public static final void setNumVertices(long numVertices) {
+ Vertex.numVertices = numVertices;
+ }
- @SuppressWarnings("unchecked")
- public void sortEdges() {
- Collections.sort((List) destEdgeList);
- }
+ /**
+ * Get the number of vertexes in the graph
+ *
+ * @return the number of vertexes in the graph
+ */
+ public static final long getNumVertices() {
+ return numVertices;
+ }
+
+ /**
+ * Set the total number of edges from the last superstep.
+ *
+ * @param numEdges
+ * Aggregate edges in the last superstep
+ */
+ public static void setNumEdges(long numEdges) {
+ Vertex.numEdges = numEdges;
+ }
+
+ /**
+ * Get the number of edges from this graph
+ *
+ * @return the number of edges in the graph
+ */
+ public static final long getNumEdges() {
+ return numEdges;
+ }
+
+ /**
+ * Pregelix internal use only
+ */
+ public static final TaskAttemptContext getContext() {
+ return context;
+ }
+
+ /**
+ * Pregelix internal use only
+ *
+ * @param context
+ */
+ public static final void setContext(TaskAttemptContext context) {
+ Vertex.context = context;
+ }
}
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/VertexDelegate.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/VertexDelegate.java
index 7267f30..4b153c1 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/VertexDelegate.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/VertexDelegate.java
@@ -44,6 +44,16 @@
private IFrameWriter aliveWriter;
private FrameTupleAppender appenderAlive;
+ /** the tuple for insert */
+ private ArrayTupleBuilder insertTb;
+ private IFrameWriter insertWriter;
+ private FrameTupleAppender appenderInsert;
+
+ /** the tuple for insert */
+ private ArrayTupleBuilder deleteTb;
+ private IFrameWriter deleteWriter;
+ private FrameTupleAppender appenderDelete;
+
/** message list */
private MsgList dummyMessageList = new MsgList();
/** whether alive message should be pushed out */
@@ -95,25 +105,70 @@
this.vertexId = vertexId;
}
+ public final void addVertex(I vertexId, Vertex vertex) {
+ try {
+ insertTb.reset();
+ DataOutput outputInsert = insertTb.getDataOutput();
+ vertexId.write(outputInsert);
+ insertTb.addFieldEndOffset();
+ vertex.write(outputInsert);
+ insertTb.addFieldEndOffset();
+ FrameTupleUtils.flushTuple(appenderInsert, insertTb, insertWriter);
+
+ /**
+ * push alive when necessary
+ */
+ if (pushAlive && !vertex.isHalted()) {
+ alive.reset();
+ DataOutput outputAlive = alive.getDataOutput();
+ vertexId.write(outputAlive);
+ alive.addFieldEndOffset();
+ dummyMessageList.write(outputAlive);
+ alive.addFieldEndOffset();
+ FrameTupleUtils.flushTuple(appenderAlive, alive, aliveWriter);
+ }
+ } catch (Exception e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
+ public final void deleteVertex(I vertexId) {
+ try {
+ deleteTb.reset();
+ DataOutput outputDelete = deleteTb.getDataOutput();
+ vertexId.write(outputDelete);
+ deleteTb.addFieldEndOffset();
+ FrameTupleUtils.flushTuple(appenderDelete, deleteTb, deleteWriter);
+ } catch (Exception e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
public final void setOutputWriters(List<IFrameWriter> outputs) {
msgWriter = outputs.get(0);
- if (outputs.size() > 1) {
- aliveWriter = outputs.get(1);
+ insertWriter = outputs.get(1);
+ deleteWriter = outputs.get(2);
+ if (outputs.size() > 3) {
+ aliveWriter = outputs.get(outputs.size() - 1);
pushAlive = true;
}
}
public final void setOutputAppenders(List<FrameTupleAppender> appenders) {
appenderMsg = appenders.get(0);
- if (appenders.size() > 1) {
- appenderAlive = appenders.get(1);
+ appenderInsert = appenders.get(1);
+ appenderDelete = appenders.get(2);
+ if (appenders.size() > 3) {
+ appenderAlive = appenders.get(appenders.size() - 1);
}
}
public final void setOutputTupleBuilders(List<ArrayTupleBuilder> tbs) {
message = tbs.get(0);
- if (tbs.size() > 1) {
- alive = tbs.get(1);
+ insertTb = tbs.get(1);
+ deleteTb = tbs.get(2);
+ if (tbs.size() > 3) {
+ alive = tbs.get(tbs.size() - 1);
}
}
}
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/io/BasicGenInputSplit.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/io/BasicGenInputSplit.java
index 7179737..ea33691 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/io/BasicGenInputSplit.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/io/BasicGenInputSplit.java
@@ -21,60 +21,64 @@
import java.io.Serializable;
import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
/**
* This InputSplit will not give any ordering or location data. It is used
* internally by BspInputFormat (which determines how many tasks to run the
* application on). Users should not use this directly.
*/
-public class BasicGenInputSplit extends InputSplit implements Writable, Serializable {
- private static final long serialVersionUID = 1L;
- /** Number of splits */
- private int numSplits = -1;
- /** Split index */
- private int splitIndex = -1;
+public class BasicGenInputSplit extends FileSplit implements Writable,
+ Serializable {
+ private static final long serialVersionUID = 1L;
+ /** Number of splits */
+ private int numSplits = -1;
+ /** Split index */
+ private int splitIndex = -1;
- public BasicGenInputSplit() {
- }
+ public BasicGenInputSplit() {
+ super(null, 0, 0, null);
+ }
- public BasicGenInputSplit(int splitIndex, int numSplits) {
- this.splitIndex = splitIndex;
- this.numSplits = numSplits;
- }
+ public BasicGenInputSplit(int splitIndex, int numSplits) {
+ super(null, 0, 0, null);
+ this.splitIndex = splitIndex;
+ this.numSplits = numSplits;
+ }
- @Override
- public long getLength() throws IOException, InterruptedException {
- return 0;
- }
+ @Override
+ public long getLength() {
+ return 0;
+ }
- @Override
- public String[] getLocations() throws IOException, InterruptedException {
- return new String[] {};
- }
+ @Override
+ public String[] getLocations() throws IOException {
+ 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;
- }
+ public int getSplitIndex() {
+ return splitIndex;
+ }
- public int getNumSplits() {
- return numSplits;
- }
+ 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/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java
index 6ef7e13..8b6d1b6 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java
@@ -60,8 +60,12 @@
public static final String NUM_VERTICE = "pregelix.numVertices";
/** num of edges */
public static final String NUM_EDGES = "pregelix.numEdges";
+ /** increase state length */
+ public static final String INCREASE_STATE_LENGTH = "pregelix.incStateLength";
/** job id */
public static final String JOB_ID = "pregelix.jobid";
+ /** frame size */
+ public static final String FRAME_SIZE = "pregelix.framesize";
/**
* Constructor that will instantiate the configuration
@@ -130,8 +134,8 @@
/**
* Set the global aggregator class (optional)
*
- * @param vertexCombinerClass
- * Determines how vertex messages are combined
+ * @param globalAggregatorClass
+ * Determines how messages are globally aggregated
*/
final public void setGlobalAggregatorClass(Class<?> globalAggregatorClass) {
getConfiguration().setClass(GLOBAL_AGGREGATOR_CLASS, globalAggregatorClass, GlobalAggregator.class);
@@ -139,11 +143,27 @@
/**
* Set the job Id
- *
- * @param vertexCombinerClass
- * Determines how vertex messages are combined
*/
final public void setJobId(String jobId) {
getConfiguration().set(JOB_ID, jobId);
}
+
+ /**
+ * Set whether the vertex state length can be dynamically increased
+ *
+ * @param jobId
+ */
+ final public void setDynamicVertexValueSize(boolean incStateLengthDynamically) {
+ getConfiguration().setBoolean(INCREASE_STATE_LENGTH, incStateLengthDynamically);
+ }
+
+ /**
+ * Set the frame size for a job
+ *
+ * @param frameSize
+ * the desired frame size
+ */
+ final public void setFrameSize(int frameSize) {
+ getConfiguration().setInt(FRAME_SIZE, frameSize);
+ }
}
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java
index 7c4853f..ff9724d 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java
@@ -410,4 +410,26 @@
throw new IllegalArgumentException("createMessageValue: Illegally accessed", e);
}
}
+
+ /**
+ * Get the job configuration parameter whether the vertex states will increase dynamically
+ *
+ * @param conf
+ * the job configuration
+ * @return the boolean setting of the parameter, by default it is false
+ */
+ public static boolean getDynamicVertexValueSize(Configuration conf) {
+ return conf.getBoolean(PregelixJob.INCREASE_STATE_LENGTH, false);
+ }
+
+ /**
+ * Get the specified frame size
+ *
+ * @param conf
+ * the job configuration
+ * @return the specified frame size; -1 if it is not set by users
+ */
+ public static int getFrameSize(Configuration conf) {
+ return conf.getInt(PregelixJob.FRAME_SIZE, -1);
+ }
}
diff --git a/pregelix/pregelix-core/pom.xml b/pregelix/pregelix-core/pom.xml
index 17ea6c3..5238068 100644
--- a/pregelix/pregelix-core/pom.xml
+++ b/pregelix/pregelix-core/pom.xml
@@ -1,4 +1,5 @@
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<artifactId>pregelix-core</artifactId>
<packaging>jar</packaging>
@@ -19,6 +20,7 @@
<plugins>
<plugin>
<artifactId>maven-jar-plugin</artifactId>
+ <version>2.4</version>
<executions>
<execution>
<id>balancer</id>
@@ -38,11 +40,6 @@
</includes>
</configuration>
</execution>
- </executions>
- </plugin>
- <plugin>
- <artifactId>maven-jar-plugin</artifactId>
- <executions>
<execution>
<id>generator</id>
<goals>
@@ -64,36 +61,19 @@
</executions>
</plugin>
<plugin>
- <artifactId>maven-jar-plugin</artifactId>
- <executions>
- <execution>
- <id>patch</id>
- <goals>
- <goal>jar</goal>
- </goals>
- <phase>package</phase>
- <configuration>
- <classifier>patch</classifier>
- <finalName>a-hadoop</finalName>
- <includes>
- <include>**/org/apache/**</include>
- </includes>
- </configuration>
- </execution>
- </executions>
- </plugin>
- <plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>appassembler-maven-plugin</artifactId>
+ <version>1.3</version>
<executions>
<execution>
<configuration>
@@ -166,25 +146,6 @@
</resources>
</configuration>
</execution>
- <execution>
- <id>copy-hadoop-patch</id>
- <!-- here the phase you need -->
- <phase>package</phase>
- <goals>
- <goal>copy-resources</goal>
- </goals>
- <configuration>
- <outputDirectory>target/appassembler/lib</outputDirectory>
- <resources>
- <resource>
- <directory>target</directory>
- <includes>
- <include>a-hadoop-patch.jar</include>
- </includes>
- </resource>
- </resources>
- </configuration>
- </execution>
</executions>
</plugin>
<plugin>
@@ -193,7 +154,8 @@
<version>2.7.2</version>
<configuration>
<forkMode>pertest</forkMode>
- <argLine>-enableassertions -Xmx512m -XX:MaxPermSize=300m -Dfile.encoding=UTF-8
+ <argLine>-enableassertions -Xmx512m -XX:MaxPermSize=300m
+ -Dfile.encoding=UTF-8
-Djava.util.logging.config.file=src/test/resources/logging.properties</argLine>
<includes>
<include>**/*TestSuite.java</include>
@@ -203,6 +165,7 @@
</plugin>
<plugin>
<artifactId>maven-clean-plugin</artifactId>
+ <version>2.5</version>
<configuration>
<filesets>
<fileset>
@@ -282,11 +245,6 @@
<groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks-data-std</artifactId>
<version>0.2.3-SNAPSHOT</version>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-core</artifactId>
- <version>0.20.2</version>
<type>jar</type>
<scope>compile</scope>
</dependency>
@@ -319,13 +277,6 @@
<scope>compile</scope>
</dependency>
<dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-test</artifactId>
- <version>0.20.2</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
<groupId>com.kenai.nbpwr</groupId>
<artifactId>org-apache-commons-io</artifactId>
<version>1.3.1-201002241208</version>
@@ -335,7 +286,7 @@
<dependency>
<groupId>edu.uci.ics.hyracks.examples</groupId>
<artifactId>hyracks-integration-tests</artifactId>
- <version>0.2.1</version>
+ <version>0.2.3-SNAPSHOT</version>
<scope>test</scope>
</dependency>
<dependency>
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/driver/Driver.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/driver/Driver.java
index 1b6f195..3a4c41b 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/driver/Driver.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/driver/Driver.java
@@ -72,22 +72,29 @@
public void runJob(PregelixJob job, Plan planChoice, String ipAddress, int port, boolean profiling)
throws HyracksException {
applicationName = exampleClass.getSimpleName() + UUID.randomUUID();
- /** add hadoop configurations */
- URL hadoopCore = job.getClass().getClassLoader().getResource("core-site.xml");
- job.getConfiguration().addResource(hadoopCore);
- URL hadoopMapRed = job.getClass().getClassLoader().getResource("mapred-site.xml");
- job.getConfiguration().addResource(hadoopMapRed);
- URL hadoopHdfs = job.getClass().getClassLoader().getResource("hdfs-site.xml");
- job.getConfiguration().addResource(hadoopHdfs);
- ClusterConfig.loadClusterConfig(ipAddress, port);
-
- LOG.info("job started");
- long start = System.currentTimeMillis();
- long end = start;
- long time = 0;
-
- this.profiling = profiling;
try {
+ /** add hadoop configurations */
+ URL hadoopCore = job.getClass().getClassLoader().getResource("core-site.xml");
+ if (hadoopCore != null) {
+ job.getConfiguration().addResource(hadoopCore);
+ }
+ URL hadoopMapRed = job.getClass().getClassLoader().getResource("mapred-site.xml");
+ if (hadoopMapRed != null) {
+ job.getConfiguration().addResource(hadoopMapRed);
+ }
+ URL hadoopHdfs = job.getClass().getClassLoader().getResource("hdfs-site.xml");
+ if (hadoopHdfs != null) {
+ job.getConfiguration().addResource(hadoopHdfs);
+ }
+ ClusterConfig.loadClusterConfig(ipAddress, port);
+
+ LOG.info("job started");
+ long start = System.currentTimeMillis();
+ long end = start;
+ long time = 0;
+
+ this.profiling = profiling;
+
switch (planChoice) {
case INNER_JOIN:
jobGen = new JobGenInnerJoin(job);
@@ -146,6 +153,16 @@
LOG.info("result writing finished " + time + "ms");
LOG.info("job finished");
} catch (Exception e) {
+ try {
+ /**
+ * destroy application if there is any exception
+ */
+ if (hcc != null) {
+ destroyApplication(applicationName);
+ }
+ } catch (Exception e2) {
+ throw new HyracksException(e2);
+ }
throw new HyracksException(e);
}
}
@@ -224,8 +241,8 @@
LOG.info("jar deployment finished " + (end - start) + "ms");
}
- public void destroyApplication(String jarFile) throws Exception {
- hcc.destroyApplication(applicationName);
+ public void destroyApplication(String appName) throws Exception {
+ hcc.destroyApplication(appName);
}
}
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java
index de29dbc..0b1be61 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java
@@ -76,8 +76,8 @@
import edu.uci.ics.pregelix.core.util.DatatypeHelper;
import edu.uci.ics.pregelix.dataflow.HDFSFileWriteOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.VertexFileScanOperatorDescriptor;
+import edu.uci.ics.pregelix.dataflow.VertexWriteOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
-import edu.uci.ics.pregelix.dataflow.std.FileWriteOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
import edu.uci.ics.pregelix.runtime.bootstrap.StorageManagerInterface;
@@ -89,8 +89,6 @@
private static final Logger LOGGER = Logger.getLogger(JobGen.class.getName());
protected static final int MB = 1048576;
protected static final float DEFAULT_BTREE_FILL_FACTOR = 1.00f;
- protected static final int frameSize = ClusterConfig.getFrameSize();
- protected static final int maxFrameSize = (int) (((long) 32 * MB) / frameSize);
protected static final int tableSize = 10485767;
protected static final String PRIMARY_INDEX = "primary";
protected final Configuration conf;
@@ -98,6 +96,8 @@
protected IIndexRegistryProvider<IIndex> treeRegistryProvider = TreeIndexRegistryProvider.INSTANCE;
protected IStorageManagerInterface storageManagerInterface = StorageManagerInterface.INSTANCE;
protected String jobId = new UUID(System.currentTimeMillis(), System.nanoTime()).toString();
+ protected int frameSize = ClusterConfig.getFrameSize();
+ protected int maxFrameNumber = (int) (((long) 32 * MB) / frameSize);
protected static final String SECONDARY_INDEX_ODD = "secondary1";
protected static final String SECONDARY_INDEX_EVEN = "secondary2";
@@ -107,6 +107,17 @@
this.giraphJob = job;
this.initJobConfiguration();
job.setJobId(jobId);
+
+ // set the frame size to be the one user specified if the user did
+ // specify.
+ int specifiedFrameSize = BspUtils.getFrameSize(job.getConfiguration());
+ if (specifiedFrameSize > 0) {
+ frameSize = specifiedFrameSize;
+ maxFrameNumber = (int) (((long) 32 * MB) / frameSize);
+ }
+ if (maxFrameNumber <= 0) {
+ maxFrameNumber = 1;
+ }
}
@SuppressWarnings({ "rawtypes", "unchecked" })
@@ -189,9 +200,10 @@
RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
vertexIdClass.getName(), vertexClass.getName());
IConfigurationFactory confFactory = new ConfigurationFactory(conf);
+ String[] readSchedule = ClusterConfig.getHdfsScheduler().getLocationConstraints(splits);
VertexFileScanOperatorDescriptor scanner = new VertexFileScanOperatorDescriptor(spec, recordDescriptor, splits,
- confFactory);
- ClusterConfig.setLocationConstraint(spec, scanner, splits);
+ readSchedule, confFactory);
+ ClusterConfig.setLocationConstraint(spec, scanner);
/**
* construct sort operator
@@ -203,7 +215,7 @@
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
.getClass());
- ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, maxFrameSize, sortFields,
+ ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, sortFields,
nkmFactory, comparatorFactories, recordDescriptor);
ClusterConfig.setLocationConstraint(spec, sorter);
@@ -264,9 +276,10 @@
RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
vertexIdClass.getName(), vertexClass.getName());
IConfigurationFactory confFactory = new ConfigurationFactory(conf);
+ String[] readSchedule = ClusterConfig.getHdfsScheduler().getLocationConstraints(splits);
VertexFileScanOperatorDescriptor scanner = new VertexFileScanOperatorDescriptor(spec, recordDescriptor, splits,
- confFactory);
- PartitionConstraintHelper.addPartitionCountConstraint(spec, scanner, splits.size());
+ readSchedule, confFactory);
+ ClusterConfig.setLocationConstraint(spec, scanner);
/**
* construct sort operator
@@ -280,7 +293,7 @@
.getClass());
ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, maxFrameLimit, sortFields,
nkmFactory, comparatorFactories, recordDescriptor);
- PartitionConstraintHelper.addPartitionCountConstraint(spec, sorter, splits.size());
+ ClusterConfig.setLocationConstraint(spec, sorter);
/**
* construct write file operator
@@ -292,7 +305,7 @@
IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(confFactory);
IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
vertexIdClass.getName(), vertexClass.getName());
- FileWriteOperatorDescriptor writer = new FileWriteOperatorDescriptor(spec, inputRdFactory,
+ VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, inputRdFactory,
resultFileSplitProvider, preHookFactory, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, new String[] { "nc1" });
PartitionConstraintHelper.addPartitionCountConstraint(spec, writer, 1);
@@ -357,7 +370,7 @@
IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(confFactory);
IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
vertexIdClass.getName(), vertexClass.getName());
- FileWriteOperatorDescriptor writer = new FileWriteOperatorDescriptor(spec, inputRdFactory,
+ VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, inputRdFactory,
resultFileSplitProvider, preHookFactory, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, new String[] { "nc1" });
PartitionConstraintHelper.addPartitionCountConstraint(spec, writer, 1);
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
index 00cdf07..9de4c04 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
@@ -39,6 +39,9 @@
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
import edu.uci.ics.pregelix.api.graph.MsgList;
import edu.uci.ics.pregelix.api.job.PregelixJob;
@@ -127,13 +130,16 @@
MsgList.class.getName());
RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
vertexIdClass.getName(), messageValueClass.getName());
+ RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+ vertexClass.getName());
+ RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
BTreeSearchFunctionUpdateOperatorDescriptor scanner = new BTreeSearchFunctionUpdateOperatorDescriptor(spec,
recordDescriptor, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
comparatorFactories, JobGenUtil.getForwardScan(iteration), null, null, true, true,
- new BTreeDataflowHelperFactory(), inputRdFactory, 4,
+ new BTreeDataflowHelperFactory(), inputRdFactory, 6,
new StartComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
- rdPartialAggregate, rdFinal);
+ rdPartialAggregate, rdInsert, rdDelete, rdFinal);
ClusterConfig.setLocationConstraint(spec, scanner);
/**
@@ -173,7 +179,7 @@
IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
.getClass());
- ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameSize, keyFields,
+ ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
nkmFactory, sortCmpFactories, rdUnnestedMessage);
ClusterConfig.setLocationConstraint(spec, localSort);
@@ -212,18 +218,54 @@
EmptySinkOperatorDescriptor emptySink = new EmptySinkOperatorDescriptor(spec);
ClusterConfig.setLocationConstraint(spec, emptySink);
+ /**
+ * add the insert operator to insert vertexes
+ */
+ TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+ spec, rdInsert, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+ comparatorFactories, fieldPermutation, IndexOp.INSERT, new BTreeDataflowHelperFactory(), null,
+ NoOpOperationCallbackProvider.INSTANCE);
+ ClusterConfig.setLocationConstraint(spec, insertOp);
+
+ /**
+ * add the delete operator to delete vertexes
+ */
+ TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+ spec, rdDelete, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+ comparatorFactories, fieldPermutation, IndexOp.DELETE, new BTreeDataflowHelperFactory(), null,
+ NoOpOperationCallbackProvider.INSTANCE);
+ ClusterConfig.setLocationConstraint(spec, deleteOp);
+
+ /** construct empty sink operator */
+ EmptySinkOperatorDescriptor emptySink3 = new EmptySinkOperatorDescriptor(spec);
+ ClusterConfig.setLocationConstraint(spec, emptySink3);
+
+ /** construct empty sink operator */
+ EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
+ ClusterConfig.setLocationConstraint(spec, emptySink4);
+
ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
rdUnnestedMessage.getFields()[0]);
- ITuplePartitionComputerFactory hashPartitionComputerFactory = new MergePartitionComputerFactory();
+ ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
+
/** connect all operators **/
spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, scanner, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, localSort, 0);
- spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 1,
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), scanner, 1,
terminateWriter, 0);
- spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 2,
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), scanner, 2,
finalAggregator, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 3, btreeBulkLoad, 0);
+
+ /**
+ * connect the insert/delete operator
+ */
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 3, insertOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 4, deleteOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
+
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 5, btreeBulkLoad, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories),
localGby, 0, globalGby, 0);
@@ -235,8 +277,10 @@
spec.addRoot(btreeBulkLoad);
spec.addRoot(terminateWriter);
spec.addRoot(finalAggregator);
+ spec.addRoot(emptySink3);
+ spec.addRoot(emptySink4);
- spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+ spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy(spec));
spec.setFrameSize(frameSize);
return spec;
}
@@ -261,6 +305,9 @@
.getClass());
RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
MsgList.class.getName());
+ RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+ vertexClass.getName());
+ RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
/**
* construct empty tuple operator
@@ -316,8 +363,8 @@
IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
spec, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
comparatorFactories, JobGenUtil.getForwardScan(iteration), keyFields, keyFields, true, true,
- new BTreeDataflowHelperFactory(), inputRdFactory, 4, new ComputeUpdateFunctionFactory(confFactory),
- preHookFactory, null, rdUnnestedMessage, rdDummy, rdPartialAggregate, rdFinal);
+ new BTreeDataflowHelperFactory(), inputRdFactory, 6, new ComputeUpdateFunctionFactory(confFactory),
+ preHookFactory, null, rdUnnestedMessage, rdDummy, rdPartialAggregate, rdInsert, rdDelete, rdFinal);
ClusterConfig.setLocationConstraint(spec, join);
/**
@@ -342,7 +389,7 @@
IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
.getClass());
- ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameSize, keyFields,
+ ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
nkmFactory, sortCmpFactories, rdUnnestedMessage);
ClusterConfig.setLocationConstraint(spec, localSort);
@@ -395,7 +442,33 @@
configurationFactory, aggRdFactory, jobId);
PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
- ITuplePartitionComputerFactory hashPartitionComputerFactory = new MergePartitionComputerFactory();
+ /**
+ * add the insert operator to insert vertexes
+ */
+ TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+ spec, rdInsert, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+ comparatorFactories, fieldPermutation, IndexOp.INSERT, new BTreeDataflowHelperFactory(), null,
+ NoOpOperationCallbackProvider.INSTANCE);
+ ClusterConfig.setLocationConstraint(spec, insertOp);
+
+ /**
+ * add the delete operator to delete vertexes
+ */
+ TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+ spec, rdDelete, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+ comparatorFactories, fieldPermutation, IndexOp.DELETE, new BTreeDataflowHelperFactory(), null,
+ NoOpOperationCallbackProvider.INSTANCE);
+ ClusterConfig.setLocationConstraint(spec, deleteOp);
+
+ /** construct empty sink operator */
+ EmptySinkOperatorDescriptor emptySink3 = new EmptySinkOperatorDescriptor(spec);
+ ClusterConfig.setLocationConstraint(spec, emptySink3);
+
+ /** construct empty sink operator */
+ EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
+ ClusterConfig.setLocationConstraint(spec, emptySink4);
+
+ ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
rdUnnestedMessage.getFields()[0]);
/** connect all operators **/
@@ -404,12 +477,20 @@
spec.connect(new OneToOneConnectorDescriptor(spec), materializeRead, 0, setUnion, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), setUnion, 0, join, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), join, 0, localSort, 0);
- spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 1,
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), join, 1,
terminateWriter, 0);
-
- spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 2,
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), join, 2,
finalAggregator, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec), join, 3, btreeBulkLoad, 0);
+
+ /**
+ * connect the insert/delete operator
+ */
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 3, insertOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 4, deleteOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
+
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 5, btreeBulkLoad, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories),
localGby, 0, globalGby, 0);
@@ -420,8 +501,10 @@
spec.addRoot(emptySink);
spec.addRoot(btreeBulkLoad);
spec.addRoot(terminateWriter);
+ spec.addRoot(emptySink3);
+ spec.addRoot(emptySink4);
- spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+ spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy(spec));
spec.setFrameSize(frameSize);
return spec;
}
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java
index 3847aa7..91c15b2 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java
@@ -39,6 +39,9 @@
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
import edu.uci.ics.pregelix.api.graph.MsgList;
import edu.uci.ics.pregelix.api.job.PregelixJob;
@@ -121,13 +124,16 @@
vertexIdClass.getName(), vertexClass.getName());
RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
vertexIdClass.getName(), messageValueClass.getName());
+ RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+ vertexClass.getName());
+ RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
BTreeSearchFunctionUpdateOperatorDescriptor scanner = new BTreeSearchFunctionUpdateOperatorDescriptor(spec,
recordDescriptor, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
comparatorFactories, JobGenUtil.getForwardScan(iteration), null, null, true, true,
- new BTreeDataflowHelperFactory(), inputRdFactory, 3,
+ new BTreeDataflowHelperFactory(), inputRdFactory, 5,
new StartComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
- rdPartialAggregate);
+ rdPartialAggregate, rdInsert, rdDelete);
ClusterConfig.setLocationConstraint(spec, scanner);
/**
@@ -139,14 +145,15 @@
IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
.getClass());
- ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameSize, keyFields,
+ ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
nkmFactory, sortCmpFactories, rdUnnestedMessage);
ClusterConfig.setLocationConstraint(spec, localSort);
/**
* construct local pre-clustered group-by operator
*/
- IAggregatorDescriptorFactory aggregatorFactory = DataflowUtils.getAccumulatingAggregatorFactory(conf, false, false);
+ IAggregatorDescriptorFactory aggregatorFactory = DataflowUtils.getAccumulatingAggregatorFactory(conf, false,
+ false);
PreclusteredGroupOperatorDescriptor localGby = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
sortCmpFactories, aggregatorFactory, rdUnnestedMessage);
ClusterConfig.setLocationConstraint(spec, localGby);
@@ -156,8 +163,8 @@
*/
RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
MsgList.class.getName());
- IAggregatorDescriptorFactory aggregatorFactoryFinal = DataflowUtils
- .getAccumulatingAggregatorFactory(conf, true, true);
+ IAggregatorDescriptorFactory aggregatorFactoryFinal = DataflowUtils.getAccumulatingAggregatorFactory(conf,
+ true, true);
PreclusteredGroupOperatorDescriptor globalGby = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
sortCmpFactories, aggregatorFactoryFinal, rdFinal);
ClusterConfig.setLocationConstraint(spec, globalGby);
@@ -182,7 +189,7 @@
TerminationStateWriterOperatorDescriptor terminateWriter = new TerminationStateWriterOperatorDescriptor(spec,
configurationFactory, jobId);
PartitionConstraintHelper.addPartitionCountConstraint(spec, terminateWriter, 1);
- ITuplePartitionComputerFactory hashPartitionComputerFactory = new MergePartitionComputerFactory();
+ ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
/**
* final aggregate write operator
@@ -193,16 +200,55 @@
configurationFactory, aggRdFactory, jobId);
PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
+ /**
+ * add the insert operator to insert vertexes
+ */
+ int[] fieldPermutation = new int[] { 0, 1 };
+ TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+ spec, rdInsert, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+ comparatorFactories, fieldPermutation, IndexOp.INSERT, new BTreeDataflowHelperFactory(), null,
+ NoOpOperationCallbackProvider.INSTANCE);
+ ClusterConfig.setLocationConstraint(spec, insertOp);
+
+ /**
+ * add the delete operator to delete vertexes
+ */
+ TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+ spec, rdDelete, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+ comparatorFactories, fieldPermutation, IndexOp.DELETE, new BTreeDataflowHelperFactory(), null,
+ NoOpOperationCallbackProvider.INSTANCE);
+ ClusterConfig.setLocationConstraint(spec, deleteOp);
+
+ /** construct empty sink operator */
+ EmptySinkOperatorDescriptor emptySink3 = new EmptySinkOperatorDescriptor(spec);
+ ClusterConfig.setLocationConstraint(spec, emptySink3);
+
+ /** construct empty sink operator */
+ EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
+ ClusterConfig.setLocationConstraint(spec, emptySink4);
+
ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
rdUnnestedMessage.getFields()[0]);
/** connect all operators **/
spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, scanner, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, localSort, 0);
- spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 1,
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), scanner, 1,
terminateWriter, 0);
- spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 2,
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), scanner, 2,
finalAggregator, 0);
+
+ /**
+ * connect the insert/delete operator
+ */
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 3, insertOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 4, deleteOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
+
+ /**
+ * connect the group-by operator
+ */
spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories),
localGby, 0, globalGby, 0);
@@ -213,8 +259,10 @@
spec.addRoot(terminateWriter);
spec.addRoot(finalAggregator);
spec.addRoot(emptySink2);
+ spec.addRoot(emptySink3);
+ spec.addRoot(emptySink4);
- spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+ spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy(spec));
spec.setFrameSize(frameSize);
return spec;
}
@@ -239,6 +287,9 @@
.getClass());
RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
MsgList.class.getName());
+ RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+ vertexClass.getName());
+ RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
/**
* construct empty tuple operator
@@ -286,9 +337,9 @@
IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
spec, storageManagerInterface, treeRegistryProvider, fileSplitProvider, interiorFrameFactory,
leafFrameFactory, typeTraits, comparatorFactories, JobGenUtil.getForwardScan(iteration), keyFields,
- keyFields, true, true, new BTreeDataflowHelperFactory(), true, nullWriterFactories, inputRdFactory, 3,
+ keyFields, true, true, new BTreeDataflowHelperFactory(), true, nullWriterFactories, inputRdFactory, 5,
new ComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
- rdPartialAggregate);
+ rdPartialAggregate, rdInsert, rdDelete);
ClusterConfig.setLocationConstraint(spec, join);
/**
@@ -299,14 +350,15 @@
IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
.getClass());
- ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameSize, keyFields,
+ ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
nkmFactory, sortCmpFactories, rdUnnestedMessage);
ClusterConfig.setLocationConstraint(spec, localSort);
/**
* construct local pre-clustered group-by operator
*/
- IAggregatorDescriptorFactory aggregatorFactory = DataflowUtils.getAccumulatingAggregatorFactory(conf, false, false);
+ IAggregatorDescriptorFactory aggregatorFactory = DataflowUtils.getAccumulatingAggregatorFactory(conf, false,
+ false);
PreclusteredGroupOperatorDescriptor localGby = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
sortCmpFactories, aggregatorFactory, rdUnnestedMessage);
ClusterConfig.setLocationConstraint(spec, localGby);
@@ -314,8 +366,8 @@
/**
* construct global group-by operator
*/
- IAggregatorDescriptorFactory aggregatorFactoryFinal = DataflowUtils
- .getAccumulatingAggregatorFactory(conf, true, true);
+ IAggregatorDescriptorFactory aggregatorFactoryFinal = DataflowUtils.getAccumulatingAggregatorFactory(conf,
+ true, true);
PreclusteredGroupOperatorDescriptor globalGby = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
sortCmpFactories, aggregatorFactoryFinal, rdFinal);
ClusterConfig.setLocationConstraint(spec, globalGby);
@@ -351,7 +403,34 @@
configurationFactory, aggRdFactory, jobId);
PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
- ITuplePartitionComputerFactory hashPartitionComputerFactory = new MergePartitionComputerFactory();
+ /**
+ * add the insert operator to insert vertexes
+ */
+ int[] fieldPermutation = new int[] { 0, 1 };
+ TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+ spec, rdInsert, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+ comparatorFactories, fieldPermutation, IndexOp.INSERT, new BTreeDataflowHelperFactory(), null,
+ NoOpOperationCallbackProvider.INSTANCE);
+ ClusterConfig.setLocationConstraint(spec, insertOp);
+
+ /**
+ * add the delete operator to delete vertexes
+ */
+ TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+ spec, rdDelete, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+ comparatorFactories, fieldPermutation, IndexOp.DELETE, new BTreeDataflowHelperFactory(), null,
+ NoOpOperationCallbackProvider.INSTANCE);
+ ClusterConfig.setLocationConstraint(spec, deleteOp);
+
+ /** construct empty sink operator */
+ EmptySinkOperatorDescriptor emptySink3 = new EmptySinkOperatorDescriptor(spec);
+ ClusterConfig.setLocationConstraint(spec, emptySink3);
+
+ /** construct empty sink operator */
+ EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
+ ClusterConfig.setLocationConstraint(spec, emptySink4);
+
+ ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
rdUnnestedMessage.getFields()[0]);
@@ -360,10 +439,19 @@
spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, materializeRead, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), materializeRead, 0, join, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), join, 0, localSort, 0);
- spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 1,
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), join, 1,
terminateWriter, 0);
- spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 2,
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), join, 2,
finalAggregator, 0);
+
+ /**
+ * connect the insert/delete operator
+ */
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 3, insertOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 4, deleteOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
+
spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories),
localGby, 0, globalGby, 0);
@@ -375,7 +463,7 @@
spec.addRoot(finalAggregator);
spec.addRoot(emptySink);
- spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+ spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy(spec));
spec.setFrameSize(frameSize);
return spec;
}
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
index ec783a7..ee1fd0f 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
@@ -38,6 +38,9 @@
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
import edu.uci.ics.pregelix.api.graph.MsgList;
import edu.uci.ics.pregelix.api.job.PregelixJob;
@@ -47,12 +50,12 @@
import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
import edu.uci.ics.pregelix.core.runtime.touchpoint.WritableComparingBinaryComparatorFactory;
import edu.uci.ics.pregelix.core.util.DataflowUtils;
+import edu.uci.ics.pregelix.dataflow.ConnectorPolicyAssignmentPolicy;
import edu.uci.ics.pregelix.dataflow.EmptySinkOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.EmptyTupleSourceOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.FinalAggregateOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.MaterializingReadOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.MaterializingWriteOperatorDescriptor;
-import edu.uci.ics.pregelix.dataflow.NonCombinerConnectorPolicyAssignmentPolicy;
import edu.uci.ics.pregelix.dataflow.TerminationStateWriterOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
import edu.uci.ics.pregelix.dataflow.std.BTreeSearchFunctionUpdateOperatorDescriptor;
@@ -123,12 +126,16 @@
vertexIdClass.getName(), vertexClass.getName());
RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
vertexIdClass.getName(), messageValueClass.getName());
+ RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+ vertexClass.getName());
+ RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
+
BTreeSearchFunctionUpdateOperatorDescriptor scanner = new BTreeSearchFunctionUpdateOperatorDescriptor(spec,
recordDescriptor, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
comparatorFactories, JobGenUtil.getForwardScan(iteration), null, null, true, true,
- new BTreeDataflowHelperFactory(), inputRdFactory, 3,
+ new BTreeDataflowHelperFactory(), inputRdFactory, 5,
new StartComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
- rdPartialAggregate);
+ rdPartialAggregate, rdInsert, rdDelete);
ClusterConfig.setLocationConstraint(spec, scanner);
/**
@@ -140,7 +147,7 @@
IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
.getClass());
- ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameSize, keyFields,
+ ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
nkmFactory, sortCmpFactories, rdUnnestedMessage);
ClusterConfig.setLocationConstraint(spec, globalSort);
@@ -185,17 +192,53 @@
configurationFactory, aggRdFactory, jobId);
PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
- ITuplePartitionComputerFactory hashPartitionComputerFactory = new MergePartitionComputerFactory();
+ /**
+ * add the insert operator to insert vertexes
+ */
+ int[] fieldPermutation = new int[] { 0, 1 };
+ TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+ spec, rdInsert, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+ comparatorFactories, fieldPermutation, IndexOp.INSERT, new BTreeDataflowHelperFactory(), null,
+ NoOpOperationCallbackProvider.INSTANCE);
+ ClusterConfig.setLocationConstraint(spec, insertOp);
+
+ /**
+ * add the delete operator to delete vertexes
+ */
+ TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+ spec, rdDelete, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+ comparatorFactories, fieldPermutation, IndexOp.DELETE, new BTreeDataflowHelperFactory(), null,
+ NoOpOperationCallbackProvider.INSTANCE);
+ ClusterConfig.setLocationConstraint(spec, deleteOp);
+
+ /** construct empty sink operator */
+ EmptySinkOperatorDescriptor emptySink3 = new EmptySinkOperatorDescriptor(spec);
+ ClusterConfig.setLocationConstraint(spec, emptySink3);
+
+ /** construct empty sink operator */
+ EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
+ ClusterConfig.setLocationConstraint(spec, emptySink4);
+
+ ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
rdUnnestedMessage.getFields()[0]);
/** connect all operators **/
spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, scanner, 0);
spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 0, globalSort, 0);
- spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 1,
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), scanner, 1,
terminateWriter, 0);
- spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 2,
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), scanner, 2,
finalAggregator, 0);
+
+ /**
+ * connect the insert/delete operator
+ */
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 3, insertOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 4, deleteOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
+
spec.connect(new OneToOneConnectorDescriptor(spec), globalSort, 0, globalGby, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), globalGby, 0, materialize, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
@@ -204,8 +247,10 @@
spec.addRoot(terminateWriter);
spec.addRoot(finalAggregator);
spec.addRoot(emptySink2);
+ spec.addRoot(emptySink3);
+ spec.addRoot(emptySink4);
- spec.setConnectorPolicyAssignmentPolicy(new NonCombinerConnectorPolicyAssignmentPolicy());
+ spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy(spec));
spec.setFrameSize(frameSize);
return spec;
}
@@ -230,6 +275,9 @@
.getClass());
RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
MsgList.class.getName());
+ RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+ vertexClass.getName());
+ RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
/**
* construct empty tuple operator
@@ -277,9 +325,9 @@
IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
spec, storageManagerInterface, treeRegistryProvider, fileSplitProvider, interiorFrameFactory,
leafFrameFactory, typeTraits, comparatorFactories, JobGenUtil.getForwardScan(iteration), keyFields,
- keyFields, true, true, new BTreeDataflowHelperFactory(), true, nullWriterFactories, inputRdFactory, 3,
+ keyFields, true, true, new BTreeDataflowHelperFactory(), true, nullWriterFactories, inputRdFactory, 5,
new ComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
- rdPartialAggregate);
+ rdPartialAggregate, rdInsert, rdDelete);
ClusterConfig.setLocationConstraint(spec, join);
/**
@@ -290,7 +338,7 @@
IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
.getClass());
- ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameSize, keyFields,
+ ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
nkmFactory, sortCmpFactories, rdUnnestedMessage);
ClusterConfig.setLocationConstraint(spec, globalSort);
@@ -334,7 +382,31 @@
configurationFactory, aggRdFactory, jobId);
PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
- ITuplePartitionComputerFactory hashPartitionComputerFactory = new MergePartitionComputerFactory();
+ int[] fieldPermutation = new int[] { 0, 1 };
+ TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+ spec, rdInsert, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+ comparatorFactories, fieldPermutation, IndexOp.INSERT, new BTreeDataflowHelperFactory(), null,
+ NoOpOperationCallbackProvider.INSTANCE);
+ ClusterConfig.setLocationConstraint(spec, insertOp);
+
+ /**
+ * add the delete operator to delete vertexes
+ */
+ TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+ spec, rdDelete, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+ comparatorFactories, fieldPermutation, IndexOp.DELETE, new BTreeDataflowHelperFactory(), null,
+ NoOpOperationCallbackProvider.INSTANCE);
+ ClusterConfig.setLocationConstraint(spec, deleteOp);
+
+ /** construct empty sink operator */
+ EmptySinkOperatorDescriptor emptySink3 = new EmptySinkOperatorDescriptor(spec);
+ ClusterConfig.setLocationConstraint(spec, emptySink3);
+
+ /** construct empty sink operator */
+ EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
+ ClusterConfig.setLocationConstraint(spec, emptySink4);
+
+ ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
rdUnnestedMessage.getFields()[0]);
@@ -343,10 +415,18 @@
spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, materializeRead, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), materializeRead, 0, join, 0);
spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 0, globalSort, 0);
- spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 1,
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), join, 1,
terminateWriter, 0);
- spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 2,
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), join, 2,
finalAggregator, 0);
+ /**
+ * connect the insert/delete operator
+ */
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 3, insertOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 4, deleteOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
+
spec.connect(new OneToOneConnectorDescriptor(spec), globalSort, 0, globalGby, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), globalGby, 0, materialize, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
@@ -356,7 +436,7 @@
spec.addRoot(finalAggregator);
spec.addRoot(emptySink);
- spec.setConnectorPolicyAssignmentPolicy(new NonCombinerConnectorPolicyAssignmentPolicy());
+ spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy(spec));
spec.setFrameSize(frameSize);
return spec;
}
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
index bb939e3..628e9ce 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
@@ -38,6 +38,9 @@
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
import edu.uci.ics.pregelix.api.graph.MsgList;
import edu.uci.ics.pregelix.api.job.PregelixJob;
@@ -47,6 +50,7 @@
import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
import edu.uci.ics.pregelix.core.runtime.touchpoint.WritableComparingBinaryComparatorFactory;
import edu.uci.ics.pregelix.core.util.DataflowUtils;
+import edu.uci.ics.pregelix.dataflow.ConnectorPolicyAssignmentPolicy;
import edu.uci.ics.pregelix.dataflow.EmptySinkOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.EmptyTupleSourceOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.FinalAggregateOperatorDescriptor;
@@ -119,12 +123,16 @@
vertexIdClass.getName(), vertexClass.getName());
RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
vertexIdClass.getName(), messageValueClass.getName());
+ RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+ vertexClass.getName());
+ RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
+
BTreeSearchFunctionUpdateOperatorDescriptor scanner = new BTreeSearchFunctionUpdateOperatorDescriptor(spec,
recordDescriptor, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
comparatorFactories, JobGenUtil.getForwardScan(iteration), null, null, true, true,
- new BTreeDataflowHelperFactory(), inputRdFactory, 3,
+ new BTreeDataflowHelperFactory(), inputRdFactory, 5,
new StartComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
- rdPartialAggregate);
+ rdPartialAggregate, rdInsert, rdDelete);
ClusterConfig.setLocationConstraint(spec, scanner);
/**
@@ -136,7 +144,7 @@
IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
.getClass());
- ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameSize, keyFields,
+ ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
nkmFactory, sortCmpFactories, rdUnnestedMessage);
ClusterConfig.setLocationConstraint(spec, localSort);
@@ -152,7 +160,7 @@
/**
* construct global sort operator
*/
- ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameSize, keyFields,
+ ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
nkmFactory, sortCmpFactories, rdUnnestedMessage);
ClusterConfig.setLocationConstraint(spec, globalSort);
@@ -187,7 +195,7 @@
TerminationStateWriterOperatorDescriptor terminateWriter = new TerminationStateWriterOperatorDescriptor(spec,
configurationFactory, jobId);
PartitionConstraintHelper.addPartitionCountConstraint(spec, terminateWriter, 1);
- ITuplePartitionComputerFactory hashPartitionComputerFactory = new MergePartitionComputerFactory();
+ ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
/**
* final aggregate write operator
@@ -198,16 +206,51 @@
configurationFactory, aggRdFactory, jobId);
PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
+ /**
+ * add the insert operator to insert vertexes
+ */
+ int[] fieldPermutation = new int[] { 0, 1 };
+ TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+ spec, rdInsert, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+ comparatorFactories, fieldPermutation, IndexOp.INSERT, new BTreeDataflowHelperFactory(), null,
+ NoOpOperationCallbackProvider.INSTANCE);
+ ClusterConfig.setLocationConstraint(spec, insertOp);
+
+ /**
+ * add the delete operator to delete vertexes
+ */
+ TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+ spec, rdDelete, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+ comparatorFactories, fieldPermutation, IndexOp.DELETE, new BTreeDataflowHelperFactory(), null,
+ NoOpOperationCallbackProvider.INSTANCE);
+ ClusterConfig.setLocationConstraint(spec, deleteOp);
+
+ /** construct empty sink operator */
+ EmptySinkOperatorDescriptor emptySink3 = new EmptySinkOperatorDescriptor(spec);
+ ClusterConfig.setLocationConstraint(spec, emptySink3);
+
+ /** construct empty sink operator */
+ EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
+ ClusterConfig.setLocationConstraint(spec, emptySink4);
+
ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
rdUnnestedMessage.getFields()[0]);
/** connect all operators **/
spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, scanner, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, localSort, 0);
- spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 1,
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), scanner, 1,
terminateWriter, 0);
- spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 2,
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), scanner, 2,
finalAggregator, 0);
+ /**
+ * connect the insert/delete operator
+ */
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 3, insertOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 4, deleteOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
+
spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), localGby, 0, globalSort, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), globalSort, 0, globalGby, 0);
@@ -218,8 +261,11 @@
spec.addRoot(terminateWriter);
spec.addRoot(finalAggregator);
spec.addRoot(emptySink2);
+ spec.addRoot(emptySink3);
+ spec.addRoot(emptySink4);
spec.setFrameSize(frameSize);
+ spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy(spec));
return spec;
}
@@ -243,6 +289,9 @@
.getClass());
RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
MsgList.class.getName());
+ RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+ vertexClass.getName());
+ RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
/**
* construct empty tuple operator
@@ -290,9 +339,9 @@
IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
spec, storageManagerInterface, treeRegistryProvider, fileSplitProvider, interiorFrameFactory,
leafFrameFactory, typeTraits, comparatorFactories, JobGenUtil.getForwardScan(iteration), keyFields,
- keyFields, true, true, new BTreeDataflowHelperFactory(), true, nullWriterFactories, inputRdFactory, 3,
+ keyFields, true, true, new BTreeDataflowHelperFactory(), true, nullWriterFactories, inputRdFactory, 5,
new ComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
- rdPartialAggregate);
+ rdPartialAggregate, rdInsert, rdDelete);
ClusterConfig.setLocationConstraint(spec, join);
/**
@@ -303,7 +352,7 @@
IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
.getClass());
- ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameSize, keyFields,
+ ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
nkmFactory, sortCmpFactories, rdUnnestedMessage);
ClusterConfig.setLocationConstraint(spec, localSort);
@@ -319,7 +368,7 @@
/**
* construct global sort operator
*/
- ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameSize, keyFields,
+ ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
nkmFactory, sortCmpFactories, rdUnnestedMessage);
ClusterConfig.setLocationConstraint(spec, globalSort);
@@ -363,7 +412,34 @@
configurationFactory, aggRdFactory, jobId);
PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
- ITuplePartitionComputerFactory hashPartitionComputerFactory = new MergePartitionComputerFactory();
+ /**
+ * add the insert operator to insert vertexes
+ */
+ int[] fieldPermutation = new int[] { 0, 1 };
+ TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+ spec, rdInsert, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+ comparatorFactories, fieldPermutation, IndexOp.INSERT, new BTreeDataflowHelperFactory(), null,
+ NoOpOperationCallbackProvider.INSTANCE);
+ ClusterConfig.setLocationConstraint(spec, insertOp);
+
+ /**
+ * add the delete operator to delete vertexes
+ */
+ TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+ spec, rdDelete, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+ comparatorFactories, fieldPermutation, IndexOp.DELETE, new BTreeDataflowHelperFactory(), null,
+ NoOpOperationCallbackProvider.INSTANCE);
+ ClusterConfig.setLocationConstraint(spec, deleteOp);
+
+ /** construct empty sink operator */
+ EmptySinkOperatorDescriptor emptySink3 = new EmptySinkOperatorDescriptor(spec);
+ ClusterConfig.setLocationConstraint(spec, emptySink3);
+
+ /** construct empty sink operator */
+ EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
+ ClusterConfig.setLocationConstraint(spec, emptySink4);
+
+ ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
rdUnnestedMessage.getFields()[0]);
@@ -372,10 +448,18 @@
spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, materializeRead, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), materializeRead, 0, join, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), join, 0, localSort, 0);
- spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 1,
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), join, 1,
terminateWriter, 0);
- spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 2,
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), join, 2,
finalAggregator, 0);
+ /**
+ * connect the insert/delete operator
+ */
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 3, insertOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 4, deleteOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
+
spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), localGby, 0, globalSort, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), globalSort, 0, globalGby, 0);
@@ -386,8 +470,11 @@
spec.addRoot(terminateWriter);
spec.addRoot(finalAggregator);
spec.addRoot(emptySink);
+ spec.addRoot(emptySink3);
+ spec.addRoot(emptySink4);
spec.setFrameSize(frameSize);
+ spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy(spec));
return spec;
}
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/clusterconfig/ClusterConfig.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/clusterconfig/ClusterConfig.java
index 8eadab9..d26e637 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/clusterconfig/ClusterConfig.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/clusterconfig/ClusterConfig.java
@@ -40,6 +40,7 @@
import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.hdfs2.scheduler.Scheduler;
public class ClusterConfig {
@@ -49,6 +50,7 @@
private static Properties clusterProperties = new Properties();
private static Map<String, List<String>> ipToNcMapping;
private static String[] stores;
+ private static Scheduler hdfsScheduler;
/**
* let tests set config path to be whatever
@@ -211,6 +213,8 @@
NCs[i] = entry.getKey();
i++;
}
+
+ hdfsScheduler = new Scheduler(ipAddress, port);
} catch (Exception e) {
throw new IllegalStateException(e);
}
@@ -218,4 +222,8 @@
loadClusterProperties();
loadStores();
}
+
+ public static Scheduler getHdfsScheduler() {
+ return hdfsScheduler;
+ }
}
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
index ed04746..cd2a864 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
@@ -14,6 +14,7 @@
*/
package edu.uci.ics.pregelix.core.util;
+import java.io.File;
import java.util.EnumSet;
import edu.uci.ics.hyracks.api.client.HyracksConnection;
@@ -44,14 +45,16 @@
private static NodeControllerService nc2;
private static IHyracksClientConnection hcc;
- public static void init() throws Exception {
+ public static void init(String topologyFilePath) throws Exception {
CCConfig ccConfig = new CCConfig();
ccConfig.clientNetIpAddress = CC_HOST;
ccConfig.clusterNetIpAddress = CC_HOST;
ccConfig.clusterNetPort = TEST_HYRACKS_CC_PORT;
ccConfig.clientNetPort = TEST_HYRACKS_CC_CLIENT_PORT;
ccConfig.defaultMaxJobAttempts = 0;
- ccConfig.jobHistorySize = 10;
+ ccConfig.jobHistorySize = 0;
+ ccConfig.profileDumpPeriod = -1;
+ ccConfig.clusterTopologyDefinition = new File(topologyFilePath);
// cluster controller
cc = new ClusterControllerService(ccConfig);
@@ -63,6 +66,7 @@
ncConfig1.clusterNetIPAddress = "localhost";
ncConfig1.ccPort = TEST_HYRACKS_CC_PORT;
ncConfig1.dataIPAddress = "127.0.0.1";
+ ncConfig1.datasetIPAddress = "127.0.0.1";
ncConfig1.nodeId = NC1_ID;
nc1 = new NodeControllerService(ncConfig1);
nc1.start();
@@ -72,6 +76,7 @@
ncConfig2.clusterNetIPAddress = "localhost";
ncConfig2.ccPort = TEST_HYRACKS_CC_PORT;
ncConfig2.dataIPAddress = "127.0.0.1";
+ ncConfig2.datasetIPAddress = "127.0.0.1";
ncConfig2.nodeId = NC2_ID;
nc2 = new NodeControllerService(ncConfig2);
nc2.start();
diff --git a/pregelix/pregelix-core/src/main/java/org/apache/hadoop/fs/Path.java b/pregelix/pregelix-core/src/main/java/org/apache/hadoop/fs/Path.java
deleted file mode 100644
index 5efdde8..0000000
--- a/pregelix/pregelix-core/src/main/java/org/apache/hadoop/fs/Path.java
+++ /dev/null
@@ -1,355 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- *
- * 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.hadoop.fs;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.net.URI;
-import java.net.URISyntaxException;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Names a file or directory in a {@link FileSystem}. Path strings use slash as
- * the directory separator. A path string is absolute if it begins with a slash.
- */
-@SuppressWarnings("rawtypes")
-public class Path implements Comparable, Serializable {
- private static final long serialVersionUID = 1L;
- /** The directory separator, a slash. */
- public static final String SEPARATOR = "/";
- public static final char SEPARATOR_CHAR = '/';
-
- public static final String CUR_DIR = ".";
-
- static final boolean WINDOWS = System.getProperty("os.name").startsWith("Windows");
-
- private URI uri; // a hierarchical uri
-
- /** Resolve a child path against a parent path. */
- public Path(String parent, String child) {
- this(new Path(parent), new Path(child));
- }
-
- /** Resolve a child path against a parent path. */
- public Path(Path parent, String child) {
- this(parent, new Path(child));
- }
-
- /** Resolve a child path against a parent path. */
- public Path(String parent, Path child) {
- this(new Path(parent), child);
- }
-
- /** Resolve a child path against a parent path. */
- public Path(Path parent, Path child) {
- // Add a slash to parent's path so resolution is compatible with URI's
- URI parentUri = parent.uri;
- String parentPath = parentUri.getPath();
- if (!(parentPath.equals("/") || parentPath.equals("")))
- try {
- parentUri = new URI(parentUri.getScheme(), parentUri.getAuthority(), parentUri.getPath() + "/", null,
- parentUri.getFragment());
- } catch (URISyntaxException e) {
- throw new IllegalArgumentException(e);
- }
- URI resolved = parentUri.resolve(child.uri);
- initialize(resolved.getScheme(), resolved.getAuthority(), normalizePath(resolved.getPath()),
- resolved.getFragment());
- }
-
- private void checkPathArg(String path) {
- // disallow construction of a Path from an empty string
- if (path == null) {
- throw new IllegalArgumentException("Can not create a Path from a null string");
- }
- if (path.length() == 0) {
- throw new IllegalArgumentException("Can not create a Path from an empty string");
- }
- }
-
- /**
- * Construct a path from a String. Path strings are URIs, but with unescaped
- * elements and some additional normalization.
- */
- public Path(String pathString) {
- checkPathArg(pathString);
-
- // We can't use 'new URI(String)' directly, since it assumes things are
- // escaped, which we don't require of Paths.
-
- // add a slash in front of paths with Windows drive letters
- if (hasWindowsDrive(pathString, false))
- pathString = "/" + pathString;
-
- // parse uri components
- String scheme = null;
- String authority = null;
-
- int start = 0;
-
- // parse uri scheme, if any
- int colon = pathString.indexOf(':');
- int slash = pathString.indexOf('/');
- if ((colon != -1) && ((slash == -1) || (colon < slash))) { // has a
- // scheme
- scheme = pathString.substring(0, colon);
- start = colon + 1;
- }
-
- // parse uri authority, if any
- if (pathString.startsWith("//", start) && (pathString.length() - start > 2)) { // has
- // authority
- int nextSlash = pathString.indexOf('/', start + 2);
- int authEnd = nextSlash > 0 ? nextSlash : pathString.length();
- authority = pathString.substring(start + 2, authEnd);
- start = authEnd;
- }
-
- // uri path is the rest of the string -- query & fragment not supported
- String path = pathString.substring(start, pathString.length());
-
- initialize(scheme, authority, path, null);
- }
-
- /** Construct a Path from components. */
- public Path(String scheme, String authority, String path) {
- checkPathArg(path);
- initialize(scheme, authority, path, null);
- }
-
- /**
- * Construct a path from a URI
- */
- public Path(URI aUri) {
- uri = aUri;
- }
-
- private void initialize(String scheme, String authority, String path, String fragment) {
- try {
- this.uri = new URI(scheme, authority, normalizePath(path), null, fragment).normalize();
- } catch (URISyntaxException e) {
- throw new IllegalArgumentException(e);
- }
- }
-
- private String normalizePath(String path) {
- // remove double slashes & backslashes
- if (path.indexOf("//") != -1) {
- path = path.replace("//", "/");
- }
- if (path.indexOf("\\") != -1) {
- path = path.replace("\\", "/");
- }
-
- // trim trailing slash from non-root path (ignoring windows drive)
- int minLength = hasWindowsDrive(path, true) ? 4 : 1;
- if (path.length() > minLength && path.endsWith("/")) {
- path = path.substring(0, path.length() - 1);
- }
-
- return path;
- }
-
- private boolean hasWindowsDrive(String path, boolean slashed) {
- if (!WINDOWS)
- return false;
- int start = slashed ? 1 : 0;
- return path.length() >= start + 2
- && (slashed ? path.charAt(0) == '/' : true)
- && path.charAt(start + 1) == ':'
- && ((path.charAt(start) >= 'A' && path.charAt(start) <= 'Z') || (path.charAt(start) >= 'a' && path
- .charAt(start) <= 'z'));
- }
-
- /** Convert this to a URI. */
- public URI toUri() {
- return uri;
- }
-
- /** Return the FileSystem that owns this Path. */
- public FileSystem getFileSystem(Configuration conf) throws IOException {
- return FileSystem.get(this.toUri(), conf);
- }
-
- /** True if the directory of this path is absolute. */
- public boolean isAbsolute() {
- int start = hasWindowsDrive(uri.getPath(), true) ? 3 : 0;
- return uri.getPath().startsWith(SEPARATOR, start);
- }
-
- /** Returns the final component of this path. */
- public String getName() {
- String path = uri.getPath();
- int slash = path.lastIndexOf(SEPARATOR);
- return path.substring(slash + 1);
- }
-
- /** Returns the parent of a path or null if at root. */
- public Path getParent() {
- String path = uri.getPath();
- int lastSlash = path.lastIndexOf('/');
- int start = hasWindowsDrive(path, true) ? 3 : 0;
- if ((path.length() == start) || // empty path
- (lastSlash == start && path.length() == start + 1)) { // at root
- return null;
- }
- String parent;
- if (lastSlash == -1) {
- parent = CUR_DIR;
- } else {
- int end = hasWindowsDrive(path, true) ? 3 : 0;
- parent = path.substring(0, lastSlash == end ? end + 1 : lastSlash);
- }
- return new Path(uri.getScheme(), uri.getAuthority(), parent);
- }
-
- /** Adds a suffix to the final name in the path. */
- public Path suffix(String suffix) {
- return new Path(getParent(), getName() + suffix);
- }
-
- public String toString() {
- // we can't use uri.toString(), which escapes everything, because we
- // want
- // illegal characters unescaped in the string, for glob processing, etc.
- StringBuffer buffer = new StringBuffer();
- if (uri.getScheme() != null) {
- buffer.append(uri.getScheme());
- buffer.append(":");
- }
- if (uri.getAuthority() != null) {
- buffer.append("//");
- buffer.append(uri.getAuthority());
- }
- if (uri.getPath() != null) {
- String path = uri.getPath();
- if (path.indexOf('/') == 0 && hasWindowsDrive(path, true) && // has
- // windows
- // drive
- uri.getScheme() == null && // but no scheme
- uri.getAuthority() == null) // or authority
- path = path.substring(1); // remove slash before drive
- buffer.append(path);
- }
- if (uri.getFragment() != null) {
- buffer.append("#");
- buffer.append(uri.getFragment());
- }
- return buffer.toString();
- }
-
- public boolean equals(Object o) {
- if (!(o instanceof Path)) {
- return false;
- }
- Path that = (Path) o;
- return this.uri.equals(that.uri);
- }
-
- public int hashCode() {
- return uri.hashCode();
- }
-
- public int compareTo(Object o) {
- Path that = (Path) o;
- return this.uri.compareTo(that.uri);
- }
-
- /** Return the number of elements in this path. */
- public int depth() {
- String path = uri.getPath();
- int depth = 0;
- int slash = path.length() == 1 && path.charAt(0) == '/' ? -1 : 0;
- while (slash != -1) {
- depth++;
- slash = path.indexOf(SEPARATOR, slash + 1);
- }
- return depth;
- }
-
- /** Returns a qualified path object. */
- public Path makeQualified(FileSystem fs) {
- Path path = this;
- if (!isAbsolute()) {
- path = new Path(fs.getWorkingDirectory(), this);
- }
-
- URI pathUri = path.toUri();
- URI fsUri = fs.getUri();
-
- String scheme = pathUri.getScheme();
- String authority = pathUri.getAuthority();
- String fragment = pathUri.getFragment();
- if (scheme != null && (authority != null || fsUri.getAuthority() == null))
- return path;
-
- if (scheme == null) {
- scheme = fsUri.getScheme();
- }
-
- if (authority == null) {
- authority = fsUri.getAuthority();
- if (authority == null) {
- authority = "";
- }
- }
-
- URI newUri = null;
- try {
- newUri = new URI(scheme, authority, normalizePath(pathUri.getPath()), null, fragment);
- } catch (URISyntaxException e) {
- throw new IllegalArgumentException(e);
- }
- return new Path(newUri);
- }
-
- /** Returns a qualified path object. */
- public Path makeQualified(URI defaultUri, Path workingDir) {
- Path path = this;
- if (!isAbsolute()) {
- path = new Path(workingDir, this);
- }
-
- URI pathUri = path.toUri();
-
- String scheme = pathUri.getScheme();
- String authority = pathUri.getAuthority();
- String fragment = pathUri.getFragment();
-
- if (scheme != null && (authority != null || defaultUri.getAuthority() == null))
- return path;
-
- if (scheme == null) {
- scheme = defaultUri.getScheme();
- }
-
- if (authority == null) {
- authority = defaultUri.getAuthority();
- if (authority == null) {
- authority = "";
- }
- }
-
- URI newUri = null;
- try {
- newUri = new URI(scheme, authority, normalizePath(pathUri.getPath()), null, fragment);
- } catch (URISyntaxException e) {
- throw new IllegalArgumentException(e);
- }
- return new Path(newUri);
- }
-}
\ No newline at end of file
diff --git a/pregelix/pregelix-core/src/main/java/org/apache/hadoop/mapreduce/InputSplit.java b/pregelix/pregelix-core/src/main/java/org/apache/hadoop/mapreduce/InputSplit.java
deleted file mode 100644
index ac72160..0000000
--- a/pregelix/pregelix-core/src/main/java/org/apache/hadoop/mapreduce/InputSplit.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- *
- * 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.hadoop.mapreduce;
-
-import java.io.IOException;
-import java.io.Serializable;
-
-/**
- * <code>InputSplit</code> represents the data to be processed by an individual {@link Mapper}.
- * <p>
- * Typically, it presents a byte-oriented view on the input and is the responsibility of {@link RecordReader} of the job to process this and present a record-oriented view.
- *
- * @see InputFormat
- * @see RecordReader
- */
-public abstract class InputSplit implements Serializable {
- private static final long serialVersionUID = 1L;
-
- /**
- * Get the size of the split, so that the input splits can be sorted by
- * size.
- *
- * @return the number of bytes in the split
- * @throws IOException
- * @throws InterruptedException
- */
- public abstract long getLength() throws IOException, InterruptedException;
-
- /**
- * Get the list of nodes by name where the data for the split would be
- * local. The locations do not need to be serialized.
- *
- * @return a new array of the node nodes.
- * @throws IOException
- * @throws InterruptedException
- */
- public abstract String[] getLocations() throws IOException, InterruptedException;
-}
\ No newline at end of file
diff --git a/pregelix/pregelix-core/src/main/resources/conf/topology-template.xml b/pregelix/pregelix-core/src/main/resources/conf/topology-template.xml
new file mode 100755
index 0000000..4710706
--- /dev/null
+++ b/pregelix/pregelix-core/src/main/resources/conf/topology-template.xml
@@ -0,0 +1,7 @@
+<cluster-topology>
+ <network-switch name="Global">
+ <network-switch name="local">
+ <terminal name="127.0.0.1"/>
+ </network-switch>
+ </network-switch>
+</cluster-topology>
\ No newline at end of file
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/getip.sh b/pregelix/pregelix-core/src/main/resources/scripts/getip.sh
index e0cdf73..a691c0f 100755
--- a/pregelix/pregelix-core/src/main/resources/scripts/getip.sh
+++ b/pregelix/pregelix-core/src/main/resources/scripts/getip.sh
@@ -6,6 +6,10 @@
then
#Get IP Address
IPADDR=`/sbin/ifconfig eth0 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+ if [ "$IPADDR" = "" ]
+ then
+ IPADDR=`/sbin/ifconfig em1 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+ fi
if [ "$IPADDR" = "" ]
then
IPADDR=`/sbin/ifconfig lo | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/pregelix b/pregelix/pregelix-core/src/main/resources/scripts/pregelix
index c3fd27b..b1a2f74 100644
--- a/pregelix/pregelix-core/src/main/resources/scripts/pregelix
+++ b/pregelix/pregelix-core/src/main/resources/scripts/pregelix
@@ -91,7 +91,7 @@
REPO="$BASEDIR"/lib
fi
-CLASSPATH=$CLASSPATH_PREFIX:"$HADOOP_HOME"/conf:"$BASEDIR"/etc:$(echo ${REPO}/*.jar | tr ' ' ':'):$1
+CLASSPATH=$CLASSPATH_PREFIX:"$HADOOP_HOME"/conf:/etc/hadoop/conf:"$BASEDIR"/etc:$1
# For Cygwin, switch paths to Windows format before running java
if $cygwin; then
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/startAllNCs.sh b/pregelix/pregelix-core/src/main/resources/scripts/startAllNCs.sh
index 629bd90..d30da26 100644
--- a/pregelix/pregelix-core/src/main/resources/scripts/startAllNCs.sh
+++ b/pregelix/pregelix-core/src/main/resources/scripts/startAllNCs.sh
@@ -2,5 +2,5 @@
for i in `cat conf/slaves`
do
- ssh $i "cd ${PREGELIX_PATH}; bin/startnc.sh"
+ ssh $i "cd ${PREGELIX_PATH}; export JAVA_HOME=${JAVA_HOME}; bin/startnc.sh"
done
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/startcc.sh b/pregelix/pregelix-core/src/main/resources/scripts/startcc.sh
index fe2551d..133b604 100644
--- a/pregelix/pregelix-core/src/main/resources/scripts/startcc.sh
+++ b/pregelix/pregelix-core/src/main/resources/scripts/startcc.sh
@@ -20,6 +20,12 @@
export JAVA_HOME=$JAVA_HOME
export JAVA_OPTS=$CCJAVA_OPTS
-#Launch hyracks cc script
+
chmod -R 755 $HYRACKS_HOME
-$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyrackscc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 3 &> $CCLOGS_DIR/cc.log &
+if [ -f "conf/topology.xml" ]; then
+#Launch hyracks cc script with topology
+$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyrackscc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 0 -cluster-topology "conf/topology.xml" &> $CCLOGS_DIR/cc.log &
+else
+#Launch hyracks cc script without toplogy
+$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyrackscc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 0 &> $CCLOGS_DIR/cc.log &
+fi
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/startnc.sh b/pregelix/pregelix-core/src/main/resources/scripts/startnc.sh
index 6e0f90e..b059aad 100644
--- a/pregelix/pregelix-core/src/main/resources/scripts/startnc.sh
+++ b/pregelix/pregelix-core/src/main/resources/scripts/startnc.sh
@@ -46,4 +46,4 @@
cd $NCTMP_DIR
#Launch hyracks nc
-$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyracksnc -cc-host $CCHOST -cc-port $CC_CLUSTERPORT -cluster-net-ip-address $IPADDR -data-ip-address $IPADDR -node-id $NODEID -iodevices "${IO_DIRS}" &> $NCLOGS_DIR/$NODEID.log &
+$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyracksnc -cc-host $CCHOST -cc-port $CC_CLUSTERPORT -cluster-net-ip-address $IPADDR -data-ip-address $IPADDR -result-ip-address $IPADDR -node-id $NODEID -iodevices "${IO_DIRS}" &> $NCLOGS_DIR/$NODEID.log &
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/stopnc.sh b/pregelix/pregelix-core/src/main/resources/scripts/stopnc.sh
index 03ce4e7..35c4794 100644
--- a/pregelix/pregelix-core/src/main/resources/scripts/stopnc.sh
+++ b/pregelix/pregelix-core/src/main/resources/scripts/stopnc.sh
@@ -5,6 +5,10 @@
PID=`ps -ef|grep ${USER}|grep java|grep 'Dapp.name=hyracksnc'|awk '{print $2}'`
if [ "$PID" == "" ]; then
+ PID=`ps -ef|grep ${USER}|grep java|grep 'hyracks'|awk '{print $2}'`
+fi
+
+if [ "$PID" == "" ]; then
USERID=`id | sed 's/^uid=//;s/(.*$//'`
PID=`ps -ef|grep ${USERID}|grep java|grep 'Dapp.name=hyracksnc'|awk '{print $2}'`
fi
diff --git a/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java b/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java
index 4dfe57d..f7cadf6 100644
--- a/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java
+++ b/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java
@@ -62,7 +62,7 @@
import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
import edu.uci.ics.pregelix.core.util.TestUtils;
-import edu.uci.ics.pregelix.dataflow.std.FileWriteOperatorDescriptor;
+import edu.uci.ics.pregelix.dataflow.VertexWriteOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.std.IndexNestedLoopJoinOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.std.ProjectOperatorDescriptor;
import edu.uci.ics.pregelix.runtime.bootstrap.StorageManagerInterface;
@@ -102,7 +102,7 @@
ClusterConfig.setStorePath(PATH_TO_CLUSTER_STORE);
ClusterConfig.setClusterPropertiesPath(PATH_TO_CLUSTER_PROPERTIES);
cleanupStores();
- PregelixHyracksIntegrationUtil.init();
+ PregelixHyracksIntegrationUtil.init("src/test/resources/topology.xml");
PregelixHyracksIntegrationUtil.createApp(HYRACKS_APP_NAME);
FileUtils.forceMkdir(new File(EXPECT_RESULT_DIR));
@@ -195,7 +195,7 @@
FileSplit[] results = new FileSplit[1];
results[0] = resultFile;
IFileSplitProvider resultFileSplitProvider = new ConstantFileSplitProvider(results);
- FileWriteOperatorDescriptor writer = new FileWriteOperatorDescriptor(spec, null, resultFileSplitProvider, null,
+ VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, null, resultFileSplitProvider, null,
null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, new String[] { NC1_ID });
PartitionConstraintHelper.addPartitionCountConstraint(spec, writer, 1);
@@ -362,7 +362,7 @@
FileSplit[] results = new FileSplit[1];
results[0] = resultFile;
IFileSplitProvider resultFileSplitProvider = new ConstantFileSplitProvider(results);
- FileWriteOperatorDescriptor writer = new FileWriteOperatorDescriptor(spec, null, resultFileSplitProvider, null,
+ VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, null, resultFileSplitProvider, null,
null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, new String[] { NC1_ID });
PartitionConstraintHelper.addPartitionCountConstraint(spec, writer, 1);
@@ -459,7 +459,7 @@
FileSplit[] results = new FileSplit[1];
results[0] = resultFile;
IFileSplitProvider resultFileSplitProvider = new ConstantFileSplitProvider(results);
- FileWriteOperatorDescriptor writer = new FileWriteOperatorDescriptor(spec, null, resultFileSplitProvider, null,
+ VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, null, resultFileSplitProvider, null,
null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, new String[] { NC1_ID });
PartitionConstraintHelper.addPartitionCountConstraint(spec, writer, 1);
@@ -566,7 +566,7 @@
FileSplit[] results = new FileSplit[1];
results[0] = resultFile;
IFileSplitProvider resultFileSplitProvider = new ConstantFileSplitProvider(results);
- FileWriteOperatorDescriptor writer = new FileWriteOperatorDescriptor(spec, null, resultFileSplitProvider, null,
+ VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, null, resultFileSplitProvider, null,
null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, new String[] { NC1_ID });
PartitionConstraintHelper.addPartitionCountConstraint(spec, writer, 1);
diff --git a/pregelix/pregelix-core/src/test/resources/hadoop/conf/mapred-site.xml b/pregelix/pregelix-core/src/test/resources/hadoop/conf/mapred-site.xml
index 1b9a4d6..f89dd79 100644
--- a/pregelix/pregelix-core/src/test/resources/hadoop/conf/mapred-site.xml
+++ b/pregelix/pregelix-core/src/test/resources/hadoop/conf/mapred-site.xml
@@ -18,8 +18,8 @@
<value>20</value>
</property>
<property>
- <name>mapred.min.split.size</name>
- <value>65536</value>
+ <name>mapred.max.split.size</name>
+ <value>4096</value>
</property>
</configuration>
diff --git a/pregelix/pregelix-core/src/test/resources/topology.xml b/pregelix/pregelix-core/src/test/resources/topology.xml
new file mode 100755
index 0000000..4710706
--- /dev/null
+++ b/pregelix/pregelix-core/src/test/resources/topology.xml
@@ -0,0 +1,7 @@
+<cluster-topology>
+ <network-switch name="Global">
+ <network-switch name="local">
+ <terminal name="127.0.0.1"/>
+ </network-switch>
+ </network-switch>
+</cluster-topology>
\ No newline at end of file
diff --git a/pregelix/pregelix-dataflow-std-base/pom.xml b/pregelix/pregelix-dataflow-std-base/pom.xml
index 53d7c22..caa3222 100644
--- a/pregelix/pregelix-dataflow-std-base/pom.xml
+++ b/pregelix/pregelix-dataflow-std-base/pom.xml
@@ -22,8 +22,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
<plugin>
@@ -42,6 +43,7 @@
</plugin>
<plugin>
<artifactId>maven-clean-plugin</artifactId>
+ <version>2.5</version>
<configuration>
<filesets>
<fileset>
diff --git a/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/IUpdateFunction.java b/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/IUpdateFunction.java
index 62f92dd..a0d365f 100644
--- a/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/IUpdateFunction.java
+++ b/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/IUpdateFunction.java
@@ -16,17 +16,19 @@
package edu.uci.ics.pregelix.dataflow.std.base;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
public interface IUpdateFunction extends IFunction {
- /**
- * update the tuple pointed by tupleRef called after process,
- * one-input-tuple-at-a-time
- *
- * @param tupleRef
- * @throws HyracksDataException
- */
- public void update(ITupleReference tupleRef) throws HyracksDataException;
+ /**
+ * update the tuple pointed by tupleRef called after process,
+ * one-input-tuple-at-a-time
+ *
+ * @param tupleRef
+ * @throws HyracksDataException
+ */
+ public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb)
+ throws HyracksDataException;
}
diff --git a/pregelix/pregelix-dataflow-std/pom.xml b/pregelix/pregelix-dataflow-std/pom.xml
index cc9a184..c5a0913 100644
--- a/pregelix/pregelix-dataflow-std/pom.xml
+++ b/pregelix/pregelix-dataflow-std/pom.xml
@@ -1,14 +1,15 @@
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<artifactId>pregelix-dataflow-std</artifactId>
<packaging>jar</packaging>
<name>pregelix-dataflow-std</name>
<parent>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>pregelix</artifactId>
- <version>0.2.3-SNAPSHOT</version>
- </parent>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>pregelix</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ </parent>
<properties>
@@ -22,8 +23,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
<plugin>
@@ -42,6 +44,7 @@
</plugin>
<plugin>
<artifactId>maven-clean-plugin</artifactId>
+ <version>2.5</version>
<configuration>
<filesets>
<fileset>
@@ -100,11 +103,13 @@
<groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks-data-std</artifactId>
<version>0.2.3-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
</dependency>
<dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-core</artifactId>
- <version>0.20.2</version>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-hdfs-core</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
<type>jar</type>
<scope>compile</scope>
</dependency>
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/BTreeSearchFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/BTreeSearchFunctionUpdateOperatorNodePushable.java
index fb84aa0..3938613 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/BTreeSearchFunctionUpdateOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/BTreeSearchFunctionUpdateOperatorNodePushable.java
@@ -43,6 +43,7 @@
import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunctionFactory;
import edu.uci.ics.pregelix.dataflow.util.FunctionProxy;
+import edu.uci.ics.pregelix.dataflow.util.UpdateBuffer;
public class BTreeSearchFunctionUpdateOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
protected TreeIndexDataflowHelper treeIndexHelper;
@@ -70,6 +71,8 @@
private final IFrameWriter[] writers;
private final FunctionProxy functionProxy;
+ private ArrayTupleBuilder cloneUpdateTb;
+ private final UpdateBuffer updateBuffer;
public BTreeSearchFunctionUpdateOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward,
@@ -94,6 +97,7 @@
this.writers = new IFrameWriter[outputArity];
this.functionProxy = new FunctionProxy(ctx, functionFactory, preHookFactory, postHookFactory, inputRdFactory,
writers);
+ this.updateBuffer = new UpdateBuffer(ctx, 2);
}
@Override
@@ -122,6 +126,9 @@
appender = new FrameTupleAppender(treeIndexHelper.getHyracksTaskContext().getFrameSize());
appender.reset(writeBuffer, true);
indexAccessor = btree.createAccessor();
+
+ cloneUpdateTb = new ArrayTupleBuilder(btree.getFieldCount());
+ updateBuffer.setFieldCount(btree.getFieldCount());
} catch (Exception e) {
treeIndexHelper.deinit();
throw new HyracksDataException(e);
@@ -136,7 +143,24 @@
while (cursor.hasNext()) {
cursor.next();
ITupleReference tuple = cursor.getTuple();
- functionProxy.functionCall(tuple);
+ functionProxy.functionCall(tuple, cloneUpdateTb);
+
+ //doing clone update
+ if (cloneUpdateTb.getSize() > 0) {
+ if (!updateBuffer.appendTuple(cloneUpdateTb)) {
+ //release the cursor/latch
+ cursor.close();
+ //batch update
+ updateBuffer.updateBTree(indexAccessor);
+
+ //search again
+ cursor.reset();
+ rangePred.setLowKey(tuple, true);
+ rangePred.setHighKey(highKey, highKeyInclusive);
+ indexAccessor.search(cursor, rangePred);
+ }
+ }
+ cloneUpdateTb.reset();
}
}
@@ -168,6 +192,8 @@
try {
try {
cursor.close();
+ //batch update
+ updateBuffer.updateBTree(indexAccessor);
} catch (Exception e) {
throw new HyracksDataException(e);
}
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/FunctionCallOperatorDescriptor.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/FunctionCallOperatorDescriptor.java
index ee3ac82..4cbd6c4 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/FunctionCallOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/FunctionCallOperatorDescriptor.java
@@ -61,22 +61,21 @@
private FrameDeserializer frameDeserializer;
private final IFrameWriter[] writers = new IFrameWriter[outputArity];
private final IFunction function = functionFactory.createFunction();
+ private ClassLoader ctxCL = Thread.currentThread().getContextClassLoader();
@Override
- public void close() throws HyracksDataException {
- if (postHookFactory != null)
- postHookFactory.createRuntimeHook().configure(ctx);
- function.close();
+ public void open() throws HyracksDataException {
+ rd0 = inputRdFactory == null ? recordDescProvider.getInputRecordDescriptor(getActivityId(), 0)
+ : inputRdFactory.createRecordDescriptor();
+ frameDeserializer = new FrameDeserializer(ctx.getFrameSize(), rd0);
+ ctxCL = Thread.currentThread().getContextClassLoader();
+ Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
for (IFrameWriter writer : writers) {
- writer.close();
+ writer.open();
}
- }
-
- @Override
- public void fail() throws HyracksDataException {
- for (IFrameWriter writer : writers) {
- writer.fail();
- }
+ if (preHookFactory != null)
+ preHookFactory.createRuntimeHook().configure(ctx);
+ function.open(ctx, rd0, writers);
}
@Override
@@ -89,17 +88,21 @@
}
@Override
- public void open() throws HyracksDataException {
- rd0 = inputRdFactory == null ? recordDescProvider.getInputRecordDescriptor(getActivityId(), 0)
- : inputRdFactory.createRecordDescriptor();
- frameDeserializer = new FrameDeserializer(ctx.getFrameSize(), rd0);
- Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
+ public void close() throws HyracksDataException {
+ if (postHookFactory != null)
+ postHookFactory.createRuntimeHook().configure(ctx);
+ function.close();
for (IFrameWriter writer : writers) {
- writer.open();
+ writer.close();
}
- if (preHookFactory != null)
- preHookFactory.createRuntimeHook().configure(ctx);
- function.open(ctx, rd0, writers);
+ Thread.currentThread().setContextClassLoader(ctxCL);
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ for (IFrameWriter writer : writers) {
+ writer.fail();
+ }
}
@Override
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java
index 75a8087..37029f3 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java
@@ -14,7 +14,6 @@
*/
package edu.uci.ics.pregelix.dataflow.std;
-import java.io.DataOutput;
import java.nio.ByteBuffer;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
@@ -44,6 +43,7 @@
import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunctionFactory;
import edu.uci.ics.pregelix.dataflow.util.FunctionProxy;
+import edu.uci.ics.pregelix.dataflow.util.UpdateBuffer;
public class IndexNestedLoopJoinFunctionUpdateOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
private TreeIndexDataflowHelper treeIndexOpHelper;
@@ -51,9 +51,6 @@
private ByteBuffer writeBuffer;
private FrameTupleAppender appender;
- private ArrayTupleBuilder tb;
- private DataOutput dos;
-
private BTree btree;
private PermutingFrameTupleReference lowKey;
private PermutingFrameTupleReference highKey;
@@ -67,17 +64,16 @@
protected ITreeIndexAccessor indexAccessor;
private RecordDescriptor recDesc;
- private final RecordDescriptor inputRecDesc;
-
private final IFrameWriter[] writers;
private final FunctionProxy functionProxy;
+ private ArrayTupleBuilder cloneUpdateTb;
+ private final UpdateBuffer updateBuffer;
public IndexNestedLoopJoinFunctionUpdateOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward,
int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
IUpdateFunctionFactory functionFactory, IRuntimeHookFactory preHookFactory,
IRuntimeHookFactory postHookFactory, IRecordDescriptorFactory inputRdFactory, int outputArity) {
- inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
treeIndexOpHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
opDesc, ctx, partition);
this.lowKeyInclusive = lowKeyInclusive;
@@ -95,6 +91,7 @@
this.writers = new IFrameWriter[outputArity];
this.functionProxy = new FunctionProxy(ctx, functionFactory, preHookFactory, postHookFactory, inputRdFactory,
writers);
+ this.updateBuffer = new UpdateBuffer(ctx, 2);
}
protected void setCursor() {
@@ -144,12 +141,12 @@
rangePred = new RangePredicate(null, null, lowKeyInclusive, highKeyInclusive, lowKeySearchCmp,
highKeySearchCmp);
writeBuffer = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
- tb = new ArrayTupleBuilder(inputRecDesc.getFields().length + btree.getFieldCount());
- dos = tb.getDataOutput();
appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksTaskContext().getFrameSize());
appender.reset(writeBuffer, true);
indexAccessor = btree.createAccessor();
+ cloneUpdateTb = new ArrayTupleBuilder(btree.getFieldCount());
+ updateBuffer.setFieldCount(btree.getFieldCount());
} catch (Exception e) {
treeIndexOpHelper.deinit();
throw new HyracksDataException(e);
@@ -158,27 +155,29 @@
private void writeSearchResults(IFrameTupleAccessor leftAccessor, int tIndex) throws Exception {
while (cursor.hasNext()) {
- tb.reset();
cursor.next();
-
ITupleReference tupleRef = cursor.getTuple();
- for (int i = 0; i < inputRecDesc.getFields().length; i++) {
- int tupleStart = leftAccessor.getTupleStartOffset(tIndex);
- int fieldStart = leftAccessor.getFieldStartOffset(tIndex, i);
- int offset = leftAccessor.getFieldSlotsLength() + tupleStart + fieldStart;
- int len = leftAccessor.getFieldEndOffset(tIndex, i) - fieldStart;
- dos.write(leftAccessor.getBuffer().array(), offset, len);
- tb.addFieldEndOffset();
- }
- for (int i = 0; i < tupleRef.getFieldCount(); i++) {
- dos.write(tupleRef.getFieldData(i), tupleRef.getFieldStart(i), tupleRef.getFieldLength(i));
- tb.addFieldEndOffset();
- }
/**
* call the update function
*/
- functionProxy.functionCall(tb, tupleRef);
+ functionProxy.functionCall(leftAccessor, tIndex, tupleRef, cloneUpdateTb);
+
+ if (cloneUpdateTb.getSize() > 0) {
+ if (!updateBuffer.appendTuple(cloneUpdateTb)) {
+ //release the cursor/latch
+ cursor.close();
+ //batch update
+ updateBuffer.updateBTree(indexAccessor);
+
+ //search again
+ cursor.reset();
+ rangePred.setLowKey(tupleRef, true);
+ rangePred.setHighKey(highKey, highKeyInclusive);
+ indexAccessor.search(cursor, rangePred);
+ }
+ }
+ cloneUpdateTb.reset();
}
}
@@ -210,6 +209,8 @@
try {
try {
cursor.close();
+ //batch update
+ updateBuffer.updateBTree(indexAccessor);
} catch (Exception e) {
throw new HyracksDataException(e);
}
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java
index c31ebd4..f7b3d62 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java
@@ -45,6 +45,7 @@
import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunctionFactory;
import edu.uci.ics.pregelix.dataflow.util.FunctionProxy;
+import edu.uci.ics.pregelix.dataflow.util.UpdateBuffer;
public class IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable extends
AbstractUnaryInputOperatorNodePushable {
@@ -53,7 +54,7 @@
private ByteBuffer writeBuffer;
private FrameTupleAppender appender;
- private ArrayTupleBuilder tb;
+ private ArrayTupleBuilder nullTupleBuilder;
private DataOutput dos;
private BTree btree;
@@ -76,6 +77,8 @@
private final IFrameWriter[] writers;
private final FunctionProxy functionProxy;
+ private ArrayTupleBuilder cloneUpdateTb;
+ private UpdateBuffer updateBuffer;
public IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward,
@@ -100,6 +103,7 @@
this.writers = new IFrameWriter[outputArity];
this.functionProxy = new FunctionProxy(ctx, functionFactory, preHookFactory, postHookFactory, inputRdFactory,
writers);
+ this.updateBuffer = new UpdateBuffer(ctx, 2);
}
protected void setCursor() {
@@ -144,8 +148,15 @@
rangePred = new RangePredicate(null, null, true, true, lowKeySearchCmp, highKeySearchCmp);
writeBuffer = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
- tb = new ArrayTupleBuilder(inputRecDesc.getFields().length + btree.getFieldCount());
- dos = tb.getDataOutput();
+
+ nullTupleBuilder = new ArrayTupleBuilder(inputRecDesc.getFields().length);
+ dos = nullTupleBuilder.getDataOutput();
+ nullTupleBuilder.reset();
+ for (int i = 0; i < inputRecDesc.getFields().length; i++) {
+ nullWriter[i].writeNull(dos);
+ nullTupleBuilder.addFieldEndOffset();
+ }
+
appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksTaskContext().getFrameSize());
appender.reset(writeBuffer, true);
@@ -164,32 +175,38 @@
match = false;
}
+ cloneUpdateTb = new ArrayTupleBuilder(btree.getFieldCount());
+ updateBuffer.setFieldCount(btree.getFieldCount());
} catch (Exception e) {
treeIndexOpHelper.deinit();
throw new HyracksDataException(e);
}
}
+ //for the join match casesos
private void writeResults(IFrameTupleAccessor leftAccessor, int tIndex, ITupleReference frameTuple)
throws Exception {
- tb.reset();
- for (int i = 0; i < inputRecDesc.getFields().length; i++) {
- int tupleStart = leftAccessor.getTupleStartOffset(tIndex);
- int fieldStart = leftAccessor.getFieldStartOffset(tIndex, i);
- int offset = leftAccessor.getFieldSlotsLength() + tupleStart + fieldStart;
- int len = leftAccessor.getFieldEndOffset(tIndex, i) - fieldStart;
- dos.write(leftAccessor.getBuffer().array(), offset, len);
- tb.addFieldEndOffset();
- }
- for (int i = 0; i < frameTuple.getFieldCount(); i++) {
- dos.write(frameTuple.getFieldData(i), frameTuple.getFieldStart(i), frameTuple.getFieldLength(i));
- tb.addFieldEndOffset();
- }
-
/**
* function call
*/
- functionProxy.functionCall(tb, frameTuple);
+ functionProxy.functionCall(leftAccessor, tIndex, frameTuple, cloneUpdateTb);
+
+ //doing clone update
+ if (cloneUpdateTb.getSize() > 0) {
+ if (!updateBuffer.appendTuple(cloneUpdateTb)) {
+ //release the cursor/latch
+ cursor.close();
+ //batch update
+ updateBuffer.updateBTree(indexAccessor);
+
+ //search again and recover the cursor
+ cursor.reset();
+ rangePred.setLowKey(frameTuple, true);
+ rangePred.setHighKey(null, true);
+ indexAccessor.search(cursor, rangePred);
+ }
+ cloneUpdateTb.reset();
+ }
}
@Override
@@ -243,6 +260,8 @@
}
try {
cursor.close();
+ //batch update
+ updateBuffer.updateBTree(indexAccessor);
} catch (Exception e) {
throw new HyracksDataException(e);
}
@@ -271,20 +290,27 @@
/** write result for outer case */
private void writeResults(ITupleReference frameTuple) throws Exception {
- tb.reset();
- for (int i = 0; i < inputRecDesc.getFields().length; i++) {
- nullWriter[i].writeNull(dos);
- tb.addFieldEndOffset();
- }
- for (int i = 0; i < frameTuple.getFieldCount(); i++) {
- dos.write(frameTuple.getFieldData(i), frameTuple.getFieldStart(i), frameTuple.getFieldLength(i));
- tb.addFieldEndOffset();
- }
-
/**
* function call
*/
- functionProxy.functionCall(tb, frameTuple);
+ functionProxy.functionCall(nullTupleBuilder, frameTuple, cloneUpdateTb);
+
+ //doing clone update
+ if (cloneUpdateTb.getSize() > 0) {
+ if (!updateBuffer.appendTuple(cloneUpdateTb)) {
+ //release the cursor/latch
+ cursor.close();
+ //batch update
+ updateBuffer.updateBTree(indexAccessor);
+
+ //search again and recover the cursor
+ cursor.reset();
+ rangePred.setLowKey(frameTuple, true);
+ rangePred.setHighKey(null, true);
+ indexAccessor.search(cursor, rangePred);
+ }
+ cloneUpdateTb.reset();
+ }
}
@Override
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java
index 0a966b5..6af60a8 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java
@@ -14,7 +14,6 @@
*/
package edu.uci.ics.pregelix.dataflow.std;
-import java.io.DataOutput;
import java.nio.ByteBuffer;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
@@ -44,6 +43,7 @@
import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunctionFactory;
import edu.uci.ics.pregelix.dataflow.util.FunctionProxy;
+import edu.uci.ics.pregelix.dataflow.util.UpdateBuffer;
public class IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
private TreeIndexDataflowHelper treeIndexOpHelper;
@@ -51,8 +51,6 @@
private ByteBuffer writeBuffer;
private FrameTupleAppender appender;
- private ArrayTupleBuilder tb;
- private DataOutput dos;
private BTree btree;
private boolean isForward;
@@ -63,8 +61,6 @@
protected ITreeIndexAccessor indexAccessor;
private RecordDescriptor recDesc;
- private final RecordDescriptor inputRecDesc;
-
private PermutingFrameTupleReference lowKey;
private PermutingFrameTupleReference highKey;
@@ -73,13 +69,14 @@
private final IFrameWriter[] writers;
private final FunctionProxy functionProxy;
+ private ArrayTupleBuilder cloneUpdateTb;
+ private UpdateBuffer updateBuffer;
public IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward,
int[] lowKeyFields, int[] highKeyFields, IUpdateFunctionFactory functionFactory,
IRuntimeHookFactory preHookFactory, IRuntimeHookFactory postHookFactory,
IRecordDescriptorFactory inputRdFactory, int outputArity) {
- inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
treeIndexOpHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
opDesc, ctx, partition);
this.isForward = isForward;
@@ -97,6 +94,7 @@
this.writers = new IFrameWriter[outputArity];
this.functionProxy = new FunctionProxy(ctx, functionFactory, preHookFactory, postHookFactory, inputRdFactory,
writers);
+ this.updateBuffer = new UpdateBuffer(ctx, 2);
}
protected void setCursor() {
@@ -123,8 +121,6 @@
lowKeySearchCmp = new MultiComparator(lowKeySearchComparators);
writeBuffer = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
- tb = new ArrayTupleBuilder(btree.getFieldCount());
- dos = tb.getDataOutput();
appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksTaskContext().getFrameSize());
appender.reset(writeBuffer, true);
@@ -142,7 +138,8 @@
currentTopTuple = cursor.getTuple();
match = false;
}
-
+ cloneUpdateTb = new ArrayTupleBuilder(btree.getFieldCount());
+ updateBuffer.setFieldCount(btree.getFieldCount());
} catch (Exception e) {
treeIndexOpHelper.deinit();
throw new HyracksDataException(e);
@@ -207,6 +204,9 @@
}
try {
cursor.close();
+
+ //batch update
+ updateBuffer.updateBTree(indexAccessor);
} catch (Exception e) {
throw new HyracksDataException(e);
}
@@ -231,29 +231,47 @@
/** write the right result */
private void writeRightResults(ITupleReference frameTuple) throws Exception {
- tb.reset();
- for (int i = 0; i < frameTuple.getFieldCount(); i++) {
- dos.write(frameTuple.getFieldData(i), frameTuple.getFieldStart(i), frameTuple.getFieldLength(i));
- tb.addFieldEndOffset();
- }
+ functionProxy.functionCall(frameTuple, cloneUpdateTb);
- functionProxy.functionCall(tb, frameTuple);
+ //doing clone update
+ if (cloneUpdateTb.getSize() > 0) {
+ if (!updateBuffer.appendTuple(cloneUpdateTb)) {
+ //release the cursor/latch
+ cursor.close();
+ //batch update
+ updateBuffer.updateBTree(indexAccessor);
+
+ //search again
+ cursor.reset();
+ rangePred.setLowKey(frameTuple, true);
+ rangePred.setHighKey(null, true);
+ indexAccessor.search(cursor, rangePred);
+ }
+ cloneUpdateTb.reset();
+ }
}
/** write the left result */
private void writeLeftResults(IFrameTupleAccessor leftAccessor, int tIndex, ITupleReference frameTuple)
throws Exception {
- tb.reset();
- for (int i = 0; i < inputRecDesc.getFields().length; i++) {
- int tupleStart = leftAccessor.getTupleStartOffset(tIndex);
- int fieldStart = leftAccessor.getFieldStartOffset(tIndex, i);
- int offset = leftAccessor.getFieldSlotsLength() + tupleStart + fieldStart;
- int len = leftAccessor.getFieldEndOffset(tIndex, i) - fieldStart;
- dos.write(leftAccessor.getBuffer().array(), offset, len);
- tb.addFieldEndOffset();
- }
+ functionProxy.functionCall(leftAccessor, tIndex, frameTuple, cloneUpdateTb);
- functionProxy.functionCall(tb, frameTuple);
+ //doing clone update
+ if (cloneUpdateTb.getSize() > 0) {
+ if (!updateBuffer.appendTuple(cloneUpdateTb)) {
+ //release the cursor/latch
+ cursor.close();
+ //batch update
+ updateBuffer.updateBTree(indexAccessor);
+
+ //search again
+ cursor.reset();
+ rangePred.setLowKey(frameTuple, true);
+ rangePred.setHighKey(null, true);
+ indexAccessor.search(cursor, rangePred);
+ }
+ cloneUpdateTb.reset();
+ }
}
@Override
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/FunctionProxy.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/FunctionProxy.java
index 4b0f4a5..99bca1a 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/FunctionProxy.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/FunctionProxy.java
@@ -15,6 +15,7 @@
package edu.uci.ics.pregelix.dataflow.util;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -36,6 +37,7 @@
private final IFrameWriter[] writers;
private TupleDeserializer tupleDe;
private RecordDescriptor inputRd;
+ private ClassLoader ctxCL;
public FunctionProxy(IHyracksTaskContext ctx, IUpdateFunctionFactory functionFactory,
IRuntimeHookFactory preHookFactory, IRuntimeHookFactory postHookFactory,
@@ -56,6 +58,7 @@
public void functionOpen() throws HyracksDataException {
inputRd = inputRdFactory.createRecordDescriptor();
tupleDe = new TupleDeserializer(inputRd);
+ ctxCL = Thread.currentThread().getContextClassLoader();
Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
for (IFrameWriter writer : writers) {
writer.open();
@@ -68,16 +71,19 @@
/**
* Call the function
*
- * @param tb
- * input data
+ * @param leftAccessor
+ * input page accessor
+ * @param leftTupleIndex
+ * the tuple index in the page
* @param updateRef
* update pointer
* @throws HyracksDataException
*/
- public void functionCall(ArrayTupleBuilder tb, ITupleReference updateRef) throws HyracksDataException {
- Object[] tuple = tupleDe.deserializeRecord(tb);
+ public void functionCall(IFrameTupleAccessor leftAccessor, int leftTupleIndex, ITupleReference right,
+ ArrayTupleBuilder cloneUpdateTb) throws HyracksDataException {
+ Object[] tuple = tupleDe.deserializeRecord(leftAccessor, leftTupleIndex, right);
function.process(tuple);
- function.update(updateRef);
+ function.update(right, cloneUpdateTb);
}
/**
@@ -86,10 +92,26 @@
* @param updateRef
* @throws HyracksDataException
*/
- public void functionCall(ITupleReference updateRef) throws HyracksDataException {
+ public void functionCall(ITupleReference updateRef, ArrayTupleBuilder cloneUpdateTb) throws HyracksDataException {
Object[] tuple = tupleDe.deserializeRecord(updateRef);
function.process(tuple);
- function.update(updateRef);
+ function.update(updateRef, cloneUpdateTb);
+ }
+
+ /**
+ * Call the function
+ *
+ * @param tb
+ * input data
+ * @param inPlaceUpdateRef
+ * update pointer
+ * @throws HyracksDataException
+ */
+ public void functionCall(ArrayTupleBuilder tb, ITupleReference inPlaceUpdateRef, ArrayTupleBuilder cloneUpdateTb)
+ throws HyracksDataException {
+ Object[] tuple = tupleDe.deserializeRecord(tb, inPlaceUpdateRef);
+ function.process(tuple);
+ function.update(inPlaceUpdateRef, cloneUpdateTb);
}
/**
@@ -104,5 +126,6 @@
for (IFrameWriter writer : writers) {
writer.close();
}
+ Thread.currentThread().setContextClassLoader(ctxCL);
}
}
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/TupleDeserializer.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/TupleDeserializer.java
index 5ae1d81..4fe83db 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/TupleDeserializer.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/TupleDeserializer.java
@@ -20,6 +20,7 @@
import java.util.logging.Level;
import java.util.logging.Logger;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -42,7 +43,7 @@
}
public Object[] deserializeRecord(ITupleReference tupleRef) throws HyracksDataException {
- for (int i = 0; i < record.length; ++i) {
+ for (int i = 0; i < tupleRef.getFieldCount(); ++i) {
byte[] data = tupleRef.getFieldData(i);
int offset = tupleRef.getFieldStart(i);
bbis.setByteArray(data, offset);
@@ -65,11 +66,65 @@
return record;
}
- public Object[] deserializeRecord(ArrayTupleBuilder tb) throws HyracksDataException {
+ public Object[] deserializeRecord(IFrameTupleAccessor left, int tIndex, ITupleReference right)
+ throws HyracksDataException {
+ byte[] data = left.getBuffer().array();
+ int tStart = left.getTupleStartOffset(tIndex) + left.getFieldSlotsLength();
+ int leftFieldCount = left.getFieldCount();
+ int fStart = tStart;
+ for (int i = 0; i < leftFieldCount; ++i) {
+ /**
+ * reset the input
+ */
+ fStart = tStart + left.getFieldStartOffset(tIndex, i);
+ bbis.setByteArray(data, fStart);
+
+ /**
+ * do deserialization
+ */
+ Object instance = recordDescriptor.getFields()[i].deserialize(di);
+ if (LOGGER.isLoggable(Level.FINEST)) {
+ LOGGER.finest(i + " " + instance);
+ }
+ record[i] = instance;
+ if (FrameConstants.DEBUG_FRAME_IO) {
+ try {
+ if (di.readInt() != FrameConstants.FRAME_FIELD_MAGIC) {
+ throw new HyracksDataException("Field magic mismatch");
+ }
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+ for (int i = leftFieldCount; i < record.length; ++i) {
+ byte[] rightData = right.getFieldData(i - leftFieldCount);
+ int rightOffset = right.getFieldStart(i - leftFieldCount);
+ bbis.setByteArray(rightData, rightOffset);
+
+ Object instance = recordDescriptor.getFields()[i].deserialize(di);
+ if (LOGGER.isLoggable(Level.FINEST)) {
+ LOGGER.finest(i + " " + instance);
+ }
+ record[i] = instance;
+ if (FrameConstants.DEBUG_FRAME_IO) {
+ try {
+ if (di.readInt() != FrameConstants.FRAME_FIELD_MAGIC) {
+ throw new HyracksDataException("Field magic mismatch");
+ }
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+ return record;
+ }
+
+ public Object[] deserializeRecord(ArrayTupleBuilder tb, ITupleReference right) throws HyracksDataException {
byte[] data = tb.getByteArray();
int[] offset = tb.getFieldEndOffsets();
int start = 0;
- for (int i = 0; i < record.length; ++i) {
+ for (int i = 0; i < offset.length; ++i) {
/**
* reset the input
*/
@@ -94,6 +149,26 @@
}
}
}
+ for (int i = offset.length; i < record.length; ++i) {
+ byte[] rightData = right.getFieldData(i - offset.length);
+ int rightOffset = right.getFieldStart(i - offset.length);
+ bbis.setByteArray(rightData, rightOffset);
+
+ Object instance = recordDescriptor.getFields()[i].deserialize(di);
+ if (LOGGER.isLoggable(Level.FINEST)) {
+ LOGGER.finest(i + " " + instance);
+ }
+ record[i] = instance;
+ if (FrameConstants.DEBUG_FRAME_IO) {
+ try {
+ if (di.readInt() != FrameConstants.FRAME_FIELD_MAGIC) {
+ throw new HyracksDataException("Field magic mismatch");
+ }
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+ }
return record;
}
}
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBuffer.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBuffer.java
new file mode 100644
index 0000000..9a30647
--- /dev/null
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBuffer.java
@@ -0,0 +1,112 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ * 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 edu.uci.ics.pregelix.dataflow.util;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+
+/**
+ * The buffer to hold updates.
+ * We do a batch update for the B-tree during index search and join so that
+ * avoid to open/close cursors frequently.
+ */
+public class UpdateBuffer {
+
+ private int currentInUse = 0;
+ private final int pageLimit;
+ private final List<ByteBuffer> buffers = new ArrayList<ByteBuffer>();
+ private final FrameTupleAppender appender;
+ private final IHyracksTaskContext ctx;
+ private final FrameTupleReference tuple = new FrameTupleReference();
+ private final int frameSize;
+ private IFrameTupleAccessor fta;
+
+ public UpdateBuffer(int numPages, IHyracksTaskContext ctx, int fieldCount) {
+ this.appender = new FrameTupleAppender(ctx.getFrameSize());
+ ByteBuffer buffer = ctx.allocateFrame();
+ this.buffers.add(buffer);
+ this.appender.reset(buffer, true);
+ this.pageLimit = numPages;
+ this.ctx = ctx;
+ this.frameSize = ctx.getFrameSize();
+ this.fta = new UpdateBufferTupleAccessor(frameSize, fieldCount);
+ }
+
+ public UpdateBuffer(IHyracksTaskContext ctx, int fieldCount) {
+ //by default, the update buffer has 1000 pages
+ this(1000, ctx, fieldCount);
+ }
+
+ public void setFieldCount(int fieldCount) {
+ if (fta.getFieldCount() != fieldCount) {
+ this.fta = new UpdateBufferTupleAccessor(frameSize, fieldCount);
+ }
+ }
+
+ public boolean appendTuple(ArrayTupleBuilder tb) throws HyracksDataException {
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ if (currentInUse + 1 < pageLimit) {
+ // move to the new buffer
+ currentInUse++;
+ allocate(currentInUse);
+ ByteBuffer buffer = buffers.get(currentInUse);
+ appender.reset(buffer, true);
+
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ throw new HyracksDataException("tuple cannot be appended to a new frame!");
+ }
+ return true;
+ } else {
+ return false;
+ }
+ } else {
+ return true;
+ }
+ }
+
+ public void updateBTree(ITreeIndexAccessor bta) throws HyracksDataException, IndexException {
+ // batch update
+ for (int i = 0; i <= currentInUse; i++) {
+ ByteBuffer buffer = buffers.get(i);
+ fta.reset(buffer);
+ for (int j = 0; j < fta.getTupleCount(); j++) {
+ tuple.reset(fta, j);
+ bta.update(tuple);
+ }
+ }
+
+ //cleanup the buffer
+ currentInUse = 0;
+ ByteBuffer buffer = buffers.get(0);
+ appender.reset(buffer, true);
+ }
+
+ private void allocate(int index) {
+ if (index >= buffers.size()) {
+ buffers.add(ctx.allocateFrame());
+ }
+ }
+}
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBufferTupleAccessor.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBufferTupleAccessor.java
new file mode 100644
index 0000000..39f1361
--- /dev/null
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBufferTupleAccessor.java
@@ -0,0 +1,81 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ * 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 edu.uci.ics.pregelix.dataflow.util;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+
+public final class UpdateBufferTupleAccessor implements IFrameTupleAccessor {
+ private final int frameSize;
+ private final int fieldCount;
+ private ByteBuffer buffer;
+
+ public UpdateBufferTupleAccessor(int frameSize, int fieldCount) {
+ this.frameSize = frameSize;
+ this.fieldCount = fieldCount;
+ }
+
+ @Override
+ public void reset(ByteBuffer buffer) {
+ this.buffer = buffer;
+ }
+
+ @Override
+ public ByteBuffer getBuffer() {
+ return buffer;
+ }
+
+ @Override
+ public int getTupleCount() {
+ return buffer.getInt(FrameHelper.getTupleCountOffset(frameSize));
+ }
+
+ @Override
+ public int getTupleStartOffset(int tupleIndex) {
+ return tupleIndex == 0 ? 0 : buffer.getInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * tupleIndex);
+ }
+
+ @Override
+ public int getTupleEndOffset(int tupleIndex) {
+ return buffer.getInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleIndex + 1));
+ }
+
+ @Override
+ public int getFieldStartOffset(int tupleIndex, int fIdx) {
+ return fIdx == 0 ? 0 : buffer.getInt(getTupleStartOffset(tupleIndex) + (fIdx - 1) * 4);
+ }
+
+ @Override
+ public int getFieldEndOffset(int tupleIndex, int fIdx) {
+ return buffer.getInt(getTupleStartOffset(tupleIndex) + fIdx * 4);
+ }
+
+ @Override
+ public int getFieldLength(int tupleIndex, int fIdx) {
+ return getFieldEndOffset(tupleIndex, fIdx) - getFieldStartOffset(tupleIndex, fIdx);
+ }
+
+ @Override
+ public int getFieldSlotsLength() {
+ return getFieldCount() * 4;
+ }
+
+ @Override
+ public int getFieldCount() {
+ return fieldCount;
+ }
+}
\ No newline at end of file
diff --git a/pregelix/pregelix-dataflow/pom.xml b/pregelix/pregelix-dataflow/pom.xml
index aaa7186..d3f396e 100644
--- a/pregelix/pregelix-dataflow/pom.xml
+++ b/pregelix/pregelix-dataflow/pom.xml
@@ -1,14 +1,15 @@
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<artifactId>pregelix-dataflow</artifactId>
<packaging>jar</packaging>
<name>pregelix-dataflow</name>
<parent>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>pregelix</artifactId>
- <version>0.2.3-SNAPSHOT</version>
- </parent>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>pregelix</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ </parent>
<properties>
@@ -22,8 +23,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
<plugin>
@@ -42,6 +44,7 @@
</plugin>
<plugin>
<artifactId>maven-clean-plugin</artifactId>
+ <version>2.5</version>
<configuration>
<filesets>
<fileset>
@@ -102,13 +105,6 @@
<version>0.2.3-SNAPSHOT</version>
</dependency>
<dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-core</artifactId>
- <version>0.20.2</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
<groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks-storage-am-common</artifactId>
<version>0.2.3-SNAPSHOT</version>
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/ConnectorPolicyAssignmentPolicy.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/ConnectorPolicyAssignmentPolicy.java
index d29afca..ae47ed8 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/ConnectorPolicyAssignmentPolicy.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/ConnectorPolicyAssignmentPolicy.java
@@ -15,25 +15,44 @@
package edu.uci.ics.pregelix.dataflow;
+import org.apache.commons.lang3.tuple.Pair;
+
import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicyAssignmentPolicy;
import edu.uci.ics.hyracks.api.dataflow.connectors.PipeliningConnectorPolicy;
+import edu.uci.ics.hyracks.api.dataflow.connectors.SendSideMaterializedBlockingConnectorPolicy;
import edu.uci.ics.hyracks.api.dataflow.connectors.SendSideMaterializedPipeliningConnectorPolicy;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
public class ConnectorPolicyAssignmentPolicy implements IConnectorPolicyAssignmentPolicy {
private static final long serialVersionUID = 1L;
- private IConnectorPolicy senderSideMaterializePolicy = new SendSideMaterializedPipeliningConnectorPolicy();
+ private IConnectorPolicy senderSideMatPipPolicy = new SendSideMaterializedPipeliningConnectorPolicy();
+ private IConnectorPolicy senderSideMatBlkPolicy = new SendSideMaterializedBlockingConnectorPolicy();
private IConnectorPolicy pipeliningPolicy = new PipeliningConnectorPolicy();
+ private JobSpecification spec;
+
+ public ConnectorPolicyAssignmentPolicy(JobSpecification spec) {
+ this.spec = spec;
+ }
@Override
public IConnectorPolicy getConnectorPolicyAssignment(IConnectorDescriptor c, int nProducers, int nConsumers,
int[] fanouts) {
if (c instanceof MToNPartitioningMergingConnectorDescriptor) {
- return senderSideMaterializePolicy;
+ return senderSideMatPipPolicy;
} else {
- return pipeliningPolicy;
+ Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> endPoints = spec
+ .getConnectorOperatorMap().get(c.getConnectorId());
+ IOperatorDescriptor consumer = endPoints.getRight().getLeft();
+ if (consumer instanceof TreeIndexInsertUpdateDeleteOperatorDescriptor) {
+ return senderSideMatBlkPolicy;
+ } else {
+ return pipeliningPolicy;
+ }
}
}
}
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/HDFSFileWriteOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/HDFSFileWriteOperatorDescriptor.java
index c25e4c6..2402748 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/HDFSFileWriteOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/HDFSFileWriteOperatorDescriptor.java
@@ -14,6 +14,8 @@
*/
package edu.uci.ics.pregelix.dataflow;
+import java.io.File;
+import java.io.FileNotFoundException;
import java.io.IOException;
import java.nio.ByteBuffer;
@@ -23,9 +25,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.JobID;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -37,6 +37,7 @@
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameDeserializer;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import edu.uci.ics.hyracks.hdfs.ContextFactory;
import edu.uci.ics.pregelix.api.graph.Vertex;
import edu.uci.ics.pregelix.api.io.VertexOutputFormat;
import edu.uci.ics.pregelix.api.io.VertexWriter;
@@ -64,21 +65,24 @@
return new AbstractUnaryInputSinkOperatorNodePushable() {
private RecordDescriptor rd0;
private FrameDeserializer frameDeserializer;
- private Configuration conf = confFactory.createConfiguration();
+ private Configuration conf;
private VertexWriter vertexWriter;
private TaskAttemptContext context;
private String TEMP_DIR = "_temporary";
+ private ClassLoader ctxCL;
+ private ContextFactory ctxFactory = new ContextFactory();
@Override
public void open() throws HyracksDataException {
rd0 = inputRdFactory == null ? recordDescProvider.getInputRecordDescriptor(getActivityId(), 0)
: inputRdFactory.createRecordDescriptor();
frameDeserializer = new FrameDeserializer(ctx.getFrameSize(), rd0);
+ ctxCL = Thread.currentThread().getContextClassLoader();
Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
+ conf = confFactory.createConfiguration();
VertexOutputFormat outputFormat = BspUtils.createVertexOutputFormat(conf);
- TaskAttemptID tid = new TaskAttemptID("", 0, true, partition, 0);
- context = new TaskAttemptContext(conf, tid);
+ context = ctxFactory.createContext(conf, partition);
try {
vertexWriter = outputFormat.createVertexWriter(context);
} catch (InterruptedException e) {
@@ -107,7 +111,7 @@
@Override
public void fail() throws HyracksDataException {
-
+ Thread.currentThread().setContextClassLoader(ctxCL);
}
@Override
@@ -124,36 +128,63 @@
private void moveFilesToFinalPath() throws HyracksDataException {
try {
- JobContext job = new JobContext(conf, new JobID("0", 0));
+ JobContext job = ctxFactory.createJobContext(conf);
Path outputPath = FileOutputFormat.getOutputPath(job);
FileSystem dfs = FileSystem.get(conf);
Path filePath = new Path(outputPath, "part-" + new Integer(partition).toString());
- FileStatus[] tempPaths = dfs.listStatus(outputPath, new PathFilter() {
- @Override
- public boolean accept(Path dir) {
- return dir.getName().endsWith(TEMP_DIR);
- }
- });
- Path tempDir = tempPaths[0].getPath();
- FileStatus[] results = dfs.listStatus(tempDir, new PathFilter() {
- @Override
- public boolean accept(Path dir) {
- return dir.getName().indexOf(context.getTaskAttemptID().toString()) >= 0;
- }
- });
- Path srcDir = results[0].getPath();
- if (!dfs.exists(srcDir))
- throw new HyracksDataException("file " + srcDir.toString() + " does not exist!");
-
- FileStatus[] srcFiles = dfs.listStatus(srcDir);
- Path srcFile = srcFiles[0].getPath();
- dfs.delete(filePath, true);
- dfs.rename(srcFile, filePath);
+ FileStatus[] results = findPartitionPaths(outputPath, dfs);
+ if (results.length >= 1) {
+ /**
+ * for Hadoop-0.20.2
+ */
+ renameFile(dfs, filePath, results);
+ } else {
+ /**
+ * for Hadoop-0.23.1
+ */
+ int jobId = job.getJobID().getId();
+ outputPath = new Path(outputPath.toString() + File.separator + TEMP_DIR + File.separator
+ + jobId);
+ results = findPartitionPaths(outputPath, dfs);
+ renameFile(dfs, filePath, results);
+ }
} catch (IOException e) {
throw new HyracksDataException(e);
+ } finally {
+ Thread.currentThread().setContextClassLoader(ctxCL);
}
}
+ private FileStatus[] findPartitionPaths(Path outputPath, FileSystem dfs) throws FileNotFoundException,
+ IOException {
+ FileStatus[] tempPaths = dfs.listStatus(outputPath, new PathFilter() {
+ @Override
+ public boolean accept(Path dir) {
+ return dir.getName().endsWith(TEMP_DIR);
+ }
+ });
+ Path tempDir = tempPaths[0].getPath();
+ FileStatus[] results = dfs.listStatus(tempDir, new PathFilter() {
+ @Override
+ public boolean accept(Path dir) {
+ return dir.getName().indexOf(context.getTaskAttemptID().toString()) >= 0;
+ }
+ });
+ return results;
+ }
+
+ private void renameFile(FileSystem dfs, Path filePath, FileStatus[] results) throws IOException,
+ HyracksDataException, FileNotFoundException {
+ Path srcDir = results[0].getPath();
+ if (!dfs.exists(srcDir))
+ throw new HyracksDataException("file " + srcDir.toString() + " does not exist!");
+
+ FileStatus[] srcFiles = dfs.listStatus(srcDir);
+ Path srcFile = srcFiles[0].getPath();
+ dfs.delete(filePath, true);
+ dfs.rename(srcFile, filePath);
+ }
+
};
}
}
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileScanOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileScanOperatorDescriptor.java
index f1b98f5..0da7baf 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileScanOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileScanOperatorDescriptor.java
@@ -17,17 +17,15 @@
import java.io.DataOutput;
import java.io.IOException;
import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
import java.util.List;
-import java.util.logging.Logger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableComparable;
import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Mapper.Context;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.mapreduce.lib.input.FileSplit;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -42,6 +40,8 @@
import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import edu.uci.ics.hyracks.hdfs.ContextFactory;
+import edu.uci.ics.hyracks.hdfs2.dataflow.FileSplitsFactory;
import edu.uci.ics.pregelix.api.graph.Vertex;
import edu.uci.ics.pregelix.api.io.VertexInputFormat;
import edu.uci.ics.pregelix.api.io.VertexReader;
@@ -50,38 +50,67 @@
@SuppressWarnings("rawtypes")
public class VertexFileScanOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
- private static final Logger LOGGER = Logger.getLogger(VertexFileScanOperatorDescriptor.class.getName());
private static final long serialVersionUID = 1L;
- private final List<InputSplit> splits;
+ private final FileSplitsFactory splitsFactory;
private final IConfigurationFactory confFactory;
private final int fieldSize = 2;
+ private final String[] scheduledLocations;
+ private final boolean[] executed;
/**
* @param spec
*/
public VertexFileScanOperatorDescriptor(JobSpecification spec, RecordDescriptor rd, List<InputSplit> splits,
- IConfigurationFactory confFactory) throws HyracksException {
+ String[] scheduledLocations, IConfigurationFactory confFactory) throws HyracksException {
super(spec, 0, 1);
- this.splits = splits;
+ List<FileSplit> fileSplits = new ArrayList<FileSplit>();
+ for (int i = 0; i < splits.size(); i++) {
+ fileSplits.add((FileSplit) splits.get(i));
+ }
+ this.splitsFactory = new FileSplitsFactory(fileSplits);
this.confFactory = confFactory;
+ this.scheduledLocations = scheduledLocations;
+ this.executed = new boolean[scheduledLocations.length];
+ Arrays.fill(executed, false);
this.recordDescriptors[0] = rd;
}
public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
throws HyracksDataException {
+ final List<FileSplit> splits = splitsFactory.getSplits();
+
return new AbstractUnaryOutputSourceOperatorNodePushable() {
- private Configuration conf = confFactory.createConfiguration();
+ private ClassLoader ctxCL;
+ private ContextFactory ctxFactory = new ContextFactory();
@Override
public void initialize() throws HyracksDataException {
+ ctxCL = Thread.currentThread().getContextClassLoader();
try {
Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
+ Configuration conf = confFactory.createConfiguration();
writer.open();
- loadVertices(ctx, partition);
+ for (int i = 0; i < scheduledLocations.length; i++) {
+ if (scheduledLocations[i].equals(ctx.getJobletContext().getApplicationContext().getNodeId())) {
+ /**
+ * pick one from the FileSplit queue
+ */
+ synchronized (executed) {
+ if (!executed[i]) {
+ executed[i] = true;
+ } else {
+ continue;
+ }
+ }
+ loadVertices(ctx, conf, i);
+ }
+ }
writer.close();
} catch (Exception e) {
throw new HyracksDataException(e);
+ } finally {
+ Thread.currentThread().setContextClassLoader(ctxCL);
}
}
@@ -96,24 +125,19 @@
* @throws InterruptedException
*/
@SuppressWarnings("unchecked")
- private void loadVertices(final IHyracksTaskContext ctx, int partitionId) throws IOException,
- ClassNotFoundException, InterruptedException, InstantiationException, IllegalAccessException {
+ private void loadVertices(final IHyracksTaskContext ctx, Configuration conf, int splitId)
+ throws IOException, ClassNotFoundException, InterruptedException, InstantiationException,
+ IllegalAccessException {
ByteBuffer frame = ctx.allocateFrame();
FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
appender.reset(frame, true);
VertexInputFormat vertexInputFormat = BspUtils.createVertexInputFormat(conf);
- TaskAttemptContext context = new TaskAttemptContext(conf, new TaskAttemptID());
- InputSplit split = splits.get(partition);
+ InputSplit split = splits.get(splitId);
+ TaskAttemptContext mapperContext = ctxFactory.createContext(conf, splitId);
- if (split instanceof FileSplit) {
- FileSplit fileSplit = (FileSplit) split;
- LOGGER.info("read file split: " + fileSplit.getPath() + " location:" + fileSplit.getLocations()[0]
- + " start:" + fileSplit.getStart() + " length:" + split.getLength() + " partition:"
- + partition);
- }
- VertexReader vertexReader = vertexInputFormat.createVertexReader(split, context);
- vertexReader.initialize(split, context);
+ VertexReader vertexReader = vertexInputFormat.createVertexReader(split, mapperContext);
+ vertexReader.initialize(split, mapperContext);
Vertex readerVertex = (Vertex) BspUtils.createVertex(conf);
ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldSize);
DataOutput dos = tb.getDataOutput();
@@ -121,8 +145,6 @@
/**
* set context
*/
- Context mapperContext = new Mapper().new Context(conf, new TaskAttemptID(), null, null, null, null,
- splits.get(partition));
Vertex.setContext(mapperContext);
/**
@@ -166,5 +188,4 @@
}
};
}
-
}
\ No newline at end of file
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/FileWriteOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexWriteOperatorDescriptor.java
similarity index 89%
rename from pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/FileWriteOperatorDescriptor.java
rename to pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexWriteOperatorDescriptor.java
index 356f06c..d7cbb3a 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/FileWriteOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexWriteOperatorDescriptor.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.pregelix.dataflow.std;
+package edu.uci.ics.pregelix.dataflow;
import java.io.FileOutputStream;
import java.io.IOException;
@@ -35,14 +35,14 @@
import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
-public class FileWriteOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+public class VertexWriteOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
private static final long serialVersionUID = 1L;
private final FileSplit[] splits;
private final IRuntimeHookFactory preHookFactory;
private final IRuntimeHookFactory postHookFactory;
private final IRecordDescriptorFactory inputRdFactory;
- public FileWriteOperatorDescriptor(JobSpecification spec, IRecordDescriptorFactory inputRdFactory,
+ public VertexWriteOperatorDescriptor(JobSpecification spec, IRecordDescriptorFactory inputRdFactory,
IFileSplitProvider fileSplitProvider, IRuntimeHookFactory preHookFactory,
IRuntimeHookFactory postHookFactory) {
super(spec, 1, 0);
@@ -60,7 +60,6 @@
private RecordDescriptor rd0;
private FrameDeserializer frameDeserializer;
private PrintWriter outputWriter;
- private final static String separator = "|";
@Override
public void open() throws HyracksDataException {
@@ -82,10 +81,7 @@
frameDeserializer.reset(frame);
while (!frameDeserializer.done()) {
Object[] tuple = frameDeserializer.deserializeRecord();
- for (int i = 0; i < tuple.length - 1; i++) {
- outputWriter.print(StringSerializationUtils.toString(tuple[i]));
- outputWriter.print(separator);
- }
+ // output the vertex
outputWriter.print(StringSerializationUtils.toString(tuple[tuple.length - 1]));
outputWriter.println();
}
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java
index 43b6d17..567e220 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java
@@ -119,23 +119,23 @@
Vertex.setNumEdges(numEdges);
giraphJobIdToSuperStep.put(giraphJobId, superStep);
giraphJobIdToMove.put(giraphJobId, false);
- LOGGER.info("start iteration " + Vertex.getCurrentSuperstep());
+ LOGGER.info("start iteration " + Vertex.getSuperstep());
}
System.gc();
}
public synchronized void endSuperStep(String giraphJobId) {
giraphJobIdToMove.put(giraphJobId, true);
- LOGGER.info("end iteration " + Vertex.getCurrentSuperstep());
+ LOGGER.info("end iteration " + Vertex.getSuperstep());
}
@Override
public FileReference createManagedWorkspaceFile(String prefix) throws HyracksDataException {
final FileReference fRef = ioManager.createWorkspaceFile(prefix);
- List<FileReference> files = iterationToFiles.get(Vertex.getCurrentSuperstep());
+ List<FileReference> files = iterationToFiles.get(Vertex.getSuperstep());
if (files == null) {
files = new ArrayList<FileReference>();
- iterationToFiles.put(Vertex.getCurrentSuperstep(), files);
+ iterationToFiles.put(Vertex.getSuperstep(), files);
}
files.add(fRef);
return fRef;
diff --git a/pregelix/pregelix-dist/pom.xml b/pregelix/pregelix-dist/pom.xml
index 4cb0387..847e843 100644
--- a/pregelix/pregelix-dist/pom.xml
+++ b/pregelix/pregelix-dist/pom.xml
@@ -19,8 +19,8 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
</configuration>
</plugin>
<plugin>
@@ -70,7 +70,7 @@
<groupId>edu.uci.ics.hyracks</groupId>
<artifactId>pregelix-core</artifactId>
<version>0.2.3-SNAPSHOT</version>
- <scope>comile</scope>
+ <scope>compile</scope>
</dependency>
<dependency>
<groupId>edu.uci.ics.hyracks</groupId>
diff --git a/pregelix/pregelix-example/data/clique/clique.txt b/pregelix/pregelix-example/data/clique/clique.txt
new file mode 100755
index 0000000..08280e3
--- /dev/null
+++ b/pregelix/pregelix-example/data/clique/clique.txt
@@ -0,0 +1,7 @@
+1 2 3 4
+2 1 3 4 5
+3 1 2 4 5
+4 1 2 3
+5 6 7
+6 5 7
+7 5 6
diff --git a/pregelix/pregelix-example/pom.xml b/pregelix/pregelix-example/pom.xml
index 6b9bc89..e643331 100644
--- a/pregelix/pregelix-example/pom.xml
+++ b/pregelix/pregelix-example/pom.xml
@@ -17,8 +17,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
<plugin>
@@ -41,6 +42,7 @@
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>appassembler-maven-plugin</artifactId>
+ <version>1.3</version>
<executions>
<execution>
<configuration>
@@ -75,7 +77,9 @@
</configuration>
</plugin>
<plugin>
+ <groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-clean-plugin</artifactId>
+ <version>2.5</version>
<configuration>
<filesets>
<fileset>
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ConnectedComponentsVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ConnectedComponentsVertex.java
index 30e88ea..74ae455 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ConnectedComponentsVertex.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ConnectedComponentsVertex.java
@@ -126,6 +126,11 @@
}
}
+ @Override
+ public String toString() {
+ return getVertexId() + " " + getVertexValue();
+ }
+
public static void main(String[] args) throws Exception {
PregelixJob job = new PregelixJob(ConnectedComponentsVertex.class.getSimpleName());
job.setVertexClass(ConnectedComponentsVertex.class);
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphMutationVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphMutationVertex.java
new file mode 100644
index 0000000..e54373f
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphMutationVertex.java
@@ -0,0 +1,111 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ * 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 edu.uci.ics.pregelix.example;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.io.VertexWriter;
+import edu.uci.ics.pregelix.api.io.text.TextVertexOutputFormat;
+import edu.uci.ics.pregelix.api.io.text.TextVertexOutputFormat.TextVertexWriter;
+import edu.uci.ics.pregelix.api.job.PregelixJob;
+import edu.uci.ics.pregelix.example.client.Client;
+import edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat;
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+/**
+ * Demonstrates the basic graph vertex insert/delete implementation.
+ */
+public class GraphMutationVertex extends Vertex<VLongWritable, DoubleWritable, FloatWritable, DoubleWritable> {
+
+ private VLongWritable vid = new VLongWritable();
+ private GraphMutationVertex newVertex = null;
+
+ @Override
+ public void compute(Iterator<DoubleWritable> msgIterator) {
+ if (Vertex.getSuperstep() == 1) {
+ if (newVertex == null) {
+ newVertex = new GraphMutationVertex();
+ }
+ if (getVertexId().get() % 2 == 0 || getVertexId().get() % 3 == 0) {
+ deleteVertex(getVertexId());
+ } else {
+ vid.set(100 * getVertexId().get());
+ newVertex.setVertexId(vid);
+ newVertex.setVertexValue(getVertexValue());
+ addVertex(vid, newVertex);
+ }
+ voteToHalt();
+ } else {
+ if (getVertexId().get() % 190 == 0) {
+ deleteVertex(getVertexId());
+ }
+ voteToHalt();
+ }
+ }
+
+ /**
+ * Simple VertexWriter that supports {@link SimplePageRankVertex}
+ */
+ public static class SimpleGraphMutationVertexWriter extends
+ TextVertexWriter<VLongWritable, DoubleWritable, FloatWritable> {
+ public SimpleGraphMutationVertexWriter(RecordWriter<Text, Text> lineRecordWriter) {
+ super(lineRecordWriter);
+ }
+
+ @Override
+ public void writeVertex(Vertex<VLongWritable, DoubleWritable, FloatWritable, ?> vertex) throws IOException,
+ InterruptedException {
+ getRecordWriter().write(new Text(vertex.getVertexId().toString()),
+ new Text(vertex.getVertexValue().toString()));
+ }
+ }
+
+ @Override
+ public String toString() {
+ return getVertexId() + " " + getVertexValue();
+ }
+
+ /**
+ * Simple VertexOutputFormat that supports {@link SimplePageRankVertex}
+ */
+ public static class SimpleGraphMutationVertexOutputFormat extends
+ TextVertexOutputFormat<VLongWritable, DoubleWritable, FloatWritable> {
+
+ @Override
+ public VertexWriter<VLongWritable, DoubleWritable, FloatWritable> createVertexWriter(TaskAttemptContext context)
+ throws IOException, InterruptedException {
+ RecordWriter<Text, Text> recordWriter = textOutputFormat.getRecordWriter(context);
+ return new SimpleGraphMutationVertexWriter(recordWriter);
+ }
+ }
+
+ public static void main(String[] args) throws Exception {
+ PregelixJob job = new PregelixJob(GraphMutationVertex.class.getSimpleName());
+ job.setVertexClass(GraphMutationVertex.class);
+ job.setVertexInputFormatClass(TextPageRankInputFormat.class);
+ job.setVertexOutputFormatClass(SimpleGraphMutationVertexOutputFormat.class);
+ Client.run(args, job);
+ }
+
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/PageRankVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/PageRankVertex.java
index 290f90e..b6d4da7 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/PageRankVertex.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/PageRankVertex.java
@@ -54,6 +54,7 @@
public static final String ITERATIONS = "HyracksPageRankVertex.iteration";
private DoubleWritable outputValue = new DoubleWritable();
private DoubleWritable tmpVertexValue = new DoubleWritable();
+ private int maxIteration = -1;
/**
* Test whether combiner is called by summing up the messages.
@@ -97,7 +98,9 @@
@Override
public void compute(Iterator<DoubleWritable> msgIterator) {
- int maxIteration = this.getContext().getConfiguration().getInt(ITERATIONS, 10);
+ if (maxIteration < 0) {
+ maxIteration = getContext().getConfiguration().getInt(ITERATIONS, 10);
+ }
if (getSuperstep() == 1) {
tmpVertexValue.set(1.0 / getNumVertices());
setVertexValue(tmpVertexValue);
@@ -123,13 +126,13 @@
/**
* Simple VertexReader that supports {@link SimplePageRankVertex}
*/
- public static class SimplePageRankVertexReader extends
+ public static class SimulatedPageRankVertexReader extends
GeneratedVertexReader<VLongWritable, DoubleWritable, FloatWritable, DoubleWritable> {
/** Class logger */
- private static final Logger LOG = Logger.getLogger(SimplePageRankVertexReader.class.getName());
+ private static final Logger LOG = Logger.getLogger(SimulatedPageRankVertexReader.class.getName());
private Map<VLongWritable, FloatWritable> edges = Maps.newHashMap();
- public SimplePageRankVertexReader() {
+ public SimulatedPageRankVertexReader() {
super();
}
@@ -162,12 +165,12 @@
/**
* Simple VertexInputFormat that supports {@link SimplePageRankVertex}
*/
- public static class SimplePageRankVertexInputFormat extends
+ public static class SimulatedPageRankVertexInputFormat extends
GeneratedVertexInputFormat<VLongWritable, DoubleWritable, FloatWritable, DoubleWritable> {
@Override
public VertexReader<VLongWritable, DoubleWritable, FloatWritable, DoubleWritable> createVertexReader(
InputSplit split, TaskAttemptContext context) throws IOException {
- return new SimplePageRankVertexReader();
+ return new SimulatedPageRankVertexReader();
}
}
@@ -188,6 +191,11 @@
}
}
+ @Override
+ public String toString() {
+ return getVertexId() + " " + getVertexValue();
+ }
+
/**
* Simple VertexOutputFormat that supports {@link SimplePageRankVertex}
*/
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ReachabilityVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ReachabilityVertex.java
index 2f0ca45..0895386 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ReachabilityVertex.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ReachabilityVertex.java
@@ -85,6 +85,7 @@
}
private ByteWritable tmpVertexValue = new ByteWritable();
+ private long sourceId = -1;
/** The source vertex id */
public static final String SOURCE_ID = "ReachibilityVertex.sourceId";
@@ -101,7 +102,7 @@
* @return True if the source id
*/
private boolean isSource(VLongWritable v) {
- return (v.get() == getContext().getConfiguration().getLong(SOURCE_ID, SOURCE_ID_DEFAULT));
+ return (v.get() == sourceId);
}
/**
@@ -115,6 +116,9 @@
@Override
public void compute(Iterator<ByteWritable> msgIterator) {
+ if (sourceId < 0) {
+ sourceId = getContext().getConfiguration().getLong(SOURCE_ID, SOURCE_ID_DEFAULT);
+ }
if (getSuperstep() == 1) {
boolean isSource = isSource(getVertexId());
if (isSource) {
@@ -161,6 +165,11 @@
voteToHalt();
}
+ @Override
+ public String toString() {
+ return getVertexId() + " " + getVertexValue();
+ }
+
private void signalTerminate() {
Configuration conf = getContext().getConfiguration();
try {
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ShortestPathsVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ShortestPathsVertex.java
index a018f08..199870e 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ShortestPathsVertex.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ShortestPathsVertex.java
@@ -126,6 +126,11 @@
}
voteToHalt();
}
+
+ @Override
+ public String toString() {
+ return getVertexId() + " " + getVertexValue();
+ }
public static void main(String[] args) throws Exception {
PregelixJob job = new PregelixJob(ShortestPathsVertex.class.getSimpleName());
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/AdjacencyListWritable.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/AdjacencyListWritable.java
new file mode 100644
index 0000000..83e0a6b
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/AdjacencyListWritable.java
@@ -0,0 +1,99 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ * 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 edu.uci.ics.pregelix.example.maximalclique;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.hadoop.io.Writable;
+
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+/**
+ * The adjacency list contains <src, list-of-neighbors>
+ */
+public class AdjacencyListWritable implements Writable {
+
+ private VLongWritable sourceVertex = new VLongWritable();
+ private Set<VLongWritable> destinationVertexes = new TreeSet<VLongWritable>();
+
+ public AdjacencyListWritable() {
+ }
+
+ public void reset() {
+ this.destinationVertexes.clear();
+ }
+
+ public void setSource(VLongWritable source) {
+ this.sourceVertex = source;
+ }
+
+ public void addNeighbor(VLongWritable neighbor) {
+ destinationVertexes.add(neighbor);
+ }
+
+ @Override
+ public void readFields(DataInput input) throws IOException {
+ sourceVertex = new VLongWritable();
+ destinationVertexes.clear();
+ sourceVertex.readFields(input);
+ int numberOfNeighbors = input.readInt();
+ for (int i = 0; i < numberOfNeighbors; i++) {
+ VLongWritable neighbor = new VLongWritable();
+ neighbor.readFields(input);
+ destinationVertexes.add(neighbor);
+ }
+ }
+
+ @Override
+ public void write(DataOutput output) throws IOException {
+ sourceVertex.write(output);
+ output.writeInt(destinationVertexes.size());
+ for (VLongWritable dest : destinationVertexes) {
+ dest.write(output);
+ }
+ }
+
+ public int numberOfNeighbors() {
+ return destinationVertexes.size();
+ }
+
+ public void removeNeighbor(VLongWritable v) {
+ destinationVertexes.remove(v);
+ }
+
+ public VLongWritable getSource() {
+ return sourceVertex;
+ }
+
+ public Iterator<VLongWritable> getNeighbors() {
+ return destinationVertexes.iterator();
+ }
+
+ public void cleanNonMatch(Collection<VLongWritable> matches) {
+ destinationVertexes.retainAll(matches);
+ }
+
+ public boolean isNeighbor(VLongWritable v) {
+ return destinationVertexes.contains(v);
+ }
+
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/CliquesWritable.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/CliquesWritable.java
new file mode 100644
index 0000000..0e22ea1
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/CliquesWritable.java
@@ -0,0 +1,138 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ * 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 edu.uci.ics.pregelix.example.maximalclique;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.io.Writable;
+
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+/**
+ * The representation of cliques stored in a vertex.
+ */
+public class CliquesWritable implements Writable {
+
+ private List<VLongWritable> cliques = new ArrayList<VLongWritable>();
+ private int sizeOfClique = 0;
+
+ public CliquesWritable(List<VLongWritable> cliques, int sizeOfClique) {
+ this.cliques = cliques;
+ this.sizeOfClique = sizeOfClique;
+ }
+
+ public CliquesWritable() {
+
+ }
+
+ /**
+ * Set the size of cliques.
+ *
+ * @param sizeOfClique
+ * the size of each maximal clique
+ */
+ public void setCliqueSize(int sizeOfClique) {
+ this.sizeOfClique = sizeOfClique;
+ }
+
+ /**
+ * Add the clique vertexes
+ *
+ * @param cliques
+ * the list of vertexes -- can contain multiple cliques
+ */
+ public void addCliques(CliquesWritable cliques) {
+ this.cliques.addAll(cliques.cliques);
+ }
+
+ /**
+ * Add the clique vertexes
+ *
+ * @param cliques
+ * the list of vertexes -- can contain multiple cliques
+ */
+ public void addCliques(List<VLongWritable> vertexes) {
+ this.cliques.addAll(vertexes);
+ }
+
+ /**
+ * @return the size of the clique
+ */
+ public int getSizeOfClique() {
+ return sizeOfClique;
+ }
+
+ /**
+ * rese the clique
+ */
+ public void reset() {
+ this.cliques.clear();
+ this.sizeOfClique = 0;
+ }
+
+ @Override
+ public void readFields(DataInput input) throws IOException {
+ cliques.clear();
+ int numCliques = input.readInt();
+ if (numCliques < 0) {
+ sizeOfClique = 0;
+ return;
+ }
+ sizeOfClique = input.readInt();
+ for (int i = 0; i < numCliques; i++) {
+ for (int j = 0; j < sizeOfClique; j++) {
+ VLongWritable vid = new VLongWritable();
+ vid.readFields(input);
+ cliques.add(vid);
+ }
+ }
+ }
+
+ @Override
+ public void write(DataOutput output) throws IOException {
+ if (sizeOfClique <= 0) {
+ output.writeInt(-1);
+ return;
+ }
+ output.writeInt(cliques.size() / sizeOfClique);
+ output.writeInt(sizeOfClique);
+
+ for (int i = 0; i < cliques.size(); i++) {
+ cliques.get(i).write(output);
+ }
+ }
+
+ @Override
+ public String toString() {
+ if (sizeOfClique == 0)
+ return "";
+ StringBuffer sb = new StringBuffer();
+ int numCliques = cliques.size() / sizeOfClique;
+ for (int i = 0; i < numCliques; i++) {
+ for (int j = 0; j < sizeOfClique - 1; j++) {
+ sb.append(cliques.get(j));
+ sb.append(",");
+ }
+ sb.append(cliques.get(sizeOfClique - 1));
+ sb.append(";");
+ }
+ return sb.toString();
+ }
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/MaximalCliqueAggregator.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/MaximalCliqueAggregator.java
new file mode 100644
index 0000000..061e9e0
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/MaximalCliqueAggregator.java
@@ -0,0 +1,65 @@
+package edu.uci.ics.pregelix.example.maximalclique;
+
+import org.apache.hadoop.io.NullWritable;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.pregelix.api.graph.GlobalAggregator;
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+/**
+ * The global aggregator aggregates the count of triangles
+ */
+public class MaximalCliqueAggregator
+ extends
+ GlobalAggregator<VLongWritable, CliquesWritable, NullWritable, AdjacencyListWritable, CliquesWritable, CliquesWritable> {
+
+ private CliquesWritable state = new CliquesWritable();
+
+ @Override
+ public void init() {
+ state.reset();
+ }
+
+ @Override
+ public void step(Vertex<VLongWritable, CliquesWritable, NullWritable, AdjacencyListWritable> v)
+ throws HyracksDataException {
+ CliquesWritable cliques = v.getVertexValue();
+ updateAggregateState(cliques);
+ }
+
+ /**
+ * Update the current aggregate state
+ *
+ * @param cliques the incoming cliques
+ */
+ private void updateAggregateState(CliquesWritable cliques) {
+ if (cliques.getSizeOfClique() > state.getSizeOfClique()) {
+ //reset the vertex state
+ state.reset();
+ state.setCliqueSize(cliques.getSizeOfClique());
+ state.addCliques(cliques);
+ } else if (cliques.getSizeOfClique() == state.getSizeOfClique()) {
+ //add the new cliques
+ state.addCliques(cliques);
+ } else {
+ return;
+ }
+ }
+
+ @Override
+ public void step(CliquesWritable partialResult) {
+ updateAggregateState(partialResult);
+ }
+
+ @Override
+ public CliquesWritable finishPartial() {
+ return state;
+ }
+
+ @Override
+ public CliquesWritable finishFinal() {
+ return state;
+ }
+
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/MaximalCliqueVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/MaximalCliqueVertex.java
new file mode 100644
index 0000000..266feb7
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/MaximalCliqueVertex.java
@@ -0,0 +1,347 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ * 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 edu.uci.ics.pregelix.example.maximalclique;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import edu.uci.ics.pregelix.api.graph.Edge;
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.io.VertexWriter;
+import edu.uci.ics.pregelix.api.io.text.TextVertexOutputFormat;
+import edu.uci.ics.pregelix.api.io.text.TextVertexOutputFormat.TextVertexWriter;
+import edu.uci.ics.pregelix.api.job.PregelixJob;
+import edu.uci.ics.pregelix.api.util.BspUtils;
+import edu.uci.ics.pregelix.dataflow.util.IterationUtils;
+import edu.uci.ics.pregelix.example.client.Client;
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+import edu.uci.ics.pregelix.example.trianglecounting.TriangleCountingVertex;
+
+/**
+ * The maximal clique example -- find maximal cliques in an undirected graph.
+ * The result cliques contains vertexes ordered by the vertex id ascendingly. The algorithm takes
+ * advantage of that property to do effective pruning.
+ */
+public class MaximalCliqueVertex extends Vertex<VLongWritable, CliquesWritable, NullWritable, AdjacencyListWritable> {
+
+ private Map<VLongWritable, AdjacencyListWritable> map = new TreeMap<VLongWritable, AdjacencyListWritable>();
+ private List<VLongWritable> vertexList = new ArrayList<VLongWritable>();
+ private Map<VLongWritable, Integer> invertedMap = new TreeMap<VLongWritable, Integer>();
+ private int largestCliqueSizeSoFar = 0;
+ private List<BitSet> currentMaximalCliques = new ArrayList<BitSet>();
+ private CliquesWritable tmpValue = new CliquesWritable();
+ private List<VLongWritable> cliques = new ArrayList<VLongWritable>();
+
+ /**
+ * Update the current maximal cliques
+ *
+ * @param values
+ * the received adjcency lists
+ */
+ private void updateCurrentMaximalCliques(Iterator<AdjacencyListWritable> values) {
+ map.clear();
+ vertexList.clear();
+ invertedMap.clear();
+ currentMaximalCliques.clear();
+ cliques.clear();
+ tmpValue.reset();
+
+ // build the initial sub graph
+ while (values.hasNext()) {
+ AdjacencyListWritable adj = values.next();
+ map.put(adj.getSource(), adj);
+ }
+ VLongWritable srcId = getVertexId();
+ map.put(srcId, new AdjacencyListWritable());
+
+ // build the vertex list (vertex id in ascending order) and the inverted list of vertexes
+ int i = 0;
+ for (VLongWritable v : map.keySet()) {
+ vertexList.add(v);
+ invertedMap.put(v, i++);
+ }
+
+ //clean up adjacency list --- remove vertexes who are not neighbors of key
+ for (AdjacencyListWritable adj : map.values()) {
+ adj.cleanNonMatch(vertexList);
+ }
+
+ // get the h-index of the subgraph --- which is the maximum depth to explore
+ int[] neighborCounts = new int[map.size()];
+ i = 0;
+ for (AdjacencyListWritable adj : map.values()) {
+ neighborCounts[i++] = adj.numberOfNeighbors();
+ }
+ Arrays.sort(neighborCounts);
+ int h = 0;
+ for (i = neighborCounts.length - 1; i >= 0; i--) {
+ if (h >= neighborCounts[i]) {
+ break;
+ }
+ h++;
+ }
+ if (h < largestCliqueSizeSoFar) {
+ return;
+ }
+
+ //start depth-first search
+ BitSet cliqueSoFar = new BitSet(h);
+ for (VLongWritable v : vertexList) {
+ cliqueSoFar.set(invertedMap.get(v));
+ searchClique(h, cliqueSoFar, 1, v);
+ cliqueSoFar.clear();
+ }
+
+ //output local maximal cliques
+ for (BitSet clique : currentMaximalCliques) {
+ int keyIndex = invertedMap.get(srcId);
+ clique.set(keyIndex);
+ generateClique(clique);
+ tmpValue.addCliques(cliques);
+ tmpValue.setCliqueSize(clique.cardinality());
+ }
+
+ //update the vertex state
+ setVertexValue(tmpValue);
+ }
+
+ /**
+ * Output a clique with vertex ids.
+ *
+ * @param clique
+ * the bitmap representation of a clique
+ */
+ private void generateClique(BitSet clique) {
+ for (int j = 0; j < clique.length();) {
+ j = clique.nextSetBit(j);
+ VLongWritable v = vertexList.get(j);
+ cliques.add(v);
+ j++;
+ }
+ }
+
+ /**
+ * find cliques using the depth-first search
+ *
+ * @param maxDepth
+ * the maximum search depth
+ * @param cliqueSoFar
+ * the the cliques found so far
+ * @param depthSoFar
+ * the current search depth
+ * @param currentSource
+ * the vertex to be added into the clique
+ */
+ private void searchClique(int maxDepth, BitSet cliqueSoFar, int depthSoFar, VLongWritable currentSource) {
+ if (depthSoFar > maxDepth) {
+ // update maximal clique info
+ updateMaximalClique(cliqueSoFar);
+ return;
+ }
+
+ AdjacencyListWritable adj = map.get(currentSource);
+ Iterator<VLongWritable> neighbors = adj.getNeighbors();
+ ++depthSoFar;
+ while (neighbors.hasNext()) {
+ VLongWritable neighbor = neighbors.next();
+ if (!isTested(neighbor, cliqueSoFar) && isClique(neighbor, cliqueSoFar)) {
+ //snapshot the clique
+ int cliqueLength = cliqueSoFar.length();
+ // expand the clique
+ cliqueSoFar.set(invertedMap.get(neighbor));
+ searchClique(maxDepth, cliqueSoFar, depthSoFar, neighbor);
+ // back to the snapshot clique
+ cliqueSoFar.set(cliqueLength, cliqueSoFar.length(), false);
+ }
+ }
+
+ // update maximal clique info
+ updateMaximalClique(cliqueSoFar);
+ }
+
+ /**
+ * Update the maximal clique to a larger one if it exists
+ *
+ * @param cliqueSoFar
+ * the clique so far, in the bitmap representation
+ */
+ private void updateMaximalClique(BitSet cliqueSoFar) {
+ int cliqueSize = cliqueSoFar.cardinality();
+ if (cliqueSize > largestCliqueSizeSoFar) {
+ currentMaximalCliques.clear();
+ currentMaximalCliques.add((BitSet) cliqueSoFar.clone());
+ largestCliqueSizeSoFar = cliqueSize;
+ } else if (cliqueSize == largestCliqueSizeSoFar) {
+ currentMaximalCliques.add((BitSet) cliqueSoFar.clone());
+ } else {
+ return;
+ }
+ }
+
+ /**
+ * Should we test the vertex newVertex?
+ *
+ * @param newVertex
+ * the vertex to be tested
+ * @param cliqueSoFar
+ * the current clique, in the bitmap representation
+ * @return true if new vertex has been tested
+ */
+ private boolean isTested(VLongWritable newVertex, BitSet cliqueSoFar) {
+ int index = invertedMap.get(newVertex);
+ int largestSetIndex = cliqueSoFar.length() - 1;
+ if (index > largestSetIndex) {
+ // we only return cliques with vertexes in the ascending order
+ // hence, the new vertex must be larger than the largesetSetIndex in the clique
+ return false;
+ } else {
+ // otherwise, we think the vertex is "tested"
+ return true;
+ }
+ }
+
+ /**
+ * Will adding the newVertex yield a bigger clique?
+ *
+ * @param newVertex
+ * the new vertex id
+ * @param cliqueSoFar
+ * the bitmap representation of the clique
+ * @return true if adding the new vertex yelds a bigger clique
+ */
+ private boolean isClique(VLongWritable newVertex, BitSet cliqueSoFar) {
+ AdjacencyListWritable adj = map.get(newVertex);
+ // check whether each existing vertex is in the neighbor set of newVertex
+ for (int i = 0; i < cliqueSoFar.length();) {
+ i = cliqueSoFar.nextSetBit(i);
+ VLongWritable v = vertexList.get(i);
+ if (!adj.isNeighbor(v)) {
+ return false;
+ }
+ i++;
+ }
+ return true;
+ }
+
+ /**
+ * For superstep 1, send outgoing mesages.
+ * For superstep 2, calculate maximal cliques.
+ * otherwise, vote to halt.
+ */
+ @Override
+ public void compute(Iterator<AdjacencyListWritable> msgIterator) {
+ if (getSuperstep() == 1) {
+ sortEdges();
+ sendOutgoingMsgs(getEdges());
+ } else if (getSuperstep() == 2) {
+ updateCurrentMaximalCliques(msgIterator);
+ } else {
+ voteToHalt();
+ }
+ }
+
+ @Override
+ public String toString() {
+ return getVertexId() + " " + getVertexValue();
+ }
+
+ private static CliquesWritable readMaximalCliqueResult(Configuration conf) {
+ try {
+ CliquesWritable result = (CliquesWritable) IterationUtils.readGlobalAggregateValue(conf,
+ BspUtils.getJobId(conf));
+ return result;
+ } catch (IOException e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
+ public static void main(String[] args) throws Exception {
+ PregelixJob job = new PregelixJob(TriangleCountingVertex.class.getSimpleName());
+ job.setVertexClass(MaximalCliqueVertex.class);
+ job.setGlobalAggregatorClass(MaximalCliqueAggregator.class);
+ job.setDynamicVertexValueSize(true);
+ job.setVertexInputFormatClass(TextMaximalCliqueInputFormat.class);
+ job.setVertexOutputFormatClass(MaximalCliqueVertexOutputFormat.class);
+ Client.run(args, job);
+ System.out.println("maximal cliques: \n" + readMaximalCliqueResult(job.getConfiguration()));
+ }
+
+ /**
+ * Send the adjacency lists
+ *
+ * @param edges
+ * the outgoing edges
+ */
+ private void sendOutgoingMsgs(List<Edge<VLongWritable, NullWritable>> edges) {
+ for (int i = 0; i < edges.size(); i++) {
+ if (edges.get(i).getDestVertexId().get() < getVertexId().get()) {
+ // only add emit for the vertexes whose id is smaller than the vertex id
+ // to avoid the duplicate removal step,
+ // because all the resulting cliques will have vertexes in the ascending order.
+ AdjacencyListWritable msg = new AdjacencyListWritable();
+ msg.setSource(getVertexId());
+ for (int j = i + 1; j < edges.size(); j++) {
+ msg.addNeighbor(edges.get(j).getDestVertexId());
+ }
+ sendMsg(edges.get(i).getDestVertexId(), msg);
+ }
+ }
+ }
+
+ /**
+ * Maximal Clique VertexWriter
+ */
+ public static class MaximalCliqueVertexWriter extends
+ TextVertexWriter<VLongWritable, CliquesWritable, NullWritable> {
+ public MaximalCliqueVertexWriter(RecordWriter<Text, Text> lineRecordWriter) {
+ super(lineRecordWriter);
+ }
+
+ @Override
+ public void writeVertex(Vertex<VLongWritable, CliquesWritable, NullWritable, ?> vertex) throws IOException,
+ InterruptedException {
+ getRecordWriter().write(new Text(vertex.getVertexId().toString()),
+ new Text(vertex.getVertexValue().toString()));
+ }
+ }
+
+ /**
+ * output format for maximal clique
+ */
+ public static class MaximalCliqueVertexOutputFormat extends
+ TextVertexOutputFormat<VLongWritable, CliquesWritable, NullWritable> {
+
+ @Override
+ public VertexWriter<VLongWritable, CliquesWritable, NullWritable> createVertexWriter(TaskAttemptContext context)
+ throws IOException, InterruptedException {
+ RecordWriter<Text, Text> recordWriter = textOutputFormat.getRecordWriter(context);
+ return new MaximalCliqueVertexWriter(recordWriter);
+ }
+
+ }
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/TextMaximalCliqueInputFormat.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/TextMaximalCliqueInputFormat.java
new file mode 100644
index 0000000..ec7b32c
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/TextMaximalCliqueInputFormat.java
@@ -0,0 +1,113 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ * 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 edu.uci.ics.pregelix.example.maximalclique;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.io.VertexReader;
+import edu.uci.ics.pregelix.api.io.text.TextVertexInputFormat;
+import edu.uci.ics.pregelix.api.io.text.TextVertexInputFormat.TextVertexReader;
+import edu.uci.ics.pregelix.api.util.BspUtils;
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+public class TextMaximalCliqueInputFormat extends
+ TextVertexInputFormat<VLongWritable, CliquesWritable, NullWritable, AdjacencyListWritable> {
+
+ @Override
+ public VertexReader<VLongWritable, CliquesWritable, NullWritable, AdjacencyListWritable> createVertexReader(
+ InputSplit split, TaskAttemptContext context) throws IOException {
+ return new TextMaximalCliqueGraphReader(textInputFormat.createRecordReader(split, context));
+ }
+}
+
+@SuppressWarnings("rawtypes")
+class TextMaximalCliqueGraphReader extends
+ TextVertexReader<VLongWritable, CliquesWritable, NullWritable, AdjacencyListWritable> {
+
+ private final static String separator = " ";
+ private Vertex vertex;
+ private VLongWritable vertexId = new VLongWritable();
+ private List<VLongWritable> pool = new ArrayList<VLongWritable>();
+ private int used = 0;
+
+ public TextMaximalCliqueGraphReader(RecordReader<LongWritable, Text> lineRecordReader) {
+ super(lineRecordReader);
+ }
+
+ @Override
+ public boolean nextVertex() throws IOException, InterruptedException {
+ return getRecordReader().nextKeyValue();
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public Vertex<VLongWritable, CliquesWritable, NullWritable, AdjacencyListWritable> getCurrentVertex()
+ throws IOException, InterruptedException {
+ used = 0;
+ if (vertex == null)
+ vertex = (Vertex) BspUtils.createVertex(getContext().getConfiguration());
+ vertex.getMsgList().clear();
+ vertex.getEdges().clear();
+
+ vertex.reset();
+ Text line = getRecordReader().getCurrentValue();
+ String[] fields = line.toString().split(separator);
+
+ if (fields.length > 0) {
+ /**
+ * set the src vertex id
+ */
+ long src = Long.parseLong(fields[0]);
+ vertexId.set(src);
+ vertex.setVertexId(vertexId);
+ long dest = -1L;
+
+ /**
+ * set up edges
+ */
+ for (int i = 1; i < fields.length; i++) {
+ dest = Long.parseLong(fields[i]);
+ VLongWritable destId = allocate();
+ destId.set(dest);
+ vertex.addEdge(destId, null);
+ }
+ }
+ return vertex;
+ }
+
+ private VLongWritable allocate() {
+ if (used >= pool.size()) {
+ VLongWritable value = new VLongWritable();
+ pool.add(value);
+ used++;
+ return value;
+ } else {
+ VLongWritable value = pool.get(used);
+ used++;
+ return value;
+ }
+ }
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TextTriangleCountingInputFormat.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TextTriangleCountingInputFormat.java
new file mode 100644
index 0000000..bb399ff
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TextTriangleCountingInputFormat.java
@@ -0,0 +1,111 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ * 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 edu.uci.ics.pregelix.example.trianglecounting;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.io.VertexReader;
+import edu.uci.ics.pregelix.api.io.text.TextVertexInputFormat;
+import edu.uci.ics.pregelix.api.io.text.TextVertexInputFormat.TextVertexReader;
+import edu.uci.ics.pregelix.api.util.BspUtils;
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+public class TextTriangleCountingInputFormat extends
+ TextVertexInputFormat<VLongWritable, VLongWritable, VLongWritable, VLongWritable> {
+
+ @Override
+ public VertexReader<VLongWritable, VLongWritable, VLongWritable, VLongWritable> createVertexReader(
+ InputSplit split, TaskAttemptContext context) throws IOException {
+ return new TextPageRankGraphReader(textInputFormat.createRecordReader(split, context));
+ }
+}
+
+@SuppressWarnings("rawtypes")
+class TextPageRankGraphReader extends TextVertexReader<VLongWritable, VLongWritable, VLongWritable, VLongWritable> {
+
+ private final static String separator = " ";
+ private Vertex vertex;
+ private VLongWritable vertexId = new VLongWritable();
+ private List<VLongWritable> pool = new ArrayList<VLongWritable>();
+ private int used = 0;
+
+ public TextPageRankGraphReader(RecordReader<LongWritable, Text> lineRecordReader) {
+ super(lineRecordReader);
+ }
+
+ @Override
+ public boolean nextVertex() throws IOException, InterruptedException {
+ return getRecordReader().nextKeyValue();
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public Vertex<VLongWritable, VLongWritable, VLongWritable, VLongWritable> getCurrentVertex() throws IOException,
+ InterruptedException {
+ used = 0;
+ if (vertex == null)
+ vertex = (Vertex) BspUtils.createVertex(getContext().getConfiguration());
+ vertex.getMsgList().clear();
+ vertex.getEdges().clear();
+
+ vertex.reset();
+ Text line = getRecordReader().getCurrentValue();
+ String[] fields = line.toString().split(separator);
+
+ if (fields.length > 0) {
+ /**
+ * set the src vertex id
+ */
+ long src = Long.parseLong(fields[0]);
+ vertexId.set(src);
+ vertex.setVertexId(vertexId);
+ long dest = -1L;
+
+ /**
+ * set up edges
+ */
+ for (int i = 1; i < fields.length; i++) {
+ dest = Long.parseLong(fields[i]);
+ VLongWritable destId = allocate();
+ destId.set(dest);
+ vertex.addEdge(destId, null);
+ }
+ }
+ // vertex.sortEdges();
+ return vertex;
+ }
+
+ private VLongWritable allocate() {
+ if (used >= pool.size()) {
+ VLongWritable value = new VLongWritable();
+ pool.add(value);
+ used++;
+ return value;
+ } else {
+ VLongWritable value = pool.get(used);
+ used++;
+ return value;
+ }
+ }
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TriangleCountingAggregator.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TriangleCountingAggregator.java
new file mode 100644
index 0000000..67b028d
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TriangleCountingAggregator.java
@@ -0,0 +1,41 @@
+package edu.uci.ics.pregelix.example.trianglecounting;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.pregelix.api.graph.GlobalAggregator;
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+/**
+ * The global aggregator aggregates the count of triangles
+ */
+public class TriangleCountingAggregator extends
+ GlobalAggregator<VLongWritable, VLongWritable, VLongWritable, VLongWritable, VLongWritable, VLongWritable> {
+
+ private VLongWritable state = new VLongWritable(0);
+
+ @Override
+ public void init() {
+ state.set(0);
+ }
+
+ @Override
+ public void step(Vertex<VLongWritable, VLongWritable, VLongWritable, VLongWritable> v) throws HyracksDataException {
+ state.set(state.get() + v.getVertexValue().get());
+ }
+
+ @Override
+ public void step(VLongWritable partialResult) {
+ state.set(state.get() + partialResult.get());
+ }
+
+ @Override
+ public VLongWritable finishPartial() {
+ return state;
+ }
+
+ @Override
+ public VLongWritable finishFinal() {
+ return state;
+ }
+
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TriangleCountingVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TriangleCountingVertex.java
new file mode 100644
index 0000000..d3db095
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TriangleCountingVertex.java
@@ -0,0 +1,153 @@
+package edu.uci.ics.pregelix.example.trianglecounting;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import edu.uci.ics.pregelix.api.graph.Edge;
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.io.VertexWriter;
+import edu.uci.ics.pregelix.api.io.text.TextVertexOutputFormat;
+import edu.uci.ics.pregelix.api.io.text.TextVertexOutputFormat.TextVertexWriter;
+import edu.uci.ics.pregelix.api.job.PregelixJob;
+import edu.uci.ics.pregelix.api.util.BspUtils;
+import edu.uci.ics.pregelix.dataflow.util.IterationUtils;
+import edu.uci.ics.pregelix.example.client.Client;
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+/**
+ * The triangle counting example -- counting the triangles in an undirected graph.
+ */
+public class TriangleCountingVertex extends Vertex<VLongWritable, VLongWritable, VLongWritable, VLongWritable> {
+
+ private VLongWritable tmpValue = new VLongWritable(0);
+ private long triangleCount = 0;
+ private Edge<VLongWritable, VLongWritable> candidateEdge = new Edge<VLongWritable, VLongWritable>(
+ new VLongWritable(0), new VLongWritable(0));
+ private EdgeComparator edgeComparator = new EdgeComparator();
+
+ @Override
+ public void compute(Iterator<VLongWritable> msgIterator) {
+ // transforms the edge list into a set to facilitate lookup
+ if (getSuperstep() == 1) {
+ // sorting edges could be avoid if the dataset already has that property
+ sortEdges();
+ List<Edge<VLongWritable, VLongWritable>> edges = this.getEdges();
+ int numEdges = edges.size();
+
+ //decoding longs
+ long src = getVertexId().get();
+ long[] dests = new long[numEdges];
+ for (int i = 0; i < numEdges; i++) {
+ dests[i] = edges.get(i).getDestVertexId().get();
+ }
+
+ //send messages -- take advantage of that each discovered
+ //triangle should have vertexes ordered by vertex id
+ for (int i = 0; i < numEdges; i++) {
+ if (dests[i] < src) {
+ for (int j = i + 1; j < numEdges; j++) {
+ //send messages -- v_j.id > v_i.id -- guaranteed by sortEdge()
+ if (dests[j] > src) {
+ sendMsg(edges.get(i).getDestVertexId(), edges.get(j).getDestVertexId());
+ }
+ }
+ }
+ }
+ }
+ if (getSuperstep() >= 2) {
+ triangleCount = 0;
+ List<Edge<VLongWritable, VLongWritable>> edges = this.getEdges();
+ while (msgIterator.hasNext()) {
+ VLongWritable msg = msgIterator.next();
+ candidateEdge.setDestVertexId(msg);
+ if (Collections.binarySearch(edges, candidateEdge, edgeComparator) >= 0) {
+ // if the msg value is a dest from this vertex
+ triangleCount++;
+ }
+ }
+
+ // set vertex value
+ tmpValue.set(triangleCount);
+ setVertexValue(tmpValue);
+ voteToHalt();
+ }
+ }
+
+ /**
+ * Triangle Counting VertexWriter
+ */
+ public static class TriangleCountingVertexWriter extends
+ TextVertexWriter<VLongWritable, VLongWritable, VLongWritable> {
+ public TriangleCountingVertexWriter(RecordWriter<Text, Text> lineRecordWriter) {
+ super(lineRecordWriter);
+ }
+
+ @Override
+ public void writeVertex(Vertex<VLongWritable, VLongWritable, VLongWritable, ?> vertex) throws IOException,
+ InterruptedException {
+ getRecordWriter().write(new Text(vertex.getVertexId().toString()),
+ new Text(vertex.getVertexValue().toString()));
+ }
+ }
+
+ @Override
+ public String toString() {
+ return getVertexId() + " " + getVertexValue();
+ }
+
+ /**
+ * output format for triangle counting
+ */
+ public static class TriangleCountingVertexOutputFormat extends
+ TextVertexOutputFormat<VLongWritable, VLongWritable, VLongWritable> {
+
+ @Override
+ public VertexWriter<VLongWritable, VLongWritable, VLongWritable> createVertexWriter(TaskAttemptContext context)
+ throws IOException, InterruptedException {
+ RecordWriter<Text, Text> recordWriter = textOutputFormat.getRecordWriter(context);
+ return new TriangleCountingVertexWriter(recordWriter);
+ }
+
+ }
+
+ private static long readTriangleCountingResult(Configuration conf) {
+ try {
+ VLongWritable count = (VLongWritable) IterationUtils
+ .readGlobalAggregateValue(conf, BspUtils.getJobId(conf));
+ return count.get();
+ } catch (IOException e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
+ public static void main(String[] args) throws Exception {
+ PregelixJob job = new PregelixJob(TriangleCountingVertex.class.getSimpleName());
+ job.setVertexClass(TriangleCountingVertex.class);
+ job.setGlobalAggregatorClass(TriangleCountingAggregator.class);
+ job.setVertexInputFormatClass(TextTriangleCountingInputFormat.class);
+ job.setVertexOutputFormatClass(TriangleCountingVertexOutputFormat.class);
+ Client.run(args, job);
+ System.out.println("triangle count: " + readTriangleCountingResult(job.getConfiguration()));
+ }
+}
+
+/**
+ * The comparator for Edge<VLongWritable, VLongWritable>.
+ */
+class EdgeComparator implements Comparator<Edge<VLongWritable, VLongWritable>> {
+
+ @Override
+ public int compare(Edge<VLongWritable, VLongWritable> left, Edge<VLongWritable, VLongWritable> right) {
+ long leftValue = left.getDestVertexId().get();
+ long rightValue = right.getDestVertexId().get();
+ return leftValue > rightValue ? 1 : (leftValue < rightValue ? -1 : 0);
+ }
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/VertexAggregator.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/VertexAggregator.java
similarity index 98%
rename from pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/VertexAggregator.java
rename to pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/VertexAggregator.java
index 68b7cca..d8f704e 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/VertexAggregator.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/VertexAggregator.java
@@ -13,7 +13,7 @@
* limitations under the License.
*/
-package edu.uci.ics.pregelix.example;
+package edu.uci.ics.pregelix.example.utils;
import java.io.IOException;
import java.util.Iterator;
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/VertexSorter.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/VertexSorter.java
similarity index 98%
rename from pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/VertexSorter.java
rename to pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/VertexSorter.java
index 1dd6922..8421088 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/VertexSorter.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/VertexSorter.java
@@ -13,7 +13,7 @@
* limitations under the License.
*/
-package edu.uci.ics.pregelix.example;
+package edu.uci.ics.pregelix.example.utils;
import java.io.IOException;
import java.util.Iterator;
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/dataload/DataLoadTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/dataload/DataLoadTest.java
index 7787347..321b5b2 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/dataload/DataLoadTest.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/dataload/DataLoadTest.java
@@ -37,7 +37,7 @@
import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
import edu.uci.ics.pregelix.example.PageRankVertex;
-import edu.uci.ics.pregelix.example.PageRankVertex.SimplePageRankVertexInputFormat;
+import edu.uci.ics.pregelix.example.PageRankVertex.SimulatedPageRankVertexInputFormat;
import edu.uci.ics.pregelix.example.util.TestUtils;
@SuppressWarnings("deprecation")
@@ -65,7 +65,7 @@
public DataLoadTest() throws Exception {
job = new PregelixJob(GIRAPH_JOB_NAME);
job.setVertexClass(PageRankVertex.class);
- job.setVertexInputFormatClass(SimplePageRankVertexInputFormat.class);
+ job.setVertexInputFormatClass(SimulatedPageRankVertexInputFormat.class);
job.getConfiguration().setClass(PregelixJob.VERTEX_INDEX_CLASS, LongWritable.class, WritableComparable.class);
job.getConfiguration().setClass(PregelixJob.VERTEX_VALUE_CLASS, DoubleWritable.class, Writable.class);
job.getConfiguration().setClass(PregelixJob.EDGE_VALUE_CLASS, FloatWritable.class, Writable.class);
@@ -76,7 +76,7 @@
ClusterConfig.setStorePath(PATH_TO_CLUSTER_STORE);
ClusterConfig.setClusterPropertiesPath(PATH_TO_CLUSTER_PROPERTIES);
cleanupStores();
- PregelixHyracksIntegrationUtil.init();
+ PregelixHyracksIntegrationUtil.init("src/test/resources/topology.xml");
PregelixHyracksIntegrationUtil.createApp(HYRACKS_APP_NAME);
LOGGER.info("Hyracks mini-cluster started");
startHDFS();
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java
index c0b4a10..ca5a1c4 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java
@@ -25,9 +25,11 @@
import edu.uci.ics.pregelix.api.job.PregelixJob;
import edu.uci.ics.pregelix.example.ConnectedComponentsVertex;
import edu.uci.ics.pregelix.example.ConnectedComponentsVertex.SimpleConnectedComponentsVertexOutputFormat;
+import edu.uci.ics.pregelix.example.GraphMutationVertex;
+import edu.uci.ics.pregelix.example.GraphMutationVertex.SimpleGraphMutationVertexOutputFormat;
import edu.uci.ics.pregelix.example.PageRankVertex;
-import edu.uci.ics.pregelix.example.PageRankVertex.SimplePageRankVertexInputFormat;
import edu.uci.ics.pregelix.example.PageRankVertex.SimplePageRankVertexOutputFormat;
+import edu.uci.ics.pregelix.example.PageRankVertex.SimulatedPageRankVertexInputFormat;
import edu.uci.ics.pregelix.example.ReachabilityVertex;
import edu.uci.ics.pregelix.example.ReachabilityVertex.SimpleReachibilityVertexOutputFormat;
import edu.uci.ics.pregelix.example.ShortestPathsVertex;
@@ -35,6 +37,14 @@
import edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat;
import edu.uci.ics.pregelix.example.inputformat.TextReachibilityVertexInputFormat;
import edu.uci.ics.pregelix.example.inputformat.TextShortestPathsInputFormat;
+import edu.uci.ics.pregelix.example.maximalclique.MaximalCliqueAggregator;
+import edu.uci.ics.pregelix.example.maximalclique.MaximalCliqueVertex;
+import edu.uci.ics.pregelix.example.maximalclique.MaximalCliqueVertex.MaximalCliqueVertexOutputFormat;
+import edu.uci.ics.pregelix.example.maximalclique.TextMaximalCliqueInputFormat;
+import edu.uci.ics.pregelix.example.trianglecounting.TextTriangleCountingInputFormat;
+import edu.uci.ics.pregelix.example.trianglecounting.TriangleCountingAggregator;
+import edu.uci.ics.pregelix.example.trianglecounting.TriangleCountingVertex;
+import edu.uci.ics.pregelix.example.trianglecounting.TriangleCountingVertex.TriangleCountingVertexOutputFormat;
public class JobGenerator {
private static String outputBase = "src/test/resources/jobs/";
@@ -44,6 +54,9 @@
private static String HDFS_INPUTPATH2 = "/webmapcomplex";
private static String HDFS_OUTPUTPAH2 = "/resultcomplex";
+ private static String HDFS_INPUTPATH3 = "/clique";
+ private static String HDFS_OUTPUTPAH3 = "/resultclique";
+
private static void generatePageRankJobReal(String jobName, String outputPath) throws IOException {
PregelixJob job = new PregelixJob(jobName);
job.setVertexClass(PageRankVertex.class);
@@ -148,7 +161,7 @@
private static void generatePageRankJob(String jobName, String outputPath) throws IOException {
PregelixJob job = new PregelixJob(jobName);
job.setVertexClass(PageRankVertex.class);
- job.setVertexInputFormatClass(SimplePageRankVertexInputFormat.class);
+ job.setVertexInputFormatClass(SimulatedPageRankVertexInputFormat.class);
job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class);
job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
FileInputFormat.setInputPaths(job, HDFS_INPUTPATH);
@@ -157,26 +170,74 @@
job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
}
+ private static void generateShortestPathJob(String jobName, String outputPath) throws IOException {
+ PregelixJob job = new PregelixJob(jobName);
+ job.setVertexClass(ShortestPathsVertex.class);
+ job.setVertexInputFormatClass(SimulatedPageRankVertexInputFormat.class);
+ job.setMessageCombinerClass(ShortestPathsVertex.SimpleMinCombiner.class);
+ job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
+ FileInputFormat.setInputPaths(job, HDFS_INPUTPATH);
+ FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH));
+ job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
+ job.getConfiguration().setLong(ShortestPathsVertex.SOURCE_ID, 0);
+ job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
+ }
+
+ private static void generatePageRankJobRealDynamic(String jobName, String outputPath) throws IOException {
+ PregelixJob job = new PregelixJob(jobName);
+ job.setVertexClass(PageRankVertex.class);
+ job.setVertexInputFormatClass(TextPageRankInputFormat.class);
+ job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
+ job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class);
+ job.setDynamicVertexValueSize(true);
+ FileInputFormat.setInputPaths(job, HDFS_INPUTPATH);
+ FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH));
+ job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
+ job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
+ }
+
+ private static void generateTriangleCountingJob(String jobName, String outputPath) throws IOException {
+ PregelixJob job = new PregelixJob(jobName);
+ job.setVertexClass(TriangleCountingVertex.class);
+ job.setGlobalAggregatorClass(TriangleCountingAggregator.class);
+ job.setVertexInputFormatClass(TextTriangleCountingInputFormat.class);
+ job.setVertexOutputFormatClass(TriangleCountingVertexOutputFormat.class);
+ FileInputFormat.setInputPaths(job, HDFS_INPUTPATH3);
+ FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH3));
+ job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
+ }
+
+ private static void generateMaximalCliqueJob(String jobName, String outputPath) throws IOException {
+ PregelixJob job = new PregelixJob(jobName);
+ job.setVertexClass(MaximalCliqueVertex.class);
+ job.setGlobalAggregatorClass(MaximalCliqueAggregator.class);
+ job.setDynamicVertexValueSize(true);
+ job.setVertexInputFormatClass(TextMaximalCliqueInputFormat.class);
+ job.setVertexOutputFormatClass(MaximalCliqueVertexOutputFormat.class);
+ FileInputFormat.setInputPaths(job, HDFS_INPUTPATH3);
+ FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH3));
+ job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
+ }
+
+ private static void generateGraphMutationJob(String jobName, String outputPath) throws IOException {
+ PregelixJob job = new PregelixJob(jobName);
+ job.setVertexClass(GraphMutationVertex.class);
+ job.setVertexInputFormatClass(TextPageRankInputFormat.class);
+ job.setVertexOutputFormatClass(SimpleGraphMutationVertexOutputFormat.class);
+ FileInputFormat.setInputPaths(job, HDFS_INPUTPATH);
+ FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH));
+ job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
+ job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
+ }
+
private static void genPageRank() throws IOException {
generatePageRankJob("PageRank", outputBase + "PageRank.xml");
generatePageRankJobReal("PageRank", outputBase + "PageRankReal.xml");
+ generatePageRankJobRealDynamic("PageRank", outputBase + "PageRankRealDynamic.xml");
generatePageRankJobRealComplex("PageRank", outputBase + "PageRankRealComplex.xml");
generatePageRankJobRealNoCombiner("PageRank", outputBase + "PageRankRealNoCombiner.xml");
}
- private static void generateShortestPathJob(String jobName, String outputPath) throws IOException {
- PregelixJob job = new PregelixJob(jobName);
- job.setVertexClass(ShortestPathsVertex.class);
- job.setVertexInputFormatClass(SimplePageRankVertexInputFormat.class);
- job.setMessageCombinerClass(ShortestPathsVertex.SimpleMinCombiner.class);
- job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
- FileInputFormat.setInputPaths(job, HDFS_INPUTPATH);
- FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH));
- job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
- job.getConfiguration().setLong(ShortestPathsVertex.SOURCE_ID, 0);
- job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
- }
-
private static void genShortestPath() throws IOException {
generateShortestPathJob("ShortestPaths", outputBase + "ShortestPaths.xml");
generateShortestPathJobReal("ShortestPaths", outputBase + "ShortestPathsReal.xml");
@@ -194,11 +255,25 @@
+ "ReachibilityRealComplexNoConnectivity.xml");
}
+ private static void genTriangleCounting() throws IOException {
+ generateTriangleCountingJob("Triangle Counting", outputBase + "TriangleCounting.xml");
+ }
+
+ private static void genMaximalClique() throws IOException {
+ generateMaximalCliqueJob("Maximal Clique", outputBase + "MaximalClique.xml");
+ }
+
+ private static void genGraphMutation() throws IOException {
+ generateGraphMutationJob("Graph Mutation", outputBase + "GraphMutation.xml");
+ }
+
public static void main(String[] args) throws IOException {
genPageRank();
genShortestPath();
genConnectedComponents();
genReachibility();
+ genTriangleCounting();
+ genMaximalClique();
+ genGraphMutation();
}
-
}
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java
index 89bce34..5a556fa 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java
@@ -30,6 +30,7 @@
import edu.uci.ics.pregelix.core.jobgen.JobGen;
import edu.uci.ics.pregelix.core.jobgen.JobGenInnerJoin;
import edu.uci.ics.pregelix.core.jobgen.JobGenOuterJoin;
+import edu.uci.ics.pregelix.core.jobgen.JobGenOuterJoinSingleSort;
import edu.uci.ics.pregelix.core.jobgen.JobGenOuterJoinSort;
import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
import edu.uci.ics.pregelix.dataflow.util.IterationUtils;
@@ -37,13 +38,16 @@
public class RunJobTestCase extends TestCase {
private static final String NC1 = "nc1";
- private static final String HYRACKS_APP_NAME = "giraph";
+ private static final String HYRACKS_APP_NAME = "pregelix";
private static String HDFS_INPUTPATH = "/webmap";
private static String HDFS_OUTPUTPAH = "/result";
private static String HDFS_INPUTPATH2 = "/webmapcomplex";
private static String HDFS_OUTPUTPAH2 = "/resultcomplex";
+ private static String HDFS_INPUTPATH3 = "/clique";
+ private static String HDFS_OUTPUTPAH3 = "/resultclique";
+
private final PregelixJob job;
private JobGen[] giraphJobGens;
private final String resultFileName;
@@ -61,21 +65,24 @@
if (inputPaths[0].toString().endsWith(HDFS_INPUTPATH)) {
FileInputFormat.setInputPaths(job, HDFS_INPUTPATH);
FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH));
- } else {
+ } else if (inputPaths[0].toString().endsWith(HDFS_INPUTPATH2)) {
FileInputFormat.setInputPaths(job, HDFS_INPUTPATH2);
FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH2));
+ } else {
+ FileInputFormat.setInputPaths(job, HDFS_INPUTPATH3);
+ FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH3));
}
job.setJobName(jobName);
this.resultFileName = resultFile;
this.expectedFileName = expectedFile;
- giraphJobGens = new JobGen[3];
+ giraphJobGens = new JobGen[4];
giraphJobGens[0] = new JobGenOuterJoin(job);
waitawhile();
giraphJobGens[1] = new JobGenInnerJoin(job);
waitawhile();
giraphJobGens[2] = new JobGenOuterJoinSort(job);
- //waitawhile();
- // giraphJobGens[3] = new JobGenOuterJoinSingleSort(job);
+ waitawhile();
+ giraphJobGens[3] = new JobGenOuterJoinSingleSort(job);
}
private void waitawhile() throws InterruptedException {
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestSuite.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestSuite.java
index 8a5c34b..fa98ebd 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestSuite.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestSuite.java
@@ -41,160 +41,176 @@
@SuppressWarnings("deprecation")
public class RunJobTestSuite extends TestSuite {
- private static final Logger LOGGER = Logger.getLogger(RunJobTestSuite.class.getName());
+ private static final Logger LOGGER = Logger.getLogger(RunJobTestSuite.class
+ .getName());
- private static final String ACTUAL_RESULT_DIR = "actual";
- private static final String EXPECTED_RESULT_DIR = "src/test/resources/expected";
- private static final String PATH_TO_HADOOP_CONF = "src/test/resources/hadoop/conf";
- private static final String PATH_TO_CLUSTER_STORE = "src/test/resources/cluster/stores.properties";
- private static final String PATH_TO_CLUSTER_PROPERTIES = "src/test/resources/cluster/cluster.properties";
- private static final String PATH_TO_JOBS = "src/test/resources/jobs/";
- private static final String PATH_TO_IGNORE = "src/test/resources/ignore.txt";
- private static final String PATH_TO_ONLY = "src/test/resources/only.txt";
- private static final String FILE_EXTENSION_OF_RESULTS = "result";
+ private static final String ACTUAL_RESULT_DIR = "actual";
+ private static final String EXPECTED_RESULT_DIR = "src/test/resources/expected";
+ private static final String PATH_TO_HADOOP_CONF = "src/test/resources/hadoop/conf";
+ private static final String PATH_TO_CLUSTER_STORE = "src/test/resources/cluster/stores.properties";
+ private static final String PATH_TO_CLUSTER_PROPERTIES = "src/test/resources/cluster/cluster.properties";
+ private static final String PATH_TO_JOBS = "src/test/resources/jobs/";
+ private static final String PATH_TO_IGNORE = "src/test/resources/ignore.txt";
+ private static final String PATH_TO_ONLY = "src/test/resources/only.txt";
+ private static final String FILE_EXTENSION_OF_RESULTS = "result";
- private static final String DATA_PATH = "data/webmap/webmap_link.txt";
- private static final String HDFS_PATH = "/webmap/";
+ private static final String DATA_PATH = "data/webmap/webmap_link.txt";
+ private static final String HDFS_PATH = "/webmap/";
- private static final String DATA_PATH2 = "data/webmapcomplex/webmap_link.txt";
- private static final String HDFS_PATH2 = "/webmapcomplex/";
+ private static final String DATA_PATH2 = "data/webmapcomplex/webmap_link.txt";
+ private static final String HDFS_PATH2 = "/webmapcomplex/";
- private static final String HYRACKS_APP_NAME = "giraph";
- private static final String HADOOP_CONF_PATH = ACTUAL_RESULT_DIR + File.separator + "conf.xml";
- private MiniDFSCluster dfsCluster;
+ private static final String DATA_PATH3 = "data/clique/clique.txt";
+ private static final String HDFS_PATH3 = "/clique/";
- private JobConf conf = new JobConf();
- private int numberOfNC = 2;
+ private static final String HYRACKS_APP_NAME = "pregelix";
+ private static final String HADOOP_CONF_PATH = ACTUAL_RESULT_DIR
+ + File.separator + "conf.xml";
+ private MiniDFSCluster dfsCluster;
- public void setUp() throws Exception {
- ClusterConfig.setStorePath(PATH_TO_CLUSTER_STORE);
- ClusterConfig.setClusterPropertiesPath(PATH_TO_CLUSTER_PROPERTIES);
- cleanupStores();
- PregelixHyracksIntegrationUtil.init();
- PregelixHyracksIntegrationUtil.createApp(HYRACKS_APP_NAME);
- LOGGER.info("Hyracks mini-cluster started");
- FileUtils.forceMkdir(new File(ACTUAL_RESULT_DIR));
- FileUtils.cleanDirectory(new File(ACTUAL_RESULT_DIR));
- startHDFS();
- }
+ private JobConf conf = new JobConf();
+ private int numberOfNC = 2;
- private void cleanupStores() throws IOException {
- FileUtils.forceMkdir(new File("teststore"));
- FileUtils.forceMkdir(new File("build"));
- FileUtils.cleanDirectory(new File("teststore"));
- FileUtils.cleanDirectory(new File("build"));
- }
+ public void setUp() throws Exception {
+ ClusterConfig.setStorePath(PATH_TO_CLUSTER_STORE);
+ ClusterConfig.setClusterPropertiesPath(PATH_TO_CLUSTER_PROPERTIES);
+ cleanupStores();
+ PregelixHyracksIntegrationUtil.init("src/test/resources/topology.xml");
+ PregelixHyracksIntegrationUtil.createApp(HYRACKS_APP_NAME);
+ LOGGER.info("Hyracks mini-cluster started");
+ FileUtils.forceMkdir(new File(ACTUAL_RESULT_DIR));
+ FileUtils.cleanDirectory(new File(ACTUAL_RESULT_DIR));
+ startHDFS();
+ }
- private void startHDFS() throws IOException {
- conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/core-site.xml"));
- conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/mapred-site.xml"));
- conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/hdfs-site.xml"));
- FileSystem lfs = FileSystem.getLocal(new Configuration());
- lfs.delete(new Path("build"), true);
- System.setProperty("hadoop.log.dir", "logs");
- dfsCluster = new MiniDFSCluster(conf, numberOfNC, true, null);
- FileSystem dfs = FileSystem.get(conf);
- Path src = new Path(DATA_PATH);
- Path dest = new Path(HDFS_PATH);
- dfs.mkdirs(dest);
- dfs.copyFromLocalFile(src, dest);
+ private void cleanupStores() throws IOException {
+ FileUtils.forceMkdir(new File("teststore"));
+ FileUtils.forceMkdir(new File("build"));
+ FileUtils.cleanDirectory(new File("teststore"));
+ FileUtils.cleanDirectory(new File("build"));
+ }
- src = new Path(DATA_PATH2);
- dest = new Path(HDFS_PATH2);
- dfs.mkdirs(dest);
- dfs.copyFromLocalFile(src, dest);
+ private void startHDFS() throws IOException {
+ conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/core-site.xml"));
+ conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/mapred-site.xml"));
+ conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/hdfs-site.xml"));
+ FileSystem lfs = FileSystem.getLocal(new Configuration());
+ lfs.delete(new Path("build"), true);
+ System.setProperty("hadoop.log.dir", "logs");
+ dfsCluster = new MiniDFSCluster(conf, numberOfNC, true, null);
+ FileSystem dfs = FileSystem.get(conf);
+ Path src = new Path(DATA_PATH);
+ Path dest = new Path(HDFS_PATH);
+ dfs.mkdirs(dest);
+ dfs.copyFromLocalFile(src, dest);
- DataOutputStream confOutput = new DataOutputStream(new FileOutputStream(new File(HADOOP_CONF_PATH)));
- conf.writeXml(confOutput);
- confOutput.flush();
- confOutput.close();
- }
+ src = new Path(DATA_PATH2);
+ dest = new Path(HDFS_PATH2);
+ dfs.mkdirs(dest);
+ dfs.copyFromLocalFile(src, dest);
- /**
- * cleanup hdfs cluster
- */
- private void cleanupHDFS() throws Exception {
- dfsCluster.shutdown();
- }
+ src = new Path(DATA_PATH3);
+ dest = new Path(HDFS_PATH3);
+ dfs.mkdirs(dest);
+ dfs.copyFromLocalFile(src, dest);
- public void tearDown() throws Exception {
- PregelixHyracksIntegrationUtil.destroyApp(HYRACKS_APP_NAME);
- PregelixHyracksIntegrationUtil.deinit();
- LOGGER.info("Hyracks mini-cluster shut down");
- cleanupHDFS();
- }
+ DataOutputStream confOutput = new DataOutputStream(
+ new FileOutputStream(new File(HADOOP_CONF_PATH)));
+ conf.writeXml(confOutput);
+ confOutput.flush();
+ confOutput.close();
+ }
- public static Test suite() throws Exception {
- List<String> ignores = getFileList(PATH_TO_IGNORE);
- List<String> onlys = getFileList(PATH_TO_ONLY);
- File testData = new File(PATH_TO_JOBS);
- File[] queries = testData.listFiles();
- RunJobTestSuite testSuite = new RunJobTestSuite();
- testSuite.setUp();
- boolean onlyEnabled = false;
+ /**
+ * cleanup hdfs cluster
+ */
+ private void cleanupHDFS() throws Exception {
+ dfsCluster.shutdown();
+ }
- if (onlys.size() > 0) {
- onlyEnabled = true;
- }
- for (File qFile : queries) {
- if (isInList(ignores, qFile.getName()))
- continue;
+ public void tearDown() throws Exception {
+ PregelixHyracksIntegrationUtil.destroyApp(HYRACKS_APP_NAME);
+ PregelixHyracksIntegrationUtil.deinit();
+ LOGGER.info("Hyracks mini-cluster shut down");
+ cleanupHDFS();
+ }
- if (qFile.isFile()) {
- if (onlyEnabled && !isInList(onlys, qFile.getName())) {
- continue;
- } else {
- String resultFileName = ACTUAL_RESULT_DIR + File.separator + jobExtToResExt(qFile.getName());
- String expectedFileName = EXPECTED_RESULT_DIR + File.separator + jobExtToResExt(qFile.getName());
- testSuite.addTest(new RunJobTestCase(HADOOP_CONF_PATH, qFile.getName(), qFile.getAbsolutePath()
- .toString(), resultFileName, expectedFileName));
- }
- }
- }
- return testSuite;
- }
+ public static Test suite() throws Exception {
+ List<String> ignores = getFileList(PATH_TO_IGNORE);
+ List<String> onlys = getFileList(PATH_TO_ONLY);
+ File testData = new File(PATH_TO_JOBS);
+ File[] queries = testData.listFiles();
+ RunJobTestSuite testSuite = new RunJobTestSuite();
+ testSuite.setUp();
+ boolean onlyEnabled = false;
- /**
- * Runs the tests and collects their result in a TestResult.
- */
- @Override
- public void run(TestResult result) {
- try {
- int testCount = countTestCases();
- for (int i = 0; i < testCount; i++) {
- // cleanupStores();
- Test each = this.testAt(i);
- if (result.shouldStop())
- break;
- runTest(each, result);
- }
- tearDown();
- } catch (Exception e) {
- throw new IllegalStateException(e);
- }
- }
+ if (onlys.size() > 0) {
+ onlyEnabled = true;
+ }
+ for (File qFile : queries) {
+ if (isInList(ignores, qFile.getName()))
+ continue;
- protected static List<String> getFileList(String ignorePath) throws FileNotFoundException, IOException {
- BufferedReader reader = new BufferedReader(new FileReader(ignorePath));
- String s = null;
- List<String> ignores = new ArrayList<String>();
- while ((s = reader.readLine()) != null) {
- ignores.add(s);
- }
- reader.close();
- return ignores;
- }
+ if (qFile.isFile()) {
+ if (onlyEnabled && !isInList(onlys, qFile.getName())) {
+ continue;
+ } else {
+ String resultFileName = ACTUAL_RESULT_DIR + File.separator
+ + jobExtToResExt(qFile.getName());
+ String expectedFileName = EXPECTED_RESULT_DIR
+ + File.separator + jobExtToResExt(qFile.getName());
+ testSuite.addTest(new RunJobTestCase(HADOOP_CONF_PATH,
+ qFile.getName(),
+ qFile.getAbsolutePath().toString(), resultFileName,
+ expectedFileName));
+ }
+ }
+ }
+ return testSuite;
+ }
- private static String jobExtToResExt(String fname) {
- int dot = fname.lastIndexOf('.');
- return fname.substring(0, dot + 1) + FILE_EXTENSION_OF_RESULTS;
- }
+ /**
+ * Runs the tests and collects their result in a TestResult.
+ */
+ @Override
+ public void run(TestResult result) {
+ try {
+ int testCount = countTestCases();
+ for (int i = 0; i < testCount; i++) {
+ // cleanupStores();
+ Test each = this.testAt(i);
+ if (result.shouldStop())
+ break;
+ runTest(each, result);
+ }
+ tearDown();
+ } catch (Exception e) {
+ throw new IllegalStateException(e);
+ }
+ }
- private static boolean isInList(List<String> onlys, String name) {
- for (String only : onlys)
- if (name.indexOf(only) >= 0)
- return true;
- return false;
- }
+ protected static List<String> getFileList(String ignorePath)
+ throws FileNotFoundException, IOException {
+ BufferedReader reader = new BufferedReader(new FileReader(ignorePath));
+ String s = null;
+ List<String> ignores = new ArrayList<String>();
+ while ((s = reader.readLine()) != null) {
+ ignores.add(s);
+ }
+ reader.close();
+ return ignores;
+ }
+
+ private static String jobExtToResExt(String fname) {
+ int dot = fname.lastIndexOf('.');
+ return fname.substring(0, dot + 1) + FILE_EXTENSION_OF_RESULTS;
+ }
+
+ private static boolean isInList(List<String> onlys, String name) {
+ for (String only : onlys)
+ if (name.indexOf(only) >= 0)
+ return true;
+ return false;
+ }
}
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/util/TestUtils.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/util/TestUtils.java
index 1b22b47..d89ec46 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/util/TestUtils.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/util/TestUtils.java
@@ -62,8 +62,8 @@
if (row1.equals(row2))
continue;
- String[] fields1 = row1.split(",");
- String[] fields2 = row2.split(",");
+ String[] fields1 = row1.split(" ");
+ String[] fields2 = row2.split(" ");
for (int j = 0; j < fields1.length; j++) {
if (fields1[j].equals(fields2[j])) {
@@ -71,8 +71,6 @@
} else if (fields1[j].indexOf('.') < 0) {
return false;
} else {
- fields1[j] = fields1[j].split("=")[1];
- fields2[j] = fields2[j].split("=")[1];
Double double1 = Double.parseDouble(fields1[j]);
Double double2 = Double.parseDouble(fields2[j]);
float float1 = (float) double1.doubleValue();
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal.result b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal.result
index b8efedc..45376e2 100644
--- a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal.result
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal.result
@@ -1,20 +1,20 @@
-0|Vertex(id=0,value=0, edges=(1,))
-1|Vertex(id=1,value=0, edges=(1,2,))
-2|Vertex(id=2,value=0, edges=(1,2,3,))
-3|Vertex(id=3,value=0, edges=(1,2,3,4,))
-4|Vertex(id=4,value=0, edges=(1,2,3,4,5,))
-5|Vertex(id=5,value=0, edges=(1,2,3,4,5,6,))
-6|Vertex(id=6,value=0, edges=(1,2,3,4,5,6,7,))
-7|Vertex(id=7,value=0, edges=(1,2,3,4,5,6,7,8,))
-8|Vertex(id=8,value=0, edges=(1,2,3,4,5,6,7,8,9,))
-9|Vertex(id=9,value=0, edges=(1,2,3,4,5,6,7,8,9,10,))
-10|Vertex(id=10,value=0, edges=(11,))
-11|Vertex(id=11,value=0, edges=(11,12,))
-12|Vertex(id=12,value=0, edges=(11,12,13,))
-13|Vertex(id=13,value=0, edges=(11,12,13,14,))
-14|Vertex(id=14,value=0, edges=(11,12,13,14,15,))
-15|Vertex(id=15,value=0, edges=(11,12,13,14,15,16,))
-16|Vertex(id=16,value=0, edges=(11,12,13,14,15,16,17,))
-17|Vertex(id=17,value=0, edges=(11,12,13,14,15,16,17,18,))
-18|Vertex(id=18,value=0, edges=(11,12,13,14,15,16,17,18,19,))
-19|Vertex(id=19,value=0, edges=(0,11,12,13,14,15,16,17,18,19,))
+0 0
+1 0
+2 0
+3 0
+4 0
+5 0
+6 0
+7 0
+8 0
+9 0
+10 0
+11 0
+12 0
+13 0
+14 0
+15 0
+16 0
+17 0
+18 0
+19 0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex.result b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex.result
index ad448b2..dbc30fc 100644
--- a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex.result
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex.result
@@ -1,23 +1,23 @@
-0|Vertex(id=0,value=0, edges=(1,50,))
-1|Vertex(id=1,value=0, edges=(1,2,))
-2|Vertex(id=2,value=0, edges=(1,2,3,))
-3|Vertex(id=3,value=0, edges=(1,2,3,4,))
-4|Vertex(id=4,value=0, edges=(1,2,3,4,5,))
-5|Vertex(id=5,value=0, edges=(1,2,3,4,5,6,))
-6|Vertex(id=6,value=0, edges=(1,2,3,4,5,6,7,))
-7|Vertex(id=7,value=0, edges=(1,2,3,4,5,6,7,8,))
-8|Vertex(id=8,value=0, edges=(1,2,3,4,5,6,7,8,9,))
-9|Vertex(id=9,value=0, edges=(1,2,3,4,5,6,7,8,9,10,))
-10|Vertex(id=10,value=0, edges=(11,99,))
-11|Vertex(id=11,value=0, edges=(11,12,101,))
-12|Vertex(id=12,value=0, edges=(11,12,13,))
-13|Vertex(id=13,value=0, edges=(11,12,13,14,))
-14|Vertex(id=14,value=0, edges=(11,12,13,14,15,))
-15|Vertex(id=15,value=0, edges=(11,12,13,14,15,16,))
-16|Vertex(id=16,value=0, edges=(11,12,13,14,15,16,17,))
-17|Vertex(id=17,value=0, edges=(11,12,13,14,15,16,17,18,))
-18|Vertex(id=18,value=0, edges=(11,12,13,14,15,16,17,18,19,))
-19|Vertex(id=19,value=0, edges=(0,11,12,13,14,15,16,17,18,19,))
-21|Vertex(id=21,value=21, edges=(22,23,24,))
-25|Vertex(id=25,value=25, edges=())
-27|Vertex(id=27,value=27, edges=())
+0 0
+1 0
+2 0
+3 0
+4 0
+5 0
+6 0
+7 0
+8 0
+9 0
+10 0
+11 0
+12 0
+13 0
+14 0
+15 0
+16 0
+17 0
+18 0
+19 0
+21 21
+25 25
+27 27
diff --git a/pregelix/pregelix-example/src/test/resources/expected/GraphMutation.result b/pregelix/pregelix-example/src/test/resources/expected/GraphMutation.result
new file mode 100644
index 0000000..a30166c
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/GraphMutation.result
@@ -0,0 +1,13 @@
+1 0.0
+5 0.0
+7 0.0
+11 0.0
+13 0.0
+17 0.0
+19 0.0
+100 0.0
+500 0.0
+700 0.0
+1100 0.0
+1300 0.0
+1700 0.0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/MaximalClique.result b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique.result
new file mode 100644
index 0000000..d238037
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique.result
@@ -0,0 +1,7 @@
+1 1,2,3,4;
+2 2,3,4;
+3
+4
+5
+6
+7
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRank.result b/pregelix/pregelix-example/src/test/resources/expected/PageRank.result
index f38e191..9c4d83a 100644
--- a/pregelix/pregelix-example/src/test/resources/expected/PageRank.result
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRank.result
@@ -1,20 +1,20 @@
-0|Vertex(id=0,value=0.008290140026154316, edges=(1,))
-1|Vertex(id=1,value=0.1535152819247165, edges=(1,2,))
-2|Vertex(id=2,value=0.14646839195826475, edges=(1,2,3,))
-3|Vertex(id=3,value=0.08125113985998214, edges=(1,2,3,4,))
-4|Vertex(id=4,value=0.03976979906329426, edges=(1,2,3,4,5,))
-5|Vertex(id=5,value=0.0225041581462058, edges=(1,2,3,4,5,6,))
-6|Vertex(id=6,value=0.015736276824953852, edges=(1,2,3,4,5,6,7,))
-7|Vertex(id=7,value=0.012542224114863661, edges=(1,2,3,4,5,6,7,8,))
-8|Vertex(id=8,value=0.010628239626209894, edges=(1,2,3,4,5,6,7,8,9,))
-9|Vertex(id=9,value=0.009294348455354817, edges=(1,2,3,4,5,6,7,8,9,10,))
-10|Vertex(id=10,value=0.008290140026154316, edges=(11,))
-11|Vertex(id=11,value=0.15351528192471647, edges=(11,12,))
-12|Vertex(id=12,value=0.14646839195826472, edges=(11,12,13,))
-13|Vertex(id=13,value=0.08125113985998214, edges=(11,12,13,14,))
-14|Vertex(id=14,value=0.03976979906329425, edges=(11,12,13,14,15,))
-15|Vertex(id=15,value=0.0225041581462058, edges=(11,12,13,14,15,16,))
-16|Vertex(id=16,value=0.015736276824953852, edges=(11,12,13,14,15,16,17,))
-17|Vertex(id=17,value=0.012542224114863661, edges=(11,12,13,14,15,16,17,18,))
-18|Vertex(id=18,value=0.010628239626209894, edges=(11,12,13,14,15,16,17,18,19,))
-19|Vertex(id=19,value=0.009294348455354817, edges=(0,11,12,13,14,15,16,17,18,19,))
+0 0.008290140026154316
+1 0.1535152819247165
+2 0.14646839195826475
+3 0.08125113985998214
+4 0.03976979906329426
+5 0.0225041581462058
+6 0.015736276824953852
+7 0.012542224114863661
+8 0.010628239626209894
+9 0.009294348455354817
+10 0.008290140026154316
+11 0.15351528192471647
+12 0.14646839195826472
+13 0.08125113985998214
+14 0.03976979906329425
+15 0.0225041581462058
+16 0.015736276824953852
+17 0.012542224114863661
+18 0.010628239626209894
+19 0.009294348455354817
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankReal.result b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal.result
index ab05d38..6432eda 100644
--- a/pregelix/pregelix-example/src/test/resources/expected/PageRankReal.result
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal.result
@@ -1,20 +1,20 @@
-0|Vertex(id=0,value=0.008290140026154316, edges=(1,))
-1|Vertex(id=1,value=0.1535152819247165, edges=(1,2,))
-2|Vertex(id=2,value=0.14646839195826475, edges=(1,2,3,))
-3|Vertex(id=3,value=0.08125113985998214, edges=(1,2,3,4,))
-4|Vertex(id=4,value=0.03976979906329426, edges=(1,2,3,4,5,))
-5|Vertex(id=5,value=0.0225041581462058, edges=(1,2,3,4,5,6,))
-6|Vertex(id=6,value=0.015736276824953852, edges=(1,2,3,4,5,6,7,))
-7|Vertex(id=7,value=0.012542224114863661, edges=(1,2,3,4,5,6,7,8,))
-8|Vertex(id=8,value=0.010628239626209894, edges=(1,2,3,4,5,6,7,8,9,))
-9|Vertex(id=9,value=0.009294348455354817, edges=(1,2,3,4,5,6,7,8,9,10,))
-10|Vertex(id=10,value=0.008290140026154316, edges=(11,))
-11|Vertex(id=11,value=0.15351528192471647, edges=(11,12,))
-12|Vertex(id=12,value=0.14646839195826472, edges=(11,12,13,))
-13|Vertex(id=13,value=0.08125113985998214, edges=(11,12,13,14,))
-14|Vertex(id=14,value=0.03976979906329426, edges=(11,12,13,14,15,))
-15|Vertex(id=15,value=0.0225041581462058, edges=(11,12,13,14,15,16,))
-16|Vertex(id=16,value=0.015736276824953852, edges=(11,12,13,14,15,16,17,))
-17|Vertex(id=17,value=0.012542224114863661, edges=(11,12,13,14,15,16,17,18,))
-18|Vertex(id=18,value=0.010628239626209894, edges=(11,12,13,14,15,16,17,18,19,))
-19|Vertex(id=19,value=0.009294348455354817, edges=(0,11,12,13,14,15,16,17,18,19,))
+0 0.008290140026154316
+1 0.1535152819247165
+2 0.14646839195826475
+3 0.08125113985998214
+4 0.03976979906329426
+5 0.0225041581462058
+6 0.015736276824953852
+7 0.012542224114863661
+8 0.010628239626209894
+9 0.009294348455354817
+10 0.008290140026154316
+11 0.15351528192471647
+12 0.14646839195826472
+13 0.08125113985998214
+14 0.03976979906329426
+15 0.0225041581462058
+16 0.015736276824953852
+17 0.012542224114863661
+18 0.010628239626209894
+19 0.009294348455354817
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex.result b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex.result
index 1fc108a..2bd09e1 100644
--- a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex.result
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex.result
@@ -1,23 +1,23 @@
-0|Vertex(id=0,value=0.0072088164890121405, edges=(1,50,))
-1|Vertex(id=1,value=0.12352056961948686, edges=(1,2,))
-2|Vertex(id=2,value=0.12045670441668178, edges=(1,2,3,))
-3|Vertex(id=3,value=0.06798545786459467, edges=(1,2,3,4,))
-4|Vertex(id=4,value=0.03387281259892814, edges=(1,2,3,4,5,))
-5|Vertex(id=5,value=0.01942600635480669, edges=(1,2,3,4,5,6,))
-6|Vertex(id=6,value=0.013661020012182747, edges=(1,2,3,4,5,6,7,))
-7|Vertex(id=7,value=0.0109034351563503, edges=(1,2,3,4,5,6,7,8,))
-8|Vertex(id=8,value=0.009241684574402657, edges=(1,2,3,4,5,6,7,8,9,))
-9|Vertex(id=9,value=0.008082028259564783, edges=(1,2,3,4,5,6,7,8,9,10,))
-10|Vertex(id=10,value=0.007208817414047232, edges=(11,99,))
-11|Vertex(id=11,value=0.07555839219845861, edges=(11,12,101,))
-12|Vertex(id=12,value=0.07249452699565352, edges=(11,12,13,))
-13|Vertex(id=13,value=0.05063539695954156, edges=(11,12,13,14,))
-14|Vertex(id=14,value=0.029644452692487822, edges=(11,12,13,14,15,))
-15|Vertex(id=15,value=0.018670183493927354, edges=(11,12,13,14,15,16,))
-16|Vertex(id=16,value=0.013558283213067561, edges=(11,12,13,14,15,16,17,))
-17|Vertex(id=17,value=0.010892790899883237, edges=(11,12,13,14,15,16,17,18,))
-18|Vertex(id=18,value=0.009240874593661061, edges=(11,12,13,14,15,16,17,18,19,))
-19|Vertex(id=19,value=0.008081987856433137, edges=(0,11,12,13,14,15,16,17,18,19,))
-21|Vertex(id=21,value=0.006521739130434782, edges=(22,23,24,))
-25|Vertex(id=25,value=0.006521739130434782, edges=())
-27|Vertex(id=27,value=0.006521739130434782, edges=())
+0 0.0072088164890121405
+1 0.12352056961948686
+2 0.12045670441668178
+3 0.06798545786459467
+4 0.03387281259892814
+5 0.01942600635480669
+6 0.013661020012182747
+7 0.0109034351563503
+8 0.009241684574402657
+9 0.008082028259564783
+10 0.007208817414047232
+11 0.07555839219845861
+12 0.07249452699565352
+13 0.05063539695954156
+14 0.029644452692487822
+15 0.018670183493927354
+16 0.013558283213067561
+17 0.010892790899883237
+18 0.009240874593661061
+19 0.008081987856433137
+21 0.006521739130434782
+25 0.006521739130434782
+27 0.006521739130434782
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic.result b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic.result
new file mode 100644
index 0000000..6432eda
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic.result
@@ -0,0 +1,20 @@
+0 0.008290140026154316
+1 0.1535152819247165
+2 0.14646839195826475
+3 0.08125113985998214
+4 0.03976979906329426
+5 0.0225041581462058
+6 0.015736276824953852
+7 0.012542224114863661
+8 0.010628239626209894
+9 0.009294348455354817
+10 0.008290140026154316
+11 0.15351528192471647
+12 0.14646839195826472
+13 0.08125113985998214
+14 0.03976979906329426
+15 0.0225041581462058
+16 0.015736276824953852
+17 0.012542224114863661
+18 0.010628239626209894
+19 0.009294348455354817
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner.result b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner.result
index ab05d38..9a747a6 100755
--- a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner.result
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner.result
@@ -1,20 +1,20 @@
-0|Vertex(id=0,value=0.008290140026154316, edges=(1,))
-1|Vertex(id=1,value=0.1535152819247165, edges=(1,2,))
-2|Vertex(id=2,value=0.14646839195826475, edges=(1,2,3,))
-3|Vertex(id=3,value=0.08125113985998214, edges=(1,2,3,4,))
-4|Vertex(id=4,value=0.03976979906329426, edges=(1,2,3,4,5,))
-5|Vertex(id=5,value=0.0225041581462058, edges=(1,2,3,4,5,6,))
-6|Vertex(id=6,value=0.015736276824953852, edges=(1,2,3,4,5,6,7,))
-7|Vertex(id=7,value=0.012542224114863661, edges=(1,2,3,4,5,6,7,8,))
-8|Vertex(id=8,value=0.010628239626209894, edges=(1,2,3,4,5,6,7,8,9,))
-9|Vertex(id=9,value=0.009294348455354817, edges=(1,2,3,4,5,6,7,8,9,10,))
-10|Vertex(id=10,value=0.008290140026154316, edges=(11,))
-11|Vertex(id=11,value=0.15351528192471647, edges=(11,12,))
-12|Vertex(id=12,value=0.14646839195826472, edges=(11,12,13,))
-13|Vertex(id=13,value=0.08125113985998214, edges=(11,12,13,14,))
-14|Vertex(id=14,value=0.03976979906329426, edges=(11,12,13,14,15,))
-15|Vertex(id=15,value=0.0225041581462058, edges=(11,12,13,14,15,16,))
-16|Vertex(id=16,value=0.015736276824953852, edges=(11,12,13,14,15,16,17,))
-17|Vertex(id=17,value=0.012542224114863661, edges=(11,12,13,14,15,16,17,18,))
-18|Vertex(id=18,value=0.010628239626209894, edges=(11,12,13,14,15,16,17,18,19,))
-19|Vertex(id=19,value=0.009294348455354817, edges=(0,11,12,13,14,15,16,17,18,19,))
+0 0.008290140026154316
+1 0.15351528192471647
+2 0.14646839195826475
+3 0.08125113985998211
+4 0.03976979906329425
+5 0.0225041581462058
+6 0.01573627682495385
+7 0.012542224114863661
+8 0.010628239626209894
+9 0.009294348455354817
+10 0.008290140026154316
+11 0.1535152819247165
+12 0.14646839195826475
+13 0.08125113985998214
+14 0.03976979906329426
+15 0.0225041581462058
+16 0.015736276824953852
+17 0.012542224114863661
+18 0.010628239626209894
+19 0.009294348455354817
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex.result b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex.result
index 74113a8..a1dfc0f 100644
--- a/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex.result
+++ b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex.result
@@ -1,23 +1,23 @@
-0|Vertex(id=0,value=2, edges=(1,50,))
-1|Vertex(id=1,value=3, edges=(1,2,))
-2|Vertex(id=2,value=1, edges=(1,2,3,))
-3|Vertex(id=3,value=1, edges=(1,2,3,4,))
-4|Vertex(id=4,value=1, edges=(1,2,3,4,5,))
-5|Vertex(id=5,value=1, edges=(1,2,3,4,5,6,))
-6|Vertex(id=6,value=1, edges=(1,2,3,4,5,6,7,))
-7|Vertex(id=7,value=1, edges=(1,2,3,4,5,6,7,8,))
-8|Vertex(id=8,value=1, edges=(1,2,3,4,5,6,7,8,9,))
-9|Vertex(id=9,value=1, edges=(1,2,3,4,5,6,7,8,9,10,))
-10|Vertex(id=10,value=3, edges=(11,99,))
-11|Vertex(id=11,value=2, edges=(11,12,101,))
-12|Vertex(id=12,value=2, edges=(11,12,13,))
-13|Vertex(id=13,value=2, edges=(11,12,13,14,))
-14|Vertex(id=14,value=2, edges=(11,12,13,14,15,))
-15|Vertex(id=15,value=2, edges=(11,12,13,14,15,16,))
-16|Vertex(id=16,value=2, edges=(11,12,13,14,15,16,17,))
-17|Vertex(id=17,value=2, edges=(11,12,13,14,15,16,17,18,))
-18|Vertex(id=18,value=2, edges=(11,12,13,14,15,16,17,18,19,))
-19|Vertex(id=19,value=2, edges=(0,11,12,13,14,15,16,17,18,19,))
-21|Vertex(id=21,value=0, edges=(22,23,24,))
-25|Vertex(id=25,value=0, edges=())
-27|Vertex(id=27,value=0, edges=())
+0 2
+1 3
+2 1
+3 1
+4 1
+5 1
+6 1
+7 1
+8 1
+9 1
+10 3
+11 2
+12 2
+13 2
+14 2
+15 2
+16 2
+17 2
+18 2
+19 2
+21 0
+25 0
+27 0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity.result b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity.result
index ea0edc2..1693fb2 100644
--- a/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity.result
+++ b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity.result
@@ -1,23 +1,23 @@
-0|Vertex(id=0,value=1, edges=(1,50,))
-1|Vertex(id=1,value=1, edges=(1,2,))
-2|Vertex(id=2,value=1, edges=(1,2,3,))
-3|Vertex(id=3,value=1, edges=(1,2,3,4,))
-4|Vertex(id=4,value=1, edges=(1,2,3,4,5,))
-5|Vertex(id=5,value=1, edges=(1,2,3,4,5,6,))
-6|Vertex(id=6,value=1, edges=(1,2,3,4,5,6,7,))
-7|Vertex(id=7,value=1, edges=(1,2,3,4,5,6,7,8,))
-8|Vertex(id=8,value=1, edges=(1,2,3,4,5,6,7,8,9,))
-9|Vertex(id=9,value=1, edges=(1,2,3,4,5,6,7,8,9,10,))
-10|Vertex(id=10,value=1, edges=(11,99,))
-11|Vertex(id=11,value=1, edges=(11,12,101,))
-12|Vertex(id=12,value=1, edges=(11,12,13,))
-13|Vertex(id=13,value=1, edges=(11,12,13,14,))
-14|Vertex(id=14,value=1, edges=(11,12,13,14,15,))
-15|Vertex(id=15,value=1, edges=(11,12,13,14,15,16,))
-16|Vertex(id=16,value=1, edges=(11,12,13,14,15,16,17,))
-17|Vertex(id=17,value=1, edges=(11,12,13,14,15,16,17,18,))
-18|Vertex(id=18,value=1, edges=(11,12,13,14,15,16,17,18,19,))
-19|Vertex(id=19,value=1, edges=(0,11,12,13,14,15,16,17,18,19,))
-21|Vertex(id=21,value=0, edges=(22,23,24,))
-25|Vertex(id=25,value=2, edges=())
-27|Vertex(id=27,value=0, edges=())
+0 1
+1 1
+2 1
+3 1
+4 1
+5 1
+6 1
+7 1
+8 1
+9 1
+10 1
+11 1
+12 1
+13 1
+14 1
+15 1
+16 1
+17 1
+18 1
+19 1
+21 0
+25 2
+27 0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths.result b/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths.result
index 7bb0ca3..46d1c73 100644
--- a/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths.result
+++ b/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths.result
@@ -1,20 +1,20 @@
-0|Vertex(id=0,value=0.0, edges=(1,))
-1|Vertex(id=1,value=0.0, edges=(1,2,))
-2|Vertex(id=2,value=100.0, edges=(1,2,3,))
-3|Vertex(id=3,value=300.0, edges=(1,2,3,4,))
-4|Vertex(id=4,value=600.0, edges=(1,2,3,4,5,))
-5|Vertex(id=5,value=1000.0, edges=(1,2,3,4,5,6,))
-6|Vertex(id=6,value=1500.0, edges=(1,2,3,4,5,6,7,))
-7|Vertex(id=7,value=2100.0, edges=(1,2,3,4,5,6,7,8,))
-8|Vertex(id=8,value=2800.0, edges=(1,2,3,4,5,6,7,8,9,))
-9|Vertex(id=9,value=3600.0, edges=(1,2,3,4,5,6,7,8,9,10,))
-10|Vertex(id=10,value=4500.0, edges=(11,))
-11|Vertex(id=11,value=5500.0, edges=(11,12,))
-12|Vertex(id=12,value=6600.0, edges=(11,12,13,))
-13|Vertex(id=13,value=7800.0, edges=(11,12,13,14,))
-14|Vertex(id=14,value=9100.0, edges=(11,12,13,14,15,))
-15|Vertex(id=15,value=10500.0, edges=(11,12,13,14,15,16,))
-16|Vertex(id=16,value=12000.0, edges=(11,12,13,14,15,16,17,))
-17|Vertex(id=17,value=13600.0, edges=(11,12,13,14,15,16,17,18,))
-18|Vertex(id=18,value=15300.0, edges=(11,12,13,14,15,16,17,18,19,))
-19|Vertex(id=19,value=17100.0, edges=(0,11,12,13,14,15,16,17,18,19,))
+0 0.0
+1 0.0
+2 100.0
+3 300.0
+4 600.0
+5 1000.0
+6 1500.0
+7 2100.0
+8 2800.0
+9 3600.0
+10 4500.0
+11 5500.0
+12 6600.0
+13 7800.0
+14 9100.0
+15 10500.0
+16 12000.0
+17 13600.0
+18 15300.0
+19 17100.0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal.result b/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal.result
index f2c31a6..b42462f 100644
--- a/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal.result
+++ b/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal.result
@@ -1,20 +1,20 @@
-0|Vertex(id=0,value=0.0, edges=(1,))
-1|Vertex(id=1,value=1.0, edges=(1,2,))
-2|Vertex(id=2,value=2.0, edges=(1,2,3,))
-3|Vertex(id=3,value=3.0, edges=(1,2,3,4,))
-4|Vertex(id=4,value=4.0, edges=(1,2,3,4,5,))
-5|Vertex(id=5,value=5.0, edges=(1,2,3,4,5,6,))
-6|Vertex(id=6,value=6.0, edges=(1,2,3,4,5,6,7,))
-7|Vertex(id=7,value=7.0, edges=(1,2,3,4,5,6,7,8,))
-8|Vertex(id=8,value=8.0, edges=(1,2,3,4,5,6,7,8,9,))
-9|Vertex(id=9,value=9.0, edges=(1,2,3,4,5,6,7,8,9,10,))
-10|Vertex(id=10,value=10.0, edges=(11,))
-11|Vertex(id=11,value=11.0, edges=(11,12,))
-12|Vertex(id=12,value=12.0, edges=(11,12,13,))
-13|Vertex(id=13,value=13.0, edges=(11,12,13,14,))
-14|Vertex(id=14,value=14.0, edges=(11,12,13,14,15,))
-15|Vertex(id=15,value=15.0, edges=(11,12,13,14,15,16,))
-16|Vertex(id=16,value=16.0, edges=(11,12,13,14,15,16,17,))
-17|Vertex(id=17,value=17.0, edges=(11,12,13,14,15,16,17,18,))
-18|Vertex(id=18,value=18.0, edges=(11,12,13,14,15,16,17,18,19,))
-19|Vertex(id=19,value=19.0, edges=(0,11,12,13,14,15,16,17,18,19,))
+0 0.0
+1 1.0
+2 2.0
+3 3.0
+4 4.0
+5 5.0
+6 6.0
+7 7.0
+8 8.0
+9 9.0
+10 10.0
+11 11.0
+12 12.0
+13 13.0
+14 14.0
+15 15.0
+16 16.0
+17 17.0
+18 18.0
+19 19.0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result
new file mode 100644
index 0000000..4818e13
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result
@@ -0,0 +1,7 @@
+1 3
+2 2
+3 0
+4 0
+5 1
+6 0
+7 0
diff --git a/pregelix/pregelix-example/src/test/resources/hadoop/conf/log4j.properties b/pregelix/pregelix-example/src/test/resources/hadoop/conf/log4j.properties
index d5e6004..3335964 100755
--- a/pregelix/pregelix-example/src/test/resources/hadoop/conf/log4j.properties
+++ b/pregelix/pregelix-example/src/test/resources/hadoop/conf/log4j.properties
@@ -76,7 +76,7 @@
# FSNamesystem Audit logging
# All audit events are logged at INFO level
#
-log4j.logger.org.apache.hadoop.fs.FSNamesystem.audit=WARN
+log4j.logger.org.apache.hadoop=FATAL
# Custom Logging levels
diff --git a/pregelix/pregelix-example/src/test/resources/hadoop/conf/mapred-site.xml b/pregelix/pregelix-example/src/test/resources/hadoop/conf/mapred-site.xml
index 1b9a4d6..71450f1 100644
--- a/pregelix/pregelix-example/src/test/resources/hadoop/conf/mapred-site.xml
+++ b/pregelix/pregelix-example/src/test/resources/hadoop/conf/mapred-site.xml
@@ -18,8 +18,8 @@
<value>20</value>
</property>
<property>
- <name>mapred.min.split.size</name>
- <value>65536</value>
+ <name>mapred.max.split.size</name>
+ <value>128</value>
</property>
</configuration>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/GraphMutation.xml b/pregelix/pregelix-example/src/test/resources/jobs/GraphMutation.xml
new file mode 100644
index 0000000..9f51f6d
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/jobs/GraphMutation.xml
@@ -0,0 +1,141 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
+<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
+<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
+<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
+<property><name>mapred.jobtracker.instrumentation</name><value>org.apache.hadoop.mapred.JobTrackerMetricsInst</value></property>
+<property><name>mapred.skip.reduce.auto.incr.proc.count</name><value>true</value></property>
+<property><name>fs.hsftp.impl</name><value>org.apache.hadoop.hdfs.HsftpFileSystem</value></property>
+<property><name>mapred.input.dir</name><value>file:/webmap</value></property>
+<property><name>mapred.submit.replication</name><value>10</value></property>
+<property><name>ipc.server.tcpnodelay</name><value>false</value></property>
+<property><name>fs.checkpoint.dir</name><value>${hadoop.tmp.dir}/dfs/namesecondary</value></property>
+<property><name>mapred.output.compression.type</name><value>RECORD</value></property>
+<property><name>mapred.job.shuffle.merge.percent</name><value>0.66</value></property>
+<property><name>mapred.child.java.opts</name><value>-Xmx200m</value></property>
+<property><name>mapred.queue.default.acl-submit-job</name><value>*</value></property>
+<property><name>keep.failed.task.files</name><value>false</value></property>
+<property><name>mapred.jobtracker.job.history.block.size</name><value>3145728</value></property>
+<property><name>io.bytes.per.checksum</name><value>512</value></property>
+<property><name>mapred.task.tracker.report.address</name><value>127.0.0.1:0</value></property>
+<property><name>hadoop.util.hash.type</name><value>murmur</value></property>
+<property><name>fs.hdfs.impl</name><value>org.apache.hadoop.hdfs.DistributedFileSystem</value></property>
+<property><name>fs.ramfs.impl</name><value>org.apache.hadoop.fs.InMemoryFileSystem</value></property>
+<property><name>mapred.jobtracker.restart.recover</name><value>false</value></property>
+<property><name>fs.hftp.impl</name><value>org.apache.hadoop.hdfs.HftpFileSystem</value></property>
+<property><name>fs.checkpoint.period</name><value>3600</value></property>
+<property><name>mapred.child.tmp</name><value>./tmp</value></property>
+<property><name>mapred.local.dir.minspacekill</name><value>0</value></property>
+<property><name>map.sort.class</name><value>org.apache.hadoop.util.QuickSort</value></property>
+<property><name>hadoop.logfile.count</name><value>10</value></property>
+<property><name>ipc.client.connection.maxidletime</name><value>10000</value></property>
+<property><name>mapred.output.dir</name><value>/result</value></property>
+<property><name>io.map.index.skip</name><value>0</value></property>
+<property><name>mapred.tasktracker.expiry.interval</name><value>600000</value></property>
+<property><name>mapred.output.compress</name><value>false</value></property>
+<property><name>io.seqfile.lazydecompress</name><value>true</value></property>
+<property><name>mapred.reduce.parallel.copies</name><value>5</value></property>
+<property><name>fs.checkpoint.size</name><value>67108864</value></property>
+<property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
+<property><name>mapred.job.name</name><value>Graph Mutation</value></property>
+<property><name>local.cache.size</name><value>10737418240</value></property>
+<property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
+<property><name>mapred.userlog.limit.kb</name><value>0</value></property>
+<property><name>fs.file.impl</name><value>org.apache.hadoop.fs.LocalFileSystem</value></property>
+<property><name>mapred.task.tracker.http.address</name><value>0.0.0.0:50060</value></property>
+<property><name>mapred.task.timeout</name><value>600000</value></property>
+<property><name>fs.kfs.impl</name><value>org.apache.hadoop.fs.kfs.KosmosFileSystem</value></property>
+<property><name>mapred.max.tracker.blacklists</name><value>4</value></property>
+<property><name>fs.s3.buffer.dir</name><value>${hadoop.tmp.dir}/s3</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.dir</name><value>/jobtracker/jobsInfo</value></property>
+<property><name>ipc.client.kill.max</name><value>10</value></property>
+<property><name>mapred.tasktracker.instrumentation</name><value>org.apache.hadoop.mapred.TaskTrackerMetricsInst</value></property>
+<property><name>mapred.reduce.tasks.speculative.execution</name><value>true</value></property>
+<property><name>io.sort.record.percent</name><value>0.05</value></property>
+<property><name>hadoop.security.authorization</name><value>false</value></property>
+<property><name>mapred.max.tracker.failures</name><value>4</value></property>
+<property><name>mapred.jobtracker.taskScheduler</name><value>org.apache.hadoop.mapred.JobQueueTaskScheduler</value></property>
+<property><name>pregelix.numVertices</name><value>20</value></property>
+<property><name>mapred.tasktracker.dns.interface</name><value>default</value></property>
+<property><name>mapred.map.tasks</name><value>2</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.hours</name><value>0</value></property>
+<property><name>fs.s3.sleepTimeSeconds</name><value>10</value></property>
+<property><name>fs.default.name</name><value>file:///</value></property>
+<property><name>tasktracker.http.threads</name><value>40</value></property>
+<property><name>mapred.tasktracker.taskmemorymanager.monitoring-interval</name><value>5000</value></property>
+<property><name>hadoop.rpc.socket.factory.class.default</name><value>org.apache.hadoop.net.StandardSocketFactory</value></property>
+<property><name>mapred.reduce.tasks</name><value>1</value></property>
+<property><name>topology.node.switch.mapping.impl</name><value>org.apache.hadoop.net.ScriptBasedMapping</value></property>
+<property><name>pregelix.vertexClass</name><value>edu.uci.ics.pregelix.example.GraphMutationVertex</value></property>
+<property><name>mapred.skip.reduce.max.skip.groups</name><value>0</value></property>
+<property><name>io.file.buffer.size</name><value>4096</value></property>
+<property><name>mapred.jobtracker.maxtasks.per.job</name><value>-1</value></property>
+<property><name>mapred.tasktracker.indexcache.mb</name><value>10</value></property>
+<property><name>mapred.tasktracker.map.tasks.maximum</name><value>2</value></property>
+<property><name>fs.har.impl.disable.cache</name><value>true</value></property>
+<property><name>mapred.task.profile.maps</name><value>0-2</value></property>
+<property><name>hadoop.native.lib</name><value>true</value></property>
+<property><name>fs.s3.block.size</name><value>67108864</value></property>
+<property><name>mapred.job.reuse.jvm.num.tasks</name><value>1</value></property>
+<property><name>mapred.job.tracker.http.address</name><value>0.0.0.0:50030</value></property>
+<property><name>mapred.tasktracker.reduce.tasks.maximum</name><value>2</value></property>
+<property><name>io.compression.codecs</name><value>org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec</value></property>
+<property><name>mapred.job.shuffle.input.buffer.percent</name><value>0.70</value></property>
+<property><name>io.seqfile.compress.blocksize</name><value>1000000</value></property>
+<property><name>mapred.queue.names</name><value>default</value></property>
+<property><name>fs.har.impl</name><value>org.apache.hadoop.fs.HarFileSystem</value></property>
+<property><name>io.mapfile.bloom.error.rate</name><value>0.005</value></property>
+<property><name>mapred.job.tracker</name><value>local</value></property>
+<property><name>io.skip.checksum.errors</name><value>false</value></property>
+<property><name>mapred.reduce.max.attempts</name><value>4</value></property>
+<property><name>fs.s3.maxRetries</name><value>4</value></property>
+<property><name>ipc.server.listen.queue.size</name><value>128</value></property>
+<property><name>fs.trash.interval</name><value>0</value></property>
+<property><name>mapred.local.dir.minspacestart</name><value>0</value></property>
+<property><name>fs.s3.impl</name><value>org.apache.hadoop.fs.s3.S3FileSystem</value></property>
+<property><name>io.seqfile.sorter.recordlimit</name><value>1000000</value></property>
+<property><name>io.mapfile.bloom.size</name><value>1048576</value></property>
+<property><name>io.sort.mb</name><value>100</value></property>
+<property><name>mapred.local.dir</name><value>${hadoop.tmp.dir}/mapred/local</value></property>
+<property><name>io.sort.factor</name><value>10</value></property>
+<property><name>mapred.task.profile</name><value>false</value></property>
+<property><name>job.end.retry.interval</name><value>30000</value></property>
+<property><name>mapred.tasktracker.procfsbasedprocesstree.sleeptime-before-sigkill</name><value>5000</value></property>
+<property><name>mapred.jobtracker.completeuserjobs.maximum</name><value>100</value></property>
+<property><name>mapred.task.profile.reduces</name><value>0-2</value></property>
+<property><name>webinterface.private.actions</name><value>false</value></property>
+<property><name>hadoop.tmp.dir</name><value>/tmp/hadoop-${user.name}</value></property>
+<property><name>mapred.output.compression.codec</name><value>org.apache.hadoop.io.compress.DefaultCodec</value></property>
+<property><name>mapred.skip.attempts.to.start.skipping</name><value>2</value></property>
+<property><name>mapred.temp.dir</name><value>${hadoop.tmp.dir}/mapred/temp</value></property>
+<property><name>mapred.merge.recordsBeforeProgress</name><value>10000</value></property>
+<property><name>mapred.map.output.compression.codec</name><value>org.apache.hadoop.io.compress.DefaultCodec</value></property>
+<property><name>mapred.compress.map.output</name><value>false</value></property>
+<property><name>io.sort.spill.percent</name><value>0.80</value></property>
+<property><name>fs.checkpoint.edits.dir</name><value>${fs.checkpoint.dir}</value></property>
+<property><name>mapred.userlog.retain.hours</name><value>24</value></property>
+<property><name>mapred.system.dir</name><value>${hadoop.tmp.dir}/mapred/system</value></property>
+<property><name>mapred.line.input.format.linespermap</name><value>1</value></property>
+<property><name>job.end.retry.attempts</name><value>0</value></property>
+<property><name>ipc.client.idlethreshold</name><value>4000</value></property>
+<property><name>pregelix.vertexOutputFormatClass</name><value>edu.uci.ics.pregelix.example.GraphMutationVertex$SimpleGraphMutationVertexOutputFormat</value></property>
+<property><name>mapred.reduce.copy.backoff</name><value>300</value></property>
+<property><name>mapred.map.tasks.speculative.execution</name><value>true</value></property>
+<property><name>mapred.inmem.merge.threshold</name><value>1000</value></property>
+<property><name>hadoop.logfile.size</name><value>10000000</value></property>
+<property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat</value></property>
+<property><name>mapred.job.queue.name</name><value>default</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.active</name><value>false</value></property>
+<property><name>mapred.reduce.slowstart.completed.maps</name><value>0.05</value></property>
+<property><name>topology.script.number.args</name><value>100</value></property>
+<property><name>mapred.skip.map.max.skip.records</name><value>0</value></property>
+<property><name>fs.ftp.impl</name><value>org.apache.hadoop.fs.ftp.FTPFileSystem</value></property>
+<property><name>mapred.task.cache.levels</name><value>2</value></property>
+<property><name>mapred.job.tracker.handler.count</name><value>10</value></property>
+<property><name>io.serializations</name><value>org.apache.hadoop.io.serializer.WritableSerialization</value></property>
+<property><name>ipc.client.connect.max.retries</name><value>10</value></property>
+<property><name>mapred.min.split.size</name><value>0</value></property>
+<property><name>mapred.map.max.attempts</name><value>4</value></property>
+<property><name>jobclient.output.filter</name><value>FAILED</value></property>
+<property><name>ipc.client.tcpnodelay</name><value>false</value></property>
+<property><name>mapred.acls.enabled</name><value>false</value></property>
+</configuration>
\ No newline at end of file
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique.xml b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique.xml
new file mode 100644
index 0000000..616c647
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique.xml
@@ -0,0 +1,142 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
+<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
+<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
+<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
+<property><name>mapred.jobtracker.instrumentation</name><value>org.apache.hadoop.mapred.JobTrackerMetricsInst</value></property>
+<property><name>mapred.skip.reduce.auto.incr.proc.count</name><value>true</value></property>
+<property><name>fs.hsftp.impl</name><value>org.apache.hadoop.hdfs.HsftpFileSystem</value></property>
+<property><name>mapred.input.dir</name><value>file:/clique</value></property>
+<property><name>mapred.submit.replication</name><value>10</value></property>
+<property><name>ipc.server.tcpnodelay</name><value>false</value></property>
+<property><name>fs.checkpoint.dir</name><value>${hadoop.tmp.dir}/dfs/namesecondary</value></property>
+<property><name>mapred.output.compression.type</name><value>RECORD</value></property>
+<property><name>mapred.job.shuffle.merge.percent</name><value>0.66</value></property>
+<property><name>mapred.child.java.opts</name><value>-Xmx200m</value></property>
+<property><name>mapred.queue.default.acl-submit-job</name><value>*</value></property>
+<property><name>keep.failed.task.files</name><value>false</value></property>
+<property><name>mapred.jobtracker.job.history.block.size</name><value>3145728</value></property>
+<property><name>io.bytes.per.checksum</name><value>512</value></property>
+<property><name>mapred.task.tracker.report.address</name><value>127.0.0.1:0</value></property>
+<property><name>hadoop.util.hash.type</name><value>murmur</value></property>
+<property><name>fs.hdfs.impl</name><value>org.apache.hadoop.hdfs.DistributedFileSystem</value></property>
+<property><name>fs.ramfs.impl</name><value>org.apache.hadoop.fs.InMemoryFileSystem</value></property>
+<property><name>mapred.jobtracker.restart.recover</name><value>false</value></property>
+<property><name>fs.hftp.impl</name><value>org.apache.hadoop.hdfs.HftpFileSystem</value></property>
+<property><name>fs.checkpoint.period</name><value>3600</value></property>
+<property><name>mapred.child.tmp</name><value>./tmp</value></property>
+<property><name>mapred.local.dir.minspacekill</name><value>0</value></property>
+<property><name>map.sort.class</name><value>org.apache.hadoop.util.QuickSort</value></property>
+<property><name>hadoop.logfile.count</name><value>10</value></property>
+<property><name>ipc.client.connection.maxidletime</name><value>10000</value></property>
+<property><name>mapred.output.dir</name><value>/resultclique</value></property>
+<property><name>io.map.index.skip</name><value>0</value></property>
+<property><name>mapred.tasktracker.expiry.interval</name><value>600000</value></property>
+<property><name>mapred.output.compress</name><value>false</value></property>
+<property><name>io.seqfile.lazydecompress</name><value>true</value></property>
+<property><name>mapred.reduce.parallel.copies</name><value>5</value></property>
+<property><name>fs.checkpoint.size</name><value>67108864</value></property>
+<property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
+<property><name>mapred.job.name</name><value>Maximal Clique</value></property>
+<property><name>local.cache.size</name><value>10737418240</value></property>
+<property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
+<property><name>mapred.userlog.limit.kb</name><value>0</value></property>
+<property><name>fs.file.impl</name><value>org.apache.hadoop.fs.LocalFileSystem</value></property>
+<property><name>mapred.task.tracker.http.address</name><value>0.0.0.0:50060</value></property>
+<property><name>mapred.task.timeout</name><value>600000</value></property>
+<property><name>fs.kfs.impl</name><value>org.apache.hadoop.fs.kfs.KosmosFileSystem</value></property>
+<property><name>mapred.max.tracker.blacklists</name><value>4</value></property>
+<property><name>fs.s3.buffer.dir</name><value>${hadoop.tmp.dir}/s3</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.dir</name><value>/jobtracker/jobsInfo</value></property>
+<property><name>ipc.client.kill.max</name><value>10</value></property>
+<property><name>mapred.tasktracker.instrumentation</name><value>org.apache.hadoop.mapred.TaskTrackerMetricsInst</value></property>
+<property><name>mapred.reduce.tasks.speculative.execution</name><value>true</value></property>
+<property><name>io.sort.record.percent</name><value>0.05</value></property>
+<property><name>hadoop.security.authorization</name><value>false</value></property>
+<property><name>mapred.max.tracker.failures</name><value>4</value></property>
+<property><name>mapred.jobtracker.taskScheduler</name><value>org.apache.hadoop.mapred.JobQueueTaskScheduler</value></property>
+<property><name>mapred.tasktracker.dns.interface</name><value>default</value></property>
+<property><name>mapred.map.tasks</name><value>2</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.hours</name><value>0</value></property>
+<property><name>fs.s3.sleepTimeSeconds</name><value>10</value></property>
+<property><name>fs.default.name</name><value>file:///</value></property>
+<property><name>tasktracker.http.threads</name><value>40</value></property>
+<property><name>mapred.tasktracker.taskmemorymanager.monitoring-interval</name><value>5000</value></property>
+<property><name>hadoop.rpc.socket.factory.class.default</name><value>org.apache.hadoop.net.StandardSocketFactory</value></property>
+<property><name>mapred.reduce.tasks</name><value>1</value></property>
+<property><name>topology.node.switch.mapping.impl</name><value>org.apache.hadoop.net.ScriptBasedMapping</value></property>
+<property><name>pregelix.vertexClass</name><value>edu.uci.ics.pregelix.example.maximalclique.MaximalCliqueVertex</value></property>
+<property><name>mapred.skip.reduce.max.skip.groups</name><value>0</value></property>
+<property><name>io.file.buffer.size</name><value>4096</value></property>
+<property><name>mapred.jobtracker.maxtasks.per.job</name><value>-1</value></property>
+<property><name>mapred.tasktracker.indexcache.mb</name><value>10</value></property>
+<property><name>mapred.tasktracker.map.tasks.maximum</name><value>2</value></property>
+<property><name>fs.har.impl.disable.cache</name><value>true</value></property>
+<property><name>mapred.task.profile.maps</name><value>0-2</value></property>
+<property><name>hadoop.native.lib</name><value>true</value></property>
+<property><name>fs.s3.block.size</name><value>67108864</value></property>
+<property><name>mapred.job.reuse.jvm.num.tasks</name><value>1</value></property>
+<property><name>mapred.job.tracker.http.address</name><value>0.0.0.0:50030</value></property>
+<property><name>mapred.tasktracker.reduce.tasks.maximum</name><value>2</value></property>
+<property><name>io.compression.codecs</name><value>org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec</value></property>
+<property><name>mapred.job.shuffle.input.buffer.percent</name><value>0.70</value></property>
+<property><name>io.seqfile.compress.blocksize</name><value>1000000</value></property>
+<property><name>mapred.queue.names</name><value>default</value></property>
+<property><name>fs.har.impl</name><value>org.apache.hadoop.fs.HarFileSystem</value></property>
+<property><name>io.mapfile.bloom.error.rate</name><value>0.005</value></property>
+<property><name>mapred.job.tracker</name><value>local</value></property>
+<property><name>io.skip.checksum.errors</name><value>false</value></property>
+<property><name>mapred.reduce.max.attempts</name><value>4</value></property>
+<property><name>fs.s3.maxRetries</name><value>4</value></property>
+<property><name>ipc.server.listen.queue.size</name><value>128</value></property>
+<property><name>fs.trash.interval</name><value>0</value></property>
+<property><name>mapred.local.dir.minspacestart</name><value>0</value></property>
+<property><name>fs.s3.impl</name><value>org.apache.hadoop.fs.s3.S3FileSystem</value></property>
+<property><name>io.seqfile.sorter.recordlimit</name><value>1000000</value></property>
+<property><name>io.mapfile.bloom.size</name><value>1048576</value></property>
+<property><name>io.sort.mb</name><value>100</value></property>
+<property><name>mapred.local.dir</name><value>${hadoop.tmp.dir}/mapred/local</value></property>
+<property><name>io.sort.factor</name><value>10</value></property>
+<property><name>mapred.task.profile</name><value>false</value></property>
+<property><name>job.end.retry.interval</name><value>30000</value></property>
+<property><name>mapred.tasktracker.procfsbasedprocesstree.sleeptime-before-sigkill</name><value>5000</value></property>
+<property><name>mapred.jobtracker.completeuserjobs.maximum</name><value>100</value></property>
+<property><name>mapred.task.profile.reduces</name><value>0-2</value></property>
+<property><name>webinterface.private.actions</name><value>false</value></property>
+<property><name>hadoop.tmp.dir</name><value>/tmp/hadoop-${user.name}</value></property>
+<property><name>mapred.output.compression.codec</name><value>org.apache.hadoop.io.compress.DefaultCodec</value></property>
+<property><name>mapred.skip.attempts.to.start.skipping</name><value>2</value></property>
+<property><name>mapred.temp.dir</name><value>${hadoop.tmp.dir}/mapred/temp</value></property>
+<property><name>mapred.merge.recordsBeforeProgress</name><value>10000</value></property>
+<property><name>mapred.map.output.compression.codec</name><value>org.apache.hadoop.io.compress.DefaultCodec</value></property>
+<property><name>mapred.compress.map.output</name><value>false</value></property>
+<property><name>io.sort.spill.percent</name><value>0.80</value></property>
+<property><name>fs.checkpoint.edits.dir</name><value>${fs.checkpoint.dir}</value></property>
+<property><name>mapred.userlog.retain.hours</name><value>24</value></property>
+<property><name>mapred.system.dir</name><value>${hadoop.tmp.dir}/mapred/system</value></property>
+<property><name>mapred.line.input.format.linespermap</name><value>1</value></property>
+<property><name>job.end.retry.attempts</name><value>0</value></property>
+<property><name>ipc.client.idlethreshold</name><value>4000</value></property>
+<property><name>pregelix.vertexOutputFormatClass</name><value>edu.uci.ics.pregelix.example.maximalclique.MaximalCliqueVertex$MaximalCliqueVertexOutputFormat</value></property>
+<property><name>mapred.reduce.copy.backoff</name><value>300</value></property>
+<property><name>mapred.map.tasks.speculative.execution</name><value>true</value></property>
+<property><name>mapred.inmem.merge.threshold</name><value>1000</value></property>
+<property><name>hadoop.logfile.size</name><value>10000000</value></property>
+<property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.maximalclique.TextMaximalCliqueInputFormat</value></property>
+<property><name>pregelix.aggregatorClass</name><value>edu.uci.ics.pregelix.example.maximalclique.MaximalCliqueAggregator</value></property>
+<property><name>mapred.job.queue.name</name><value>default</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.active</name><value>false</value></property>
+<property><name>pregelix.incStateLength</name><value>true</value></property>
+<property><name>mapred.reduce.slowstart.completed.maps</name><value>0.05</value></property>
+<property><name>topology.script.number.args</name><value>100</value></property>
+<property><name>mapred.skip.map.max.skip.records</name><value>0</value></property>
+<property><name>fs.ftp.impl</name><value>org.apache.hadoop.fs.ftp.FTPFileSystem</value></property>
+<property><name>mapred.task.cache.levels</name><value>2</value></property>
+<property><name>mapred.job.tracker.handler.count</name><value>10</value></property>
+<property><name>io.serializations</name><value>org.apache.hadoop.io.serializer.WritableSerialization</value></property>
+<property><name>ipc.client.connect.max.retries</name><value>10</value></property>
+<property><name>mapred.min.split.size</name><value>0</value></property>
+<property><name>mapred.map.max.attempts</name><value>4</value></property>
+<property><name>jobclient.output.filter</name><value>FAILED</value></property>
+<property><name>ipc.client.tcpnodelay</name><value>false</value></property>
+<property><name>mapred.acls.enabled</name><value>false</value></property>
+</configuration>
\ No newline at end of file
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml
index e425b38..744e5b0 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml
@@ -123,7 +123,7 @@
<property><name>mapred.map.tasks.speculative.execution</name><value>true</value></property>
<property><name>mapred.inmem.merge.threshold</name><value>1000</value></property>
<property><name>hadoop.logfile.size</name><value>10000000</value></property>
-<property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.PageRankVertex$SimplePageRankVertexInputFormat</value></property>
+<property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.PageRankVertex$SimulatedPageRankVertexInputFormat</value></property>
<property><name>mapred.job.queue.name</name><value>default</value></property>
<property><name>mapred.job.tracker.persist.jobstatus.active</name><value>false</value></property>
<property><name>mapred.reduce.slowstart.completed.maps</name><value>0.05</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml
new file mode 100644
index 0000000..c1a04ae
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml
@@ -0,0 +1,143 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
+<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
+<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
+<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
+<property><name>mapred.jobtracker.instrumentation</name><value>org.apache.hadoop.mapred.JobTrackerMetricsInst</value></property>
+<property><name>mapred.skip.reduce.auto.incr.proc.count</name><value>true</value></property>
+<property><name>fs.hsftp.impl</name><value>org.apache.hadoop.hdfs.HsftpFileSystem</value></property>
+<property><name>mapred.input.dir</name><value>file:/webmap</value></property>
+<property><name>mapred.submit.replication</name><value>10</value></property>
+<property><name>ipc.server.tcpnodelay</name><value>false</value></property>
+<property><name>fs.checkpoint.dir</name><value>${hadoop.tmp.dir}/dfs/namesecondary</value></property>
+<property><name>mapred.output.compression.type</name><value>RECORD</value></property>
+<property><name>mapred.job.shuffle.merge.percent</name><value>0.66</value></property>
+<property><name>mapred.child.java.opts</name><value>-Xmx200m</value></property>
+<property><name>mapred.queue.default.acl-submit-job</name><value>*</value></property>
+<property><name>keep.failed.task.files</name><value>false</value></property>
+<property><name>mapred.jobtracker.job.history.block.size</name><value>3145728</value></property>
+<property><name>io.bytes.per.checksum</name><value>512</value></property>
+<property><name>mapred.task.tracker.report.address</name><value>127.0.0.1:0</value></property>
+<property><name>hadoop.util.hash.type</name><value>murmur</value></property>
+<property><name>fs.hdfs.impl</name><value>org.apache.hadoop.hdfs.DistributedFileSystem</value></property>
+<property><name>fs.ramfs.impl</name><value>org.apache.hadoop.fs.InMemoryFileSystem</value></property>
+<property><name>mapred.jobtracker.restart.recover</name><value>false</value></property>
+<property><name>fs.hftp.impl</name><value>org.apache.hadoop.hdfs.HftpFileSystem</value></property>
+<property><name>fs.checkpoint.period</name><value>3600</value></property>
+<property><name>mapred.child.tmp</name><value>./tmp</value></property>
+<property><name>mapred.local.dir.minspacekill</name><value>0</value></property>
+<property><name>map.sort.class</name><value>org.apache.hadoop.util.QuickSort</value></property>
+<property><name>hadoop.logfile.count</name><value>10</value></property>
+<property><name>ipc.client.connection.maxidletime</name><value>10000</value></property>
+<property><name>mapred.output.dir</name><value>/result</value></property>
+<property><name>io.map.index.skip</name><value>0</value></property>
+<property><name>mapred.tasktracker.expiry.interval</name><value>600000</value></property>
+<property><name>mapred.output.compress</name><value>false</value></property>
+<property><name>io.seqfile.lazydecompress</name><value>true</value></property>
+<property><name>mapred.reduce.parallel.copies</name><value>5</value></property>
+<property><name>fs.checkpoint.size</name><value>67108864</value></property>
+<property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
+<property><name>mapred.job.name</name><value>PageRank</value></property>
+<property><name>local.cache.size</name><value>10737418240</value></property>
+<property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
+<property><name>mapred.userlog.limit.kb</name><value>0</value></property>
+<property><name>fs.file.impl</name><value>org.apache.hadoop.fs.LocalFileSystem</value></property>
+<property><name>mapred.task.tracker.http.address</name><value>0.0.0.0:50060</value></property>
+<property><name>mapred.task.timeout</name><value>600000</value></property>
+<property><name>fs.kfs.impl</name><value>org.apache.hadoop.fs.kfs.KosmosFileSystem</value></property>
+<property><name>mapred.max.tracker.blacklists</name><value>4</value></property>
+<property><name>fs.s3.buffer.dir</name><value>${hadoop.tmp.dir}/s3</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.dir</name><value>/jobtracker/jobsInfo</value></property>
+<property><name>ipc.client.kill.max</name><value>10</value></property>
+<property><name>mapred.tasktracker.instrumentation</name><value>org.apache.hadoop.mapred.TaskTrackerMetricsInst</value></property>
+<property><name>mapred.reduce.tasks.speculative.execution</name><value>true</value></property>
+<property><name>io.sort.record.percent</name><value>0.05</value></property>
+<property><name>hadoop.security.authorization</name><value>false</value></property>
+<property><name>mapred.max.tracker.failures</name><value>4</value></property>
+<property><name>mapred.jobtracker.taskScheduler</name><value>org.apache.hadoop.mapred.JobQueueTaskScheduler</value></property>
+<property><name>pregelix.numVertices</name><value>20</value></property>
+<property><name>mapred.tasktracker.dns.interface</name><value>default</value></property>
+<property><name>mapred.map.tasks</name><value>2</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.hours</name><value>0</value></property>
+<property><name>fs.s3.sleepTimeSeconds</name><value>10</value></property>
+<property><name>fs.default.name</name><value>file:///</value></property>
+<property><name>tasktracker.http.threads</name><value>40</value></property>
+<property><name>mapred.tasktracker.taskmemorymanager.monitoring-interval</name><value>5000</value></property>
+<property><name>hadoop.rpc.socket.factory.class.default</name><value>org.apache.hadoop.net.StandardSocketFactory</value></property>
+<property><name>mapred.reduce.tasks</name><value>1</value></property>
+<property><name>topology.node.switch.mapping.impl</name><value>org.apache.hadoop.net.ScriptBasedMapping</value></property>
+<property><name>pregelix.vertexClass</name><value>edu.uci.ics.pregelix.example.PageRankVertex</value></property>
+<property><name>mapred.skip.reduce.max.skip.groups</name><value>0</value></property>
+<property><name>io.file.buffer.size</name><value>4096</value></property>
+<property><name>mapred.jobtracker.maxtasks.per.job</name><value>-1</value></property>
+<property><name>mapred.tasktracker.indexcache.mb</name><value>10</value></property>
+<property><name>mapred.tasktracker.map.tasks.maximum</name><value>2</value></property>
+<property><name>fs.har.impl.disable.cache</name><value>true</value></property>
+<property><name>mapred.task.profile.maps</name><value>0-2</value></property>
+<property><name>hadoop.native.lib</name><value>true</value></property>
+<property><name>fs.s3.block.size</name><value>67108864</value></property>
+<property><name>mapred.job.reuse.jvm.num.tasks</name><value>1</value></property>
+<property><name>mapred.job.tracker.http.address</name><value>0.0.0.0:50030</value></property>
+<property><name>mapred.tasktracker.reduce.tasks.maximum</name><value>2</value></property>
+<property><name>io.compression.codecs</name><value>org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec</value></property>
+<property><name>mapred.job.shuffle.input.buffer.percent</name><value>0.70</value></property>
+<property><name>io.seqfile.compress.blocksize</name><value>1000000</value></property>
+<property><name>mapred.queue.names</name><value>default</value></property>
+<property><name>fs.har.impl</name><value>org.apache.hadoop.fs.HarFileSystem</value></property>
+<property><name>io.mapfile.bloom.error.rate</name><value>0.005</value></property>
+<property><name>mapred.job.tracker</name><value>local</value></property>
+<property><name>io.skip.checksum.errors</name><value>false</value></property>
+<property><name>mapred.reduce.max.attempts</name><value>4</value></property>
+<property><name>fs.s3.maxRetries</name><value>4</value></property>
+<property><name>ipc.server.listen.queue.size</name><value>128</value></property>
+<property><name>fs.trash.interval</name><value>0</value></property>
+<property><name>mapred.local.dir.minspacestart</name><value>0</value></property>
+<property><name>fs.s3.impl</name><value>org.apache.hadoop.fs.s3.S3FileSystem</value></property>
+<property><name>io.seqfile.sorter.recordlimit</name><value>1000000</value></property>
+<property><name>io.mapfile.bloom.size</name><value>1048576</value></property>
+<property><name>io.sort.mb</name><value>100</value></property>
+<property><name>mapred.local.dir</name><value>${hadoop.tmp.dir}/mapred/local</value></property>
+<property><name>io.sort.factor</name><value>10</value></property>
+<property><name>mapred.task.profile</name><value>false</value></property>
+<property><name>job.end.retry.interval</name><value>30000</value></property>
+<property><name>mapred.tasktracker.procfsbasedprocesstree.sleeptime-before-sigkill</name><value>5000</value></property>
+<property><name>mapred.jobtracker.completeuserjobs.maximum</name><value>100</value></property>
+<property><name>mapred.task.profile.reduces</name><value>0-2</value></property>
+<property><name>webinterface.private.actions</name><value>false</value></property>
+<property><name>hadoop.tmp.dir</name><value>/tmp/hadoop-${user.name}</value></property>
+<property><name>pregelix.combinerClass</name><value>edu.uci.ics.pregelix.example.PageRankVertex$SimpleSumCombiner</value></property>
+<property><name>mapred.output.compression.codec</name><value>org.apache.hadoop.io.compress.DefaultCodec</value></property>
+<property><name>mapred.skip.attempts.to.start.skipping</name><value>2</value></property>
+<property><name>mapred.temp.dir</name><value>${hadoop.tmp.dir}/mapred/temp</value></property>
+<property><name>mapred.merge.recordsBeforeProgress</name><value>10000</value></property>
+<property><name>mapred.map.output.compression.codec</name><value>org.apache.hadoop.io.compress.DefaultCodec</value></property>
+<property><name>mapred.compress.map.output</name><value>false</value></property>
+<property><name>io.sort.spill.percent</name><value>0.80</value></property>
+<property><name>fs.checkpoint.edits.dir</name><value>${fs.checkpoint.dir}</value></property>
+<property><name>mapred.userlog.retain.hours</name><value>24</value></property>
+<property><name>mapred.system.dir</name><value>${hadoop.tmp.dir}/mapred/system</value></property>
+<property><name>mapred.line.input.format.linespermap</name><value>1</value></property>
+<property><name>job.end.retry.attempts</name><value>0</value></property>
+<property><name>ipc.client.idlethreshold</name><value>4000</value></property>
+<property><name>pregelix.vertexOutputFormatClass</name><value>edu.uci.ics.pregelix.example.PageRankVertex$SimplePageRankVertexOutputFormat</value></property>
+<property><name>mapred.reduce.copy.backoff</name><value>300</value></property>
+<property><name>mapred.map.tasks.speculative.execution</name><value>true</value></property>
+<property><name>mapred.inmem.merge.threshold</name><value>1000</value></property>
+<property><name>hadoop.logfile.size</name><value>10000000</value></property>
+<property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat</value></property>
+<property><name>mapred.job.queue.name</name><value>default</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.active</name><value>false</value></property>
+<property><name>pregelix.incStateLength</name><value>true</value></property>
+<property><name>mapred.reduce.slowstart.completed.maps</name><value>0.05</value></property>
+<property><name>topology.script.number.args</name><value>100</value></property>
+<property><name>mapred.skip.map.max.skip.records</name><value>0</value></property>
+<property><name>fs.ftp.impl</name><value>org.apache.hadoop.fs.ftp.FTPFileSystem</value></property>
+<property><name>mapred.task.cache.levels</name><value>2</value></property>
+<property><name>mapred.job.tracker.handler.count</name><value>10</value></property>
+<property><name>io.serializations</name><value>org.apache.hadoop.io.serializer.WritableSerialization</value></property>
+<property><name>ipc.client.connect.max.retries</name><value>10</value></property>
+<property><name>mapred.min.split.size</name><value>0</value></property>
+<property><name>mapred.map.max.attempts</name><value>4</value></property>
+<property><name>jobclient.output.filter</name><value>FAILED</value></property>
+<property><name>ipc.client.tcpnodelay</name><value>false</value></property>
+<property><name>mapred.acls.enabled</name><value>false</value></property>
+</configuration>
\ No newline at end of file
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ShortestPaths.xml b/pregelix/pregelix-example/src/test/resources/jobs/ShortestPaths.xml
index 3719247..9e791e2 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ShortestPaths.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ShortestPaths.xml
@@ -124,7 +124,7 @@
<property><name>mapred.map.tasks.speculative.execution</name><value>true</value></property>
<property><name>mapred.inmem.merge.threshold</name><value>1000</value></property>
<property><name>hadoop.logfile.size</name><value>10000000</value></property>
-<property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.PageRankVertex$SimplePageRankVertexInputFormat</value></property>
+<property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.PageRankVertex$SimulatedPageRankVertexInputFormat</value></property>
<property><name>mapred.job.queue.name</name><value>default</value></property>
<property><name>mapred.job.tracker.persist.jobstatus.active</name><value>false</value></property>
<property><name>mapred.reduce.slowstart.completed.maps</name><value>0.05</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/TriangleCounting.xml b/pregelix/pregelix-example/src/test/resources/jobs/TriangleCounting.xml
new file mode 100644
index 0000000..0f44f4d
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/jobs/TriangleCounting.xml
@@ -0,0 +1,141 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
+<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
+<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
+<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
+<property><name>mapred.jobtracker.instrumentation</name><value>org.apache.hadoop.mapred.JobTrackerMetricsInst</value></property>
+<property><name>mapred.skip.reduce.auto.incr.proc.count</name><value>true</value></property>
+<property><name>fs.hsftp.impl</name><value>org.apache.hadoop.hdfs.HsftpFileSystem</value></property>
+<property><name>mapred.input.dir</name><value>file:/clique</value></property>
+<property><name>mapred.submit.replication</name><value>10</value></property>
+<property><name>ipc.server.tcpnodelay</name><value>false</value></property>
+<property><name>fs.checkpoint.dir</name><value>${hadoop.tmp.dir}/dfs/namesecondary</value></property>
+<property><name>mapred.output.compression.type</name><value>RECORD</value></property>
+<property><name>mapred.job.shuffle.merge.percent</name><value>0.66</value></property>
+<property><name>mapred.child.java.opts</name><value>-Xmx200m</value></property>
+<property><name>mapred.queue.default.acl-submit-job</name><value>*</value></property>
+<property><name>keep.failed.task.files</name><value>false</value></property>
+<property><name>mapred.jobtracker.job.history.block.size</name><value>3145728</value></property>
+<property><name>io.bytes.per.checksum</name><value>512</value></property>
+<property><name>mapred.task.tracker.report.address</name><value>127.0.0.1:0</value></property>
+<property><name>hadoop.util.hash.type</name><value>murmur</value></property>
+<property><name>fs.hdfs.impl</name><value>org.apache.hadoop.hdfs.DistributedFileSystem</value></property>
+<property><name>fs.ramfs.impl</name><value>org.apache.hadoop.fs.InMemoryFileSystem</value></property>
+<property><name>mapred.jobtracker.restart.recover</name><value>false</value></property>
+<property><name>fs.hftp.impl</name><value>org.apache.hadoop.hdfs.HftpFileSystem</value></property>
+<property><name>fs.checkpoint.period</name><value>3600</value></property>
+<property><name>mapred.child.tmp</name><value>./tmp</value></property>
+<property><name>mapred.local.dir.minspacekill</name><value>0</value></property>
+<property><name>map.sort.class</name><value>org.apache.hadoop.util.QuickSort</value></property>
+<property><name>hadoop.logfile.count</name><value>10</value></property>
+<property><name>ipc.client.connection.maxidletime</name><value>10000</value></property>
+<property><name>mapred.output.dir</name><value>/resultclique</value></property>
+<property><name>io.map.index.skip</name><value>0</value></property>
+<property><name>mapred.tasktracker.expiry.interval</name><value>600000</value></property>
+<property><name>mapred.output.compress</name><value>false</value></property>
+<property><name>io.seqfile.lazydecompress</name><value>true</value></property>
+<property><name>mapred.reduce.parallel.copies</name><value>5</value></property>
+<property><name>fs.checkpoint.size</name><value>67108864</value></property>
+<property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
+<property><name>mapred.job.name</name><value>Triangle Counting</value></property>
+<property><name>local.cache.size</name><value>10737418240</value></property>
+<property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
+<property><name>mapred.userlog.limit.kb</name><value>0</value></property>
+<property><name>fs.file.impl</name><value>org.apache.hadoop.fs.LocalFileSystem</value></property>
+<property><name>mapred.task.tracker.http.address</name><value>0.0.0.0:50060</value></property>
+<property><name>mapred.task.timeout</name><value>600000</value></property>
+<property><name>fs.kfs.impl</name><value>org.apache.hadoop.fs.kfs.KosmosFileSystem</value></property>
+<property><name>mapred.max.tracker.blacklists</name><value>4</value></property>
+<property><name>fs.s3.buffer.dir</name><value>${hadoop.tmp.dir}/s3</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.dir</name><value>/jobtracker/jobsInfo</value></property>
+<property><name>ipc.client.kill.max</name><value>10</value></property>
+<property><name>mapred.tasktracker.instrumentation</name><value>org.apache.hadoop.mapred.TaskTrackerMetricsInst</value></property>
+<property><name>mapred.reduce.tasks.speculative.execution</name><value>true</value></property>
+<property><name>io.sort.record.percent</name><value>0.05</value></property>
+<property><name>hadoop.security.authorization</name><value>false</value></property>
+<property><name>mapred.max.tracker.failures</name><value>4</value></property>
+<property><name>mapred.jobtracker.taskScheduler</name><value>org.apache.hadoop.mapred.JobQueueTaskScheduler</value></property>
+<property><name>mapred.tasktracker.dns.interface</name><value>default</value></property>
+<property><name>mapred.map.tasks</name><value>2</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.hours</name><value>0</value></property>
+<property><name>fs.s3.sleepTimeSeconds</name><value>10</value></property>
+<property><name>fs.default.name</name><value>file:///</value></property>
+<property><name>tasktracker.http.threads</name><value>40</value></property>
+<property><name>mapred.tasktracker.taskmemorymanager.monitoring-interval</name><value>5000</value></property>
+<property><name>hadoop.rpc.socket.factory.class.default</name><value>org.apache.hadoop.net.StandardSocketFactory</value></property>
+<property><name>mapred.reduce.tasks</name><value>1</value></property>
+<property><name>topology.node.switch.mapping.impl</name><value>org.apache.hadoop.net.ScriptBasedMapping</value></property>
+<property><name>pregelix.vertexClass</name><value>edu.uci.ics.pregelix.example.trianglecounting.TriangleCountingVertex</value></property>
+<property><name>mapred.skip.reduce.max.skip.groups</name><value>0</value></property>
+<property><name>io.file.buffer.size</name><value>4096</value></property>
+<property><name>mapred.jobtracker.maxtasks.per.job</name><value>-1</value></property>
+<property><name>mapred.tasktracker.indexcache.mb</name><value>10</value></property>
+<property><name>mapred.tasktracker.map.tasks.maximum</name><value>2</value></property>
+<property><name>fs.har.impl.disable.cache</name><value>true</value></property>
+<property><name>mapred.task.profile.maps</name><value>0-2</value></property>
+<property><name>hadoop.native.lib</name><value>true</value></property>
+<property><name>fs.s3.block.size</name><value>67108864</value></property>
+<property><name>mapred.job.reuse.jvm.num.tasks</name><value>1</value></property>
+<property><name>mapred.job.tracker.http.address</name><value>0.0.0.0:50030</value></property>
+<property><name>mapred.tasktracker.reduce.tasks.maximum</name><value>2</value></property>
+<property><name>io.compression.codecs</name><value>org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec</value></property>
+<property><name>mapred.job.shuffle.input.buffer.percent</name><value>0.70</value></property>
+<property><name>io.seqfile.compress.blocksize</name><value>1000000</value></property>
+<property><name>mapred.queue.names</name><value>default</value></property>
+<property><name>fs.har.impl</name><value>org.apache.hadoop.fs.HarFileSystem</value></property>
+<property><name>io.mapfile.bloom.error.rate</name><value>0.005</value></property>
+<property><name>mapred.job.tracker</name><value>local</value></property>
+<property><name>io.skip.checksum.errors</name><value>false</value></property>
+<property><name>mapred.reduce.max.attempts</name><value>4</value></property>
+<property><name>fs.s3.maxRetries</name><value>4</value></property>
+<property><name>ipc.server.listen.queue.size</name><value>128</value></property>
+<property><name>fs.trash.interval</name><value>0</value></property>
+<property><name>mapred.local.dir.minspacestart</name><value>0</value></property>
+<property><name>fs.s3.impl</name><value>org.apache.hadoop.fs.s3.S3FileSystem</value></property>
+<property><name>io.seqfile.sorter.recordlimit</name><value>1000000</value></property>
+<property><name>io.mapfile.bloom.size</name><value>1048576</value></property>
+<property><name>io.sort.mb</name><value>100</value></property>
+<property><name>mapred.local.dir</name><value>${hadoop.tmp.dir}/mapred/local</value></property>
+<property><name>io.sort.factor</name><value>10</value></property>
+<property><name>mapred.task.profile</name><value>false</value></property>
+<property><name>job.end.retry.interval</name><value>30000</value></property>
+<property><name>mapred.tasktracker.procfsbasedprocesstree.sleeptime-before-sigkill</name><value>5000</value></property>
+<property><name>mapred.jobtracker.completeuserjobs.maximum</name><value>100</value></property>
+<property><name>mapred.task.profile.reduces</name><value>0-2</value></property>
+<property><name>webinterface.private.actions</name><value>false</value></property>
+<property><name>hadoop.tmp.dir</name><value>/tmp/hadoop-${user.name}</value></property>
+<property><name>mapred.output.compression.codec</name><value>org.apache.hadoop.io.compress.DefaultCodec</value></property>
+<property><name>mapred.skip.attempts.to.start.skipping</name><value>2</value></property>
+<property><name>mapred.temp.dir</name><value>${hadoop.tmp.dir}/mapred/temp</value></property>
+<property><name>mapred.merge.recordsBeforeProgress</name><value>10000</value></property>
+<property><name>mapred.map.output.compression.codec</name><value>org.apache.hadoop.io.compress.DefaultCodec</value></property>
+<property><name>mapred.compress.map.output</name><value>false</value></property>
+<property><name>io.sort.spill.percent</name><value>0.80</value></property>
+<property><name>fs.checkpoint.edits.dir</name><value>${fs.checkpoint.dir}</value></property>
+<property><name>mapred.userlog.retain.hours</name><value>24</value></property>
+<property><name>mapred.system.dir</name><value>${hadoop.tmp.dir}/mapred/system</value></property>
+<property><name>mapred.line.input.format.linespermap</name><value>1</value></property>
+<property><name>job.end.retry.attempts</name><value>0</value></property>
+<property><name>ipc.client.idlethreshold</name><value>4000</value></property>
+<property><name>pregelix.vertexOutputFormatClass</name><value>edu.uci.ics.pregelix.example.trianglecounting.TriangleCountingVertex$TriangleCountingVertexOutputFormat</value></property>
+<property><name>mapred.reduce.copy.backoff</name><value>300</value></property>
+<property><name>mapred.map.tasks.speculative.execution</name><value>true</value></property>
+<property><name>mapred.inmem.merge.threshold</name><value>1000</value></property>
+<property><name>hadoop.logfile.size</name><value>10000000</value></property>
+<property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.trianglecounting.TextTriangleCountingInputFormat</value></property>
+<property><name>pregelix.aggregatorClass</name><value>edu.uci.ics.pregelix.example.trianglecounting.TriangleCountingAggregator</value></property>
+<property><name>mapred.job.queue.name</name><value>default</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.active</name><value>false</value></property>
+<property><name>mapred.reduce.slowstart.completed.maps</name><value>0.05</value></property>
+<property><name>topology.script.number.args</name><value>100</value></property>
+<property><name>mapred.skip.map.max.skip.records</name><value>0</value></property>
+<property><name>fs.ftp.impl</name><value>org.apache.hadoop.fs.ftp.FTPFileSystem</value></property>
+<property><name>mapred.task.cache.levels</name><value>2</value></property>
+<property><name>mapred.job.tracker.handler.count</name><value>10</value></property>
+<property><name>io.serializations</name><value>org.apache.hadoop.io.serializer.WritableSerialization</value></property>
+<property><name>ipc.client.connect.max.retries</name><value>10</value></property>
+<property><name>mapred.min.split.size</name><value>0</value></property>
+<property><name>mapred.map.max.attempts</name><value>4</value></property>
+<property><name>jobclient.output.filter</name><value>FAILED</value></property>
+<property><name>ipc.client.tcpnodelay</name><value>false</value></property>
+<property><name>mapred.acls.enabled</name><value>false</value></property>
+</configuration>
\ No newline at end of file
diff --git a/pregelix/pregelix-example/src/test/resources/topology.xml b/pregelix/pregelix-example/src/test/resources/topology.xml
new file mode 100755
index 0000000..2a6c380
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/topology.xml
@@ -0,0 +1,7 @@
+<cluster-topology>
+ <network-switch name="Global">
+ <network-switch name="local">
+ <terminal name="127.1.0.1"/>
+ </network-switch>
+ </network-switch>
+</cluster-topology>
\ No newline at end of file
diff --git a/pregelix/pregelix-runtime/pom.xml b/pregelix/pregelix-runtime/pom.xml
index d12cb36..e75f98b 100644
--- a/pregelix/pregelix-runtime/pom.xml
+++ b/pregelix/pregelix-runtime/pom.xml
@@ -22,8 +22,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
<plugin>
@@ -42,6 +43,7 @@
</plugin>
<plugin>
<artifactId>maven-clean-plugin</artifactId>
+ <version>2.5</version>
<configuration>
<filesets>
<fileset>
@@ -116,13 +118,6 @@
<version>0.2.3-SNAPSHOT</version>
</dependency>
<dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-core</artifactId>
- <version>0.20.2</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
<groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks-storage-am-common</artifactId>
<version>0.2.3-SNAPSHOT</version>
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/NCBootstrapImpl.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/NCBootstrapImpl.java
index e7caeaf..76c725e 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/NCBootstrapImpl.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/NCBootstrapImpl.java
@@ -34,7 +34,7 @@
@Override
public void stop() throws Exception {
- LOGGER.info("Stopping Giraph NC Bootstrap");
+ LOGGER.info("Stopping NC Bootstrap");
RuntimeContext rCtx = (RuntimeContext) appCtx.getApplicationObject();
rCtx.close();
}
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/ComputeUpdateFunctionFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/ComputeUpdateFunctionFactory.java
index 105d3e2..f7958d9 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/ComputeUpdateFunctionFactory.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/ComputeUpdateFunctionFactory.java
@@ -21,6 +21,7 @@
import java.util.ArrayList;
import java.util.List;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.Writable;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
@@ -58,6 +59,8 @@
private final ArrayTupleBuilder tbAlive = new ArrayTupleBuilder(2);
private final ArrayTupleBuilder tbTerminate = new ArrayTupleBuilder(1);
private final ArrayTupleBuilder tbGlobalAggregate = new ArrayTupleBuilder(1);
+ private final ArrayTupleBuilder tbInsert = new ArrayTupleBuilder(2);
+ private final ArrayTupleBuilder tbDelete = new ArrayTupleBuilder(1);
// for writing out to message channel
private IFrameWriter writerMsg;
@@ -82,6 +85,16 @@
private ByteBuffer bufferGlobalAggregate;
private GlobalAggregator aggregator;
+ // for writing out to insert vertex channel
+ private IFrameWriter writerInsert;
+ private FrameTupleAppender appenderInsert;
+ private ByteBuffer bufferInsert;
+
+ // for writing out to delete vertex channel
+ private IFrameWriter writerDelete;
+ private FrameTupleAppender appenderDelete;
+ private ByteBuffer bufferDelete;
+
private Vertex vertex;
private ResetableByteArrayOutputStream bbos = new ResetableByteArrayOutputStream();
private DataOutput output = new DataOutputStream(bbos);
@@ -90,11 +103,15 @@
private final List<IFrameWriter> writers = new ArrayList<IFrameWriter>();
private final List<FrameTupleAppender> appenders = new ArrayList<FrameTupleAppender>();
private final List<ArrayTupleBuilder> tbs = new ArrayList<ArrayTupleBuilder>();
+ private Configuration conf;
+ private boolean dynamicStateLength;
@Override
public void open(IHyracksTaskContext ctx, RecordDescriptor rd, IFrameWriter... writers)
throws HyracksDataException {
- this.aggregator = BspUtils.createGlobalAggregator(confFactory.createConfiguration());
+ this.conf = confFactory.createConfiguration();
+ this.dynamicStateLength = BspUtils.getDynamicVertexValueSize(conf);
+ this.aggregator = BspUtils.createGlobalAggregator(conf);
this.aggregator.init();
this.writerMsg = writers[0];
@@ -114,8 +131,22 @@
this.appenderGlobalAggregate = new FrameTupleAppender(ctx.getFrameSize());
this.appenderGlobalAggregate.reset(bufferGlobalAggregate, true);
- if (writers.length > 3) {
- this.writerAlive = writers[3];
+ this.writerInsert = writers[3];
+ this.bufferInsert = ctx.allocateFrame();
+ this.appenderInsert = new FrameTupleAppender(ctx.getFrameSize());
+ this.appenderInsert.reset(bufferInsert, true);
+ this.writers.add(writerInsert);
+ this.appenders.add(appenderInsert);
+
+ this.writerDelete = writers[4];
+ this.bufferDelete = ctx.allocateFrame();
+ this.appenderDelete = new FrameTupleAppender(ctx.getFrameSize());
+ this.appenderDelete.reset(bufferDelete, true);
+ this.writers.add(writerDelete);
+ this.appenders.add(appenderDelete);
+
+ if (writers.length > 5) {
+ this.writerAlive = writers[5];
this.bufferAlive = ctx.allocateFrame();
this.appenderAlive = new FrameTupleAppender(ctx.getFrameSize());
this.appenderAlive.reset(bufferAlive, true);
@@ -125,6 +156,8 @@
}
tbs.add(tbMsg);
+ tbs.add(tbInsert);
+ tbs.add(tbDelete);
tbs.add(tbAlive);
}
@@ -155,7 +188,7 @@
/**
* this partition should not terminate
*/
- if (terminate && (!vertex.isHalted() || vertex.hasMessage()))
+ if (terminate && (!vertex.isHalted() || vertex.hasMessage() || vertex.createdNewLiveVertex()))
terminate = false;
aggregator.step(vertex);
@@ -164,6 +197,9 @@
@Override
public void close() throws HyracksDataException {
FrameTupleUtils.flushTuplesFinal(appenderMsg, writerMsg);
+ FrameTupleUtils.flushTuplesFinal(appenderInsert, writerInsert);
+ FrameTupleUtils.flushTuplesFinal(appenderDelete, writerDelete);
+
if (pushAlive)
FrameTupleUtils.flushTuplesFinal(appenderAlive, writerAlive);
if (!terminate) {
@@ -177,7 +213,8 @@
private void writeOutGlobalAggregate() throws HyracksDataException {
try {
/**
- * get partial aggregate result and flush to the final aggregator
+ * get partial aggregate result and flush to the final
+ * aggregator
*/
Writable agg = aggregator.finishPartial();
agg.write(tbGlobalAggregate.getDataOutput());
@@ -203,15 +240,27 @@
}
@Override
- public void update(ITupleReference tupleRef) throws HyracksDataException {
+ public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb) throws HyracksDataException {
try {
if (vertex != null && vertex.hasUpdate()) {
- int fieldCount = tupleRef.getFieldCount();
- for (int i = 1; i < fieldCount; i++) {
- byte[] data = tupleRef.getFieldData(i);
- int offset = tupleRef.getFieldStart(i);
- bbos.setByteArray(data, offset);
- vertex.write(output);
+ if (!dynamicStateLength) {
+ // in-place update
+ int fieldCount = tupleRef.getFieldCount();
+ for (int i = 1; i < fieldCount; i++) {
+ byte[] data = tupleRef.getFieldData(i);
+ int offset = tupleRef.getFieldStart(i);
+ bbos.setByteArray(data, offset);
+ vertex.write(output);
+ }
+ } else {
+ // write the vertex id
+ DataOutput tbOutput = cloneUpdateTb.getDataOutput();
+ vertex.getVertexId().write(tbOutput);
+ cloneUpdateTb.addFieldEndOffset();
+
+ // write the vertex value
+ vertex.write(tbOutput);
+ cloneUpdateTb.addFieldEndOffset();
}
}
} catch (IOException e) {
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/StartComputeUpdateFunctionFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/StartComputeUpdateFunctionFactory.java
index f72b059..0cf64a0 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/StartComputeUpdateFunctionFactory.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/StartComputeUpdateFunctionFactory.java
@@ -21,6 +21,7 @@
import java.util.ArrayList;
import java.util.List;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.Writable;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
@@ -58,6 +59,8 @@
private final ArrayTupleBuilder tbAlive = new ArrayTupleBuilder(2);
private final ArrayTupleBuilder tbTerminate = new ArrayTupleBuilder(1);
private final ArrayTupleBuilder tbGlobalAggregate = new ArrayTupleBuilder(1);
+ private final ArrayTupleBuilder tbInsert = new ArrayTupleBuilder(2);
+ private final ArrayTupleBuilder tbDelete = new ArrayTupleBuilder(1);
// for writing out to message channel
private IFrameWriter writerMsg;
@@ -76,12 +79,22 @@
private ByteBuffer bufferGlobalAggregate;
private GlobalAggregator aggregator;
- //for writing out the global aggregate
+ // for writing out the global aggregate
private IFrameWriter writerTerminate;
private FrameTupleAppender appenderTerminate;
private ByteBuffer bufferTerminate;
private boolean terminate = true;
+ // for writing out to insert vertex channel
+ private IFrameWriter writerInsert;
+ private FrameTupleAppender appenderInsert;
+ private ByteBuffer bufferInsert;
+
+ // for writing out to delete vertex channel
+ private IFrameWriter writerDelete;
+ private FrameTupleAppender appenderDelete;
+ private ByteBuffer bufferDelete;
+
// dummy empty msgList
private MsgList msgList = new MsgList();
private ArrayIterator msgIterator = new ArrayIterator();
@@ -93,11 +106,15 @@
private final List<IFrameWriter> writers = new ArrayList<IFrameWriter>();
private final List<FrameTupleAppender> appenders = new ArrayList<FrameTupleAppender>();
private final List<ArrayTupleBuilder> tbs = new ArrayList<ArrayTupleBuilder>();
+ private Configuration conf;
+ private boolean dynamicStateLength;
@Override
public void open(IHyracksTaskContext ctx, RecordDescriptor rd, IFrameWriter... writers)
throws HyracksDataException {
- this.aggregator = BspUtils.createGlobalAggregator(confFactory.createConfiguration());
+ this.conf = confFactory.createConfiguration();
+ this.dynamicStateLength = BspUtils.getDynamicVertexValueSize(conf);
+ this.aggregator = BspUtils.createGlobalAggregator(conf);
this.aggregator.init();
this.writerMsg = writers[0];
@@ -117,8 +134,22 @@
this.appenderGlobalAggregate = new FrameTupleAppender(ctx.getFrameSize());
this.appenderGlobalAggregate.reset(bufferGlobalAggregate, true);
- if (writers.length > 3) {
- this.writerAlive = writers[3];
+ this.writerInsert = writers[3];
+ this.bufferInsert = ctx.allocateFrame();
+ this.appenderInsert = new FrameTupleAppender(ctx.getFrameSize());
+ this.appenderInsert.reset(bufferInsert, true);
+ this.writers.add(writerInsert);
+ this.appenders.add(appenderInsert);
+
+ this.writerDelete = writers[4];
+ this.bufferDelete = ctx.allocateFrame();
+ this.appenderDelete = new FrameTupleAppender(ctx.getFrameSize());
+ this.appenderDelete.reset(bufferDelete, true);
+ this.writers.add(writerDelete);
+ this.appenders.add(appenderDelete);
+
+ if (writers.length > 5) {
+ this.writerAlive = writers[5];
this.bufferAlive = ctx.allocateFrame();
this.appenderAlive = new FrameTupleAppender(ctx.getFrameSize());
this.appenderAlive.reset(bufferAlive, true);
@@ -129,6 +160,8 @@
msgList.reset(msgIterator);
tbs.add(tbMsg);
+ tbs.add(tbInsert);
+ tbs.add(tbDelete);
tbs.add(tbAlive);
}
@@ -156,7 +189,7 @@
/**
* this partition should not terminate
*/
- if (terminate && (!vertex.isHalted() || vertex.hasMessage()))
+ if (terminate && (!vertex.isHalted() || vertex.hasMessage() || vertex.createdNewLiveVertex()))
terminate = false;
/**
@@ -168,20 +201,24 @@
@Override
public void close() throws HyracksDataException {
FrameTupleUtils.flushTuplesFinal(appenderMsg, writerMsg);
+ FrameTupleUtils.flushTuplesFinal(appenderInsert, writerInsert);
+ FrameTupleUtils.flushTuplesFinal(appenderDelete, writerDelete);
+
if (pushAlive)
FrameTupleUtils.flushTuplesFinal(appenderAlive, writerAlive);
if (!terminate) {
writeOutTerminationState();
}
-
- /**write out global aggregate value*/
+
+ /** write out global aggregate value */
writeOutGlobalAggregate();
}
private void writeOutGlobalAggregate() throws HyracksDataException {
try {
/**
- * get partial aggregate result and flush to the final aggregator
+ * get partial aggregate result and flush to the final
+ * aggregator
*/
Writable agg = aggregator.finishPartial();
agg.write(tbGlobalAggregate.getDataOutput());
@@ -207,15 +244,27 @@
}
@Override
- public void update(ITupleReference tupleRef) throws HyracksDataException {
+ public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb) throws HyracksDataException {
try {
if (vertex != null && vertex.hasUpdate()) {
- int fieldCount = tupleRef.getFieldCount();
- for (int i = 1; i < fieldCount; i++) {
- byte[] data = tupleRef.getFieldData(i);
- int offset = tupleRef.getFieldStart(i);
- bbos.setByteArray(data, offset);
- vertex.write(output);
+ if (!dynamicStateLength) {
+ // in-place update
+ int fieldCount = tupleRef.getFieldCount();
+ for (int i = 1; i < fieldCount; i++) {
+ byte[] data = tupleRef.getFieldData(i);
+ int offset = tupleRef.getFieldStart(i);
+ bbos.setByteArray(data, offset);
+ vertex.write(output);
+ }
+ } else {
+ // write the vertex id
+ DataOutput tbOutput = cloneUpdateTb.getDataOutput();
+ vertex.getVertexId().write(tbOutput);
+ cloneUpdateTb.addFieldEndOffset();
+
+ // write the vertex value
+ vertex.write(tbOutput);
+ cloneUpdateTb.addFieldEndOffset();
}
}
} catch (IOException e) {
@@ -224,5 +273,4 @@
}
};
}
-
}
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RuntimeHookFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RuntimeHookFactory.java
index f7d0018..c025f85 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RuntimeHookFactory.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RuntimeHookFactory.java
@@ -15,12 +15,12 @@
package edu.uci.ics.pregelix.runtime.touchpoint;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Mapper.Context;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.TaskAttemptID;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.hdfs.ContextFactory;
import edu.uci.ics.pregelix.api.graph.Vertex;
import edu.uci.ics.pregelix.api.util.BspUtils;
import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
@@ -40,14 +40,13 @@
public IRuntimeHook createRuntimeHook() {
return new IRuntimeHook() {
+ private ContextFactory ctxFactory = new ContextFactory();
- @SuppressWarnings({ "rawtypes", "unchecked" })
@Override
public void configure(IHyracksTaskContext ctx) throws HyracksDataException {
Configuration conf = confFactory.createConfiguration();
try {
- Context mapperContext = new Mapper().new Context(conf, new TaskAttemptID(), null, null, null, null,
- null);
+ TaskAttemptContext mapperContext = ctxFactory.createContext(conf, new TaskAttemptID());
Vertex.setContext(mapperContext);
BspUtils.setDefaultConfiguration(conf);
} catch (Exception e) {