You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ec...@apache.org on 2013/08/16 00:04:26 UTC

svn commit: r1514514 [1/3] - in /hbase/branches/0.95: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/ hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/ hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbas...

Author: eclark
Date: Thu Aug 15 22:04:25 2013
New Revision: 1514514

URL: http://svn.apache.org/r1514514
Log:
HBASE-9194 Break HMaster metrics into multiple contexts

Added:
    hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSource.java
    hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystemSource.java
    hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSource.java
    hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/
    hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSource.java
    hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java
    hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFilesystemSourceImpl.java
    hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSourceImpl.java
    hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/
    hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java
    hbase/branches/0.95/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsAssignmentManagerSource
    hbase/branches/0.95/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsMasterFileSystemSource
    hbase/branches/0.95/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsSnapshotSource
    hbase/branches/0.95/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.balancer.MetricsBalancerSource
    hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java
    hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFilesystemSourceImpl.java
    hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSourceImpl.java
    hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/
    hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java
    hbase/branches/0.95/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsAssignmentManagerSource
    hbase/branches/0.95/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsMasterFileSystemSource
    hbase/branches/0.95/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsSnapshotSource
    hbase/branches/0.95/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.balancer.MetricsBalancerSource
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java.orig
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManager.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystem.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshot.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancer.java
Modified:
    hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java
    hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java
    hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMaster.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/CloneSnapshotHandler.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/EnabledTableSnapshotHandler.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/RestoreSnapshotHandler.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java

Added: hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSource.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSource.java?rev=1514514&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSource.java (added)
+++ hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSource.java Thu Aug 15 22:04:25 2013
@@ -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.hadoop.hbase.master;
+
+import org.apache.hadoop.hbase.metrics.BaseSource;
+
+public interface MetricsAssignmentManagerSource extends BaseSource {
+
+  /**
+   * The name of the metrics
+   */
+  String METRICS_NAME = "AssignmentManger";
+
+  /**
+   * The context metrics will be under.
+   */
+  String METRICS_CONTEXT = "master";
+
+  /**
+   * The name of the metrics context that metrics will be under in jmx
+   */
+  String METRICS_JMX_CONTEXT = "Master,sub=" + METRICS_NAME;
+
+  /**
+   * Description
+   */
+  String METRICS_DESCRIPTION = "Metrics about HBase master assingment manager.";
+
+  String RIT_COUNT_NAME = "ritCount";
+  String RIT_COUNT_OVER_THRESHOLD_NAME = "ritCountOverThreshold";
+  String RIT_OLDEST_AGE_NAME = "ritOldestAge";
+  String ASSIGN_TIME_NAME = "assign";
+  String BULK_ASSIGN_TIME_NAME = "bulkAssign";
+
+  void updateAssignmentTime(long time);
+
+  void updateBulkAssignTime(long time);
+
+  /**
+   * Set the number of regions in transition.
+   *
+   * @param ritCount count of the regions in transition.
+   */
+  void setRIT(int ritCount);
+
+  /**
+   * Set the count of the number of regions that have been in transition over the threshold time.
+   *
+   * @param ritCountOverThreshold number of regions in transition for longer than threshold.
+   */
+  void setRITCountOverThreshold(int ritCountOverThreshold);
+
+  /**
+   * Set the oldest region in transition.
+   *
+   * @param age age of the oldest RIT.
+   */
+  void setRITOldestAge(long age);
+}

Added: hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystemSource.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystemSource.java?rev=1514514&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystemSource.java (added)
+++ hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystemSource.java Thu Aug 15 22:04:25 2013
@@ -0,0 +1,64 @@
+/**
+ * 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.hadoop.hbase.master;
+
+import org.apache.hadoop.hbase.metrics.BaseSource;
+
+public interface MetricsMasterFileSystemSource extends BaseSource {
+
+  /**
+   * The name of the metrics
+   */
+  String METRICS_NAME = "FileSystem";
+
+  /**
+   * The context metrics will be under.
+   */
+  String METRICS_CONTEXT = "master";
+
+  /**
+   * The name of the metrics context that metrics will be under in jmx
+   */
+  String METRICS_JMX_CONTEXT = "Master,sub=" + METRICS_NAME;
+
+  /**
+   * Description
+   */
+  String METRICS_DESCRIPTION = "Metrics about HBase master file system.";
+
+  String META_SPLIT_TIME_NAME = "metaHlogSplitTime";
+  String META_SPLIT_SIZE_NAME = "metaHlogSplitSize";
+  String SPLIT_TIME_NAME = "hlogSplitTime";
+  String SPLIT_SIZE_NAME = "hlogSplitSize";
+
+  String META_SPLIT_TIME_DESC = "Time it takes to finish splitMetaLog()";
+  String META_SPLIT_SIZE_DESC = "Size of META HLog files being split";
+  String SPLIT_TIME_DESC = "Time it takes to finish HLog.splitLog()";
+  String SPLIT_SIZE_DESC = "Size of HLog files being split";
+
+
+  void updateMetaWALSplitTime(long time);
+
+  void updateMetaWALSplitSize(long size);
+
+  void updateSplitTime(long time);
+
+  void updateSplitSize(long size);
+
+}

Modified: hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java?rev=1514514&r1=1514513&r2=1514514&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java (original)
+++ hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java Thu Aug 15 22:04:25 2013
@@ -55,17 +55,9 @@ public interface MetricsMasterSource ext
   String SERVER_NAME_NAME = "serverName";
   String CLUSTER_ID_NAME = "clusterId";
   String IS_ACTIVE_MASTER_NAME = "isActiveMaster";
-  String SPLIT_TIME_NAME = "hlogSplitTime";
-  String SPLIT_SIZE_NAME = "hlogSplitSize";
-  String SNAPSHOT_TIME_NAME = "snapshotTime";
-  String SNAPSHOT_RESTORE_TIME_NAME = "snapshotRestoreTime";
-  String SNAPSHOT_CLONE_TIME_NAME = "snapshotCloneTime";
-  String META_SPLIT_TIME_NAME = "metaHlogSplitTime";
-  String META_SPLIT_SIZE_NAME = "metaHlogSplitSize";
+
+
   String CLUSTER_REQUESTS_NAME = "clusterRequests";
-  String RIT_COUNT_NAME = "ritCount";
-  String RIT_COUNT_OVER_THRESHOLD_NAME = "ritCountOverThreshold";
-  String RIT_OLDEST_AGE_NAME = "ritOldestAge";
   String MASTER_ACTIVE_TIME_DESC = "Master Active Time";
   String MASTER_START_TIME_DESC = "Master Start Time";
   String AVERAGE_LOAD_DESC = "AverageLoad";
@@ -75,13 +67,8 @@ public interface MetricsMasterSource ext
   String SERVER_NAME_DESC = "Server Name";
   String CLUSTER_ID_DESC = "Cluster Id";
   String IS_ACTIVE_MASTER_DESC = "Is Active Master";
-  String SPLIT_TIME_DESC = "Time it takes to finish HLog.splitLog()";
-  String SPLIT_SIZE_DESC = "Size of HLog files being split";
-  String SNAPSHOT_TIME_DESC = "Time it takes to finish snapshot()";
-  String SNAPSHOT_RESTORE_TIME_DESC = "Time it takes to finish restoreSnapshot()";
-  String SNAPSHOT_CLONE_TIME_DESC = "Time it takes to finish cloneSnapshot()";
-  String META_SPLIT_TIME_DESC = "Time it takes to finish splitMetaLog()";
-  String META_SPLIT_SIZE_DESC = "Size of META HLog files being split";
+
+
 
   /**
    * Increment the number of requests the cluster has seen.
@@ -90,39 +77,7 @@ public interface MetricsMasterSource ext
    */
   void incRequests(final int inc);
 
-  /**
-   * Set the number of regions in transition.
-   *
-   * @param ritCount count of the regions in transition.
-   */
-  void setRIT(int ritCount);
-
-  /**
-   * Set the count of the number of regions that have been in transition over the threshold time.
-   *
-   * @param ritCountOverThreshold number of regions in transition for longer than threshold.
-   */
-  void setRITCountOverThreshold(int ritCountOverThreshold);
-
-  /**
-   * Set the oldest region in transition.
-   *
-   * @param age age of the oldest RIT.
-   */
-  void setRITOldestAge(long age);
-
-  void updateSplitTime(long time);
-
-  void updateSplitSize(long size);
-
-  void updateSnapshotTime(long time);
-
-  void updateSnapshotCloneTime(long time);
-
-  void updateSnapshotRestoreTime(long time);
   
-  void updateMetaWALSplitTime(long time);
 
-  void updateMetaWALSplitSize(long size);
 
 }

Added: hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSource.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSource.java?rev=1514514&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSource.java (added)
+++ hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSource.java Thu Aug 15 22:04:25 2013
@@ -0,0 +1,56 @@
+/**
+ * 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.hadoop.hbase.master;
+
+import org.apache.hadoop.hbase.metrics.BaseSource;
+
+public interface MetricsSnapshotSource extends BaseSource {
+  /**
+   * The name of the metrics
+   */
+  String METRICS_NAME = "Snapshots";
+
+  /**
+   * The context metrics will be under.
+   */
+  String METRICS_CONTEXT = "master";
+
+  /**
+   * The name of the metrics context that metrics will be under in jmx
+   */
+  String METRICS_JMX_CONTEXT = "Master,sub=" + METRICS_NAME;
+
+  /**
+   * Description
+   */
+  String METRICS_DESCRIPTION = "Metrics about HBase master server";
+
+  String SNAPSHOT_TIME_NAME = "snapshotTime";
+  String SNAPSHOT_RESTORE_TIME_NAME = "snapshotRestoreTime";
+  String SNAPSHOT_CLONE_TIME_NAME = "snapshotCloneTime";
+  String SNAPSHOT_TIME_DESC = "Time it takes to finish snapshot()";
+  String SNAPSHOT_RESTORE_TIME_DESC = "Time it takes to finish restoreSnapshot()";
+  String SNAPSHOT_CLONE_TIME_DESC = "Time it takes to finish cloneSnapshot()";
+
+  void updateSnapshotTime(long time);
+
+  void updateSnapshotCloneTime(long time);
+
+  void updateSnapshotRestoreTime(long time);
+}

Added: hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSource.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSource.java?rev=1514514&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSource.java (added)
+++ hbase/branches/0.95/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSource.java Thu Aug 15 22:04:25 2013
@@ -0,0 +1,51 @@
+/**
+ * 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.hadoop.hbase.master.balancer;
+
+import org.apache.hadoop.hbase.metrics.BaseSource;
+
+public interface MetricsBalancerSource extends BaseSource  {
+
+  /**
+   * The name of the metrics
+   */
+  String METRICS_NAME = "Balancer";
+
+  /**
+   * The context metrics will be under.
+   */
+  String METRICS_CONTEXT = "master";
+
+  /**
+   * The name of the metrics context that metrics will be under in jmx
+   */
+  String METRICS_JMX_CONTEXT = "Master,sub=" + METRICS_NAME;
+
+  String BALANCE_CLUSTER = "balancerCluster";
+  String MISC_INVOATION_COUNT = "miscInvocationCount";
+
+  /**
+   * Description
+   */
+  String METRICS_DESCRIPTION = "Metrics about HBase master balancer";
+
+  void updateBalanceCluster(long time);
+
+  void incrMiscInvocations();
+}

Added: hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java?rev=1514514&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java (added)
+++ hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java Thu Aug 15 22:04:25 2013
@@ -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.hadoop.hbase.master;
+
+import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+import org.apache.hadoop.metrics2.lib.MetricMutableGaugeLong;
+import org.apache.hadoop.metrics2.lib.MetricMutableHistogram;
+
+public class MetricsAssignmentManagerSourceImpl extends BaseSourceImpl implements MetricsAssignmentManagerSource {
+
+  private MetricMutableGaugeLong ritGauge;
+  private MetricMutableGaugeLong ritCountOverThresholdGauge;
+  private MetricMutableGaugeLong ritOldestAgeGauge;
+  private MetricMutableHistogram assignTimeHisto;
+  private MetricMutableHistogram bulkAssignTimeHisto;
+
+  public MetricsAssignmentManagerSourceImpl() {
+    this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);
+  }
+
+  public MetricsAssignmentManagerSourceImpl(String metricsName,
+                                            String metricsDescription,
+                                            String metricsContext, String metricsJmxContext) {
+    super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
+  }
+
+  @Override
+  public void init() {
+    super.init();
+    ritGauge = metricsRegistry.newGauge(RIT_COUNT_NAME, "", 0l);
+    ritCountOverThresholdGauge = metricsRegistry.newGauge(RIT_COUNT_OVER_THRESHOLD_NAME, "", 0l);
+    ritOldestAgeGauge = metricsRegistry.newGauge(RIT_OLDEST_AGE_NAME, "", 0l);
+    assignTimeHisto = metricsRegistry.newHistogram(ASSIGN_TIME_NAME);
+    bulkAssignTimeHisto = metricsRegistry.newHistogram(BULK_ASSIGN_TIME_NAME);
+  }
+
+  @Override
+  public void updateAssignmentTime(long time) {
+    assignTimeHisto.add(time);
+  }
+
+  @Override
+  public void updateBulkAssignTime(long time) {
+    bulkAssignTimeHisto.add(time);
+  }
+
+  public void setRIT(int ritCount) {
+    ritGauge.set(ritCount);
+  }
+
+  public void setRITCountOverThreshold(int ritCount) {
+    ritCountOverThresholdGauge.set(ritCount);
+  }
+
+  public void setRITOldestAge(long ritCount) {
+    ritOldestAgeGauge.set(ritCount);
+  }
+}

Added: hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFilesystemSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFilesystemSourceImpl.java?rev=1514514&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFilesystemSourceImpl.java (added)
+++ hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFilesystemSourceImpl.java Thu Aug 15 22:04:25 2013
@@ -0,0 +1,69 @@
+/**
+ * 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.hadoop.hbase.master;
+
+import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+import org.apache.hadoop.metrics2.lib.MetricMutableHistogram;
+
+public class MetricsMasterFilesystemSourceImpl extends BaseSourceImpl implements MetricsMasterFileSystemSource {
+
+  private MetricMutableHistogram splitSizeHisto;
+  private MetricMutableHistogram splitTimeHisto;
+  private MetricMutableHistogram metaSplitTimeHisto;
+  private MetricMutableHistogram metaSplitSizeHisto;
+
+  public MetricsMasterFilesystemSourceImpl() {
+    this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);
+  }
+
+  public MetricsMasterFilesystemSourceImpl(String metricsName,
+                                           String metricsDescription,
+                                           String metricsContext, String metricsJmxContext) {
+    super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
+  }
+
+  @Override
+  public void init() {
+    splitSizeHisto = metricsRegistry.newHistogram(SPLIT_SIZE_NAME, SPLIT_SIZE_DESC);
+    splitTimeHisto = metricsRegistry.newHistogram(SPLIT_TIME_NAME, SPLIT_TIME_DESC);
+    metaSplitTimeHisto = metricsRegistry.newHistogram(META_SPLIT_TIME_NAME, META_SPLIT_TIME_DESC);
+    metaSplitSizeHisto = metricsRegistry.newHistogram(META_SPLIT_SIZE_NAME, META_SPLIT_SIZE_DESC);
+  }
+
+  @Override
+  public void updateSplitTime(long time) {
+    splitTimeHisto.add(time);
+  }
+
+  @Override
+  public void updateSplitSize(long size) {
+    splitSizeHisto.add(size);
+  }
+
+
+  @Override
+  public void updateMetaWALSplitTime(long time) {
+    metaSplitTimeHisto.add(time);
+  }
+
+  @Override
+  public void updateMetaWALSplitSize(long size) {
+    metaSplitSizeHisto.add(size);
+  }
+}

Modified: hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java?rev=1514514&r1=1514513&r2=1514514&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java (original)
+++ hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java Thu Aug 15 22:04:25 2013
@@ -40,16 +40,7 @@ public class MetricsMasterSourceImpl
 
   private final MetricsMasterWrapper masterWrapper;
   private MetricMutableCounterLong clusterRequestsCounter;
-  private MetricMutableGaugeLong ritGauge;
-  private MetricMutableGaugeLong ritCountOverThresholdGauge;
-  private MetricMutableGaugeLong ritOldestAgeGauge;
-  private MetricMutableHistogram splitTimeHisto;
-  private MetricMutableHistogram splitSizeHisto;
-  private MetricMutableStat snapshotTimeHisto;
-  private MetricMutableStat snapshotCloneTimeHisto;
-  private MetricMutableStat snapshotRestoreTimeHisto;
-  private MetricMutableHistogram metaSplitTimeHisto;
-  private MetricMutableHistogram metaSplitSizeHisto;
+
 
   public MetricsMasterSourceImpl(MetricsMasterWrapper masterWrapper) {
     this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT, masterWrapper);
@@ -68,72 +59,12 @@ public class MetricsMasterSourceImpl
   public void init() {
     super.init();
     clusterRequestsCounter = metricsRegistry.newCounter(CLUSTER_REQUESTS_NAME, "", 0l);
-    ritGauge = metricsRegistry.newGauge(RIT_COUNT_NAME, "", 0l);
-    ritCountOverThresholdGauge = metricsRegistry.newGauge(RIT_COUNT_OVER_THRESHOLD_NAME, "", 0l);
-    ritOldestAgeGauge = metricsRegistry.newGauge(RIT_OLDEST_AGE_NAME, "", 0l);
-    splitSizeHisto = metricsRegistry.newHistogram(SPLIT_SIZE_NAME, SPLIT_SIZE_DESC);
-    splitTimeHisto = metricsRegistry.newHistogram(SPLIT_TIME_NAME, SPLIT_TIME_DESC);
-    snapshotTimeHisto = metricsRegistry.newStat(
-        SNAPSHOT_TIME_NAME, SNAPSHOT_TIME_DESC, "Ops", "Time", true);
-    snapshotCloneTimeHisto = metricsRegistry.newStat(
-        SNAPSHOT_CLONE_TIME_NAME, SNAPSHOT_CLONE_TIME_DESC, "Ops", "Time", true);
-    snapshotRestoreTimeHisto = metricsRegistry.newStat(
-        SNAPSHOT_RESTORE_TIME_NAME, SNAPSHOT_RESTORE_TIME_DESC, "Ops", "Time", true);
-    metaSplitTimeHisto = metricsRegistry.newHistogram(META_SPLIT_TIME_NAME, META_SPLIT_TIME_DESC);
-    metaSplitSizeHisto = metricsRegistry.newHistogram(META_SPLIT_SIZE_NAME, META_SPLIT_SIZE_DESC);
   }
 
   public void incRequests(final int inc) {
     this.clusterRequestsCounter.incr(inc);
   }
 
-  public void setRIT(int ritCount) {
-    ritGauge.set(ritCount);
-  }
-
-  public void setRITCountOverThreshold(int ritCount) {
-    ritCountOverThresholdGauge.set(ritCount);
-  }
-
-  public void setRITOldestAge(long ritCount) {
-    ritOldestAgeGauge.set(ritCount);
-  }
-
-  @Override
-  public void updateSplitTime(long time) {
-    splitTimeHisto.add(time);
-  }
-
-  @Override
-  public void updateSplitSize(long size) {
-    splitSizeHisto.add(size);
-  }
-
-  @Override
-  public void updateSnapshotTime(long time) {
-    snapshotTimeHisto.add(time);
-  }
-
-  @Override
-  public void updateSnapshotCloneTime(long time) {
-    snapshotCloneTimeHisto.add(time);
-  }
-
-  @Override
-  public void updateSnapshotRestoreTime(long time) {
-    snapshotRestoreTimeHisto.add(time);
-  }
-
-  @Override
-  public void updateMetaWALSplitTime(long time) {
-    metaSplitTimeHisto.add(time);
-  }
-
-  @Override
-  public void updateMetaWALSplitSize(long size) {
-    metaSplitSizeHisto.add(size);
-  }
-
   /**
    * Method to export all the metrics.
    *

Added: hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSourceImpl.java?rev=1514514&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSourceImpl.java (added)
+++ hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSourceImpl.java Thu Aug 15 22:04:25 2013
@@ -0,0 +1,64 @@
+/**
+ * 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.hadoop.hbase.master;
+
+import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+import org.apache.hadoop.metrics2.lib.MetricMutableHistogram;
+
+public class MetricsSnapshotSourceImpl extends BaseSourceImpl implements MetricsSnapshotSource {
+
+  private MetricMutableHistogram snapshotTimeHisto;
+  private MetricMutableHistogram snapshotCloneTimeHisto;
+  private MetricMutableHistogram snapshotRestoreTimeHisto;
+
+  public MetricsSnapshotSourceImpl() {
+    this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);
+  }
+
+  public MetricsSnapshotSourceImpl(String metricsName,
+                                   String metricsDescription,
+                                   String metricsContext, String metricsJmxContext) {
+    super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
+  }
+
+  @Override
+  public void init() {
+    snapshotTimeHisto = metricsRegistry.newHistogram(
+        SNAPSHOT_TIME_NAME, SNAPSHOT_TIME_DESC);
+    snapshotCloneTimeHisto = metricsRegistry.newHistogram(
+        SNAPSHOT_CLONE_TIME_NAME, SNAPSHOT_CLONE_TIME_DESC);
+    snapshotRestoreTimeHisto = metricsRegistry.newHistogram(
+        SNAPSHOT_RESTORE_TIME_NAME, SNAPSHOT_RESTORE_TIME_DESC);
+  }
+
+  @Override
+  public void updateSnapshotTime(long time) {
+    snapshotTimeHisto.add(time);
+  }
+
+  @Override
+  public void updateSnapshotCloneTime(long time) {
+    snapshotCloneTimeHisto.add(time);
+  }
+
+  @Override
+  public void updateSnapshotRestoreTime(long time) {
+    snapshotRestoreTimeHisto.add(time);
+  }
+}

Added: hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java?rev=1514514&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java (added)
+++ hbase/branches/0.95/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java Thu Aug 15 22:04:25 2013
@@ -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.hadoop.hbase.master.balancer;
+
+import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+import org.apache.hadoop.metrics2.lib.MetricMutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MetricMutableHistogram;
+
+public class MetricsBalancerSourceImpl extends BaseSourceImpl implements MetricsBalancerSource{
+
+  private MetricMutableHistogram blanceClusterHisto;
+  private MetricMutableCounterLong miscCount;
+
+  public MetricsBalancerSourceImpl() {
+    this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);
+  }
+
+  public MetricsBalancerSourceImpl(String metricsName,
+                                   String metricsDescription,
+                                   String metricsContext, String metricsJmxContext) {
+    super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
+  }
+
+
+  @Override
+  public void init() {
+    blanceClusterHisto = metricsRegistry.newHistogram(BALANCE_CLUSTER);
+    miscCount = metricsRegistry.newCounter(MISC_INVOATION_COUNT, "", 0L);
+
+  }
+
+  @Override
+  public void updateBalanceCluster(long time) {
+    blanceClusterHisto.add(time);
+  }
+
+  @Override
+  public void incrMiscInvocations() {
+     miscCount.incr();
+  }
+}

Added: hbase/branches/0.95/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsAssignmentManagerSource
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsAssignmentManagerSource?rev=1514514&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsAssignmentManagerSource (added)
+++ hbase/branches/0.95/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsAssignmentManagerSource Thu Aug 15 22:04:25 2013
@@ -0,0 +1 @@
+org.apache.hadoop.hbase.master.MetricsAssignmentManagerSourceImpl

Added: hbase/branches/0.95/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsMasterFileSystemSource
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsMasterFileSystemSource?rev=1514514&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsMasterFileSystemSource (added)
+++ hbase/branches/0.95/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsMasterFileSystemSource Thu Aug 15 22:04:25 2013
@@ -0,0 +1 @@
+org.apache.hadoop.hbase.master.MetricsMasterFilesystemSourceImpl
\ No newline at end of file

Added: hbase/branches/0.95/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsSnapshotSource
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsSnapshotSource?rev=1514514&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsSnapshotSource (added)
+++ hbase/branches/0.95/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsSnapshotSource Thu Aug 15 22:04:25 2013
@@ -0,0 +1 @@
+org.apache.hadoop.hbase.master.MetricsSnapshotSourceImpl

Added: hbase/branches/0.95/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.balancer.MetricsBalancerSource
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.balancer.MetricsBalancerSource?rev=1514514&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.balancer.MetricsBalancerSource (added)
+++ hbase/branches/0.95/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.balancer.MetricsBalancerSource Thu Aug 15 22:04:25 2013
@@ -0,0 +1 @@
+org.apache.hadoop.hbase.master.balancer.MetricsBalancerSourceImpl
\ No newline at end of file

Added: hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java?rev=1514514&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java (added)
+++ hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java Thu Aug 15 22:04:25 2013
@@ -0,0 +1,73 @@
+/**
+ * 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.hadoop.hbase.master;
+
+import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
+import org.apache.hadoop.metrics2.lib.MutableHistogram;
+
+
+public class MetricsAssignmentManagerSourceImpl extends BaseSourceImpl implements MetricsAssignmentManagerSource {
+
+  private MutableGaugeLong ritGauge;
+  private MutableGaugeLong ritCountOverThresholdGauge;
+  private MutableGaugeLong ritOldestAgeGauge;
+  private MutableHistogram assignTimeHisto;
+  private MutableHistogram bulkAssignTimeHisto;
+
+  public MetricsAssignmentManagerSourceImpl() {
+    this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);
+  }
+
+  public MetricsAssignmentManagerSourceImpl(String metricsName,
+                                            String metricsDescription,
+                                            String metricsContext, String metricsJmxContext) {
+    super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
+  }
+
+  public void init() {
+    ritGauge = metricsRegistry.newGauge(RIT_COUNT_NAME, "", 0l);
+    ritCountOverThresholdGauge = metricsRegistry.newGauge(RIT_COUNT_OVER_THRESHOLD_NAME, "", 0l);
+    ritOldestAgeGauge = metricsRegistry.newGauge(RIT_OLDEST_AGE_NAME, "", 0l);
+    assignTimeHisto = metricsRegistry.newHistogram(ASSIGN_TIME_NAME);
+    bulkAssignTimeHisto = metricsRegistry.newHistogram(BULK_ASSIGN_TIME_NAME);
+  }
+
+  @Override
+  public void updateAssignmentTime(long time) {
+    assignTimeHisto.add(time);
+  }
+
+  @Override
+  public void updateBulkAssignTime(long time) {
+    bulkAssignTimeHisto.add(time);
+  }
+
+  public void setRIT(int ritCount) {
+    ritGauge.set(ritCount);
+  }
+
+  public void setRITCountOverThreshold(int ritCount) {
+    ritCountOverThresholdGauge.set(ritCount);
+  }
+
+  public void setRITOldestAge(long ritCount) {
+    ritOldestAgeGauge.set(ritCount);
+  }
+}

Added: hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFilesystemSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFilesystemSourceImpl.java?rev=1514514&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFilesystemSourceImpl.java (added)
+++ hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFilesystemSourceImpl.java Thu Aug 15 22:04:25 2013
@@ -0,0 +1,69 @@
+/**
+ * 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.hadoop.hbase.master;
+
+import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+import org.apache.hadoop.metrics2.lib.MutableHistogram;
+
+public class MetricsMasterFilesystemSourceImpl extends BaseSourceImpl implements MetricsMasterFileSystemSource {
+
+  private MutableHistogram splitSizeHisto;
+  private MutableHistogram splitTimeHisto;
+  private MutableHistogram metaSplitTimeHisto;
+  private MutableHistogram metaSplitSizeHisto;
+
+  public MetricsMasterFilesystemSourceImpl() {
+    this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);
+  }
+
+  public MetricsMasterFilesystemSourceImpl(String metricsName,
+                                           String metricsDescription,
+                                           String metricsContext, String metricsJmxContext) {
+    super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
+  }
+
+  @Override
+  public void init() {
+    splitSizeHisto = metricsRegistry.newHistogram(SPLIT_SIZE_NAME, SPLIT_SIZE_DESC);
+    splitTimeHisto = metricsRegistry.newHistogram(SPLIT_TIME_NAME, SPLIT_TIME_DESC);
+    metaSplitTimeHisto = metricsRegistry.newHistogram(META_SPLIT_TIME_NAME, META_SPLIT_TIME_DESC);
+    metaSplitSizeHisto = metricsRegistry.newHistogram(META_SPLIT_SIZE_NAME, META_SPLIT_SIZE_DESC);
+  }
+
+  @Override
+  public void updateSplitTime(long time) {
+    splitTimeHisto.add(time);
+  }
+
+  @Override
+  public void updateSplitSize(long size) {
+    splitSizeHisto.add(size);
+  }
+
+
+  @Override
+  public void updateMetaWALSplitTime(long time) {
+    metaSplitTimeHisto.add(time);
+  }
+
+  @Override
+  public void updateMetaWALSplitSize(long size) {
+    metaSplitSizeHisto.add(size);
+  }
+}

Modified: hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java?rev=1514514&r1=1514513&r2=1514514&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java (original)
+++ hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java Thu Aug 15 22:04:25 2013
@@ -37,16 +37,6 @@ public class MetricsMasterSourceImpl
 
   private final MetricsMasterWrapper masterWrapper;
   private MutableCounterLong clusterRequestsCounter;
-  private MutableGaugeLong ritGauge;
-  private MutableGaugeLong ritCountOverThresholdGauge;
-  private MutableGaugeLong ritOldestAgeGauge;
-  private MutableHistogram splitTimeHisto;
-  private MutableHistogram splitSizeHisto;
-  private MutableStat snapshotTimeHisto;
-  private MutableStat snapshotCloneTimeHisto;
-  private MutableStat snapshotRestoreTimeHisto;
-  private MutableHistogram metaSplitTimeHisto;
-  private MutableHistogram metaSplitSizeHisto;
 
   public MetricsMasterSourceImpl(MetricsMasterWrapper masterWrapper) {
     this(METRICS_NAME,
@@ -70,72 +60,12 @@ public class MetricsMasterSourceImpl
   public void init() {
     super.init();
     clusterRequestsCounter = metricsRegistry.newCounter(CLUSTER_REQUESTS_NAME, "", 0l);
-    ritGauge = metricsRegistry.newGauge(RIT_COUNT_NAME, "", 0l);
-    ritCountOverThresholdGauge = metricsRegistry.newGauge(RIT_COUNT_OVER_THRESHOLD_NAME, "", 0l);
-    ritOldestAgeGauge = metricsRegistry.newGauge(RIT_OLDEST_AGE_NAME, "", 0l);
-    splitSizeHisto = metricsRegistry.newHistogram(SPLIT_SIZE_NAME, SPLIT_SIZE_DESC);
-    splitTimeHisto = metricsRegistry.newHistogram(SPLIT_TIME_NAME, SPLIT_TIME_DESC);
-    snapshotTimeHisto = metricsRegistry.newStat(
-        SNAPSHOT_TIME_NAME, SNAPSHOT_TIME_DESC, "Ops", "Time", true);
-    snapshotCloneTimeHisto = metricsRegistry.newStat(
-        SNAPSHOT_CLONE_TIME_NAME, SNAPSHOT_CLONE_TIME_DESC, "Ops", "Time", true);
-    snapshotRestoreTimeHisto = metricsRegistry.newStat(
-        SNAPSHOT_RESTORE_TIME_NAME, SNAPSHOT_RESTORE_TIME_DESC, "Ops", "Time", true);
-    metaSplitTimeHisto = metricsRegistry.newHistogram(META_SPLIT_TIME_NAME, META_SPLIT_TIME_DESC);
-    metaSplitSizeHisto = metricsRegistry.newHistogram(META_SPLIT_SIZE_NAME, META_SPLIT_SIZE_DESC);
   }
 
   public void incRequests(final int inc) {
     this.clusterRequestsCounter.incr(inc);
   }
 
-  public void setRIT(int ritCount) {
-    ritGauge.set(ritCount);
-  }
-
-  public void setRITCountOverThreshold(int ritCount) {
-    ritCountOverThresholdGauge.set(ritCount);
-  }
-
-  public void setRITOldestAge(long ritCount) {
-    ritOldestAgeGauge.set(ritCount);
-  }
-
-  @Override
-  public void updateSplitTime(long time) {
-    splitTimeHisto.add(time);
-  }
-
-  @Override
-  public void updateSplitSize(long size) {
-    splitSizeHisto.add(size);
-  }
-
-  @Override
-  public void updateSnapshotTime(long time) {
-    snapshotTimeHisto.add(time);
-  }
-
-  @Override
-  public void updateSnapshotCloneTime(long time) {
-    snapshotCloneTimeHisto.add(time);
-  }
-
-  @Override
-  public void updateSnapshotRestoreTime(long time) {
-    snapshotRestoreTimeHisto.add(time);
-  }
-
-  @Override
-  public void updateMetaWALSplitTime(long time) {
-    metaSplitTimeHisto.add(time);
-  }
-
-  @Override
-  public void updateMetaWALSplitSize(long size) {
-    metaSplitSizeHisto.add(size);
-  }
-
   @Override
   public void getMetrics(MetricsCollector metricsCollector, boolean all) {
 

Added: hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSourceImpl.java?rev=1514514&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSourceImpl.java (added)
+++ hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSourceImpl.java Thu Aug 15 22:04:25 2013
@@ -0,0 +1,64 @@
+/**
+ * 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.hadoop.hbase.master;
+
+import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+import org.apache.hadoop.metrics2.lib.MutableHistogram;
+
+public class MetricsSnapshotSourceImpl extends BaseSourceImpl implements MetricsSnapshotSource {
+
+  private MutableHistogram snapshotTimeHisto;
+  private MutableHistogram snapshotCloneTimeHisto;
+  private MutableHistogram snapshotRestoreTimeHisto;
+
+  public MetricsSnapshotSourceImpl() {
+    this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);
+  }
+
+  public MetricsSnapshotSourceImpl(String metricsName,
+                                   String metricsDescription,
+                                   String metricsContext, String metricsJmxContext) {
+    super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
+  }
+
+  @Override
+  public void init() {
+    snapshotTimeHisto = metricsRegistry.newHistogram(
+        SNAPSHOT_TIME_NAME, SNAPSHOT_TIME_DESC);
+    snapshotCloneTimeHisto = metricsRegistry.newHistogram(
+        SNAPSHOT_CLONE_TIME_NAME, SNAPSHOT_CLONE_TIME_DESC);
+    snapshotRestoreTimeHisto = metricsRegistry.newHistogram(
+        SNAPSHOT_RESTORE_TIME_NAME, SNAPSHOT_RESTORE_TIME_DESC);
+  }
+
+  @Override
+  public void updateSnapshotTime(long time) {
+    snapshotTimeHisto.add(time);
+  }
+
+  @Override
+  public void updateSnapshotCloneTime(long time) {
+    snapshotCloneTimeHisto.add(time);
+  }
+
+  @Override
+  public void updateSnapshotRestoreTime(long time) {
+    snapshotRestoreTimeHisto.add(time);
+  }
+}

Added: hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java?rev=1514514&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java (added)
+++ hbase/branches/0.95/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java Thu Aug 15 22:04:25 2013
@@ -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.hadoop.hbase.master.balancer;
+
+import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableHistogram;
+
+public class MetricsBalancerSourceImpl extends BaseSourceImpl implements MetricsBalancerSource{
+
+  private MutableHistogram blanceClusterHisto;
+  private MutableCounterLong miscCount;
+
+  public MetricsBalancerSourceImpl() {
+    this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);
+  }
+
+  public MetricsBalancerSourceImpl(String metricsName,
+                                   String metricsDescription,
+                                   String metricsContext, String metricsJmxContext) {
+    super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
+  }
+
+
+  @Override
+  public void init() {
+    blanceClusterHisto = metricsRegistry.newHistogram(BALANCE_CLUSTER);
+    miscCount = metricsRegistry.newCounter(MISC_INVOATION_COUNT, "", 0L);
+
+  }
+
+  @Override
+  public void updateBalanceCluster(long time) {
+    blanceClusterHisto.add(time);
+  }
+
+  @Override
+  public void incrMiscInvocations() {
+     miscCount.incr();
+  }
+}

Added: hbase/branches/0.95/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsAssignmentManagerSource
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsAssignmentManagerSource?rev=1514514&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsAssignmentManagerSource (added)
+++ hbase/branches/0.95/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsAssignmentManagerSource Thu Aug 15 22:04:25 2013
@@ -0,0 +1 @@
+org.apache.hadoop.hbase.master.MetricsAssignmentManagerSourceImpl

Added: hbase/branches/0.95/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsMasterFileSystemSource
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsMasterFileSystemSource?rev=1514514&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsMasterFileSystemSource (added)
+++ hbase/branches/0.95/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsMasterFileSystemSource Thu Aug 15 22:04:25 2013
@@ -0,0 +1 @@
+org.apache.hadoop.hbase.master.MetricsMasterFilesystemSourceImpl
\ No newline at end of file

Added: hbase/branches/0.95/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsSnapshotSource
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsSnapshotSource?rev=1514514&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsSnapshotSource (added)
+++ hbase/branches/0.95/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsSnapshotSource Thu Aug 15 22:04:25 2013
@@ -0,0 +1 @@
+org.apache.hadoop.hbase.master.MetricsSnapshotSourceImpl

Added: hbase/branches/0.95/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.balancer.MetricsBalancerSource
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.balancer.MetricsBalancerSource?rev=1514514&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.balancer.MetricsBalancerSource (added)
+++ hbase/branches/0.95/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.balancer.MetricsBalancerSource Thu Aug 15 22:04:25 2013
@@ -0,0 +1 @@
+org.apache.hadoop.hbase.master.balancer.MetricsBalancerSourceImpl
\ No newline at end of file

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java?rev=1514514&r1=1514513&r2=1514514&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java Thu Aug 15 22:04:25 2013
@@ -123,6 +123,8 @@ public class AssignmentManager extends Z
 
   private LoadBalancer balancer;
 
+  private final MetricsAssignmentManager metricsAssignmentManager;
+
   private final TableLockManager tableLockManager;
 
   final private KeyLocker<String> locker = new KeyLocker<String>();
@@ -182,9 +184,6 @@ public class AssignmentManager extends Z
   private List<EventType> ignoreStatesRSOffline = Arrays.asList(
       EventType.RS_ZK_REGION_FAILED_OPEN, EventType.RS_ZK_REGION_CLOSED);
 
-  // metrics instance to send metrics for RITs
-  MetricsMaster metricsMaster;
-
   private final RegionStates regionStates;
 
   // The threshold to use bulk assigning. Using bulk assignment
@@ -273,7 +272,6 @@ public class AssignmentManager extends Z
     int maxThreads = conf.getInt("hbase.assignment.threads.max", 30);
     this.threadPoolExecutorService = Threads.getBoundedCachedThreadPool(
       maxThreads, 60L, TimeUnit.SECONDS, Threads.newDaemonThreadFactory("AM."));
-    this.metricsMaster = metricsMaster;// can be null only with tests.
     this.regionStates = new RegionStates(server, serverManager);
 
     this.bulkAssignWaitTillAllAssigned =
@@ -286,6 +284,8 @@ public class AssignmentManager extends Z
     zkEventWorkers = Threads.getBoundedCachedThreadPool(workers, 60L,
             TimeUnit.SECONDS, threadFactory);
     this.tableLockManager = tableLockManager;
+
+    this.metricsAssignmentManager = new MetricsAssignmentManager();
   }
 
   void startTimeOutMonitor() {
@@ -1533,170 +1533,175 @@ public class AssignmentManager extends Z
    * @return true if successful
    */
   boolean assign(final ServerName destination, final List<HRegionInfo> regions) {
-    int regionCount = regions.size();
-    if (regionCount == 0) {
-      return true;
-    }
-    LOG.debug("Assigning " + regionCount + " region(s) to " + destination.toString());
-    Set<String> encodedNames = new HashSet<String>(regionCount);
-    for (HRegionInfo region : regions) {
-      encodedNames.add(region.getEncodedName());
-    }
-
-    List<HRegionInfo> failedToOpenRegions = new ArrayList<HRegionInfo>();
-    Map<String, Lock> locks = locker.acquireLocks(encodedNames);
+    long startTime = EnvironmentEdgeManager.currentTimeMillis();
     try {
-      AtomicInteger counter = new AtomicInteger(0);
-      Map<String, Integer> offlineNodesVersions = new ConcurrentHashMap<String, Integer>();
-      OfflineCallback cb = new OfflineCallback(
-        watcher, destination, counter, offlineNodesVersions);
-      Map<String, RegionPlan> plans = new HashMap<String, RegionPlan>(regions.size());
-      List<RegionState> states = new ArrayList<RegionState>(regions.size());
+      int regionCount = regions.size();
+      if (regionCount == 0) {
+        return true;
+      }
+      LOG.debug("Assigning " + regionCount + " region(s) to " + destination.toString());
+      Set<String> encodedNames = new HashSet<String>(regionCount);
       for (HRegionInfo region : regions) {
-        String encodedRegionName = region.getEncodedName();
-        RegionState state = forceRegionStateToOffline(region, true);
-        if (state != null && asyncSetOfflineInZooKeeper(state, cb, destination)) {
-          RegionPlan plan = new RegionPlan(region, state.getServerName(), destination);
-          plans.put(encodedRegionName, plan);
-          states.add(state);
-        } else {
-          LOG.warn("failed to force region state to offline or "
-            + "failed to set it offline in ZK, will reassign later: " + region);
-          failedToOpenRegions.add(region); // assign individually later
-          Lock lock = locks.remove(encodedRegionName);
-          lock.unlock();
-        }
+        encodedNames.add(region.getEncodedName());
       }
 
-      // Wait until all unassigned nodes have been put up and watchers set.
-      int total = states.size();
-      for (int oldCounter = 0; !server.isStopped();) {
-        int count = counter.get();
-        if (oldCounter != count) {
-          LOG.info(destination.toString() + " unassigned znodes=" + count +
-            " of total=" + total);
-          oldCounter = count;
+      List<HRegionInfo> failedToOpenRegions = new ArrayList<HRegionInfo>();
+      Map<String, Lock> locks = locker.acquireLocks(encodedNames);
+      try {
+        AtomicInteger counter = new AtomicInteger(0);
+        Map<String, Integer> offlineNodesVersions = new ConcurrentHashMap<String, Integer>();
+        OfflineCallback cb = new OfflineCallback(
+          watcher, destination, counter, offlineNodesVersions);
+        Map<String, RegionPlan> plans = new HashMap<String, RegionPlan>(regions.size());
+        List<RegionState> states = new ArrayList<RegionState>(regions.size());
+        for (HRegionInfo region : regions) {
+          String encodedRegionName = region.getEncodedName();
+          RegionState state = forceRegionStateToOffline(region, true);
+          if (state != null && asyncSetOfflineInZooKeeper(state, cb, destination)) {
+            RegionPlan plan = new RegionPlan(region, state.getServerName(), destination);
+            plans.put(encodedRegionName, plan);
+            states.add(state);
+          } else {
+            LOG.warn("failed to force region state to offline or "
+              + "failed to set it offline in ZK, will reassign later: " + region);
+            failedToOpenRegions.add(region); // assign individually later
+            Lock lock = locks.remove(encodedRegionName);
+            lock.unlock();
+          }
         }
-        if (count >= total) break;
-        Threads.sleep(5);
-      }
 
-      if (server.isStopped()) {
-        return false;
-      }
-
-      // Add region plans, so we can updateTimers when one region is opened so
-      // that unnecessary timeout on RIT is reduced.
-      this.addPlans(plans);
-
-      List<Triple<HRegionInfo, Integer, List<ServerName>>> regionOpenInfos =
-        new ArrayList<Triple<HRegionInfo, Integer, List<ServerName>>>(states.size());
-      for (RegionState state: states) {
-        HRegionInfo region = state.getRegion();
-        String encodedRegionName = region.getEncodedName();
-        Integer nodeVersion = offlineNodesVersions.get(encodedRegionName);
-        if (nodeVersion == null || nodeVersion == -1) {
-          LOG.warn("failed to offline in zookeeper: " + region);
-          failedToOpenRegions.add(region); // assign individually later
-          Lock lock = locks.remove(encodedRegionName);
-          lock.unlock();
-        } else {
-          regionStates.updateRegionState(region,
-            RegionState.State.PENDING_OPEN, destination);
-          List<ServerName> favoredNodes = ServerName.EMPTY_SERVER_LIST;
-          if (this.shouldAssignRegionsWithFavoredNodes) {
-            favoredNodes = ((FavoredNodeLoadBalancer)this.balancer).getFavoredNodes(region);
+        // Wait until all unassigned nodes have been put up and watchers set.
+        int total = states.size();
+        for (int oldCounter = 0; !server.isStopped();) {
+          int count = counter.get();
+          if (oldCounter != count) {
+            LOG.info(destination.toString() + " unassigned znodes=" + count +
+              " of total=" + total);
+            oldCounter = count;
+          }
+          if (count >= total) break;
+          Threads.sleep(5);
+        }
+
+        if (server.isStopped()) {
+          return false;
+        }
+
+        // Add region plans, so we can updateTimers when one region is opened so
+        // that unnecessary timeout on RIT is reduced.
+        this.addPlans(plans);
+
+        List<Triple<HRegionInfo, Integer, List<ServerName>>> regionOpenInfos =
+          new ArrayList<Triple<HRegionInfo, Integer, List<ServerName>>>(states.size());
+        for (RegionState state: states) {
+          HRegionInfo region = state.getRegion();
+          String encodedRegionName = region.getEncodedName();
+          Integer nodeVersion = offlineNodesVersions.get(encodedRegionName);
+          if (nodeVersion == null || nodeVersion == -1) {
+            LOG.warn("failed to offline in zookeeper: " + region);
+            failedToOpenRegions.add(region); // assign individually later
+            Lock lock = locks.remove(encodedRegionName);
+            lock.unlock();
+          } else {
+            regionStates.updateRegionState(region,
+              RegionState.State.PENDING_OPEN, destination);
+            List<ServerName> favoredNodes = ServerName.EMPTY_SERVER_LIST;
+            if (this.shouldAssignRegionsWithFavoredNodes) {
+              favoredNodes = ((FavoredNodeLoadBalancer)this.balancer).getFavoredNodes(region);
+            }
+            regionOpenInfos.add(new Triple<HRegionInfo, Integer,  List<ServerName>>(
+              region, nodeVersion, favoredNodes));
           }
-          regionOpenInfos.add(new Triple<HRegionInfo, Integer,  List<ServerName>>(
-            region, nodeVersion, favoredNodes));
         }
-      }
 
-      // Move on to open regions.
-      try {
-        // Send OPEN RPC. If it fails on a IOE or RemoteException, the
-        // TimeoutMonitor will pick up the pieces.
-        long maxWaitTime = System.currentTimeMillis() +
-          this.server.getConfiguration().
-            getLong("hbase.regionserver.rpc.startup.waittime", 60000);
-        for (int i = 1; i <= maximumAttempts && !server.isStopped(); i++) {
-          try {
-            List<RegionOpeningState> regionOpeningStateList = serverManager
-              .sendRegionOpen(destination, regionOpenInfos);
-            if (regionOpeningStateList == null) {
-              // Failed getting RPC connection to this server
-              return false;
-            }
-            for (int k = 0, n = regionOpeningStateList.size(); k < n; k++) {
-              RegionOpeningState openingState = regionOpeningStateList.get(k);
-              if (openingState != RegionOpeningState.OPENED) {
-                HRegionInfo region = regionOpenInfos.get(k).getFirst();
-                if (openingState == RegionOpeningState.ALREADY_OPENED) {
-                  processAlreadyOpenedRegion(region, destination);
-                } else if (openingState == RegionOpeningState.FAILED_OPENING) {
-                  // Failed opening this region, reassign it later
-                  failedToOpenRegions.add(region);
-                } else {
-                  LOG.warn("THIS SHOULD NOT HAPPEN: unknown opening state "
-                    + openingState + " in assigning region " + region);
+        // Move on to open regions.
+        try {
+          // Send OPEN RPC. If it fails on a IOE or RemoteException, the
+          // TimeoutMonitor will pick up the pieces.
+          long maxWaitTime = System.currentTimeMillis() +
+            this.server.getConfiguration().
+              getLong("hbase.regionserver.rpc.startup.waittime", 60000);
+          for (int i = 1; i <= maximumAttempts && !server.isStopped(); i++) {
+            try {
+              List<RegionOpeningState> regionOpeningStateList = serverManager
+                .sendRegionOpen(destination, regionOpenInfos);
+              if (regionOpeningStateList == null) {
+                // Failed getting RPC connection to this server
+                return false;
+              }
+              for (int k = 0, n = regionOpeningStateList.size(); k < n; k++) {
+                RegionOpeningState openingState = regionOpeningStateList.get(k);
+                if (openingState != RegionOpeningState.OPENED) {
+                  HRegionInfo region = regionOpenInfos.get(k).getFirst();
+                  if (openingState == RegionOpeningState.ALREADY_OPENED) {
+                    processAlreadyOpenedRegion(region, destination);
+                  } else if (openingState == RegionOpeningState.FAILED_OPENING) {
+                    // Failed opening this region, reassign it later
+                    failedToOpenRegions.add(region);
+                  } else {
+                    LOG.warn("THIS SHOULD NOT HAPPEN: unknown opening state "
+                      + openingState + " in assigning region " + region);
+                  }
                 }
               }
-            }
-            break;
-          } catch (IOException e) {
-            if (e instanceof RemoteException) {
-              e = ((RemoteException)e).unwrapRemoteException();
-            }
-            if (e instanceof RegionServerStoppedException) {
-              LOG.warn("The region server was shut down, ", e);
-              // No need to retry, the region server is a goner.
-              return false;
-            } else if (e instanceof ServerNotRunningYetException) {
-              long now = System.currentTimeMillis();
-              if (now < maxWaitTime) {
-                LOG.debug("Server is not yet up; waiting up to " +
-                  (maxWaitTime - now) + "ms", e);
-                Thread.sleep(100);
-                i--; // reset the try count
-                continue;
+              break;
+            } catch (IOException e) {
+              if (e instanceof RemoteException) {
+                e = ((RemoteException)e).unwrapRemoteException();
               }
-            } else if (e instanceof java.net.SocketTimeoutException
-                && this.serverManager.isServerOnline(destination)) {
-              // In case socket is timed out and the region server is still online,
-              // the openRegion RPC could have been accepted by the server and
-              // just the response didn't go through.  So we will retry to
-              // open the region on the same server.
-              if (LOG.isDebugEnabled()) {
-                LOG.debug("Bulk assigner openRegion() to " + destination
-                  + " has timed out, but the regions might"
-                  + " already be opened on it.", e);
+              if (e instanceof RegionServerStoppedException) {
+                LOG.warn("The region server was shut down, ", e);
+                // No need to retry, the region server is a goner.
+                return false;
+              } else if (e instanceof ServerNotRunningYetException) {
+                long now = System.currentTimeMillis();
+                if (now < maxWaitTime) {
+                  LOG.debug("Server is not yet up; waiting up to " +
+                    (maxWaitTime - now) + "ms", e);
+                  Thread.sleep(100);
+                  i--; // reset the try count
+                  continue;
+                }
+              } else if (e instanceof java.net.SocketTimeoutException
+                  && this.serverManager.isServerOnline(destination)) {
+                // In case socket is timed out and the region server is still online,
+                // the openRegion RPC could have been accepted by the server and
+                // just the response didn't go through.  So we will retry to
+                // open the region on the same server.
+                if (LOG.isDebugEnabled()) {
+                  LOG.debug("Bulk assigner openRegion() to " + destination
+                    + " has timed out, but the regions might"
+                    + " already be opened on it.", e);
+                }
+                continue;
               }
-              continue;
+              throw e;
             }
-            throw e;
           }
+        } catch (IOException e) {
+          // Can be a socket timeout, EOF, NoRouteToHost, etc
+          LOG.info("Unable to communicate with the region server in order" +
+            " to assign regions", e);
+          return false;
+        } catch (InterruptedException e) {
+          throw new RuntimeException(e);
+        }
+      } finally {
+        for (Lock lock : locks.values()) {
+          lock.unlock();
         }
-      } catch (IOException e) {
-        // Can be a socket timeout, EOF, NoRouteToHost, etc
-        LOG.info("Unable to communicate with the region server in order" +
-          " to assign regions", e);
-        return false;
-      } catch (InterruptedException e) {
-        throw new RuntimeException(e);
-      }
-    } finally {
-      for (Lock lock : locks.values()) {
-        lock.unlock();
       }
-    }
 
-    if (!failedToOpenRegions.isEmpty()) {
-      for (HRegionInfo region : failedToOpenRegions) {
-        invokeAssign(region);
+      if (!failedToOpenRegions.isEmpty()) {
+        for (HRegionInfo region : failedToOpenRegions) {
+          invokeAssign(region);
+        }
       }
+      LOG.debug("Bulk assigning done for " + destination.toString());
+      return true;
+    } finally {
+      metricsAssignmentManager.updateBulkAssignTime(EnvironmentEdgeManager.currentTimeMillis() - startTime);
     }
-    LOG.debug("Bulk assigning done for " + destination.toString());
-    return true;
   }
 
   /**
@@ -1821,6 +1826,8 @@ public class AssignmentManager extends Z
    */
   private void assign(RegionState state,
       final boolean setOfflineInZK, final boolean forceNewPlan) {
+    long startTime = EnvironmentEdgeManager.currentTimeMillis();
+    try {
     RegionState currentState = state;
     int versionOfOfflineNode = -1;
     RegionPlan plan = null;
@@ -2029,6 +2036,9 @@ public class AssignmentManager extends Z
     if (!tomActivated) {
       regionStates.updateRegionState(region, RegionState.State.FAILED_OPEN);
     }
+    } finally {
+      metricsAssignmentManager.updateAssignmentTime(EnvironmentEdgeManager.currentTimeMillis() - startTime);
+    }
   }
 
   private void processAlreadyOpenedRegion(HRegionInfo region, ServerName sn) {
@@ -2804,10 +2814,10 @@ public class AssignmentManager extends Z
         oldestRITTime = ritTime;
       }
     }
-    if (this.metricsMaster != null) {
-      this.metricsMaster.updateRITOldestAge(oldestRITTime);
-      this.metricsMaster.updateRITCount(totalRITs);
-      this.metricsMaster.updateRITCountOverThreshold(totalRITsOverThreshold);
+    if (this.metricsAssignmentManager != null) {
+      this.metricsAssignmentManager.updateRITOldestAge(oldestRITTime);
+      this.metricsAssignmentManager.updateRITCount(totalRITs);
+      this.metricsAssignmentManager.updateRITCountOverThreshold(totalRITsOverThreshold);
     }
   }
 

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1514514&r1=1514513&r2=1514514&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Thu Aug 15 22:04:25 2013
@@ -758,7 +758,7 @@ MasterServices, Server {
 
     this.masterActiveTime = System.currentTimeMillis();
     // TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
-    this.fileSystemManager = new MasterFileSystem(this, this, metricsMaster, masterRecovery);
+    this.fileSystemManager = new MasterFileSystem(this, this, masterRecovery);
 
     this.tableDescriptors =
       new FSTableDescriptors(this.fileSystemManager.getFileSystem(),