Author: maja
Date: Mon Nov 12 21:20:42 2012
New Revision: 1408464

URL: http://svn.apache.org/viewvc?rev=1408464&view=rev
Log:
GIRAPH-415: Refactor / cleanup Hadoop Counters

Added:
    giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/
    
giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/GiraphHadoopCounter.java
    
giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/GiraphStats.java
    
giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/GiraphTimers.java
    
giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/HadoopCountersBase.java
    
giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/package-info.java
    
giraph/trunk/giraph/src/main/java/org/apache/giraph/metrics/NoOpMetricsRegistry.java
Removed:
    
giraph/trunk/giraph/src/main/java/org/apache/giraph/metrics/EmptyMetricsRegistry.java
Modified:
    giraph/trunk/CHANGELOG
    
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/BspServiceMaster.java
    
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/EdgeListVertex.java
    giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/HashMapVertex.java
    
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/IntIntNullIntVertex.java
    
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/LongDoubleFloatDoubleVertex.java
    giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/MasterThread.java
    
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/SimpleMutableVertex.java
    giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/SimpleVertex.java
    giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/Vertex.java
    
giraph/trunk/giraph/src/main/java/org/apache/giraph/metrics/GiraphMetricsRegistry.java
    
giraph/trunk/giraph/src/main/java/org/apache/giraph/metrics/package-info.java

Modified: giraph/trunk/CHANGELOG
URL: 
http://svn.apache.org/viewvc/giraph/trunk/CHANGELOG?rev=1408464&r1=1408463&r2=1408464&view=diff
==============================================================================
--- giraph/trunk/CHANGELOG (original)
+++ giraph/trunk/CHANGELOG Mon Nov 12 21:20:42 2012
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 0.2.0 - unreleased
+  GIRAPH-415: Refactor / cleanup Hadoop Counters (nitay via majakabiljo)
+
   GIRAPH-413: Make building Facebook profile easier. (nitay via aching)
 
   GIRAPH-407: Metrics Update (nitay via apresta)

Added: 
giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/GiraphHadoopCounter.java
URL: 
http://svn.apache.org/viewvc/giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/GiraphHadoopCounter.java?rev=1408464&view=auto
==============================================================================
--- 
giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/GiraphHadoopCounter.java
 (added)
+++ 
giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/GiraphHadoopCounter.java
 Mon Nov 12 21:20:42 2012
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.counters;
+
+import com.google.common.base.Objects;
+import org.apache.hadoop.mapreduce.Counter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * Wrapper around Hadoop Counter to make it easier to use.
+ */
+public class GiraphHadoopCounter {
+  /** Hadoop Counter we're wrapping. */
+  private Counter counter;
+
+  /**
+   * Create wrapping a Hadoop Counter.
+   *
+   * @param counter Hadoop Counter to wrap.
+   */
+  public GiraphHadoopCounter(Counter counter) {
+    this.counter = counter;
+  }
+
+  /**
+   * Get underlying Hadoop Counter we're wrapping.
+   *
+   * @return Hadoop Counter being wrapped.
+   */
+  public Counter getHadoopCounter() {
+    return counter;
+  }
+
+  @Override
+  public int hashCode() {
+    return counter.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object genericRight) {
+    if (genericRight == null) {
+      return false;
+    }
+    if (getClass() != genericRight.getClass()) {
+      return false;
+    }
+    GiraphHadoopCounter right = (GiraphHadoopCounter) genericRight;
+    return Objects.equal(counter, right.counter);
+  }
+
+  /**
+   * Set counter to value. Should be greater than current value.
+   *
+   * @param value long value to set to.
+   */
+  public void setValue(long value) {
+    increment(value - getValue());
+  }
+
+  /**
+   * Increment counter value by 1.
+   */
+  public void increment() {
+    increment(1);
+  }
+
+  /**
+   * Increment counter value.
+   *
+   * @param incr amount to increment by.
+   */
+  public void increment(long incr) {
+    counter.increment(incr);
+  }
+
+  /**
+   * Get counter value
+   *
+   * @return long value of counter
+   */
+  public long getValue() {
+    return counter.getValue();
+  }
+
+  /**
+   * Get counter display name.
+   *
+   * @return String Hadoop counter display name.
+   */
+  public String getDisplayName() {
+    return counter.getDisplayName();
+  }
+
+  /**
+   * Get counter name.
+   *
+   * @return String Hadoop counter name.
+   */
+  public String getName() {
+    return counter.getName();
+  }
+
+  /**
+   * Write to Hadoop output.
+   *
+   * @param out DataOutput to write to.
+   * @throws IOException if something goes wrong.
+   */
+  public void write(DataOutput out) throws IOException {
+    counter.write(out);
+  }
+
+  /**
+   * Read from Hadoop input.
+   *
+   * @param in DataInput to read from.
+   * @throws IOException if something goes wrong reading.
+   */
+  public void readFields(DataInput in) throws IOException {
+    counter.readFields(in);
+  }
+}

Added: 
giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/GiraphStats.java
URL: 
http://svn.apache.org/viewvc/giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/GiraphStats.java?rev=1408464&view=auto
==============================================================================
--- 
giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/GiraphStats.java 
(added)
+++ 
giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/GiraphStats.java 
Mon Nov 12 21:20:42 2012
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.counters;
+
+import org.apache.hadoop.mapreduce.Mapper.Context;
+
+import java.util.Arrays;
+import java.util.Iterator;
+
+/**
+ * Hadoop Counters in group "Giraph Stats". General statistics about job.
+ */
+public class GiraphStats extends HadoopCountersBase {
+  /** Counter group name for the Giraph statistics */
+  public static final String GROUP_NAME = "Giraph Stats";
+
+  /** Singleton instance for everyone to use */
+  private static GiraphStats INSTANCE;
+
+  /** Superstep counter */
+  private static final int SUPERSTEP = 0;
+  /** Vertex counter */
+  private static final int VERTICES = 1;
+  /** Finished vertex counter */
+  private static final int FINISHED_VERTICES = 2;
+  /** Edge counter */
+  private static final int EDGES = 3;
+  /** Sent messages counter */
+  private static final int SENT_MESSAGES = 4;
+  /** Workers on this superstep */
+  private static final int CURRENT_WORKERS = 5;
+  /** Current master task partition */
+  private static final int CURRENT_MASTER_TASK_PARTITION = 6;
+  /** Last checkpointed superstep */
+  private static final int LAST_CHECKPOINTED_SUPERSTEP = 7;
+  /** Number of counters in this class */
+  private static final int NUM_COUNTERS = 8;
+
+  /** All the counters stored */
+  private final GiraphHadoopCounter[] counters;
+
+  /**
+   * Create with Hadoop Context.
+   *
+   * @param context Hadoop Context to use.
+   */
+  private GiraphStats(Context context) {
+    super(context, GROUP_NAME);
+    counters = new GiraphHadoopCounter[NUM_COUNTERS];
+    counters[SUPERSTEP] = getCounter("Superstep");
+    counters[VERTICES] = getCounter("Aggregate vertices");
+    counters[FINISHED_VERTICES] = getCounter("Aggregate finished vertices");
+    counters[EDGES] = getCounter("Aggregate edges");
+    counters[SENT_MESSAGES] = getCounter("Sent messages");
+    counters[CURRENT_WORKERS] = getCounter("Current workers");
+    counters[CURRENT_MASTER_TASK_PARTITION] =
+        getCounter("Current master task partition");
+    counters[LAST_CHECKPOINTED_SUPERSTEP] =
+        getCounter("Last checkpointed superstep");
+  }
+
+  /**
+   * Initialize with Hadoop Context.
+   *
+   * @param context Hadoop Context to use.
+   */
+  public static void init(Context context) {
+    INSTANCE = new GiraphStats(context);
+  }
+
+  /**
+   * Get singleton instance.
+   *
+   * @return GiraphStats singleton
+   */
+  public static GiraphStats getInstance() {
+    return INSTANCE;
+  }
+
+  /**
+   * Get SuperstepCounter counter
+   *
+   * @return SuperstepCounter counter
+   */
+  public GiraphHadoopCounter getSuperstepCounter() {
+    return counters[SUPERSTEP];
+  }
+
+  /**
+   * Get Vertices counter
+   *
+   * @return Vertices counter
+   */
+  public GiraphHadoopCounter getVertices() {
+    return counters[VERTICES];
+  }
+
+  /**
+   * Get FinishedVertexes counter
+   *
+   * @return FinishedVertexes counter
+   */
+  public GiraphHadoopCounter getFinishedVertexes() {
+    return counters[FINISHED_VERTICES];
+  }
+
+  /**
+   * Get Edges counter
+   *
+   * @return Edges counter
+   */
+  public GiraphHadoopCounter getEdges() {
+    return counters[EDGES];
+  }
+
+  /**
+   * Get SentMessages counter
+   *
+   * @return SentMessages counter
+   */
+  public GiraphHadoopCounter getSentMessages() {
+    return counters[SENT_MESSAGES];
+  }
+
+  /**
+   * Get CurrentWorkers counter
+   *
+   * @return CurrentWorkers counter
+   */
+  public GiraphHadoopCounter getCurrentWorkers() {
+    return counters[CURRENT_WORKERS];
+  }
+
+  /**
+   * Get CurrentMasterTaskPartition counter
+   *
+   * @return CurrentMasterTaskPartition counter
+   */
+  public GiraphHadoopCounter getCurrentMasterTaskPartition() {
+    return counters[CURRENT_MASTER_TASK_PARTITION];
+  }
+
+  /**
+   * Get LastCheckpointedSuperstep counter
+   *
+   * @return LastCheckpointedSuperstep counter
+   */
+  public GiraphHadoopCounter getLastCheckpointedSuperstep() {
+    return counters[LAST_CHECKPOINTED_SUPERSTEP];
+  }
+
+  @Override
+  public Iterator<GiraphHadoopCounter> iterator() {
+    return Arrays.asList(counters).iterator();
+  }
+}

Added: 
giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/GiraphTimers.java
URL: 
http://svn.apache.org/viewvc/giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/GiraphTimers.java?rev=1408464&view=auto
==============================================================================
--- 
giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/GiraphTimers.java 
(added)
+++ 
giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/GiraphTimers.java 
Mon Nov 12 21:20:42 2012
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.counters;
+
+import org.apache.hadoop.mapreduce.Mapper.Context;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Maps;
+
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.Map;
+
+/**
+ * Hadoop Counters in group "Giraph Timers" for timing things.
+ */
+public class GiraphTimers extends HadoopCountersBase {
+  /** Counter group name for the giraph timers */
+  public static final String GROUP_NAME = "Giraph Timers";
+
+  /** Singleton instance for everyone to use */
+  private static GiraphTimers INSTANCE;
+
+  /** Setup time in msec */
+  private static final int SETUP_MS = 0;
+  /** Total time in msec */
+  private static final int TOTAL_MS = 1;
+  /** Shutdown time in msec */
+  private static final int SHUTDOWN_MS = 2;
+  /** How many whole job counters we have */
+  private static final int NUM_COUNTERS = 3;
+
+  /** superstep time in msec */
+  private final Map<Long, GiraphHadoopCounter> superstepMsec;
+
+  /** Whole job counters stored in this class */
+  private final GiraphHadoopCounter[] jobCounters;
+
+  /**
+   * Internal use only. Create using Hadoop Context
+   *
+   * @param context Hadoop Context to use.
+   */
+  private GiraphTimers(Context context) {
+    super(context, GROUP_NAME);
+    jobCounters = new GiraphHadoopCounter[NUM_COUNTERS];
+    jobCounters[SETUP_MS] = getCounter("Setup (milliseconds)");
+    jobCounters[TOTAL_MS] = getCounter("Total (milliseconds)");
+    jobCounters[SHUTDOWN_MS] = getCounter("Shutdown (milliseconds)");
+    superstepMsec = Maps.newHashMap();
+  }
+
+  /**
+   * Instantiate with Hadoop Context.
+   *
+   * @param context Hadoop Context to use.
+   */
+  public static void init(Context context) {
+    INSTANCE = new GiraphTimers(context);
+  }
+
+  /**
+   * Get singleton instance.
+   *
+   * @return singleton GiraphTimers instance.
+   */
+  public static GiraphTimers getInstance() {
+    return INSTANCE;
+  }
+
+  /**
+   * Get counter for setup time in milliseconds
+   *
+   * @return Counter for setup time in milliseconds
+   */
+  public GiraphHadoopCounter getSetupMs() {
+    return jobCounters[SETUP_MS];
+  }
+
+  /**
+   * Get counter for superstep time in milliseconds
+   *
+   * @param superstep Integer superstep number.
+   * @return Counter for setup time in milliseconds
+   */
+  public GiraphHadoopCounter getSuperstepMs(long superstep) {
+    GiraphHadoopCounter counter = superstepMsec.get(superstep);
+    if (counter == null) {
+      String counterPrefix;
+      if (superstep == -1) {
+        counterPrefix = "Vertex input superstep";
+      } else {
+        counterPrefix = "Superstep " + superstep;
+      }
+      counter = getCounter(counterPrefix + " (milliseconds)");
+      superstepMsec.put(superstep, counter);
+    }
+    return counter;
+  }
+
+  /**
+   * Get counter for total time in milliseconds.
+   *
+   * @return Counter for total time in milliseconds.
+   */
+  public GiraphHadoopCounter getTotalMs() {
+    return jobCounters[TOTAL_MS];
+  }
+
+  /**
+   * Get counter for shutdown time in milliseconds.
+   *
+   * @return Counter for shutdown time in milliseconds.
+   */
+  public GiraphHadoopCounter getShutdownMs() {
+    return jobCounters[SHUTDOWN_MS];
+  }
+
+  /**
+   * Get map of superstep to msec counter.
+   *
+   * @return mapping of superstep to msec counter.
+   */
+  public Map<Long, GiraphHadoopCounter> superstepCounters() {
+    return superstepMsec;
+  }
+
+  /**
+   * Get Iterable through job counters.
+   *
+   * @return Iterable of job counters.
+   */
+  public Iterable<GiraphHadoopCounter> jobCounters() {
+    return Arrays.asList(jobCounters);
+  }
+
+  public Iterator<GiraphHadoopCounter> iterator() {
+    return Iterators.concat(jobCounters().iterator(),
+        superstepCounters().values().iterator());
+  }
+}

Added: 
giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/HadoopCountersBase.java
URL: 
http://svn.apache.org/viewvc/giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/HadoopCountersBase.java?rev=1408464&view=auto
==============================================================================
--- 
giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/HadoopCountersBase.java
 (added)
+++ 
giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/HadoopCountersBase.java
 Mon Nov 12 21:20:42 2012
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.counters;
+
+import org.apache.hadoop.mapreduce.Mapper.Context;
+
+/**
+ * Base class for groups of Hadoop Counters.
+ */
+public abstract class HadoopCountersBase
+    implements Iterable<GiraphHadoopCounter> {
+  /** Hadoop Context used to create Counters */
+  private final Context context;
+  /** group to put counters under */
+  private final String groupName;
+
+  /**
+   * Initialize with Hadoop Context and group name.
+   *
+   * @param context Hadoop Context to use.
+   * @param groupName String group name to use.
+   */
+  protected HadoopCountersBase(Context context, String groupName) {
+    this.context = context;
+    this.groupName = groupName;
+  }
+
+  /**
+   * Get Hadoop Context
+   *
+   * @return Context object used by Hadoop
+   */
+  public Context getContext() {
+    return context;
+  }
+
+  /**
+   * Get or create counter with given name and class's group name.
+   *
+   * @param name String name of counter
+   * @return GiraphHadoopCounter found or created
+   */
+  protected GiraphHadoopCounter getCounter(String name) {
+    return new GiraphHadoopCounter(context.getCounter(groupName, name));
+  }
+}

Added: 
giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/package-info.java
URL: 
http://svn.apache.org/viewvc/giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/package-info.java?rev=1408464&view=auto
==============================================================================
--- 
giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/package-info.java 
(added)
+++ 
giraph/trunk/giraph/src/main/java/org/apache/giraph/counters/package-info.java 
Mon Nov 12 21:20:42 2012
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Package of all the Giraph Hadoop counters related data.
+ */
+package org.apache.giraph.counters;

Modified: 
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/BspServiceMaster.java
URL: 
http://svn.apache.org/viewvc/giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/BspServiceMaster.java?rev=1408464&r1=1408463&r2=1408464&view=diff
==============================================================================
--- 
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/BspServiceMaster.java 
(original)
+++ 
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/BspServiceMaster.java 
Mon Nov 12 21:20:42 2012
@@ -19,12 +19,14 @@
 package org.apache.giraph.graph;
 
 import org.apache.giraph.GiraphConfiguration;
+import org.apache.giraph.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.bsp.ApplicationState;
 import org.apache.giraph.bsp.BspInputFormat;
 import org.apache.giraph.bsp.CentralizedServiceMaster;
 import org.apache.giraph.bsp.SuperstepState;
 import org.apache.giraph.comm.MasterClientServer;
 import org.apache.giraph.comm.netty.NettyMasterClientServer;
+import org.apache.giraph.counters.GiraphStats;
 import org.apache.giraph.graph.GraphMapper.MapFunctions;
 import org.apache.giraph.graph.partition.MasterGraphPartitioner;
 import org.apache.giraph.graph.partition.PartitionOwner;
@@ -51,7 +53,6 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.RunningJob;
-import org.apache.hadoop.mapreduce.Counter;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.log4j.Logger;
@@ -99,8 +100,6 @@ public class BspServiceMaster<I extends 
     extends BspService<I, V, E, M>
     implements CentralizedServiceMaster<I, V, E, M>,
     ResetSuperstepMetricsObserver {
-  /** Counter group name for the Giraph statistics */
-  public static final String GIRAPH_STATS_COUNTER_GROUP_NAME = "Giraph Stats";
   /** Print worker names only if there are 10 workers left */
   public static final int MAX_PRINTABLE_REMAINING_WORKERS = 10;
   /** How many threads to use when writing input splits to zookeeper*/
@@ -112,22 +111,6 @@ public class BspServiceMaster<I extends 
   private static final Time TIME = SystemTime.getInstance();
   /** Class logger */
   private static final Logger LOG = Logger.getLogger(BspServiceMaster.class);
-  /** Superstep counter */
-  private Counter superstepCounter = null;
-  /** Vertex counter */
-  private Counter vertexCounter = null;
-  /** Finished vertex counter */
-  private Counter finishedVertexCounter = null;
-  /** Edge counter */
-  private Counter edgeCounter = null;
-  /** Sent messages counter */
-  private Counter sentMessagesCounter = null;
-  /** Workers on this superstep */
-  private Counter currentWorkersCounter = null;
-  /** Current master task partition */
-  private Counter currentMasterTaskPartitionCounter = null;
-  /** Last checkpointed superstep */
-  private Counter lastCheckpointedSuperstepCounter = null;
   /** Am I the master? */
   private boolean isMaster = false;
   /** Max number of workers */
@@ -189,26 +172,24 @@ public class BspServiceMaster<I extends 
     superstepStateChanged = new PredicateLock(context);
     registerBspEvent(superstepStateChanged);
 
-    maxWorkers =
-        getConfiguration().getInt(GiraphConfiguration.MAX_WORKERS, -1);
-    minWorkers =
-        getConfiguration().getInt(GiraphConfiguration.MIN_WORKERS, -1);
-    minPercentResponded =
-        getConfiguration().getFloat(GiraphConfiguration.MIN_PERCENT_RESPONDED,
-            100.0f);
-    msecsPollPeriod =
-        getConfiguration().getInt(GiraphConfiguration.POLL_MSECS,
-                                  GiraphConfiguration.POLL_MSECS_DEFAULT);
-    maxPollAttempts =
-        getConfiguration().getInt(GiraphConfiguration.POLL_ATTEMPTS,
-            GiraphConfiguration.POLL_ATTEMPTS_DEFAULT);
-    partitionLongTailMinPrint = getConfiguration().getInt(
+    ImmutableClassesGiraphConfiguration<I, V, E, M> conf = getConfiguration();
+
+    maxWorkers = conf.getInt(GiraphConfiguration.MAX_WORKERS, -1);
+    minWorkers = conf.getInt(GiraphConfiguration.MIN_WORKERS, -1);
+    minPercentResponded = conf.getFloat(
+        GiraphConfiguration.MIN_PERCENT_RESPONDED, 100.0f);
+    msecsPollPeriod = conf.getInt(GiraphConfiguration.POLL_MSECS,
+        GiraphConfiguration.POLL_MSECS_DEFAULT);
+    maxPollAttempts = conf.getInt(GiraphConfiguration.POLL_ATTEMPTS,
+        GiraphConfiguration.POLL_ATTEMPTS_DEFAULT);
+    partitionLongTailMinPrint = conf.getInt(
         GiraphConfiguration.PARTITION_LONG_TAIL_MIN_PRINT,
         GiraphConfiguration.PARTITION_LONG_TAIL_MIN_PRINT_DEFAULT);
     masterGraphPartitioner =
         getGraphPartitionerFactory().createMasterGraphPartitioner();
 
     GiraphMetrics.getInstance().addSuperstepResetObserver(this);
+    GiraphStats.init(context);
   }
 
   @Override
@@ -227,7 +208,7 @@ public class BspServiceMaster<I extends 
       jobState.put(JSONOBJ_APPLICATION_ATTEMPT_KEY, applicationAttempt);
       jobState.put(JSONOBJ_SUPERSTEP_KEY, desiredSuperstep);
     } catch (JSONException e) {
-      throw new RuntimeException("setJobState: Coudn't put " +
+      throw new RuntimeException("setJobState: Couldn't put " +
           state.toString());
     }
     if (LOG.isInfoEnabled()) {
@@ -745,24 +726,10 @@ public class BspServiceMaster<I extends 
     // In that case, the input splits are not set, they will be faked by
     // the checkpoint files.  Each checkpoint file will be an input split
     // and the input split
-    superstepCounter = getContext().getCounter(
-        GIRAPH_STATS_COUNTER_GROUP_NAME, "Superstep");
-    vertexCounter = getContext().getCounter(
-        GIRAPH_STATS_COUNTER_GROUP_NAME, "Aggregate vertices");
-    finishedVertexCounter = getContext().getCounter(
-        GIRAPH_STATS_COUNTER_GROUP_NAME, "Aggregate finished vertices");
-    edgeCounter = getContext().getCounter(
-        GIRAPH_STATS_COUNTER_GROUP_NAME, "Aggregate edges");
-    sentMessagesCounter = getContext().getCounter(
-        GIRAPH_STATS_COUNTER_GROUP_NAME, "Sent messages");
-    currentWorkersCounter = getContext().getCounter(
-        GIRAPH_STATS_COUNTER_GROUP_NAME, "Current workers");
-    currentMasterTaskPartitionCounter = getContext().getCounter(
-        GIRAPH_STATS_COUNTER_GROUP_NAME, "Current master task partition");
-    lastCheckpointedSuperstepCounter = getContext().getCounter(
-        GIRAPH_STATS_COUNTER_GROUP_NAME, "Last checkpointed superstep");
+
     if (getRestartedSuperstep() != UNSET_SUPERSTEP) {
-      superstepCounter.increment(getRestartedSuperstep());
+      GiraphStats.getInstance().getSuperstepCounter().
+        setValue(getRestartedSuperstep());
     }
   }
 
@@ -812,9 +779,8 @@ public class BspServiceMaster<I extends 
               myBid + "'");
         }
         if (masterChildArr.get(0).equals(myBid)) {
-          currentMasterTaskPartitionCounter.increment(
-              getTaskPartition() -
-                  currentMasterTaskPartitionCounter.getValue());
+          GiraphStats.getInstance().getCurrentMasterTaskPartition().
+              setValue(getTaskPartition());
           masterCompute = getConfiguration().createMasterCompute();
           aggregatorHandler = new MasterAggregatorHandler(getConfiguration(),
               getContext());
@@ -961,8 +927,8 @@ public class BspServiceMaster<I extends 
     masterCompute.write(finalizedOutputStream);
     finalizedOutputStream.close();
     lastCheckpointedSuperstep = superstep;
-    lastCheckpointedSuperstepCounter.increment(superstep -
-        lastCheckpointedSuperstepCounter.getValue());
+    GiraphStats.getInstance().
+        getLastCheckpointedSuperstep().setValue(superstep);
   }
 
   /**
@@ -1386,8 +1352,8 @@ public class BspServiceMaster<I extends 
 
     commService.openConnections();
 
-    currentWorkersCounter.increment(chosenWorkerInfoList.size() -
-        currentWorkersCounter.getValue());
+    GiraphStats.getInstance().
+        getCurrentWorkers().setValue(chosenWorkerInfoList.size());
     assignPartitionOwners(allPartitionStatsList,
         chosenWorkerInfoList,
         masterGraphPartitioner);
@@ -1465,7 +1431,7 @@ public class BspServiceMaster<I extends 
     incrCachedSuperstep();
     // Counter starts at zero, so no need to increment
     if (getSuperstep() > 0) {
-      superstepCounter.increment(1);
+      GiraphStats.getInstance().getSuperstepCounter().increment();
     }
     SuperstepState superstepState;
     if (globalStats.getHaltComputation()) {
@@ -1487,8 +1453,10 @@ public class BspServiceMaster<I extends 
     // The master.compute() should run logically before the workers, so
     // increase the superstep counter it uses by one
     GraphState<I, V, E, M> graphState =
-        new GraphState<I, V, E, M>(superstep + 1, vertexCounter.getValue(),
-            edgeCounter.getValue(), getContext(), getGraphMapper(), null, 
null);
+        new GraphState<I, V, E, M>(superstep + 1,
+            GiraphStats.getInstance().getVertices().getValue(),
+            GiraphStats.getInstance().getEdges().getValue(),
+            getContext(), getGraphMapper(), null, null);
     masterCompute.setGraphState(graphState);
     if (superstep == INPUT_SUPERSTEP) {
       try {
@@ -1748,18 +1716,11 @@ public class BspServiceMaster<I extends 
    * @param globalStats Global statistics which holds new counter values
    */
   private void updateCounters(GlobalStats globalStats) {
-    vertexCounter.increment(
-        globalStats.getVertexCount() -
-            vertexCounter.getValue());
-    finishedVertexCounter.increment(
-        globalStats.getFinishedVertexCount() -
-            finishedVertexCounter.getValue());
-    edgeCounter.increment(
-        globalStats.getEdgeCount() -
-            edgeCounter.getValue());
-    sentMessagesCounter.increment(
-        globalStats.getMessageCount() -
-            sentMessagesCounter.getValue());
+    GiraphStats gs = GiraphStats.getInstance();
+    gs.getVertices().setValue(globalStats.getVertexCount());
+    gs.getFinishedVertexes().setValue(globalStats.getFinishedVertexCount());
+    gs.getEdges().setValue(globalStats.getEdgeCount());
+    gs.getSentMessages().setValue(globalStats.getMessageCount());
   }
 
   /**

Modified: 
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/EdgeListVertex.java
URL: 
http://svn.apache.org/viewvc/giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/EdgeListVertex.java?rev=1408464&r1=1408463&r2=1408464&view=diff
==============================================================================
--- 
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/EdgeListVertex.java 
(original)
+++ 
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/EdgeListVertex.java 
Mon Nov 12 21:20:42 2012
@@ -114,12 +114,7 @@ public abstract class EdgeListVertex<I e
       edgeList.add(new Edge<I, E>(targetVertexId, edgeValue));
     }
 
-    boolean halt = in.readBoolean();
-    if (halt) {
-      voteToHalt();
-    } else {
-      wakeUp();
-    }
+    readHaltBoolean(in);
   }
 
   @Override

Modified: 
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/HashMapVertex.java
URL: 
http://svn.apache.org/viewvc/giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/HashMapVertex.java?rev=1408464&r1=1408463&r2=1408464&view=diff
==============================================================================
--- 
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/HashMapVertex.java 
(original)
+++ 
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/HashMapVertex.java 
Mon Nov 12 21:20:42 2012
@@ -137,12 +137,7 @@ public abstract class HashMapVertex<I ex
       edgeMap.put(targetVertexId, edgeValue);
     }
 
-    boolean halt = in.readBoolean();
-    if (halt) {
-      voteToHalt();
-    } else {
-      wakeUp();
-    }
+    readHaltBoolean(in);
   }
 
   @Override

Modified: 
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/IntIntNullIntVertex.java
URL: 
http://svn.apache.org/viewvc/giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/IntIntNullIntVertex.java?rev=1408464&r1=1408463&r2=1408464&view=diff
==============================================================================
--- 
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/IntIntNullIntVertex.java
 (original)
+++ 
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/IntIntNullIntVertex.java
 Mon Nov 12 21:20:42 2012
@@ -94,11 +94,6 @@ public abstract class IntIntNullIntVerte
     for (int n = 0; n < numEdges; n++) {
       neighbors[n] = in.readInt();
     }
-    boolean halt = in.readBoolean();
-    if (halt) {
-      voteToHalt();
-    } else {
-      wakeUp();
-    }
+    readHaltBoolean(in);
   }
 }

Modified: 
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/LongDoubleFloatDoubleVertex.java
URL: 
http://svn.apache.org/viewvc/giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/LongDoubleFloatDoubleVertex.java?rev=1408464&r1=1408463&r2=1408464&view=diff
==============================================================================
--- 
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/LongDoubleFloatDoubleVertex.java
 (original)
+++ 
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/LongDoubleFloatDoubleVertex.java
 Mon Nov 12 21:20:42 2012
@@ -139,12 +139,7 @@ public abstract class LongDoubleFloatDou
       float edgeValue = in.readFloat();
       edgeMap.put(targetVertexId, edgeValue);
     }
-    boolean halt = in.readBoolean();
-    if (halt) {
-      voteToHalt();
-    } else {
-      wakeUp();
-    }
+    readHaltBoolean(in);
   }
 
   @Override

Modified: 
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/MasterThread.java
URL: 
http://svn.apache.org/viewvc/giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/MasterThread.java?rev=1408464&r1=1408463&r2=1408464&view=diff
==============================================================================
--- giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/MasterThread.java 
(original)
+++ giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/MasterThread.java 
Mon Nov 12 21:20:42 2012
@@ -22,6 +22,7 @@ import org.apache.giraph.GiraphConfigura
 import org.apache.giraph.bsp.ApplicationState;
 import org.apache.giraph.bsp.CentralizedServiceMaster;
 import org.apache.giraph.bsp.SuperstepState;
+import org.apache.giraph.counters.GiraphTimers;
 import org.apache.giraph.metrics.GiraphMetrics;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
@@ -75,6 +76,7 @@ public class MasterThread<I extends Writ
     super(MasterThread.class.getName());
     this.bspServiceMaster = bspServiceMaster;
     this.context = context;
+    GiraphTimers.init(context);
     superstepCounterOn = context.getConfiguration().getBoolean(
         GiraphConfiguration.USE_SUPERSTEP_COUNTERS,
         GiraphConfiguration.USE_SUPERSTEP_COUNTERS_DEFAULT);
@@ -102,9 +104,7 @@ public class MasterThread<I extends Writ
             (bspServiceMaster.createVertexInputSplits() != -1 &&
                 bspServiceMaster.createEdgeInputSplits() != -1)) {
           long setupMillis = System.currentTimeMillis() - startMillis;
-          context.getCounter(GIRAPH_TIMERS_COUNTER_GROUP_NAME,
-              "Setup (milliseconds)").
-              increment(setupMillis);
+          GiraphTimers.getInstance().getSetupMs().increment(setupMillis);
           setupSecs = setupMillis / 1000.0d;
           SuperstepState superstepState = SuperstepState.INITIAL;
           long cachedSuperstep = BspService.UNSET_SUPERSTEP;
@@ -126,16 +126,8 @@ public class MasterThread<I extends Writ
                   bspServiceMaster.getSuperstep());
             }
             if (superstepCounterOn) {
-              String counterPrefix;
-              if (cachedSuperstep == -1) {
-                counterPrefix = "Input superstep";
-              } else {
-                counterPrefix = "Superstep " + cachedSuperstep;
-              }
-              context.getCounter(GIRAPH_TIMERS_COUNTER_GROUP_NAME,
-                  counterPrefix +
-                      " (milliseconds)").
-                  increment(superstepMillis);
+              GiraphTimers.getInstance().getSuperstepMs(cachedSuperstep).
+                increment(superstepMillis);
             }
 
             // If a worker failed, restart from a known good superstep
@@ -150,10 +142,8 @@ public class MasterThread<I extends Writ
       }
       bspServiceMaster.cleanup();
       if (!superstepSecsMap.isEmpty()) {
-        context.getCounter(
-            GIRAPH_TIMERS_COUNTER_GROUP_NAME,
-            "Shutdown (milliseconds)").
-            increment(System.currentTimeMillis() - endMillis);
+        GiraphTimers.getInstance().getShutdownMs().
+          increment(System.currentTimeMillis() - endMillis);
         if (LOG.isInfoEnabled()) {
           LOG.info("setup: Took " + setupSecs + " seconds.");
         }
@@ -178,10 +168,8 @@ public class MasterThread<I extends Writ
               ((System.currentTimeMillis() - startMillis) /
               1000.0d) + " seconds.");
         }
-        context.getCounter(
-            GIRAPH_TIMERS_COUNTER_GROUP_NAME,
-            "Total (milliseconds)").
-            increment(System.currentTimeMillis() - startMillis);
+        GiraphTimers.getInstance().getTotalMs().
+          increment(System.currentTimeMillis() - startMillis);
       }
     } catch (IOException e) {
       LOG.error("masterThread: Master algorithm failed with " +

Modified: 
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/SimpleMutableVertex.java
URL: 
http://svn.apache.org/viewvc/giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/SimpleMutableVertex.java?rev=1408464&r1=1408463&r2=1408464&view=diff
==============================================================================
--- 
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/SimpleMutableVertex.java
 (original)
+++ 
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/SimpleMutableVertex.java
 Mon Nov 12 21:20:42 2012
@@ -119,12 +119,7 @@ public abstract class SimpleMutableVerte
 
     initialize(vertexId, vertexValue, edges);
 
-    boolean halt = in.readBoolean();
-    if (halt) {
-      voteToHalt();
-    } else {
-      wakeUp();
-    }
+    readHaltBoolean(in);
   }
 
   @Override

Modified: 
giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/SimpleVertex.java
URL: 
http://svn.apache.org/viewvc/giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/SimpleVertex.java?rev=1408464&r1=1408463&r2=1408464&view=diff
==============================================================================
--- giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/SimpleVertex.java 
(original)
+++ giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/SimpleVertex.java 
Mon Nov 12 21:20:42 2012
@@ -25,6 +25,7 @@ import org.apache.hadoop.io.WritableComp
 import com.google.common.base.Function;
 import com.google.common.collect.Iterables;
 
+import javax.annotation.Nullable;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
@@ -32,8 +33,6 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 
-import javax.annotation.Nullable;
-
 /**
  * Vertex with no edge values.
  * @param <I> Vertex id
@@ -97,12 +96,7 @@ public abstract class SimpleVertex<I ext
 
     initialize(vertexId, vertexValue, edges);
 
-    boolean halt = in.readBoolean();
-    if (halt) {
-      voteToHalt();
-    } else {
-      wakeUp();
-    }
+    readHaltBoolean(in);
   }
 
   @Override

Modified: giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/Vertex.java
URL: 
http://svn.apache.org/viewvc/giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/Vertex.java?rev=1408464&r1=1408463&r2=1408464&view=diff
==============================================================================
--- giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/Vertex.java 
(original)
+++ giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/Vertex.java Mon 
Nov 12 21:20:42 2012
@@ -360,6 +360,16 @@ public abstract class Vertex<I extends W
 
     initialize(vertexId, vertexValue, edges);
 
+    readHaltBoolean(in);
+  }
+
+  /**
+   * Helper method for subclasses which implement their own readFields() to 
use.
+   *
+   * @param in DataInput to read from.
+   * @throws IOException If anything goes wrong during read.
+   */
+  protected void readHaltBoolean(DataInput in) throws IOException {
     halt = in.readBoolean();
   }
 

Modified: 
giraph/trunk/giraph/src/main/java/org/apache/giraph/metrics/GiraphMetricsRegistry.java
URL: 
http://svn.apache.org/viewvc/giraph/trunk/giraph/src/main/java/org/apache/giraph/metrics/GiraphMetricsRegistry.java?rev=1408464&r1=1408463&r2=1408464&view=diff
==============================================================================
--- 
giraph/trunk/giraph/src/main/java/org/apache/giraph/metrics/GiraphMetricsRegistry.java
 (original)
+++ 
giraph/trunk/giraph/src/main/java/org/apache/giraph/metrics/GiraphMetricsRegistry.java
 Mon Nov 12 21:20:42 2012
@@ -50,7 +50,7 @@ public class GiraphMetricsRegistry {
    * Create no-op empty registry that makes no-op metrics.
    */
   public GiraphMetricsRegistry() {
-    registry = new EmptyMetricsRegistry();
+    registry = new NoOpMetricsRegistry();
     jmxReporter = null;
   }
 
@@ -67,7 +67,7 @@ public class GiraphMetricsRegistry {
       jmxReporter = new JmxReporter(registry);
       jmxReporter.start();
     } else {
-      registry = new EmptyMetricsRegistry();
+      registry = new NoOpMetricsRegistry();
       jmxReporter = null;
     }
   }

Added: 
giraph/trunk/giraph/src/main/java/org/apache/giraph/metrics/NoOpMetricsRegistry.java
URL: 
http://svn.apache.org/viewvc/giraph/trunk/giraph/src/main/java/org/apache/giraph/metrics/NoOpMetricsRegistry.java?rev=1408464&view=auto
==============================================================================
--- 
giraph/trunk/giraph/src/main/java/org/apache/giraph/metrics/NoOpMetricsRegistry.java
 (added)
+++ 
giraph/trunk/giraph/src/main/java/org/apache/giraph/metrics/NoOpMetricsRegistry.java
 Mon Nov 12 21:20:42 2012
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.metrics;
+
+import com.google.common.collect.Maps;
+import com.yammer.metrics.core.Counter;
+import com.yammer.metrics.core.NoOpCounter;
+import com.yammer.metrics.core.NoOpGuage;
+import com.yammer.metrics.core.NoOpHistogram;
+import com.yammer.metrics.core.NoOpMeter;
+import com.yammer.metrics.core.NoOpTimer;
+import com.yammer.metrics.core.Gauge;
+import com.yammer.metrics.core.Histogram;
+import com.yammer.metrics.core.Meter;
+import com.yammer.metrics.core.Metric;
+import com.yammer.metrics.core.MetricName;
+import com.yammer.metrics.core.MetricPredicate;
+import com.yammer.metrics.core.MetricsRegistry;
+import com.yammer.metrics.core.MetricsRegistryListener;
+import com.yammer.metrics.core.Timer;
+
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * An empty MetricsRegistry, used to easily turn off all metrics without
+ * affecting client code.
+ */
+public class NoOpMetricsRegistry extends MetricsRegistry {
+  @Override
+  public <T> Gauge<T> newGauge(Class<?> klass, String name, Gauge<T> metric) {
+    return NoOpGuage.INSTANCE;
+  }
+
+  @Override
+  public <T> Gauge<T> newGauge(Class<?> klass, String name, String scope,
+                               Gauge<T> metric) {
+    return NoOpGuage.INSTANCE;
+  }
+
+  @Override
+  public <T> Gauge<T> newGauge(MetricName metricName, Gauge<T> metric) {
+    return NoOpGuage.INSTANCE;
+  }
+
+  @Override
+  public Counter newCounter(Class<?> klass, String name) {
+    return NoOpCounter.INSTANCE;
+  }
+
+  @Override
+  public Counter newCounter(Class<?> klass, String name, String scope) {
+    return NoOpCounter.INSTANCE;
+  }
+
+  @Override
+  public Counter newCounter(MetricName metricName) {
+    return NoOpCounter.INSTANCE;
+  }
+
+  @Override
+  public Histogram newHistogram(Class<?> klass, String name, boolean biased) {
+    return NoOpHistogram.INSTANCE;
+  }
+
+  @Override
+  public Histogram newHistogram(Class<?> klass, String name, String scope,
+                                boolean biased) {
+    return NoOpHistogram.INSTANCE;
+  }
+
+  @Override
+  public Histogram newHistogram(Class<?> klass, String name) {
+    return NoOpHistogram.INSTANCE;
+  }
+
+  @Override
+  public Histogram newHistogram(Class<?> klass, String name, String scope) {
+    return NoOpHistogram.INSTANCE;
+  }
+
+  @Override
+  public Histogram newHistogram(MetricName metricName, boolean biased) {
+    return NoOpHistogram.INSTANCE;
+  }
+
+  @Override
+  public Meter newMeter(Class<?> klass, String name, String eventType,
+                        TimeUnit unit) {
+    return NoOpMeter.INSTANCE;
+  }
+
+  @Override
+  public Meter newMeter(
+    Class<?> klass, String name, String scope, String eventType, TimeUnit unit
+  ) {
+    return NoOpMeter.INSTANCE;
+  }
+
+  @Override
+  public Meter newMeter(MetricName metricName, String eventType,
+                        TimeUnit unit) {
+    return NoOpMeter.INSTANCE;
+  }
+
+  @Override
+  public Timer newTimer(Class<?> klass, String name) {
+    return NoOpTimer.INSTANCE;
+  }
+
+  @Override
+  public Timer newTimer(Class<?> klass, String name, TimeUnit durationUnit,
+                        TimeUnit rateUnit) {
+    return NoOpTimer.INSTANCE;
+  }
+
+  @Override
+  public Timer newTimer(Class<?> klass, String name, String scope) {
+    return NoOpTimer.INSTANCE;
+  }
+
+  @Override
+  public Timer newTimer(Class<?> klass, String name, String scope,
+                        TimeUnit durationUnit, TimeUnit rateUnit
+  ) {
+    return NoOpTimer.INSTANCE;
+  }
+
+  @Override
+  public Timer newTimer(MetricName metricName, TimeUnit durationUnit,
+                        TimeUnit rateUnit) {
+    return NoOpTimer.INSTANCE;
+  }
+
+  @Override
+  public Map<MetricName, Metric> allMetrics() {
+    return Maps.newHashMap();
+  }
+
+  @Override
+  public SortedMap<String, SortedMap<MetricName, Metric>> groupedMetrics() {
+    return Maps.newTreeMap();
+  }
+
+  @Override
+  public SortedMap<String, SortedMap<MetricName, Metric>>
+  groupedMetrics(MetricPredicate predicate) {
+    return Maps.newTreeMap();
+  }
+
+  @Override
+  public void shutdown() { }
+
+  @Override
+  public ScheduledExecutorService newScheduledThreadPool(int poolSize,
+                                                         String name) {
+    return null;
+  }
+
+  @Override
+  public void removeMetric(Class<?> klass, String name) { }
+
+  @Override
+  public void removeMetric(Class<?> klass, String name, String scope) { }
+
+  @Override
+  public void removeMetric(MetricName name) { }
+
+  @Override
+  public void addListener(MetricsRegistryListener listener) { }
+
+  @Override
+  public void removeListener(MetricsRegistryListener listener) { }
+
+  @Override
+  protected ConcurrentMap<MetricName, Metric> newMetricsMap() {
+    return Maps.newConcurrentMap();
+  }
+}

Modified: 
giraph/trunk/giraph/src/main/java/org/apache/giraph/metrics/package-info.java
URL: 
http://svn.apache.org/viewvc/giraph/trunk/giraph/src/main/java/org/apache/giraph/metrics/package-info.java?rev=1408464&r1=1408463&r2=1408464&view=diff
==============================================================================
--- 
giraph/trunk/giraph/src/main/java/org/apache/giraph/metrics/package-info.java 
(original)
+++ 
giraph/trunk/giraph/src/main/java/org/apache/giraph/metrics/package-info.java 
Mon Nov 12 21:20:42 2012
@@ -16,6 +16,6 @@
  * limitations under the License.
  */
 /**
- * Package of all helper integration test objects.
+ * Package of all the Giraph metrics related data.
  */
 package org.apache.giraph.metrics;


Reply via email to