You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by el...@apache.org on 2014/12/06 00:12:49 UTC

[1/2] accumulo git commit: ACCUMULO-1817 Introduce usage of Hadoop Metrics2 by default for JMX and other metrics sinks (graphite and ganglia)

Repository: accumulo
Updated Branches:
  refs/heads/master 856496138 -> 2821a4815


http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMetricsUtil.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMetricsUtil.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMetricsUtil.java
new file mode 100644
index 0000000..5905aea
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMetricsUtil.java
@@ -0,0 +1,138 @@
+/*
+ * 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.accumulo.tserver.metrics;
+
+import org.apache.accumulo.tserver.TabletServer;
+import org.apache.accumulo.tserver.tablet.Tablet;
+
+/**
+ * Wrapper around extracting metrics from a TabletServer instance
+ *
+ * Necessary to support both old custom JMX metrics and Hadoop Metrics2
+ */
+public class TabletServerMetricsUtil {
+
+  private final TabletServer tserver;
+
+  public TabletServerMetricsUtil(TabletServer tserver) {
+    this.tserver = tserver;
+  }
+
+  public long getEntries() {
+    long result = 0;
+    for (Tablet tablet : tserver.getOnlineTablets()) {
+      result += tablet.getNumEntries();
+    }
+    return result;
+  }
+
+  public long getEntriesInMemory() {
+    long result = 0;
+    for (Tablet tablet : tserver.getOnlineTablets()) {
+      result += tablet.getNumEntriesInMemory();
+    }
+    return result;
+  }
+
+  public long getIngest() {
+    long result = 0;
+    for (Tablet tablet : tserver.getOnlineTablets()) {
+      result += tablet.getNumEntriesInMemory();
+    }
+    return result;
+  }
+
+  public int getMajorCompactions() {
+    int result = 0;
+    for (Tablet tablet : tserver.getOnlineTablets()) {
+      if (tablet.isMajorCompactionRunning())
+        result++;
+    }
+    return result;
+  }
+
+  public int getMajorCompactionsQueued() {
+    int result = 0;
+    for (Tablet tablet : tserver.getOnlineTablets()) {
+      if (tablet.isMajorCompactionQueued())
+        result++;
+    }
+    return result;
+  }
+
+  public int getMinorCompactions() {
+    int result = 0;
+    for (Tablet tablet : tserver.getOnlineTablets()) {
+      if (tablet.isMinorCompactionRunning())
+        result++;
+    }
+    return result;
+  }
+
+  public int getMinorCompactionsQueued() {
+    int result = 0;
+    for (Tablet tablet : tserver.getOnlineTablets()) {
+      if (tablet.isMinorCompactionQueued())
+        result++;
+    }
+    return result;
+  }
+
+  public int getOnlineCount() {
+    return tserver.getOnlineTablets().size();
+  }
+
+  public int getOpeningCount() {
+    return tserver.getOpeningCount();
+  }
+
+  public long getQueries() {
+    long result = 0;
+    for (Tablet tablet : tserver.getOnlineTablets()) {
+      result += tablet.totalQueries();
+    }
+    return result;
+  }
+
+  public int getUnopenedCount() {
+    return tserver.getUnopenedCount();
+  }
+
+  public String getName() {
+    return tserver.getClientAddressString();
+  }
+
+  public long getTotalMinorCompactions() {
+    return tserver.getTotalMinorCompactions();
+  }
+
+  public double getHoldTime() {
+    return tserver.getHoldTimeMillis() / 1000.;
+  }
+
+  public double getAverageFilesPerTablet() {
+    int count = 0;
+    long result = 0;
+    for (Tablet tablet : tserver.getOnlineTablets()) {
+      result += tablet.getDatafiles().size();
+      count++;
+    }
+    if (count == 0)
+      return 0;
+    return result / (double) count;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetrics.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetrics.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetrics.java
index a89bb45..f4d61ca 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetrics.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetrics.java
@@ -21,14 +21,14 @@ import javax.management.ObjectName;
 import org.apache.accumulo.server.metrics.AbstractMetricsImpl;
 
 public class TabletServerMinCMetrics extends AbstractMetricsImpl implements TabletServerMinCMetricsMBean {
-  
+
   static final org.apache.log4j.Logger log = org.apache.log4j.Logger.getLogger(TabletServerMinCMetrics.class);
-  
+
   private static final String METRICS_PREFIX = "tserver.minc";
-  
+
   private static ObjectName OBJECT_NAME = null;
-  
-  public TabletServerMinCMetrics() {
+
+  TabletServerMinCMetrics() {
     super();
     reset();
     try {
@@ -37,52 +37,61 @@ public class TabletServerMinCMetrics extends AbstractMetricsImpl implements Tabl
       log.error("Exception setting MBean object name", e);
     }
   }
-  
+
   @Override
   protected ObjectName getObjectName() {
     return OBJECT_NAME;
   }
-  
+
   @Override
   protected String getMetricsPrefix() {
     return METRICS_PREFIX;
   }
-  
+
+  @Override
   public long getMinorCompactionMinTime() {
-    return this.getMetricMin(minc);
+    return this.getMetricMin(MINC);
   }
-  
+
+  @Override
   public long getMinorCompactionAvgTime() {
-    return this.getMetricAvg(minc);
+    return this.getMetricAvg(MINC);
   }
-  
+
+  @Override
   public long getMinorCompactionCount() {
-    return this.getMetricCount(minc);
+    return this.getMetricCount(MINC);
   }
-  
+
+  @Override
   public long getMinorCompactionMaxTime() {
-    return this.getMetricMax(minc);
+    return this.getMetricMax(MINC);
   }
-  
+
+  @Override
   public long getMinorCompactionQueueAvgTime() {
-    return this.getMetricAvg(queue);
+    return this.getMetricAvg(QUEUE);
   }
-  
+
+  @Override
   public long getMinorCompactionQueueCount() {
-    return this.getMetricCount(queue);
+    return this.getMetricCount(QUEUE);
   }
-  
+
+  @Override
   public long getMinorCompactionQueueMaxTime() {
-    return this.getMetricMax(queue);
+    return this.getMetricMax(QUEUE);
   }
-  
+
+  @Override
   public long getMinorCompactionQueueMinTime() {
-    return this.getMetricMin(minc);
+    return this.getMetricMin(MINC);
   }
-  
+
+  @Override
   public void reset() {
     createMetric("minc");
     createMetric("queue");
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetricsKeys.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetricsKeys.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetricsKeys.java
new file mode 100644
index 0000000..29d4846
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetricsKeys.java
@@ -0,0 +1,27 @@
+/*
+ * 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.accumulo.tserver.metrics;
+
+/**
+ * Keys to reference minor compaction metrics
+ */
+public interface TabletServerMinCMetricsKeys {
+
+  String MINC = "minc";
+  String QUEUE = "queue";
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetricsMBean.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetricsMBean.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetricsMBean.java
index 0dc6cf3..2af820c 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetricsMBean.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetricsMBean.java
@@ -16,27 +16,24 @@
  */
 package org.apache.accumulo.tserver.metrics;
 
-public interface TabletServerMinCMetricsMBean {
-  
-  static final String minc = "minc";
-  static final String queue = "queue";
-  
+public interface TabletServerMinCMetricsMBean extends TabletServerMinCMetricsKeys {
+
   long getMinorCompactionCount();
-  
+
   long getMinorCompactionAvgTime();
-  
+
   long getMinorCompactionMinTime();
-  
+
   long getMinorCompactionMaxTime();
-  
+
   long getMinorCompactionQueueCount();
-  
+
   long getMinorCompactionQueueAvgTime();
-  
+
   long getMinorCompactionQueueMinTime();
-  
+
   long getMinorCompactionQueueMaxTime();
-  
+
   void reset();
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetrics.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetrics.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetrics.java
index 142f171..6bcabfd 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetrics.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetrics.java
@@ -21,14 +21,14 @@ import javax.management.ObjectName;
 import org.apache.accumulo.server.metrics.AbstractMetricsImpl;
 
 public class TabletServerScanMetrics extends AbstractMetricsImpl implements TabletServerScanMetricsMBean {
-  
+
   static final org.apache.log4j.Logger log = org.apache.log4j.Logger.getLogger(TabletServerScanMetrics.class);
-  
+
   public static final String METRICS_PREFIX = "tserver.scan";
-  
+
   static ObjectName OBJECT_NAME = null;
-  
-  public TabletServerScanMetrics() {
+
+  TabletServerScanMetrics() {
     super();
     reset();
     try {
@@ -37,52 +37,61 @@ public class TabletServerScanMetrics extends AbstractMetricsImpl implements Tabl
       log.error("Exception setting MBean object name", e);
     }
   }
-  
+
   @Override
   protected ObjectName getObjectName() {
     return OBJECT_NAME;
   }
-  
+
   @Override
   protected String getMetricsPrefix() {
     return METRICS_PREFIX;
   }
-  
+
+  @Override
   public long getResultAvgSize() {
-    return this.getMetricAvg(resultSize);
+    return this.getMetricAvg(RESULT_SIZE);
   }
-  
+
+  @Override
   public long getResultCount() {
-    return this.getMetricCount(resultSize);
+    return this.getMetricCount(RESULT_SIZE);
   }
-  
+
+  @Override
   public long getResultMaxSize() {
-    return this.getMetricMax(resultSize);
+    return this.getMetricMax(RESULT_SIZE);
   }
-  
+
+  @Override
   public long getResultMinSize() {
-    return this.getMetricMin(resultSize);
+    return this.getMetricMin(RESULT_SIZE);
   }
-  
+
+  @Override
   public long getScanAvgTime() {
-    return this.getMetricAvg(scan);
+    return this.getMetricAvg(SCAN);
   }
-  
+
+  @Override
   public long getScanCount() {
-    return this.getMetricCount(scan);
+    return this.getMetricCount(SCAN);
   }
-  
+
+  @Override
   public long getScanMaxTime() {
-    return this.getMetricMax(scan);
+    return this.getMetricMax(SCAN);
   }
-  
+
+  @Override
   public long getScanMinTime() {
-    return this.getMetricMin(scan);
+    return this.getMetricMin(SCAN);
   }
-  
+
+  @Override
   public void reset() {
-    createMetric(scan);
-    createMetric(resultSize);
+    createMetric(SCAN);
+    createMetric(RESULT_SIZE);
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetricsKeys.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetricsKeys.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetricsKeys.java
new file mode 100644
index 0000000..7d33a84
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetricsKeys.java
@@ -0,0 +1,27 @@
+/*
+ * 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.accumulo.tserver.metrics;
+
+/**
+ * Keys for referencing scan metrics
+ */
+public interface TabletServerScanMetricsKeys {
+
+  String SCAN = "scan";
+  String RESULT_SIZE = "result";
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetricsMBean.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetricsMBean.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetricsMBean.java
index b532cfa..e262fc4 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetricsMBean.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetricsMBean.java
@@ -16,27 +16,24 @@
  */
 package org.apache.accumulo.tserver.metrics;
 
-public interface TabletServerScanMetricsMBean {
-  
-  static final String scan = "scan";
-  static final String resultSize = "result";
-  
+public interface TabletServerScanMetricsMBean extends TabletServerScanMetricsKeys {
+
   long getScanCount();
-  
+
   long getScanAvgTime();
-  
+
   long getScanMinTime();
-  
+
   long getScanMaxTime();
-  
+
   long getResultCount();
-  
+
   long getResultAvgSize();
-  
+
   long getResultMinSize();
-  
+
   long getResultMaxSize();
-  
+
   void reset();
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetrics.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetrics.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetrics.java
index b23fb1a..48b78b0 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetrics.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetrics.java
@@ -21,14 +21,14 @@ import javax.management.ObjectName;
 import org.apache.accumulo.server.metrics.AbstractMetricsImpl;
 
 public class TabletServerUpdateMetrics extends AbstractMetricsImpl implements TabletServerUpdateMetricsMBean {
-  
+
   static final org.apache.log4j.Logger log = org.apache.log4j.Logger.getLogger(TabletServerUpdateMetrics.class);
-  
+
   private static final String METRICS_PREFIX = "tserver.update";
-  
+
   private static ObjectName OBJECT_NAME = null;
-  
-  public TabletServerUpdateMetrics() {
+
+  TabletServerUpdateMetrics() {
     super();
     reset();
     try {
@@ -38,97 +38,116 @@ public class TabletServerUpdateMetrics extends AbstractMetricsImpl implements Ta
       log.error("Exception setting MBean object name", e);
     }
   }
-  
+
   @Override
   protected ObjectName getObjectName() {
     return OBJECT_NAME;
   }
-  
+
   @Override
   protected String getMetricsPrefix() {
     return METRICS_PREFIX;
   }
-  
+
+  @Override
   public long getPermissionErrorCount() {
-    return this.getMetricCount(permissionErrors);
+    return this.getMetricCount(PERMISSION_ERRORS);
   }
-  
+
+  @Override
   public long getUnknownTabletErrorCount() {
-    return this.getMetricCount(unknownTabletErrors);
+    return this.getMetricCount(UNKNOWN_TABLET_ERRORS);
   }
-  
+
+  @Override
   public long getMutationArrayAvgSize() {
-    return this.getMetricAvg(mutationArraySize);
+    return this.getMetricAvg(MUTATION_ARRAY_SIZE);
   }
-  
+
+  @Override
   public long getMutationArrayMinSize() {
-    return this.getMetricMin(mutationArraySize);
+    return this.getMetricMin(MUTATION_ARRAY_SIZE);
   }
-  
+
+  @Override
   public long getMutationArrayMaxSize() {
-    return this.getMetricMax(mutationArraySize);
+    return this.getMetricMax(MUTATION_ARRAY_SIZE);
   }
-  
+
+  @Override
   public long getCommitPrepCount() {
-    return this.getMetricCount(commitPrep);
+    return this.getMetricCount(COMMIT_PREP);
   }
-  
+
+  @Override
   public long getCommitPrepMinTime() {
-    return this.getMetricMin(commitPrep);
+    return this.getMetricMin(COMMIT_PREP);
   }
-  
+
+  @Override
   public long getCommitPrepMaxTime() {
-    return this.getMetricMax(commitPrep);
+    return this.getMetricMax(COMMIT_PREP);
   }
-  
+
+  @Override
   public long getCommitPrepAvgTime() {
-    return this.getMetricAvg(commitPrep);
+    return this.getMetricAvg(COMMIT_PREP);
   }
-  
+
+  @Override
   public long getConstraintViolationCount() {
-    return this.getMetricCount(constraintViolations);
+    return this.getMetricCount(CONSTRAINT_VIOLATIONS);
   }
-  
+
+  @Override
   public long getWALogWriteCount() {
-    return this.getMetricCount(waLogWriteTime);
+    return this.getMetricCount(WALOG_WRITE_TIME);
   }
-  
+
+  @Override
   public long getWALogWriteMinTime() {
-    return this.getMetricMin(waLogWriteTime);
+    return this.getMetricMin(WALOG_WRITE_TIME);
   }
-  
+
+  @Override
   public long getWALogWriteMaxTime() {
-    return this.getMetricMax(waLogWriteTime);
+    return this.getMetricMax(WALOG_WRITE_TIME);
   }
-  
+
+  @Override
   public long getWALogWriteAvgTime() {
-    return this.getMetricAvg(waLogWriteTime);
+    return this.getMetricAvg(WALOG_WRITE_TIME);
   }
-  
+
+  @Override
   public long getCommitCount() {
-    return this.getMetricCount(commitTime);
+    return this.getMetricCount(COMMIT_TIME);
   }
-  
+
+  @Override
   public long getCommitMinTime() {
-    return this.getMetricMin(commitTime);
+    return this.getMetricMin(COMMIT_TIME);
   }
-  
+
+  @Override
   public long getCommitMaxTime() {
-    return this.getMetricMax(commitTime);
+    return this.getMetricMax(COMMIT_TIME);
   }
-  
+
+  @Override
   public long getCommitAvgTime() {
-    return this.getMetricAvg(commitTime);
+    return this.getMetricAvg(COMMIT_TIME);
   }
-  
+
+  @Override
   public void reset() {
-    createMetric(permissionErrors);
-    createMetric(unknownTabletErrors);
-    createMetric(mutationArraySize);
-    createMetric(commitPrep);
-    createMetric(constraintViolations);
-    createMetric(waLogWriteTime);
-    createMetric(commitTime);
-  }
-  
+    createMetric(PERMISSION_ERRORS);
+    createMetric(UNKNOWN_TABLET_ERRORS);
+    createMetric(MUTATION_ARRAY_SIZE);
+    createMetric(COMMIT_PREP);
+    createMetric(CONSTRAINT_VIOLATIONS);
+    createMetric(WALOG_WRITE_TIME);
+    createMetric(COMMIT_TIME);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetricsKeys.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetricsKeys.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetricsKeys.java
new file mode 100644
index 0000000..f06751b
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetricsKeys.java
@@ -0,0 +1,32 @@
+/*
+ * 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.accumulo.tserver.metrics;
+
+/**
+ * Keys to identify which update metric is being altered
+ */
+public interface TabletServerUpdateMetricsKeys {
+
+  static String PERMISSION_ERRORS = "permissionErrors";
+  static String UNKNOWN_TABLET_ERRORS = "unknownTabletErrors";
+  static String MUTATION_ARRAY_SIZE = "mutationArraysSize";
+  static String COMMIT_PREP = "commitPrep";
+  static String CONSTRAINT_VIOLATIONS = "constraintViolations";
+  static String WALOG_WRITE_TIME = "waLogWriteTime";
+  static String COMMIT_TIME = "commitTime";
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetricsMBean.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetricsMBean.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetricsMBean.java
index 0292947..867f2f7 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetricsMBean.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetricsMBean.java
@@ -16,51 +16,43 @@
  */
 package org.apache.accumulo.tserver.metrics;
 
-public interface TabletServerUpdateMetricsMBean {
-  
-  final static String permissionErrors = "permissionErrors";
-  final static String unknownTabletErrors = "unknownTabletErrors";
-  final static String mutationArraySize = "mutationArraysSize";
-  final static String commitPrep = "commitPrep";
-  final static String constraintViolations = "constraintViolations";
-  final static String waLogWriteTime = "waLogWriteTime";
-  final static String commitTime = "commitTime";
-  
+public interface TabletServerUpdateMetricsMBean extends TabletServerUpdateMetricsKeys {
+
   long getPermissionErrorCount();
-  
+
   long getUnknownTabletErrorCount();
-  
+
   long getMutationArrayAvgSize();
-  
+
   long getMutationArrayMinSize();
-  
+
   long getMutationArrayMaxSize();
-  
+
   long getCommitPrepCount();
-  
+
   long getCommitPrepMinTime();
-  
+
   long getCommitPrepMaxTime();
-  
+
   long getCommitPrepAvgTime();
-  
+
   long getConstraintViolationCount();
-  
+
   long getWALogWriteCount();
-  
+
   long getWALogWriteMinTime();
-  
+
   long getWALogWriteMaxTime();
-  
+
   long getWALogWriteAvgTime();
-  
+
   long getCommitCount();
-  
+
   long getCommitMinTime();
-  
+
   long getCommitMaxTime();
-  
+
   long getCommitAvgTime();
-  
+
   void reset();
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index 82e2be2..c0c8b53 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -98,6 +98,7 @@ import org.apache.accumulo.server.fs.VolumeUtil;
 import org.apache.accumulo.server.fs.VolumeUtil.TabletFiles;
 import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.master.tableOps.UserCompactionConfig;
+import org.apache.accumulo.server.metrics.Metrics;
 import org.apache.accumulo.server.problems.ProblemReport;
 import org.apache.accumulo.server.problems.ProblemReports;
 import org.apache.accumulo.server.problems.ProblemType;
@@ -972,13 +973,13 @@ public class Tablet implements TabletCommitter {
       if (!failed) {
         lastMinorCompactionFinishTime = System.currentTimeMillis();
       }
-      TabletServerMinCMetrics minCMetrics = getTabletServer().getMinCMetrics();
+      Metrics minCMetrics = getTabletServer().getMinCMetrics();
       if (minCMetrics.isEnabled())
-        minCMetrics.add(TabletServerMinCMetrics.minc, (lastMinorCompactionFinishTime - start));
+        minCMetrics.add(TabletServerMinCMetrics.MINC, (lastMinorCompactionFinishTime - start));
       if (hasQueueTime) {
         timer.updateTime(Operation.MINOR, queued, start, count, failed);
         if (minCMetrics.isEnabled())
-          minCMetrics.add(TabletServerMinCMetrics.queue, (start - queued));
+          minCMetrics.add(TabletServerMinCMetrics.QUEUE, (start - queued));
       } else
         timer.updateTime(Operation.MINOR, start, count, failed);
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
index ef18efd..4094c5f 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
@@ -102,7 +102,7 @@ public class ZombieTServer {
     TransactionWatcher watcher = new TransactionWatcher();
     final ThriftClientHandler tch = new ThriftClientHandler(context, watcher);
     Processor<Iface> processor = new Processor<Iface>(tch);
-    ServerAddress serverPort = TServerUtils.startTServer(HostAndPort.fromParts("0.0.0.0", port), processor, "ZombieTServer", "walking dead", 2, 1, 1000,
+    ServerAddress serverPort = TServerUtils.startTServer(context.getConfiguration(), HostAndPort.fromParts("0.0.0.0", port), processor, "ZombieTServer", "walking dead", 2, 1, 1000,
         10 * 1024 * 1024, null, -1);
     
     String addressString = serverPort.address.toString();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java b/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
index ae07925..671ead6 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
@@ -247,17 +247,17 @@ public class NullTserver {
     Opts opts = new Opts();
     opts.parseArgs(NullTserver.class.getName(), args);
 
+    // modify metadata
+    ZooKeeperInstance zki = new ZooKeeperInstance(new ClientConfiguration().withInstance(opts.iname).withZkHosts(opts.keepers));
+    AccumuloServerContext context = new AccumuloServerContext(new ServerConfigurationFactory(zki));
+
     TransactionWatcher watcher = new TransactionWatcher();
     ThriftClientHandler tch = new ThriftClientHandler(new AccumuloServerContext(new ServerConfigurationFactory(HdfsZooInstance.getInstance())), watcher);
     Processor<Iface> processor = new Processor<Iface>(tch);
-    TServerUtils.startTServer(HostAndPort.fromParts("0.0.0.0", opts.port), processor, "NullTServer", "null tserver", 2, 1, 1000, 10 * 1024 * 1024, null, -1);
+    TServerUtils.startTServer(context.getConfiguration(), HostAndPort.fromParts("0.0.0.0", opts.port), processor, "NullTServer", "null tserver", 2, 1, 1000, 10 * 1024 * 1024, null, -1);
 
     HostAndPort addr = HostAndPort.fromParts(InetAddress.getLocalHost().getHostName(), opts.port);
 
-    // modify metadata
-    ZooKeeperInstance zki = new ZooKeeperInstance(new ClientConfiguration().withInstance(opts.iname).withZkHosts(opts.keepers));
-    AccumuloServerContext context = new AccumuloServerContext(new ServerConfigurationFactory(zki));
-
     String tableId = Tables.getTableId(zki, opts.tableName);
 
     // read the locations for the table


[2/2] accumulo git commit: ACCUMULO-1817 Introduce usage of Hadoop Metrics2 by default for JMX and other metrics sinks (graphite and ganglia)

Posted by el...@apache.org.
ACCUMULO-1817 Introduce usage of Hadoop Metrics2 by default for JMX and other metrics sinks (graphite and ganglia)

Preserves "old" custom JMX MBean registration via Property.general.legacy.metrics, otherwise, Metrics2
classes are used. Example hadoop-metrics2-accumulo.properties configuration file is provided, which
enables nothing but serves as a default for users to enable file collection, graphite, or ganglia.

Cleaned up a bit of ill-formatted code, consolidated utility methods and constants across both implementations,
and introduced factory methods to create each type of metric collection class we provide.


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/2821a481
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/2821a481
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/2821a481

Branch: refs/heads/master
Commit: 2821a481563a7411a170c6776add917693c056c7
Parents: 8564961
Author: Josh Elser <el...@apache.org>
Authored: Fri Dec 5 18:05:43 2014 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Fri Dec 5 18:05:43 2014 -0500

----------------------------------------------------------------------
 .../hadoop-metrics2-accumulo.properties         |  86 ++++++++++++
 .../org/apache/accumulo/core/conf/Property.java |   2 +
 .../server/metrics/AbstractMetricsImpl.java     |  91 ++++++------
 .../apache/accumulo/server/metrics/Metrics.java |  34 +++++
 .../server/metrics/Metrics2ThriftMetrics.java   |  65 +++++++++
 .../accumulo/server/metrics/MetricsFactory.java |  53 +++++++
 .../accumulo/server/metrics/ThriftMetrics.java  |  43 +++---
 .../server/metrics/ThriftMetricsKeys.java       |  27 ++++
 .../server/metrics/ThriftMetricsMBean.java      |  25 ++--
 .../accumulo/server/util/TServerUtils.java      |  18 ++-
 .../accumulo/gc/SimpleGarbageCollector.java     |   2 +-
 .../java/org/apache/accumulo/master/Master.java |  11 +-
 .../master/metrics/MasterMetricsFactory.java    |  57 ++++++++
 .../metrics/Metrics2ReplicationMetrics.java     | 127 +++++++++++++++++
 .../master/metrics/ReplicationMetrics.java      |  13 +-
 .../apache/accumulo/tserver/TabletServer.java   |  84 ++++++-----
 .../metrics/Metrics2TabletServerMetrics.java    | 106 ++++++++++++++
 .../Metrics2TabletServerMinCMetrics.java        |  74 ++++++++++
 .../Metrics2TabletServerScanMetrics.java        |  75 ++++++++++
 .../Metrics2TabletServerUpdateMetrics.java      |  94 +++++++++++++
 .../tserver/metrics/TabletServerMBeanImpl.java  | 102 +++++---------
 .../metrics/TabletServerMetricsFactory.java     |  92 +++++++++++++
 .../metrics/TabletServerMetricsKeys.java        |  38 +++++
 .../metrics/TabletServerMetricsUtil.java        | 138 +++++++++++++++++++
 .../metrics/TabletServerMinCMetrics.java        |  59 ++++----
 .../metrics/TabletServerMinCMetricsKeys.java    |  27 ++++
 .../metrics/TabletServerMinCMetricsMBean.java   |  25 ++--
 .../metrics/TabletServerScanMetrics.java        |  63 +++++----
 .../metrics/TabletServerScanMetricsKeys.java    |  27 ++++
 .../metrics/TabletServerScanMetricsMBean.java   |  25 ++--
 .../metrics/TabletServerUpdateMetrics.java      | 125 ++++++++++-------
 .../metrics/TabletServerUpdateMetricsKeys.java  |  32 +++++
 .../metrics/TabletServerUpdateMetricsMBean.java |  48 +++----
 .../apache/accumulo/tserver/tablet/Tablet.java  |   7 +-
 .../accumulo/test/functional/ZombieTServer.java |   2 +-
 .../test/performance/thrift/NullTserver.java    |  10 +-
 36 files changed, 1535 insertions(+), 372 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/assemble/conf/templates/hadoop-metrics2-accumulo.properties
----------------------------------------------------------------------
diff --git a/assemble/conf/templates/hadoop-metrics2-accumulo.properties b/assemble/conf/templates/hadoop-metrics2-accumulo.properties
new file mode 100644
index 0000000..7e3c7c0
--- /dev/null
+++ b/assemble/conf/templates/hadoop-metrics2-accumulo.properties
@@ -0,0 +1,86 @@
+# 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.
+
+# Poll collectors every ten seconds
+*.period=10
+
+#
+# MetricSink configuration
+# 
+# Uncomment a sink (and configuration) to enable it. By default,
+# no sinks are enabled.
+#
+
+#
+# Configure file logging
+#
+
+# File sink for all metrics
+# accumulo.sink.file-all.class=org.apache.hadoop.metrics2.sink.FileSink
+# accumulo.sink.file-all.filename=all.metrics
+
+# File sink for tserver metrics
+# accumulo.sink.file-tserver.class=org.apache.hadoop.metrics2.sink.FileSink
+# accumulo.sink.file-tserver.context=tserver
+# accumulo.sink.file-tserver.filename=tserver.metrics
+
+# File sink for master metrics
+# accumulo.sink.file-master.class=org.apache.hadoop.metrics2.sink.FileSink
+# accumulo.sink.file-master.context=master
+# accumulo.sink.file-master.filename=master.metrics
+
+# File sink for thrift server metrics
+# accumulo.sink.file-thrift.class=org.apache.hadoop.metrics2.sink.FileSink
+# accumulo.sink.file-thrift.context=thrift
+# accumulo.sink.file-thrift.filename=thrift.metrics
+
+#
+# Configure Graphite
+#
+
+# Configure all sources to be written to a Graphite instance
+# accumulo.sink.graphite.class=org.apache.hadoop.metrics2.sink.GraphiteSink
+# accumulo.sink.graphite.server_host=localhost
+# accumulo.sink.graphite.server_port=2003
+
+#
+# Configure Ganglia
+#
+
+# for Ganglia 3.0 support
+# accumulo.sink.ganglia.class=org.apache.hadoop.metrics2.sink.ganglia.GangliaSink30
+
+# for Ganglia 3.1 support
+# accumulo.sink.ganglia.class=org.apache.hadoop.metrics2.sink.ganglia.GangliaSink31
+
+# accumulo.sink.ganglia.period=10
+
+# default for supportsparse is false
+# accumulo.sink.ganglia.supportsparse=true
+
+# accumulo.sink.ganglia.slope=jvm.metrics.gcCount=zero,jvm.metrics.memHeapUsedM=both
+# accumulo.sink.ganglia.dmax=jvm.metrics.threadsBlocked=70,jvm.metrics.memHeapUsedM=40
+
+# Tag values to use for the ganglia prefix. If not defined no tags are used.
+# If '*' all tags are used. If specifiying multiple tags separate them with
+# commas. Note that the last segment of the property name is the context name.
+#
+# accumulo.sink.ganglia.tagsForPrefix.jvm=ProcesName
+# accumulo.sink.ganglia.tagsForPrefix.tserver=
+# accumulo.sink.ganglia.tagsForPrefix.master=
+# accumulo.sink.ganglia.tagsForPrefix.thrift=
+
+# Ganglia host(s)
+# accumulo.sink.ganglia.servers=yourgangliahost_1:8649,yourgangliahost_2:8649

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/core/src/main/java/org/apache/accumulo/core/conf/Property.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index c2c3587..cc7d548 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -168,6 +168,8 @@ public enum Property {
   GENERAL_VOLUME_CHOOSER("general.volume.chooser", "org.apache.accumulo.server.fs.RandomVolumeChooser", PropertyType.CLASSNAME,
       "The class that will be used to select which volume will be used to create new files."),
   GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS("general.security.credential.provider.paths", "", PropertyType.STRING, "Comma-separated list of paths to CredentialProviders"),
+  GENERAL_LEGACY_METRICS("general.legacy.metrics", "false", PropertyType.BOOLEAN,
+      "Use the old metric infrastructure configured by accumulo-metrics.xml, instead of Hadoop Metrics2"),
 
   // properties that are specific to master server behavior
   MASTER_PREFIX("master.", null, PropertyType.PREFIX, "Properties in this category affect the behavior of the master server"),

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/base/src/main/java/org/apache/accumulo/server/metrics/AbstractMetricsImpl.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/metrics/AbstractMetricsImpl.java b/server/base/src/main/java/org/apache/accumulo/server/metrics/AbstractMetricsImpl.java
index 54ca8de..657fc31 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/metrics/AbstractMetricsImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/metrics/AbstractMetricsImpl.java
@@ -36,87 +36,87 @@ import org.apache.commons.lang.builder.ToStringBuilder;
 import org.apache.commons.lang.builder.ToStringStyle;
 import org.apache.commons.lang.time.DateUtils;
 
-public abstract class AbstractMetricsImpl {
-  
+public abstract class AbstractMetricsImpl implements Metrics {
+
   public class Metric {
-    
+
     private long count = 0;
     private long avg = 0;
     private long min = 0;
     private long max = 0;
-    
+
     public long getCount() {
       return count;
     }
-    
+
     public long getAvg() {
       return avg;
     }
-    
+
     public long getMin() {
       return min;
     }
-    
+
     public long getMax() {
       return max;
     }
-    
+
     public void incCount() {
       count++;
     }
-    
+
     public void addAvg(long a) {
       if (a < 0)
         return;
       avg = (long) ((avg * .8) + (a * .2));
     }
-    
+
     public void addMin(long a) {
       if (a < 0)
         return;
       min = Math.min(min, a);
     }
-    
+
     public void addMax(long a) {
       if (a < 0)
         return;
       max = Math.max(max, a);
     }
-    
+
     @Override
     public String toString() {
       return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE).append("count", count).append("average", avg).append("minimum", min).append("maximum", max).toString();
     }
-    
+
   }
-  
+
   static final org.apache.log4j.Logger log = org.apache.log4j.Logger.getLogger(AbstractMetricsImpl.class);
-  
+
   private static ConcurrentHashMap<String,Metric> registry = new ConcurrentHashMap<String,Metric>();
-  
+
   private boolean currentlyLogging = false;
-  
+
   private File logDir = null;
-  
+
   private String metricsPrefix = null;
-  
+
   private Date today = new Date();
-  
+
   private File logFile = null;
-  
+
   private Writer logWriter = null;
-  
+
   private SimpleDateFormat formatter = new SimpleDateFormat("yyyyMMdd");
-  
+
   private SimpleDateFormat logFormatter = new SimpleDateFormat("yyyyMMddhhmmssz");
-  
+
   private MetricsConfiguration config = null;
-  
+
   public AbstractMetricsImpl() {
     this.metricsPrefix = getMetricsPrefix();
     config = new MetricsConfiguration(metricsPrefix);
   }
-  
+
   /**
    * Registers a StandardMBean with the MBean Server
    */
@@ -126,13 +126,14 @@ public abstract class AbstractMetricsImpl {
     if (null == getObjectName())
       throw new IllegalArgumentException("MBean object name must be set.");
     mbs.registerMBean(mbean, getObjectName());
-    
+
     setupLogging();
   }
-  
+
   /**
    * Registers this MBean with the MBean Server
    */
+  @Override
   public void register() throws Exception {
     // Register this object with the MBeanServer
     MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
@@ -141,31 +142,31 @@ public abstract class AbstractMetricsImpl {
     mbs.registerMBean(this, getObjectName());
     setupLogging();
   }
-  
+
   public void createMetric(String name) {
     registry.put(name, new Metric());
   }
-  
+
   public Metric getMetric(String name) {
     return registry.get(name);
   }
-  
+
   public long getMetricCount(String name) {
     return registry.get(name).getCount();
   }
-  
+
   public long getMetricAvg(String name) {
     return registry.get(name).getAvg();
   }
-  
+
   public long getMetricMin(String name) {
     return registry.get(name).getMin();
   }
-  
+
   public long getMetricMax(String name) {
     return registry.get(name).getMax();
   }
-  
+
   private void setupLogging() throws IOException {
     if (null == config.getMetricsConfiguration())
       return;
@@ -176,7 +177,7 @@ public abstract class AbstractMetricsImpl {
       if (null != mDir) {
         File dir = new File(mDir);
         if (!dir.isDirectory())
-          if (!dir.mkdir()) 
+          if (!dir.mkdir())
             log.warn("Could not create log directory: " + dir);
         logDir = dir;
         // Create new log file
@@ -185,7 +186,7 @@ public abstract class AbstractMetricsImpl {
       currentlyLogging = true;
     }
   }
-  
+
   private void startNewLog() throws IOException {
     if (null != logWriter) {
       logWriter.flush();
@@ -201,7 +202,7 @@ public abstract class AbstractMetricsImpl {
     }
     logWriter = new OutputStreamWriter(new FileOutputStream(logFile, true), UTF_8);
   }
-  
+
   private void writeToLog(String name) throws IOException {
     if (null == logWriter)
       return;
@@ -213,7 +214,8 @@ public abstract class AbstractMetricsImpl {
     }
     logWriter.append(logFormatter.format(now)).append(" Metric: ").append(name).append(": ").append(registry.get(name).toString()).append("\n");
   }
-  
+
+  @Override
   public void add(String name, long time) {
     if (isEnabled()) {
       registry.get(name).incCount();
@@ -248,15 +250,16 @@ public abstract class AbstractMetricsImpl {
       }
     }
   }
-  
+
+  @Override
   public boolean isEnabled() {
     return config.isEnabled();
   }
-  
+
   protected abstract ObjectName getObjectName();
-  
+
   protected abstract String getMetricsPrefix();
-  
+
   @Override
   protected void finalize() {
     if (null != logWriter) {
@@ -270,5 +273,5 @@ public abstract class AbstractMetricsImpl {
     }
     logFile = null;
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/base/src/main/java/org/apache/accumulo/server/metrics/Metrics.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/metrics/Metrics.java b/server/base/src/main/java/org/apache/accumulo/server/metrics/Metrics.java
new file mode 100644
index 0000000..f451266
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/metrics/Metrics.java
@@ -0,0 +1,34 @@
+/*
+ * 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.accumulo.server.metrics;
+
+/**
+ *
+ */
+public interface Metrics {
+
+  String PREFIX = "Accumulo";
+  String THRIFT_NAME = "Thrift";
+  String TSERVER_NAME = "TabletServer";
+  String MASTER_NAME = "Master";
+
+  void register() throws Exception;
+
+  void add(String name, long time);
+
+  boolean isEnabled();
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/base/src/main/java/org/apache/accumulo/server/metrics/Metrics2ThriftMetrics.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/metrics/Metrics2ThriftMetrics.java b/server/base/src/main/java/org/apache/accumulo/server/metrics/Metrics2ThriftMetrics.java
new file mode 100644
index 0000000..d166899
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/metrics/Metrics2ThriftMetrics.java
@@ -0,0 +1,65 @@
+/*
+ * 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.accumulo.server.metrics;
+
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.lib.Interns;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+
+/**
+ *
+ */
+public class Metrics2ThriftMetrics implements Metrics, MetricsSource, ThriftMetricsKeys {
+  public static final String CONTEXT = "thrift";
+
+  private final MetricsSystem system;
+  private final MetricsRegistry registry;
+  private final String record, name, desc;
+
+  Metrics2ThriftMetrics(MetricsSystem system, String serverName, String threadName) {
+    this.system = system;
+    this.record = serverName;
+    this.name = THRIFT_NAME + ",sub=" + serverName;
+    this.desc = "Thrift Server Metrics - " + serverName + " " + threadName;
+    this.registry = new MetricsRegistry(Interns.info(name, desc));
+  }
+
+  @Override
+  public void add(String name, long time) {
+    registry.add(name, time);
+  }
+
+  @Override
+  public void register() {
+    system.register(name, desc, this);
+  }
+
+  @Override
+  public boolean isEnabled() {
+    return true;
+  }
+
+  @Override
+  public void getMetrics(MetricsCollector collector, boolean all) {
+    MetricsRecordBuilder builder = collector.addRecord(record).setContext(CONTEXT);
+
+    registry.snapshot(builder, all);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/base/src/main/java/org/apache/accumulo/server/metrics/MetricsFactory.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/metrics/MetricsFactory.java b/server/base/src/main/java/org/apache/accumulo/server/metrics/MetricsFactory.java
new file mode 100644
index 0000000..7c06971
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/metrics/MetricsFactory.java
@@ -0,0 +1,53 @@
+/*
+ * 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.accumulo.server.metrics;
+
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+
+import com.google.common.base.Preconditions;
+
+/**
+ *
+ */
+public class MetricsFactory {
+
+  private final boolean useOldMetrics;
+  private final MetricsSystem metricsSystem;
+
+  public MetricsFactory(AccumuloConfiguration conf) {
+    Preconditions.checkNotNull(conf);
+    useOldMetrics = conf.getBoolean(Property.GENERAL_LEGACY_METRICS);
+
+    if (useOldMetrics) {
+      metricsSystem = null;
+    } else {
+      metricsSystem = DefaultMetricsSystem.initialize(Metrics.PREFIX);
+    }
+  }
+
+  public Metrics createThriftMetrics(String serverName, String threadName) {
+    if (useOldMetrics) {
+      return new ThriftMetrics(serverName, threadName);
+    }
+
+    return new Metrics2ThriftMetrics(metricsSystem, serverName, threadName);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/base/src/main/java/org/apache/accumulo/server/metrics/ThriftMetrics.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/metrics/ThriftMetrics.java b/server/base/src/main/java/org/apache/accumulo/server/metrics/ThriftMetrics.java
index cf7d3d5..d87d055 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/metrics/ThriftMetrics.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/metrics/ThriftMetrics.java
@@ -20,14 +20,14 @@ import javax.management.ObjectName;
 
 
 public class ThriftMetrics extends AbstractMetricsImpl implements ThriftMetricsMBean {
-  
+
   static final org.apache.log4j.Logger log = org.apache.log4j.Logger.getLogger(ThriftMetrics.class);
-  
+
   private static final String METRICS_PREFIX = "thrift";
-  
+
   private ObjectName OBJECT_NAME = null;
-  
-  public ThriftMetrics(String serverName, String threadName) {
+
+  ThriftMetrics(String serverName, String threadName) {
     super();
     reset();
     try {
@@ -36,52 +36,61 @@ public class ThriftMetrics extends AbstractMetricsImpl implements ThriftMetricsM
       log.error("Exception setting MBean object name", e);
     }
   }
-  
+
   @Override
   protected ObjectName getObjectName() {
     return OBJECT_NAME;
   }
-  
+
   @Override
   protected String getMetricsPrefix() {
     return METRICS_PREFIX;
   }
-  
+
+  @Override
   public void reset() {
     createMetric(idle);
     createMetric(execute);
   }
-  
+
+  @Override
   public long getExecutionAvgTime() {
     return this.getMetricAvg(execute);
   }
-  
+
+  @Override
   public long getExecutionCount() {
     return this.getMetricCount(execute);
   }
-  
+
+  @Override
   public long getExecutionMaxTime() {
     return this.getMetricMax(execute);
   }
-  
+
+  @Override
   public long getExecutionMinTime() {
     return this.getMetricMin(execute);
   }
-  
+
+  @Override
   public long getIdleAvgTime() {
     return this.getMetricAvg(idle);
   }
-  
+
+  @Override
   public long getIdleCount() {
     return this.getMetricCount(idle);
   }
-  
+
+  @Override
   public long getIdleMaxTime() {
     return this.getMetricMax(idle);
   }
-  
+
+  @Override
   public long getIdleMinTime() {
     return this.getMetricMin(idle);
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/base/src/main/java/org/apache/accumulo/server/metrics/ThriftMetricsKeys.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/metrics/ThriftMetricsKeys.java b/server/base/src/main/java/org/apache/accumulo/server/metrics/ThriftMetricsKeys.java
new file mode 100644
index 0000000..f527657
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/metrics/ThriftMetricsKeys.java
@@ -0,0 +1,27 @@
+/*
+ * 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.accumulo.server.metrics;
+
+/**
+ *
+ */
+public interface ThriftMetricsKeys {
+
+  static String idle = "idle";
+  static String execute = "execute";
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/base/src/main/java/org/apache/accumulo/server/metrics/ThriftMetricsMBean.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/metrics/ThriftMetricsMBean.java b/server/base/src/main/java/org/apache/accumulo/server/metrics/ThriftMetricsMBean.java
index 403cffe..101c7d7 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/metrics/ThriftMetricsMBean.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/metrics/ThriftMetricsMBean.java
@@ -16,27 +16,24 @@
  */
 package org.apache.accumulo.server.metrics;
 
-public interface ThriftMetricsMBean {
-  
-  static final String idle = "idle";
-  static final String execute = "execute";
-  
+public interface ThriftMetricsMBean extends ThriftMetricsKeys {
+
   long getIdleCount();
-  
+
   long getIdleMinTime();
-  
+
   long getIdleMaxTime();
-  
+
   long getIdleAvgTime();
-  
+
   long getExecutionCount();
-  
+
   long getExecutionMinTime();
-  
+
   long getExecutionMaxTime();
-  
+
   long getExecutionAvgTime();
-  
+
   void reset();
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/base/src/main/java/org/apache/accumulo/server/util/TServerUtils.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/TServerUtils.java b/server/base/src/main/java/org/apache/accumulo/server/util/TServerUtils.java
index d30f101..f1156d4 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/TServerUtils.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/TServerUtils.java
@@ -28,6 +28,7 @@ import java.util.Random;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ThreadPoolExecutor;
 
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.util.Daemon;
 import org.apache.accumulo.core.util.LoggingRunnable;
@@ -37,6 +38,8 @@ import org.apache.accumulo.core.util.TBufferedSocket;
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.server.AccumuloServerContext;
+import org.apache.accumulo.server.metrics.Metrics;
+import org.apache.accumulo.server.metrics.MetricsFactory;
 import org.apache.accumulo.server.metrics.ThriftMetrics;
 import org.apache.accumulo.server.util.time.SimpleTimer;
 import org.apache.log4j.Logger;
@@ -70,7 +73,7 @@ public class TServerUtils {
 
   /**
    * Start a server, at the given port, or higher, if that port is not available.
-   * 
+   *
    * @param portHintProperty
    *          the port to attempt to open, can be zero, meaning "any available port"
    * @param processor
@@ -100,7 +103,7 @@ public class TServerUtils {
     if (portSearchProperty != null)
       portSearch = service.getConfiguration().getBoolean(portSearchProperty);
     // create the TimedProcessor outside the port search loop so we don't try to register the same metrics mbean more than once
-    TServerUtils.TimedProcessor timedProcessor = new TServerUtils.TimedProcessor(processor, serverName, threadName);
+    TServerUtils.TimedProcessor timedProcessor = new TServerUtils.TimedProcessor(service.getConfiguration(), processor, serverName, threadName);
     Random random = new Random();
     for (int j = 0; j < 100; j++) {
 
@@ -143,14 +146,15 @@ public class TServerUtils {
   public static class TimedProcessor implements TProcessor {
 
     final TProcessor other;
-    ThriftMetrics metrics = null;
+    Metrics metrics = null;
     long idleStart = 0;
 
-    TimedProcessor(TProcessor next, String serverName, String threadName) {
+    TimedProcessor(AccumuloConfiguration conf, TProcessor next, String serverName, String threadName) {
       this.other = next;
       // Register the metrics MBean
+      MetricsFactory factory = new MetricsFactory(conf);
+      metrics = factory.createThriftMetrics(serverName, threadName);
       try {
-        metrics = new ThriftMetrics(serverName, threadName);
         metrics.register();
       } catch (Exception e) {
         log.error("Exception registering MBean with MBean Server", e);
@@ -280,9 +284,9 @@ public class TServerUtils {
     return new ServerAddress(createThreadPoolServer(transport, processor), address);
   }
 
-  public static ServerAddress startTServer(HostAndPort address, TProcessor processor, String serverName, String threadName, int numThreads, int numSTThreads,
+  public static ServerAddress startTServer(AccumuloConfiguration conf, HostAndPort address, TProcessor processor, String serverName, String threadName, int numThreads, int numSTThreads,
       long timeBetweenThreadChecks, long maxMessageSize, SslConnectionParams sslParams, long sslSocketTimeout) throws TTransportException {
-    return startTServer(address, new TimedProcessor(processor, serverName, threadName), serverName, threadName, numThreads, numSTThreads,
+    return startTServer(address, new TimedProcessor(conf, processor, serverName, threadName), serverName, threadName, numThreads, numSTThreads,
         timeBetweenThreadChecks, maxMessageSize, sslParams, sslSocketTimeout);
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
----------------------------------------------------------------------
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
index a223eb9..371c94d 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
@@ -716,7 +716,7 @@ public class SimpleGarbageCollector extends AccumuloServerContext implements Ifa
     HostAndPort result = HostAndPort.fromParts(opts.getAddress(), port);
     log.debug("Starting garbage collector listening on " + result);
     try {
-      return TServerUtils.startTServer(result, processor, this.getClass().getSimpleName(), "GC Monitor Service", 2,
+      return TServerUtils.startTServer(getConfiguration(), result, processor, this.getClass().getSimpleName(), "GC Monitor Service", 2,
           getConfiguration().getCount(Property.GENERAL_SIMPLETIMER_THREADPOOL_SIZE), 1000, maxMessageSize, getServerSslParams(), 0).address;
     } catch (Exception ex) {
       log.fatal(ex, ex);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/master/src/main/java/org/apache/accumulo/master/Master.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/Master.java b/server/master/src/main/java/org/apache/accumulo/master/Master.java
index d5d5145..6e81354 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/Master.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/Master.java
@@ -82,7 +82,7 @@ import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooLock.LockLossReason;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
-import org.apache.accumulo.master.metrics.ReplicationMetrics;
+import org.apache.accumulo.master.metrics.MasterMetricsFactory;
 import org.apache.accumulo.master.recovery.RecoveryManager;
 import org.apache.accumulo.master.replication.MasterReplicationCoordinator;
 import org.apache.accumulo.master.replication.ReplicationDriver;
@@ -115,6 +115,7 @@ import org.apache.accumulo.server.master.state.TabletServerState;
 import org.apache.accumulo.server.master.state.TabletState;
 import org.apache.accumulo.server.master.state.ZooStore;
 import org.apache.accumulo.server.master.state.ZooTabletStateStore;
+import org.apache.accumulo.server.metrics.Metrics;
 import org.apache.accumulo.server.replication.ZooKeeperInitialization;
 import org.apache.accumulo.server.security.AuditedSecurityOperation;
 import org.apache.accumulo.server.security.SecurityOperation;
@@ -1126,11 +1127,13 @@ public class Master extends AccumuloServerContext implements LiveTServerSet.List
     ZooReaderWriter.getInstance().putPersistentData(ZooUtil.getRoot(getInstance()) + Constants.ZMASTER_REPLICATION_COORDINATOR_ADDR,
         replAddress.address.toString().getBytes(UTF_8), NodeExistsPolicy.OVERWRITE);
 
+    // Register replication metrics
+    MasterMetricsFactory factory = new MasterMetricsFactory(getConfiguration(), this);
+    Metrics replicationMetrics = factory.createReplicationMetrics();
     try {
-      ReplicationMetrics beanImpl = new ReplicationMetrics(this);
-      beanImpl.register();
+      replicationMetrics.register();
     } catch (Exception e) {
-      log.error("Error registering Replication metrics with JMX", e);
+      log.error("Failed to register replication metrics", e);
     }
 
     while (clientService.isServing()) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/master/src/main/java/org/apache/accumulo/master/metrics/MasterMetricsFactory.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/metrics/MasterMetricsFactory.java b/server/master/src/main/java/org/apache/accumulo/master/metrics/MasterMetricsFactory.java
new file mode 100644
index 0000000..6d86d09
--- /dev/null
+++ b/server/master/src/main/java/org/apache/accumulo/master/metrics/MasterMetricsFactory.java
@@ -0,0 +1,57 @@
+/*
+ * 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.accumulo.master.metrics;
+
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.master.Master;
+import org.apache.accumulo.server.metrics.Metrics;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+
+import com.google.common.base.Preconditions;
+
+/**
+ *
+ */
+public class MasterMetricsFactory {
+
+  private final boolean useOldMetrics;
+  private final MetricsSystem metricsSystem;
+  private final Master master;
+
+  public MasterMetricsFactory(AccumuloConfiguration conf, Master master) {
+    Preconditions.checkNotNull(conf);
+    useOldMetrics = conf.getBoolean(Property.GENERAL_LEGACY_METRICS);
+    this.master = master;
+
+    if (useOldMetrics) {
+      metricsSystem = null;
+    } else {
+      metricsSystem = DefaultMetricsSystem.initialize(Metrics.PREFIX);
+    }
+  }
+
+  public Metrics createReplicationMetrics() {
+    if (useOldMetrics) {
+      return new ReplicationMetrics(master);
+    }
+
+    return new Metrics2ReplicationMetrics(master, metricsSystem);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/master/src/main/java/org/apache/accumulo/master/metrics/Metrics2ReplicationMetrics.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/metrics/Metrics2ReplicationMetrics.java b/server/master/src/main/java/org/apache/accumulo/master/metrics/Metrics2ReplicationMetrics.java
new file mode 100644
index 0000000..9e35c61
--- /dev/null
+++ b/server/master/src/main/java/org/apache/accumulo/master/metrics/Metrics2ReplicationMetrics.java
@@ -0,0 +1,127 @@
+/*
+ * 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.accumulo.master.metrics;
+
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.core.master.state.tables.TableState;
+import org.apache.accumulo.core.replication.ReplicationTable;
+import org.apache.accumulo.core.replication.ReplicationTarget;
+import org.apache.accumulo.master.Master;
+import org.apache.accumulo.server.metrics.Metrics;
+import org.apache.accumulo.server.replication.ReplicationUtil;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.lib.Interns;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+
+/**
+ *
+ */
+public class Metrics2ReplicationMetrics implements Metrics, MetricsSource {
+  public static final String NAME = MASTER_NAME + ",sub=Replication", DESCRIPTION = "Data-Center Replication Metrics", CONTEXT = "master",
+      RECORD = "MasterReplication";
+  public static final String PENDING_FILES = "filesPendingReplication", NUM_PEERS = "numPeers", MAX_REPLICATION_THREADS = "maxReplicationThreads";
+
+  private final Master master;
+  private final MetricsSystem system;
+  private final MetricsRegistry registry;
+  private final ReplicationUtil replicationUtil;
+
+  Metrics2ReplicationMetrics(Master master, MetricsSystem system) {
+    this.master = master;
+    this.system = system;
+
+    registry = new MetricsRegistry(Interns.info(NAME, DESCRIPTION));
+    replicationUtil = new ReplicationUtil(master);
+  }
+
+  protected void snapshot() {
+    registry.add(PENDING_FILES, getNumFilesPendingReplication());
+    registry.add(NUM_PEERS, getNumConfiguredPeers());
+    registry.add(MAX_REPLICATION_THREADS, getMaxReplicationThreads());
+  }
+
+  @Override
+  public void getMetrics(MetricsCollector collector, boolean all) {
+    MetricsRecordBuilder builder = collector.addRecord(RECORD).setContext(CONTEXT);
+
+    snapshot();
+
+    registry.snapshot(builder, all);
+  }
+
+  @Override
+  public void register() throws Exception {
+    system.register(NAME, DESCRIPTION, this);
+  }
+
+  @Override
+  public void add(String name, long time) {
+    throw new UnsupportedOperationException("add() is not implemented");
+  }
+
+  @Override
+  public boolean isEnabled() {
+    return true;
+  }
+
+  protected int getNumFilesPendingReplication() {
+    if (TableState.ONLINE != Tables.getTableState(master.getInstance(), ReplicationTable.ID)) {
+      return 0;
+    }
+
+    // Get all of the configured replication peers
+    Map<String,String> peers = replicationUtil.getPeers();
+
+    // A quick lookup to see if have any replication peer configured
+    if (peers.isEmpty()) {
+      return 0;
+    }
+
+    // The total set of configured targets
+    Set<ReplicationTarget> allConfiguredTargets = replicationUtil.getReplicationTargets();
+
+    // Number of files per target we have to replicate
+    Map<ReplicationTarget,Long> targetCounts = replicationUtil.getPendingReplications();
+
+    int filesPending = 0;
+
+    // Sum pending replication over all targets
+    for (ReplicationTarget configuredTarget : allConfiguredTargets) {
+      Long numFiles = targetCounts.get(configuredTarget);
+
+      if (null != numFiles) {
+        filesPending += numFiles;
+      }
+    }
+
+    return filesPending;
+  }
+
+  protected int getNumConfiguredPeers() {
+    return replicationUtil.getPeers().size();
+  }
+
+  protected int getMaxReplicationThreads() {
+    return replicationUtil.getMaxReplicationThreads(master.getMasterMonitorInfo());
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/master/src/main/java/org/apache/accumulo/master/metrics/ReplicationMetrics.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/metrics/ReplicationMetrics.java b/server/master/src/main/java/org/apache/accumulo/master/metrics/ReplicationMetrics.java
index 39112d0..c6405de 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/metrics/ReplicationMetrics.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/metrics/ReplicationMetrics.java
@@ -19,11 +19,8 @@ package org.apache.accumulo.master.metrics;
 import java.util.Map;
 import java.util.Set;
 
-import javax.management.MalformedObjectNameException;
 import javax.management.ObjectName;
 
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.replication.ReplicationTable;
@@ -31,21 +28,27 @@ import org.apache.accumulo.core.replication.ReplicationTarget;
 import org.apache.accumulo.master.Master;
 import org.apache.accumulo.server.metrics.AbstractMetricsImpl;
 import org.apache.accumulo.server.replication.ReplicationUtil;
+import org.apache.log4j.Logger;
 
 /**
  * JMX bindings to expose 'high-level' metrics about Replication
  */
 public class ReplicationMetrics extends AbstractMetricsImpl implements ReplicationMetricsMBean {
+  private static final Logger log = Logger.getLogger(ReplicationMetrics.class);
   private static final String METRICS_PREFIX = "replication";
 
   private Master master;
   private ObjectName objectName = null;
   private ReplicationUtil replicationUtil;
 
-  public ReplicationMetrics(Master master) throws MalformedObjectNameException, AccumuloException, AccumuloSecurityException {
+  ReplicationMetrics(Master master) {
     super();
     this.master = master;
-    objectName = new ObjectName("accumulo.server.metrics:service=Replication Metrics,name=ReplicationMBean,instance=" + Thread.currentThread().getName());
+    try {
+      objectName = new ObjectName("accumulo.server.metrics:service=Replication Metrics,name=ReplicationMBean,instance=" + Thread.currentThread().getName());
+    } catch (Exception e) {
+      log.error("Exception setting MBean object name", e);
+    }
     replicationUtil = new ReplicationUtil(master);
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index f9f5b4c..c45d5cd 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -16,6 +16,9 @@
  */
 package org.apache.accumulo.tserver;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.server.problems.ProblemType.TABLET_LOAD;
+
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
@@ -49,9 +52,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReentrantLock;
 
-import javax.management.StandardMBean;
-
-import com.google.common.net.HostAndPort;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -167,6 +167,7 @@ import org.apache.accumulo.server.master.state.TabletLocationState.BadLocationSt
 import org.apache.accumulo.server.master.state.TabletStateStore;
 import org.apache.accumulo.server.master.state.ZooTabletStateStore;
 import org.apache.accumulo.server.master.tableOps.UserCompactionConfig;
+import org.apache.accumulo.server.metrics.Metrics;
 import org.apache.accumulo.server.problems.ProblemReport;
 import org.apache.accumulo.server.problems.ProblemReports;
 import org.apache.accumulo.server.replication.ZooKeeperInitialization;
@@ -200,9 +201,7 @@ import org.apache.accumulo.tserver.log.TabletServerLogger;
 import org.apache.accumulo.tserver.mastermessage.MasterMessage;
 import org.apache.accumulo.tserver.mastermessage.SplitReportMessage;
 import org.apache.accumulo.tserver.mastermessage.TabletStatusMessage;
-import org.apache.accumulo.tserver.metrics.TabletServerMBean;
-import org.apache.accumulo.tserver.metrics.TabletServerMBeanImpl;
-import org.apache.accumulo.tserver.metrics.TabletServerMinCMetrics;
+import org.apache.accumulo.tserver.metrics.TabletServerMetricsFactory;
 import org.apache.accumulo.tserver.metrics.TabletServerScanMetrics;
 import org.apache.accumulo.tserver.metrics.TabletServerUpdateMetrics;
 import org.apache.accumulo.tserver.replication.ReplicationServicerHandler;
@@ -239,8 +238,7 @@ import org.apache.thrift.server.TServer;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NoNodeException;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.accumulo.server.problems.ProblemType.TABLET_LOAD;
+import com.google.common.net.HostAndPort;
 
 public class TabletServer extends AccumuloServerContext implements Runnable {
   private static final Logger log = Logger.getLogger(TabletServer.class);
@@ -255,9 +253,12 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
 
   private final TabletServerLogger logger;
 
-  private final TabletServerMinCMetrics mincMetrics = new TabletServerMinCMetrics();
+  private final TabletServerMetricsFactory metricsFactory;
+  private final Metrics updateMetrics;
+  private final Metrics scanMetrics;
+  private final Metrics mincMetrics;
 
-  public TabletServerMinCMetrics getMinCMetrics() {
+  public Metrics getMinCMetrics() {
     return mincMetrics;
   }
 
@@ -338,13 +339,15 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
     logger = new TabletServerLogger(this, walogMaxSize, syncCounter, flushCounter);
     this.resourceManager = new TabletServerResourceManager(this, fs);
     this.security = AuditedSecurityOperation.getInstance(this);
+
+    metricsFactory = new TabletServerMetricsFactory(aconf);
+    updateMetrics = metricsFactory.createUpdateMetrics();
+    scanMetrics = metricsFactory.createScanMetrics();
+    mincMetrics = metricsFactory.createMincMetrics();
   }
 
   private final SessionManager sessionManager;
 
-  private final TabletServerUpdateMetrics updateMetrics = new TabletServerUpdateMetrics();
-
-  private final TabletServerScanMetrics scanMetrics = new TabletServerScanMetrics();
 
   private final WriteTracker writeTracker = new WriteTracker();
 
@@ -352,19 +355,12 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
 
   private final AtomicLong totalQueuedMutationSize = new AtomicLong(0);
   private final ReentrantLock recoveryLock = new ReentrantLock(true);
-  
+
   private class ThriftClientHandler extends ClientServiceHandler implements TabletClientService.Iface {
 
     ThriftClientHandler() {
       super(TabletServer.this, watcher, fs);
       log.debug(ThriftClientHandler.class.getName() + " created");
-      // Register the metrics MBean
-      try {
-        updateMetrics.register();
-        scanMetrics.register();
-      } catch (Exception e) {
-        log.error("Exception registering MBean with MBean Server", e);
-      }
     }
 
     @Override
@@ -538,8 +534,8 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
         log.debug(String.format("ScanSess tid %s %s %,d entries in %.2f secs, nbTimes = [%s] ", TServerUtils.clientAddress.get(), ss.extent.getTableId()
             .toString(), ss.entriesReturned, (t2 - ss.startTime) / 1000.0, ss.nbTimes.toString()));
         if (scanMetrics.isEnabled()) {
-          scanMetrics.add(TabletServerScanMetrics.scan, t2 - ss.startTime);
-          scanMetrics.add(TabletServerScanMetrics.resultSize, ss.entriesReturned);
+          scanMetrics.add(TabletServerScanMetrics.SCAN, t2 - ss.startTime);
+          scanMetrics.add(TabletServerScanMetrics.RESULT_SIZE, ss.entriesReturned);
         }
       }
     }
@@ -661,7 +657,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
       Durability durability = DurabilityImpl.fromThrift(tdurabilty);
       security.authenticateUser(credentials, credentials);
       if (updateMetrics.isEnabled())
-        updateMetrics.add(TabletServerUpdateMetrics.permissionErrors, 0);
+        updateMetrics.add(TabletServerUpdateMetrics.PERMISSION_ERRORS, 0);
 
       UpdateSession us = new UpdateSession(new TservConstraintEnv(security, credentials), credentials, durability);
       long sid = sessionManager.createSession(us, false);
@@ -693,7 +689,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
             // failures
             us.failures.put(keyExtent, 0l);
             if (updateMetrics.isEnabled())
-              updateMetrics.add(TabletServerUpdateMetrics.unknownTabletErrors, 0);
+              updateMetrics.add(TabletServerUpdateMetrics.UNKNOWN_TABLET_ERRORS, 0);
           }
         } else {
           log.warn("Denying access to table " + keyExtent.getTableId() + " for user " + us.getUser());
@@ -702,7 +698,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
           us.currentTablet = null;
           us.authFailures.put(keyExtent, SecurityErrorCode.PERMISSION_DENIED);
           if (updateMetrics.isEnabled())
-            updateMetrics.add(TabletServerUpdateMetrics.permissionErrors, 0);
+            updateMetrics.add(TabletServerUpdateMetrics.PERMISSION_ERRORS, 0);
           return;
         }
       } catch (ThriftSecurityException e) {
@@ -712,7 +708,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
         us.currentTablet = null;
         us.authFailures.put(keyExtent, e.getCode());
         if (updateMetrics.isEnabled())
-          updateMetrics.add(TabletServerUpdateMetrics.permissionErrors, 0);
+          updateMetrics.add(TabletServerUpdateMetrics.PERMISSION_ERRORS, 0);
         return;
       }
     }
@@ -774,7 +770,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
           if (mutations.size() > 0) {
             try {
               if (updateMetrics.isEnabled())
-                updateMetrics.add(TabletServerUpdateMetrics.mutationArraySize, mutations.size());
+                updateMetrics.add(TabletServerUpdateMetrics.MUTATION_ARRAY_SIZE, mutations.size());
 
               CommitSession commitSession = tablet.prepareMutationsForCommit(us.cenv, mutations);
               if (commitSession == null) {
@@ -790,7 +786,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
             } catch (TConstraintViolationException e) {
               us.violations.add(e.getViolations());
               if (updateMetrics.isEnabled())
-                updateMetrics.add(TabletServerUpdateMetrics.constraintViolations, 0);
+                updateMetrics.add(TabletServerUpdateMetrics.CONSTRAINT_VIOLATIONS, 0);
 
               if (e.getNonViolators().size() > 0) {
                 // only log and commit mutations if there were some
@@ -893,17 +889,17 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
 
     private void updateWalogWriteTime(long time) {
       if (updateMetrics.isEnabled())
-        updateMetrics.add(TabletServerUpdateMetrics.waLogWriteTime, time);
+        updateMetrics.add(TabletServerUpdateMetrics.WALOG_WRITE_TIME, time);
     }
 
     private void updateAvgCommitTime(long time, int size) {
       if (updateMetrics.isEnabled())
-        updateMetrics.add(TabletServerUpdateMetrics.commitTime, (long) ((time) / (double) size));
+        updateMetrics.add(TabletServerUpdateMetrics.COMMIT_TIME, (long) ((time) / (double) size));
     }
 
     private void updateAvgPrepTime(long time, int size) {
       if (updateMetrics.isEnabled())
-        updateMetrics.add(TabletServerUpdateMetrics.commitPrep, (long) ((time) / (double) size));
+        updateMetrics.add(TabletServerUpdateMetrics.COMMIT_PREP, (long) ((time) / (double) size));
     }
 
     @Override
@@ -2208,7 +2204,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
       recoveryLock.unlock();
     }
   }
-  
+
   public void addLoggersToMetadata(List<DfsLogger> logs, KeyExtent extent, int id) {
     if (!this.onlineTablets.containsKey(extent)) {
       log.info("Not adding " + logs.size() + " logs for extent " + extent + " as alias " + id + " tablet is offline");
@@ -2380,6 +2376,18 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
       throw new RuntimeException(e);
     }
 
+    Metrics tserverMetrics = metricsFactory.createTabletServerMetrics(this);
+
+    // Register MBeans
+    try {
+      tserverMetrics.register();
+      mincMetrics.register();
+      scanMetrics.register();
+      updateMetrics.register();
+    } catch (Exception e) {
+      log.error("Error registering with JMX", e);
+    }
+
     try {
       clientAddress = startTabletClientService();
     } catch (UnknownHostException e1) {
@@ -2429,16 +2437,6 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
     };
     SimpleTimer.getInstance(aconf).schedule(replicationWorkThreadPoolResizer, 10000, 30000);
 
-    try {
-      // Do this because interface not in same package.
-      TabletServerMBeanImpl beanImpl = new TabletServerMBeanImpl(this);
-      StandardMBean mbean = new StandardMBean(beanImpl, TabletServerMBean.class, false);
-      beanImpl.register(mbean);
-      mincMetrics.register();
-    } catch (Exception e) {
-      log.error("Error registering with JMX", e);
-    }
-
     String masterHost;
     while (!serverStopRequested) {
       // send all of the pending messages

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/Metrics2TabletServerMetrics.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/Metrics2TabletServerMetrics.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/Metrics2TabletServerMetrics.java
new file mode 100644
index 0000000..b470a8f
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/Metrics2TabletServerMetrics.java
@@ -0,0 +1,106 @@
+/*
+ * 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.accumulo.tserver.metrics;
+
+import org.apache.accumulo.server.metrics.Metrics;
+import org.apache.accumulo.tserver.TabletServer;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.lib.Interns;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
+
+/**
+ *
+ */
+public class Metrics2TabletServerMetrics implements Metrics, MetricsSource, TabletServerMetricsKeys {
+  public static final String NAME = TSERVER_NAME + ",sub=General", DESCRIPTION = "General TabletServer Metrics", CONTEXT = "tserver", RECORD = "general";
+
+  private final TabletServerMetricsUtil util;
+  private final MetricsSystem system;
+  private final MetricsRegistry registry;
+
+  private final MutableGaugeLong entries, entriesInMemory, activeMajcs, queuedMajcs, activeMincs, queuedMincs, onlineTablets, openingTablets, unopenedTablets,
+      queries, totalMincs;
+
+  // Use TabletServerMetricsFactory
+  Metrics2TabletServerMetrics(TabletServer tserver, MetricsSystem system) {
+    util = new TabletServerMetricsUtil(tserver);
+    this.system = system;
+    this.registry = new MetricsRegistry(Interns.info(NAME, DESCRIPTION));
+
+    entries = registry.newGauge(Interns.info(ENTRIES, "Number of entries"), 0l);
+    entriesInMemory = registry.newGauge(Interns.info(ENTRIES_IN_MEM, "Number of entries in memory"), 0l);
+    activeMajcs = registry.newGauge(Interns.info(ACTIVE_MAJCS, "Number of active major compactions"), 0l);
+    queuedMajcs = registry.newGauge(Interns.info(QUEUED_MAJCS, "Number of queued major compactions"), 0l);
+    activeMincs = registry.newGauge(Interns.info(ACTIVE_MINCS, "Number of active minor compactions"), 0l);
+    queuedMincs = registry.newGauge(Interns.info(QUEUED_MINCS, "Number of queued minor compactions"), 0l);
+    onlineTablets = registry.newGauge(Interns.info(ONLINE_TABLETS, "Number of online tablets"), 0l);
+    openingTablets = registry.newGauge(Interns.info(OPENING_TABLETS, "Number of opening tablets"), 0l);
+    unopenedTablets = registry.newGauge(Interns.info(UNOPENED_TABLETS, "Number of unopened tablets"), 0l);
+    queries = registry.newGauge(Interns.info(QUERIES, "Number of queries"), 0l);
+    totalMincs = registry.newGauge(Interns.info(TOTAL_MINCS, "Total number of minor compactions performed"), 0l);
+  }
+
+  @Override
+  public void add(String name, long time) {
+    throw new UnsupportedOperationException("add() is not implemented");
+  }
+
+  @Override
+  public void register() {
+    system.register(NAME, DESCRIPTION, this);
+  }
+
+  @Override
+  public boolean isEnabled() {
+    return true;
+  }
+
+  protected void snapshot() {
+
+    entries.set(util.getEntries());
+    entriesInMemory.set(util.getEntriesInMemory());
+    activeMajcs.set(util.getMajorCompactions());
+    queuedMajcs.set(util.getMajorCompactionsQueued());
+    activeMincs.set(util.getMinorCompactions());
+    queuedMincs.set(util.getMinorCompactionsQueued());
+    onlineTablets.set(util.getOnlineCount());
+    openingTablets.set(util.getOpeningCount());
+    unopenedTablets.set(util.getUnopenedCount());
+    queries.set(util.getQueries());
+    totalMincs.set(util.getTotalMinorCompactions());
+  }
+
+  @Override
+  public void getMetrics(MetricsCollector collector, boolean all) {
+    MetricsRecordBuilder builder = collector.addRecord(RECORD).setContext(CONTEXT);
+
+    // Update each MutableMetric with the new value
+    snapshot();
+
+    // Add then all to the builder
+    registry.snapshot(builder, all);
+
+    // TODO Some day, MetricsRegistry will also support the MetricsGaugeDouble or allow us to instantiate it directly
+    builder.addGauge(Interns.info(FILES_PER_TABLET, "Number of files per tablet"), util.getAverageFilesPerTablet());
+    builder.addGauge(Interns.info(HOLD_TIME, "Time commits held"), util.getHoldTime());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/Metrics2TabletServerMinCMetrics.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/Metrics2TabletServerMinCMetrics.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/Metrics2TabletServerMinCMetrics.java
new file mode 100644
index 0000000..cd06c8e
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/Metrics2TabletServerMinCMetrics.java
@@ -0,0 +1,74 @@
+/*
+ * 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.accumulo.tserver.metrics;
+
+import org.apache.accumulo.server.metrics.Metrics;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.lib.Interns;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MutableStat;
+
+/**
+ *
+ */
+public class Metrics2TabletServerMinCMetrics implements Metrics, MetricsSource, TabletServerMinCMetricsKeys {
+  public static final String NAME = TSERVER_NAME + ",sub=MinorCompactions", DESCRIPTION = "TabletServer Minor Compaction Metrics", CONTEXT = "tserver",
+      RECORD = "MinorCompactions";
+
+  private final MetricsSystem system;
+  private final MetricsRegistry registry;
+  private final MutableStat activeMinc, queuedMinc;
+
+  // Use TabletServerMetricsFactory
+  Metrics2TabletServerMinCMetrics(MetricsSystem system) {
+    this.system = system;
+    this.registry = new MetricsRegistry(Interns.info(NAME, DESCRIPTION));
+
+    activeMinc = registry.newStat(MINC, "Minor compactions", "Ops", "Count", true);
+    queuedMinc = registry.newStat(QUEUE, "Queued minor compactions", "Ops", "Count", true);
+  }
+
+  @Override
+  public void add(String name, long value) {
+    if (MINC.equals(name)) {
+      activeMinc.add(value);
+    } else if (QUEUE.equals(name)) {
+      queuedMinc.add(value);
+    }
+  }
+
+  @Override
+  public void register() {
+    system.register(NAME, DESCRIPTION, this);
+  }
+
+  @Override
+  public boolean isEnabled() {
+    return true;
+  }
+
+  @Override
+  public void getMetrics(MetricsCollector collector, boolean all) {
+    MetricsRecordBuilder builder = collector.addRecord(RECORD).setContext(CONTEXT);
+
+    registry.snapshot(builder, all);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/Metrics2TabletServerScanMetrics.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/Metrics2TabletServerScanMetrics.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/Metrics2TabletServerScanMetrics.java
new file mode 100644
index 0000000..0c72cb5
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/Metrics2TabletServerScanMetrics.java
@@ -0,0 +1,75 @@
+/*
+ * 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.accumulo.tserver.metrics;
+
+import org.apache.accumulo.server.metrics.Metrics;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.lib.Interns;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MutableStat;
+
+/**
+ *
+ */
+public class Metrics2TabletServerScanMetrics implements Metrics, MetricsSource, TabletServerScanMetricsKeys {
+  public static final String NAME = TSERVER_NAME + ",sub=Scans", DESCRIPTION = "TabletServer Scan Metrics", CONTEXT = "tserver", RECORD = "Scans";
+
+  private final MetricsSystem system;
+  private final MetricsRegistry registry;
+  private final MutableStat scans, resultsPerScan;
+
+  // Use TabletServerMetricsFactory
+  Metrics2TabletServerScanMetrics(MetricsSystem system) {
+    this.system = system;
+    this.registry = new MetricsRegistry(Interns.info(NAME, DESCRIPTION));
+
+    scans = registry.newStat(SCAN, "Scans", "Ops", "Count", true);
+    resultsPerScan = registry.newStat(RESULT_SIZE, "Results per scan", "Ops", "Count", true);
+  }
+
+  @Override
+  public void add(String name, long value) {
+    if (SCAN.equals(name)) {
+      scans.add(value);
+    } else if (RESULT_SIZE.equals(name)) {
+      resultsPerScan.add(value);
+    } else {
+      throw new RuntimeException("Could not find metric to update for name " + name);
+    }
+  }
+
+  @Override
+  public void register() {
+    system.register(NAME, DESCRIPTION, this);
+  }
+
+  @Override
+  public boolean isEnabled() {
+    return true;
+  }
+
+  @Override
+  public void getMetrics(MetricsCollector collector, boolean all) {
+    MetricsRecordBuilder builder = collector.addRecord(RECORD).setContext(CONTEXT);
+
+    registry.snapshot(builder, all);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/Metrics2TabletServerUpdateMetrics.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/Metrics2TabletServerUpdateMetrics.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/Metrics2TabletServerUpdateMetrics.java
new file mode 100644
index 0000000..e1c91c6
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/Metrics2TabletServerUpdateMetrics.java
@@ -0,0 +1,94 @@
+/*
+ * 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.accumulo.tserver.metrics;
+
+import org.apache.accumulo.server.metrics.Metrics;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.lib.Interns;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableStat;
+
+/**
+ *
+ */
+public class Metrics2TabletServerUpdateMetrics implements Metrics, MetricsSource, TabletServerUpdateMetricsKeys {
+  public static final String NAME = TSERVER_NAME + ",sub=Updates", DESCRIPTION = "TabletServer Update Metrics", CONTEXT = "tserver", RECORD = "Updates";
+
+  private final MetricsSystem system;
+  private final MetricsRegistry registry;
+
+  private final MutableCounterLong permissionErrorsCounter, unknownTabletErrorsCounter, constraintViolationsCounter;
+  private final MutableStat commitPrepStat, walogWriteTimeStat, commitTimeStat, mutationArraySizeStat;
+
+  // Use TabletServerMetricsFactory
+  Metrics2TabletServerUpdateMetrics(MetricsSystem system) {
+    this.system = system;
+    this.registry = new MetricsRegistry(Interns.info(NAME, DESCRIPTION));
+
+    permissionErrorsCounter = registry.newCounter(Interns.info(PERMISSION_ERRORS, "Permission Errors"), 0l);
+    unknownTabletErrorsCounter = registry.newCounter(Interns.info(UNKNOWN_TABLET_ERRORS, "Unknown Tablet Errors"), 0l);
+    constraintViolationsCounter = registry.newCounter(Interns.info(CONSTRAINT_VIOLATIONS, "Table Constraint Violations"), 0l);
+
+    commitPrepStat = registry.newStat(COMMIT_PREP, "preparing to commit mutations", "Ops", "Time", true);
+    walogWriteTimeStat = registry.newStat(WALOG_WRITE_TIME, "writing mutations to WAL", "Ops", "Time", true);
+    commitTimeStat = registry.newStat(COMMIT_TIME, "committing mutations", "Ops", "Time", true);
+    mutationArraySizeStat = registry.newStat(MUTATION_ARRAY_SIZE, "mutation array", "ops", "Size", true);
+  }
+
+  @Override
+  public void add(String name, long value) {
+    if (PERMISSION_ERRORS.equals(name)) {
+      permissionErrorsCounter.incr(value);
+    } else if (UNKNOWN_TABLET_ERRORS.equals(name)) {
+      unknownTabletErrorsCounter.incr(value);
+    } else if (MUTATION_ARRAY_SIZE.equals(name)) {
+      mutationArraySizeStat.add(value);
+    } else if (COMMIT_PREP.equals(name)) {
+      commitPrepStat.add(value);
+    } else if (CONSTRAINT_VIOLATIONS.equals(name)) {
+      constraintViolationsCounter.incr(value);
+    } else if (WALOG_WRITE_TIME.equals(name)) {
+      walogWriteTimeStat.add(value);
+    } else if (COMMIT_TIME.equals(name)) {
+      commitTimeStat.add(value);
+    } else {
+      throw new RuntimeException("Cannot process metric with name " + name);
+    }
+  }
+
+  @Override
+  public void register() {
+    system.register(NAME, DESCRIPTION, this);
+  }
+
+  @Override
+  public boolean isEnabled() {
+    return true;
+  }
+
+  @Override
+  public void getMetrics(MetricsCollector collector, boolean all) {
+    MetricsRecordBuilder builder = collector.addRecord(RECORD).setContext(CONTEXT);
+
+    registry.snapshot(builder, all);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMBeanImpl.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMBeanImpl.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMBeanImpl.java
index 3970379..b472065 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMBeanImpl.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMBeanImpl.java
@@ -18,31 +18,39 @@ package org.apache.accumulo.tserver.metrics;
 
 import javax.management.MalformedObjectNameException;
 import javax.management.ObjectName;
+import javax.management.StandardMBean;
 
 import org.apache.accumulo.server.metrics.AbstractMetricsImpl;
 import org.apache.accumulo.tserver.TabletServer;
-import org.apache.accumulo.tserver.tablet.Tablet;
+import org.apache.log4j.Logger;
 
 public class TabletServerMBeanImpl extends AbstractMetricsImpl implements TabletServerMBean {
-
+  private static final Logger log = Logger.getLogger(TabletServerMBeanImpl.class);
   private static final String METRICS_PREFIX = "tserver";
   private static ObjectName OBJECT_NAME = null;
 
-  final TabletServer server;
-  
-  public TabletServerMBeanImpl(TabletServer server) throws MalformedObjectNameException {
-    this.server = server;
-    OBJECT_NAME = new ObjectName("accumulo.server.metrics:service=TServerInfo,name=TabletServerMBean,instance=" + Thread.currentThread().getName());
+  private final TabletServerMetricsUtil util;
+
+  TabletServerMBeanImpl(TabletServer server) {
+    util = new TabletServerMetricsUtil(server);
+    try {
+      OBJECT_NAME = new ObjectName("accumulo.server.metrics:service=TServerInfo,name=TabletServerMBean,instance=" + Thread.currentThread().getName());
+    } catch (MalformedObjectNameException e) {
+      log.error("Exception setting MBean object name", e);
+    }
   }
-  
+
+  @Override
+  public void register() throws Exception {
+    // Do this because interface not in same package.
+    StandardMBean mbean = new StandardMBean(this, TabletServerMBean.class, false);
+    register(mbean);
+  }
+
   @Override
   public long getEntries() {
     if (isEnabled()) {
-      long result = 0;
-      for (Tablet tablet : server.getOnlineTablets()) {
-        result += tablet.getNumEntries();
-      }
-      return result;
+      return util.getEntries();
     }
     return 0;
   }
@@ -50,11 +58,7 @@ public class TabletServerMBeanImpl extends AbstractMetricsImpl implements Tablet
   @Override
   public long getEntriesInMemory() {
     if (isEnabled()) {
-      long result = 0;
-      for (Tablet tablet : server.getOnlineTablets()) {
-        result += tablet.getNumEntriesInMemory();
-      }
-      return result;
+      return util.getEntriesInMemory();
     }
     return 0;
   }
@@ -62,11 +66,7 @@ public class TabletServerMBeanImpl extends AbstractMetricsImpl implements Tablet
   @Override
   public long getIngest() {
     if (isEnabled()) {
-      long result = 0;
-      for (Tablet tablet : server.getOnlineTablets()) {
-        result += tablet.getNumEntriesInMemory();
-      }
-      return result;
+      return util.getIngest();
     }
     return 0;
   }
@@ -74,12 +74,7 @@ public class TabletServerMBeanImpl extends AbstractMetricsImpl implements Tablet
   @Override
   public int getMajorCompactions() {
     if (isEnabled()) {
-      int result = 0;
-      for (Tablet tablet : server.getOnlineTablets()) {
-        if (tablet.isMajorCompactionRunning())
-          result++;
-      }
-      return result;
+      return util.getMajorCompactions();
     }
     return 0;
   }
@@ -87,12 +82,7 @@ public class TabletServerMBeanImpl extends AbstractMetricsImpl implements Tablet
   @Override
   public int getMajorCompactionsQueued() {
     if (isEnabled()) {
-      int result = 0;
-      for (Tablet tablet : server.getOnlineTablets()) {
-        if (tablet.isMajorCompactionQueued())
-          result++;
-      }
-      return result;
+      return util.getMajorCompactionsQueued();
     }
     return 0;
   }
@@ -100,12 +90,7 @@ public class TabletServerMBeanImpl extends AbstractMetricsImpl implements Tablet
   @Override
   public int getMinorCompactions() {
     if (isEnabled()) {
-      int result = 0;
-      for (Tablet tablet : server.getOnlineTablets()) {
-        if (tablet.isMinorCompactionRunning())
-          result++;
-      }
-      return result;
+      return util.getMinorCompactions();
     }
     return 0;
   }
@@ -113,12 +98,7 @@ public class TabletServerMBeanImpl extends AbstractMetricsImpl implements Tablet
   @Override
   public int getMinorCompactionsQueued() {
     if (isEnabled()) {
-      int result = 0;
-      for (Tablet tablet : server.getOnlineTablets()) {
-        if (tablet.isMinorCompactionQueued())
-          result++;
-      }
-      return result;
+      return util.getMinorCompactionsQueued();
     }
     return 0;
   }
@@ -126,25 +106,21 @@ public class TabletServerMBeanImpl extends AbstractMetricsImpl implements Tablet
   @Override
   public int getOnlineCount() {
     if (isEnabled())
-      return server.getOnlineTablets().size();
+      return util.getOnlineCount();
     return 0;
   }
 
   @Override
   public int getOpeningCount() {
     if (isEnabled())
-      return server.getOpeningCount();
+      return util.getOpeningCount();
     return 0;
   }
 
   @Override
   public long getQueries() {
     if (isEnabled()) {
-      long result = 0;
-      for (Tablet tablet : server.getOnlineTablets()) {
-        result += tablet.totalQueries();
-      }
-      return result;
+      return util.getQueries();
     }
     return 0;
   }
@@ -152,43 +128,35 @@ public class TabletServerMBeanImpl extends AbstractMetricsImpl implements Tablet
   @Override
   public int getUnopenedCount() {
     if (isEnabled())
-      return server.getUnopenedCount();
+      return util.getUnopenedCount();
     return 0;
   }
 
   @Override
   public String getName() {
     if (isEnabled())
-      return server.getClientAddressString();
+      return util.getName();
     return "";
   }
 
   @Override
   public long getTotalMinorCompactions() {
     if (isEnabled())
-      return server.getTotalMinorCompactions();
+      return util.getTotalMinorCompactions();
     return 0;
   }
 
   @Override
   public double getHoldTime() {
     if (isEnabled())
-      return server.getHoldTimeMillis() / 1000.;
+      return util.getHoldTime();
     return 0;
   }
 
   @Override
   public double getAverageFilesPerTablet() {
     if (isEnabled()) {
-      int count = 0;
-      long result = 0;
-      for (Tablet tablet : server.getOnlineTablets()) {
-        result += tablet.getDatafiles().size();
-        count++;
-      }
-      if (count == 0)
-        return 0;
-      return result / (double) count;
+      return util.getAverageFilesPerTablet();
     }
     return 0;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMetricsFactory.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMetricsFactory.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMetricsFactory.java
new file mode 100644
index 0000000..7ca9486
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMetricsFactory.java
@@ -0,0 +1,92 @@
+/*
+ * 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.accumulo.tserver.metrics;
+
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.server.metrics.Metrics;
+import org.apache.accumulo.tserver.TabletServer;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Factory to create Metrics instances for various TabletServer functions.
+ *
+ * Necessary shim to support both the custom JMX metrics from &lt;1.7.0 and the new Hadoop Metrics2 implementations.
+ */
+public class TabletServerMetricsFactory {
+
+  private final boolean useOldMetrics;
+  private final MetricsSystem metricsSystem;
+
+  public TabletServerMetricsFactory(AccumuloConfiguration conf) {
+    Preconditions.checkNotNull(conf);
+    useOldMetrics = conf.getBoolean(Property.GENERAL_LEGACY_METRICS);
+
+    if (useOldMetrics) {
+      metricsSystem = null;
+    } else {
+      metricsSystem = DefaultMetricsSystem.initialize(Metrics.PREFIX);
+    }
+  }
+
+  /**
+   * Create Metrics to track MinorCompactions
+   */
+  public Metrics createMincMetrics() {
+    if (useOldMetrics) {
+      return new TabletServerMinCMetrics();
+    }
+
+    return new Metrics2TabletServerMinCMetrics(metricsSystem);
+  }
+
+  /**
+   * Create Metrics to track TabletServer state
+   */
+  public Metrics createTabletServerMetrics(TabletServer tserver) {
+    if (useOldMetrics) {
+      return new TabletServerMBeanImpl(tserver);
+    }
+
+    return new Metrics2TabletServerMetrics(tserver, metricsSystem);
+  }
+
+  /**
+   * Create Metrics to track scans
+   */
+  public Metrics createScanMetrics() {
+    if (useOldMetrics) {
+      return new TabletServerScanMetrics();
+    }
+
+    return new Metrics2TabletServerScanMetrics(metricsSystem);
+  }
+
+  /**
+   * Create Metrics to track updates (writes)
+   */
+  public Metrics createUpdateMetrics() {
+    if (useOldMetrics) {
+      return new TabletServerUpdateMetrics();
+    }
+
+    return new Metrics2TabletServerUpdateMetrics(metricsSystem);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMetricsKeys.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMetricsKeys.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMetricsKeys.java
new file mode 100644
index 0000000..1d8d1af
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMetricsKeys.java
@@ -0,0 +1,38 @@
+/*
+ * 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.accumulo.tserver.metrics;
+
+/**
+ * Keys for general tablet server metrics
+ */
+public interface TabletServerMetricsKeys {
+
+  String ENTRIES = "entries";
+  String ENTRIES_IN_MEM = "entriesInMem";
+  String HOLD_TIME = "holdTime";
+  String FILES_PER_TABLET = "filesPerTablet";
+  String ACTIVE_MAJCS = "activeMajCs";
+  String QUEUED_MAJCS = "queuedMajCs";
+  String ACTIVE_MINCS = "activeMinCs";
+  String QUEUED_MINCS = "queuedMinCs";
+  String ONLINE_TABLETS = "onlineTablets";
+  String OPENING_TABLETS = "openingTablets";
+  String UNOPENED_TABLETS = "unopenedTablets";
+  String QUERIES = "queries";
+  String TOTAL_MINCS = "totalMinCs";
+
+}