add the triangle counting example

git-svn-id: https://hyracks.googlecode.com/svn/branches/fullstack_staging@2680 123451ca-8445-de46-9d55-352943316053
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/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..a76837e 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
@@ -97,7 +97,7 @@
 
     @Override
     public void compute(Iterator<DoubleWritable> msgIterator) {
-        int maxIteration = this.getContext().getConfiguration().getInt(ITERATIONS, 10);
+        int maxIteration = getContext().getConfiguration().getInt(ITERATIONS, 10);
         if (getSuperstep() == 1) {
             tmpVertexValue.set(1.0 / getNumVertices());
             setVertexValue(tmpVertexValue);
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/TriangleCountingAggregator.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/TriangleCountingAggregator.java
new file mode 100644
index 0000000..4089758
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/TriangleCountingAggregator.java
@@ -0,0 +1,41 @@
+package edu.uci.ics.pregelix.example;
+
+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/TriangleCountingVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/TriangleCountingVertex.java
new file mode 100644
index 0000000..7e404ca
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/TriangleCountingVertex.java
@@ -0,0 +1,143 @@
+package edu.uci.ics.pregelix.example;
+
+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.inputformat.TextPageRankInputFormat;
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+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()));
+        }
+    }
+
+    /**
+     * 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(TextPageRankInputFormat.class);
+        job.setVertexOutputFormatClass(TriangleCountingVertexOutputFormat.class);
+        Client.run(args, job);
+        System.out.println("triangle count: " + readTriangleCountingResult(job.getConfiguration()));
+    }
+}
+
+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/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 2c580c4..23b019f 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
@@ -31,6 +31,9 @@
 import edu.uci.ics.pregelix.example.ReachabilityVertex;
 import edu.uci.ics.pregelix.example.ReachabilityVertex.SimpleReachibilityVertexOutputFormat;
 import edu.uci.ics.pregelix.example.ShortestPathsVertex;
+import edu.uci.ics.pregelix.example.TriangleCountingAggregator;
+import edu.uci.ics.pregelix.example.TriangleCountingVertex;
+import edu.uci.ics.pregelix.example.TriangleCountingVertex.TriangleCountingVertexOutputFormat;
 import edu.uci.ics.pregelix.example.inputformat.TextConnectedComponentsInputFormat;
 import edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat;
 import edu.uci.ics.pregelix.example.inputformat.TextReachibilityVertexInputFormat;
@@ -44,6 +47,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);
@@ -183,6 +189,17 @@
         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(TextPageRankInputFormat.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 genPageRank() throws IOException {
         generatePageRankJob("PageRank", outputBase + "PageRank.xml");
         generatePageRankJobReal("PageRank", outputBase + "PageRankReal.xml");
@@ -208,11 +225,15 @@
                 + "ReachibilityRealComplexNoConnectivity.xml");
     }
 
+    private static void genTriangleCounting() throws IOException {
+        generateTriangleCountingJob("Triangle Counting", outputBase + "TriangleCounting.xml");
+    }
+
     public static void main(String[] args) throws IOException {
         genPageRank();
         genShortestPath();
         genConnectedComponents();
         genReachibility();
+        genTriangleCounting();
     }
-
 }
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..7a5bba6 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;
@@ -44,6 +45,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 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..b5abd94 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
@@ -59,6 +59,9 @@
     private static final String DATA_PATH2 = "data/webmapcomplex/webmap_link.txt";
     private static final String HDFS_PATH2 = "/webmapcomplex/";
 
+    private static final String DATA_PATH3 = "data/clique/clique.txt";
+    private static final String HDFS_PATH3 = "/clique/";
+
     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;
@@ -104,6 +107,11 @@
         dfs.mkdirs(dest);
         dfs.copyFromLocalFile(src, dest);
 
+        src = new Path(DATA_PATH3);
+        dest = new Path(HDFS_PATH3);
+        dfs.mkdirs(dest);
+        dfs.copyFromLocalFile(src, dest);
+
         DataOutputStream confOutput = new DataOutputStream(new FileOutputStream(new File(HADOOP_CONF_PATH)));
         conf.writeXml(confOutput);
         confOutput.flush();
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..8d2050b
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result
@@ -0,0 +1,7 @@
+1|Vertex(id=1,value=3, edges=(2,3,4,))
+2|Vertex(id=2,value=2, edges=(1,3,4,5,))
+3|Vertex(id=3,value=0, edges=(1,2,4,5,))
+4|Vertex(id=4,value=0, edges=(1,2,3,))
+5|Vertex(id=5,value=1, edges=(6,7,))
+6|Vertex(id=6,value=0, edges=(5,7,))
+7|Vertex(id=7,value=0, edges=(5,6,))
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..f785686
--- /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.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.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.inputformat.TextPageRankInputFormat</value></property>
+<property><name>pregelix.aggregatorClass</name><value>edu.uci.ics.pregelix.example.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