add new example for Counters usage
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
index cf753bb..1fe7616 100644
--- 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
@@ -133,7 +133,7 @@
 
     }
 
-    private static long readTriangleCountingResult(Configuration conf) {
+    protected static long readTriangleCountingResult(Configuration conf) {
         try {
             VLongWritable count = (VLongWritable) IterationUtils.readGlobalAggregateValue(conf,
                     BspUtils.getJobId(conf), TriangleCountingAggregator.class.getName());
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TriangleCountingWithAggregateHadoopCountersVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TriangleCountingWithAggregateHadoopCountersVertex.java
new file mode 100644
index 0000000..7cf2840
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TriangleCountingWithAggregateHadoopCountersVertex.java
@@ -0,0 +1,77 @@
+/*
+ * Copyright 2009-2013 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 org.apache.hadoop.mapreduce.Counters;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.job.PregelixJob;
+import edu.uci.ics.pregelix.api.util.BspUtils;
+import edu.uci.ics.pregelix.api.util.HadoopCountersAggregator;
+import edu.uci.ics.pregelix.example.client.Client;
+import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+/**
+ * The triangle counting example -- counting the triangles in an undirected graph.
+ */
+public class TriangleCountingWithAggregateHadoopCountersVertex extends TriangleCountingVertex {
+
+    public static void main(String[] args) throws Exception {
+        PregelixJob job = new PregelixJob(TriangleCountingWithAggregateHadoopCountersVertex.class.getSimpleName());
+        job.setVertexClass(TriangleCountingWithAggregateHadoopCountersVertex.class);
+        job.addGlobalAggregatorClass(TriangleCountingAggregator.class);
+        job.setCounterAggregatorClass(TriangleHadoopCountersAggregator.class);
+        job.setVertexInputFormatClass(TextTriangleCountingInputFormat.class);
+        job.setVertexOutputFormatClass(TriangleCountingVertexOutputFormat.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+        Client.run(args, job);
+        System.out.println("triangle count in last iteration: " + readTriangleCountingResult(job.getConfiguration()));
+        System.out.println("aggregate counter (including all iterations):\n" + BspUtils.getCounters(job));
+    }
+
+    public static class TriangleHadoopCountersAggregator extends
+            HadoopCountersAggregator<VLongWritable, VLongWritable, VLongWritable, VLongWritable, Counters> {
+        private Counters counters;
+
+        @Override
+        public void init() {
+            counters = new Counters();
+        }
+
+        @Override
+        public void step(Vertex<VLongWritable, VLongWritable, VLongWritable, VLongWritable> v)
+                throws HyracksDataException {
+            counters.findCounter("TriangleCounting", "total-ids").increment(1);
+            counters.findCounter("TriangleCounting", "total-values").increment(v.getVertexValue().get());
+        }
+
+        @Override
+        public void step(Counters partialResult) {
+            counters.incrAllCounters(partialResult);
+        }
+
+        @Override
+        public Counters finishPartial() {
+            return counters;
+        }
+
+        @Override
+        public Counters finishFinal() {
+            return counters;
+        }
+    }
+}
\ No newline at end of file
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 6ccefd2..fcfeb95 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,6 +25,7 @@
 import edu.uci.ics.pregelix.api.job.PregelixJob;
 import edu.uci.ics.pregelix.api.util.ConservativeCheckpointHook;
 import edu.uci.ics.pregelix.api.util.DefaultVertexPartitioner;
+import edu.uci.ics.pregelix.api.util.HadoopCountersGlobalAggregateHook;
 import edu.uci.ics.pregelix.example.ConnectedComponentsVertex;
 import edu.uci.ics.pregelix.example.ConnectedComponentsVertex.SimpleConnectedComponentsVertexOutputFormat;
 import edu.uci.ics.pregelix.example.EarlyTerminationVertex;
@@ -53,6 +54,8 @@
 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;
+import edu.uci.ics.pregelix.example.trianglecounting.TriangleCountingWithAggregateHadoopCountersVertex.TriangleHadoopCountersAggregator;
+import edu.uci.ics.pregelix.example.trianglecounting.TriangleCountingWithAggregateHadoopCountersVertex;
 
 public class JobGenerator {
     private static String outputBase = "src/test/resources/jobs/";
@@ -239,6 +242,19 @@
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH3));
         job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
     }
+    
+    private static void generateTriangleCountingWithHadoopCountersJob(String jobName, String outputPath) throws IOException {
+        PregelixJob job = new PregelixJob(jobName);
+        job.setVertexClass(TriangleCountingWithAggregateHadoopCountersVertex.class);
+        job.addGlobalAggregatorClass(TriangleCountingAggregator.class);
+        job.setCounterAggregatorClass(TriangleCountingWithAggregateHadoopCountersVertex.TriangleHadoopCountersAggregator.class);
+        job.setVertexInputFormatClass(TextTriangleCountingInputFormat.class);
+        job.setVertexOutputFormatClass(TriangleCountingVertexOutputFormat.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.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);
@@ -375,6 +391,7 @@
 
     private static void genTriangleCounting() throws IOException {
         generateTriangleCountingJob("Triangle Counting", outputBase + "TriangleCounting.xml");
+//        generateTriangleCountingWithHadoopCountersJob("Triangle Counting", outputBase + "TriangleCountingWithHadoopCounters.xml");
     }
 
     private static void genMaximalClique() throws IOException {