You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2009/07/30 20:14:01 UTC

svn commit: r799397 - in /hadoop/hbase/branches/0.19: ./ src/java/org/apache/hadoop/hbase/ipc/ src/java/org/apache/hadoop/hbase/master/metrics/ src/java/org/apache/hadoop/hbase/regionserver/metrics/

Author: stack
Date: Thu Jul 30 18:14:00 2009
New Revision: 799397

URL: http://svn.apache.org/viewvc?rev=799397&view=rev
Log:
HBASE-1722 Add support for exporting HBase metrics via JMX

Added:
    hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/ipc/HBaseRPCStatistics.java
    hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/ipc/HBaseRPCStatisticsMBean.java
    hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/metrics/MasterStatistics.java
    hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/metrics/MasterStatisticsMBean.java
    hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerStatistics.java
    hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerStatisticsMBean.java
Modified:
    hadoop/hbase/branches/0.19/CHANGES.txt
    hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/ipc/HBaseRpcMetrics.java
    hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/metrics/MasterMetrics.java
    hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java

Modified: hadoop/hbase/branches/0.19/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/CHANGES.txt?rev=799397&r1=799396&r2=799397&view=diff
==============================================================================
--- hadoop/hbase/branches/0.19/CHANGES.txt (original)
+++ hadoop/hbase/branches/0.19/CHANGES.txt Thu Jul 30 18:14:00 2009
@@ -30,6 +30,10 @@
                too much
    HBASE-1683  OOME on master splitting logs; stuck, won't go down
 
+  IMPROVEMENTS
+   HBASE-1722  Add support for exporting HBase metrics via JMX
+               (Gary Helmling via Stack)
+
 Release 0.19.3 - May 27th, 2009
   BUG FIXES
    HBASE-1413  fall back to filesystem block size default if

Added: hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/ipc/HBaseRPCStatistics.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/ipc/HBaseRPCStatistics.java?rev=799397&view=auto
==============================================================================
--- hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/ipc/HBaseRPCStatistics.java (added)
+++ hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/ipc/HBaseRPCStatistics.java Thu Jul 30 18:14:00 2009
@@ -0,0 +1,133 @@
+/**
+ * Copyright 2009 The Apache Software Foundation
+ *
+ * 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.ipc;
+
+import javax.management.ObjectName;
+
+import org.apache.hadoop.metrics.util.MBeanUtil;
+
+/**
+ * Exports HBase RPC statistics recorded in {@link HBaseRpcMetrics} as an MBean
+ * for JMX monitoring.
+ */
+public class HBaseRPCStatistics implements HBaseRPCStatisticsMBean {
+  private final HBaseRpcMetrics rpcMetrics;
+  private final ObjectName mbeanName;
+  private final HBaseServer server;
+
+  public HBaseRPCStatistics(HBaseRpcMetrics metrics, 
+      String hostName, String port, HBaseServer hbaseServer) {
+    rpcMetrics = metrics;
+
+    String name = String.format("RPCStatistics-%s", 
+        (port != null ? port : "unknown"));
+
+    mbeanName = MBeanUtil.registerMBean("HBase", name, this);
+    server = hbaseServer;
+  }
+
+  public void shutdown() {
+    if (mbeanName != null)
+      MBeanUtil.unregisterMBean(mbeanName);
+  }
+
+  /**
+   * Returns average RPC processing time since reset was last called
+   */
+  @Override
+  public long getRpcProcessingTimeAverage() {
+    return rpcMetrics.rpcProcessingTime.getPreviousIntervalAverageTime();
+  }
+
+  /**
+   * The maximum RPC processing time for current update interval.
+   */
+  @Override
+  public long getRpcProcessingTimeMax() {
+    return rpcMetrics.rpcProcessingTime.getMaxTime();
+  }
+
+  /**
+   * The minimum RPC operation processing time since reset was last called
+   */
+  @Override
+  public long getRpcProcessingTimeMin() {
+    return rpcMetrics.rpcProcessingTime.getMinTime();
+  }
+
+  /**
+   * The number of RPC operations in the last sampling interval
+   */
+  @Override
+  public int getRpcNumOps() {
+    return rpcMetrics.rpcProcessingTime.getPreviousIntervalNumOps();
+  }
+
+  /**
+   * The average RPC operation queued time in the last sampling interval
+   */
+  @Override
+  public long getRpcQueueTimeAverage() {
+    return rpcMetrics.rpcQueueTime.getPreviousIntervalAverageTime();
+  }
+
+  /**
+   * The maximum RPC operation queued time since reset was last called
+   */
+  @Override
+  public long getRpcQueueTimeMax() {
+    return rpcMetrics.rpcQueueTime.getMaxTime();
+  }
+
+  /**
+   * The minimum RPC operation queued time since reset was last called
+   */
+  @Override
+  public long getRpcQueueTimeMin() {
+    return rpcMetrics.rpcQueueTime.getMinTime();
+  }
+
+  /**
+   * The number of RPC calls in the queue
+   */
+  @Override
+  public int getCallQueueLen() {
+    return server.getCallQueueLen();
+  }
+
+  /**
+   * The number of current RPC connections
+   */
+  @Override
+  public int getNumOpenConnections() {
+    return server.getNumOpenConnections();
+  }
+
+  /**
+   * Resets minimum and maximum values for RPC queue
+   * and processing timers.
+   */
+  @Override
+  public void resetAllMinMax() {
+    rpcMetrics.rpcProcessingTime.resetMinMax();
+    rpcMetrics.rpcQueueTime.resetMinMax();
+  }
+
+}

Added: hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/ipc/HBaseRPCStatisticsMBean.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/ipc/HBaseRPCStatisticsMBean.java?rev=799397&view=auto
==============================================================================
--- hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/ipc/HBaseRPCStatisticsMBean.java (added)
+++ hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/ipc/HBaseRPCStatisticsMBean.java Thu Jul 30 18:14:00 2009
@@ -0,0 +1,78 @@
+/**
+ * Copyright 2009 The Apache Software Foundation
+ *
+ * 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.ipc;
+
+/**
+ * JMX MBean interface for exporting statistics on HBase RPC calls
+ */
+public interface HBaseRPCStatisticsMBean {
+
+  /**
+   * Returns average RPC processing time since reset was last called
+   */
+  public long getRpcProcessingTimeAverage();
+
+  /**
+   * The maximum RPC processing time for current update interval.
+   */
+  public long getRpcProcessingTimeMax();
+
+  /**
+   * The minimum RPC operation processing time since reset was last called
+   */
+  public long getRpcProcessingTimeMin();
+
+  /**
+   * The number of RPC operations in the last sampling interval
+   */
+  public int getRpcNumOps();
+
+  /**
+   * The average RPC operation queued time in the last sampling interval
+   */
+  public long getRpcQueueTimeAverage();
+
+  /**
+   * The maximum RPC operation queued time since reset was last called
+   */
+  public long getRpcQueueTimeMax();
+
+  /**
+   * The minimum RPC operation queued time since reset was last called
+   */
+  public long getRpcQueueTimeMin();
+
+  /**
+   * Reset all min max times
+   */
+  void resetAllMinMax();
+  
+  /**
+   * The number of open RPC conections
+   * @return the number of open rpc connections
+   */
+  public int getNumOpenConnections();
+  
+  /**
+   * The number of rpc calls in the queue.
+   * @return The number of rpc calls in the queue.
+   */
+  public int getCallQueueLen();  
+}

Modified: hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/ipc/HBaseRpcMetrics.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/ipc/HBaseRpcMetrics.java?rev=799397&r1=799396&r2=799397&view=diff
==============================================================================
--- hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/ipc/HBaseRpcMetrics.java (original)
+++ hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/ipc/HBaseRpcMetrics.java Thu Jul 30 18:14:00 2009
@@ -47,6 +47,7 @@
 public class HBaseRpcMetrics implements Updater {
   private MetricsRecord metricsRecord;
   private static Log LOG = LogFactory.getLog(HBaseRpcMetrics.class);
+  private final HBaseRPCStatistics rpcStatistics;
   
   public HBaseRpcMetrics(String hostName, String port, HBaseServer server) {
     MetricsContext context = MetricsUtil.getContext("rpc");
@@ -58,6 +59,8 @@
         + hostName + ", port=" + port);
 
     context.registerUpdater(this);
+    
+    rpcStatistics = new HBaseRPCStatistics(this, hostName, port, server);
   }
   
   
@@ -98,6 +101,7 @@
   }
 
   public void shutdown() {
-    // Nothing to do
+    if (rpcStatistics != null)
+      rpcStatistics.shutdown();
   }
 }
\ No newline at end of file

Modified: hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/metrics/MasterMetrics.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/metrics/MasterMetrics.java?rev=799397&r1=799396&r2=799397&view=diff
==============================================================================
--- hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/metrics/MasterMetrics.java (original)
+++ hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/metrics/MasterMetrics.java Thu Jul 30 18:14:00 2009
@@ -37,7 +37,9 @@
 public class MasterMetrics implements Updater {
   private final Log LOG = LogFactory.getLog(this.getClass());
   private final MetricsRecord metricsRecord;
-  
+  private final MasterStatistics masterStatistics;
+
+
   /*
    * Count of requests to the cluster since last call to metrics update
    */
@@ -51,11 +53,16 @@
     metricsRecord.setTag("Master", name);
     context.registerUpdater(this);
     JvmMetrics.init("Master", name);
+
+    // expose the MBean for metrics
+    masterStatistics = new MasterStatistics(this);
+
     LOG.info("Initialized");
   }
   
   public void shutdown() {
-    // nought to do.
+    if (masterStatistics != null)
+      masterStatistics.shutdown();
   }
     
   /**

Added: hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/metrics/MasterStatistics.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/metrics/MasterStatistics.java?rev=799397&view=auto
==============================================================================
--- hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/metrics/MasterStatistics.java (added)
+++ hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/metrics/MasterStatistics.java Thu Jul 30 18:14:00 2009
@@ -0,0 +1,49 @@
+/**
+ * Copyright 2009 The Apache Software Foundation
+ *
+ * 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.metrics;
+
+import javax.management.ObjectName;
+
+import org.apache.hadoop.metrics.util.MBeanUtil;
+
+/**
+ * Exports the {@link MasterMetrics} statistics as an MBean
+ * for JMX.
+ */
+public class MasterStatistics implements MasterStatisticsMBean {
+  private final MasterMetrics metrics;
+  private final ObjectName mbeanName;
+
+  public MasterStatistics(MasterMetrics masterMetrics) {
+    metrics = masterMetrics;
+    mbeanName = MBeanUtil.registerMBean("Master", "MasterStatistics", this);    
+  }
+
+  public void shutdown() {
+    if (mbeanName != null)
+      MBeanUtil.unregisterMBean(mbeanName);
+  }
+
+  @Override
+  public int getClusterRequests() {
+    return metrics.getRequests();
+  }
+
+}

Added: hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/metrics/MasterStatisticsMBean.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/metrics/MasterStatisticsMBean.java?rev=799397&view=auto
==============================================================================
--- hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/metrics/MasterStatisticsMBean.java (added)
+++ hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/metrics/MasterStatisticsMBean.java Thu Jul 30 18:14:00 2009
@@ -0,0 +1,33 @@
+/**
+ * Copyright 2009 The Apache Software Foundation
+ *
+ * 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.metrics;
+
+/**
+ * JMX interface for runtime statistics exported by the HBase Master,
+ * mirroring {@link MasterMetrics}.
+ */
+public interface MasterStatisticsMBean {
+
+  /**
+   * Returns the number of requests to the cluster since the
+   * last metrics update.
+   */
+  public int getClusterRequests();
+}

Modified: hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java?rev=799397&r1=799396&r2=799397&view=diff
==============================================================================
--- hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java (original)
+++ hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java Thu Jul 30 18:14:00 2009
@@ -46,7 +46,9 @@
   private final MetricsRecord metricsRecord;
   private long lastUpdate = System.currentTimeMillis();
   private static final int MB = 1024*1024;
-  
+
+  private final RegionServerStatistics statistics;
+
   /**
    * Count of regions carried by this regionserver
    */
@@ -87,13 +89,18 @@
     context.registerUpdater(this);
     // Add jvmmetrics.
     JvmMetrics.init("RegionServer", name);
+
+    // export for JMX
+    statistics = new RegionServerStatistics(this, name);
+
     LOG.info("Initialized");
   }
-  
+
   public void shutdown() {
-    // nought to do.
+    if (statistics != null)
+      statistics.shutdown();
   }
-    
+
   /**
    * Since this object is a registered updater, this method will be called
    * periodically, e.g. every 5 seconds.
@@ -110,7 +117,7 @@
     this.metricsRecord.update();
     this.lastUpdate = System.currentTimeMillis();
   }
-  
+
   public void resetAllMinMax() {
     // Nothing to do
   }

Added: hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerStatistics.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerStatistics.java?rev=799397&view=auto
==============================================================================
--- hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerStatistics.java (added)
+++ hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerStatistics.java Thu Jul 30 18:14:00 2009
@@ -0,0 +1,87 @@
+/**
+ * Copyright 2009 The Apache Software Foundation
+ *
+ * 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.regionserver.metrics;
+
+import javax.management.ObjectName;
+
+import org.apache.hadoop.metrics.util.MBeanUtil;
+
+/**
+ * Exports metrics recorded by {@link RegionServerMetrics} as an MBean
+ * for JMX monitoring.
+ */
+public class RegionServerStatistics implements RegionServerStatisticsMBean {
+
+  private final RegionServerMetrics rsMetrics;
+  private final ObjectName mbeanName;
+
+  public RegionServerStatistics(RegionServerMetrics metrics, String rsName) {
+    rsMetrics = metrics;
+    mbeanName = MBeanUtil.registerMBean("RegionServer", "RegionServerStatistics", this);    
+  }
+
+  public void shutdown() {
+    if (mbeanName != null)
+      MBeanUtil.unregisterMBean(mbeanName);
+  }
+
+  /**
+   * Count of regions carried by this regionserver
+   */
+  public int getRegionsNum() {
+    return rsMetrics.regions.get();
+  }
+
+  /**
+   * Count of requests to the regionservers since last call to metrics update
+   */
+  public float getRequestsRate() {
+    return rsMetrics.getRequests();
+  }
+
+  /**
+   * Count of stores open on the regionserver.
+   */
+  public int getStoresNum() {
+    return rsMetrics.stores.get();
+  }
+
+  /**
+   * Count of storefiles open on the regionserver.
+   */
+  public int getStoreFilesNum() {
+    return rsMetrics.storefiles.get();
+  }
+
+  /**
+   * Sum of all the storefile index sizes in this regionserver in MB
+   */
+  public int getStoreFileIndexSizeMB() {
+    return rsMetrics.storefileIndexSizeMB.get();
+  }
+
+  /**
+   * Sum of all the memcache sizes in this regionserver in MB
+   */
+  public int getMemcacheSizeMB() {
+    return rsMetrics.memcacheSizeMB.get();
+  }
+
+}

Added: hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerStatisticsMBean.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerStatisticsMBean.java?rev=799397&view=auto
==============================================================================
--- hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerStatisticsMBean.java (added)
+++ hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerStatisticsMBean.java Thu Jul 30 18:14:00 2009
@@ -0,0 +1,58 @@
+/**
+ * Copyright 2009 The Apache Software Foundation
+ *
+ * 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.regionserver.metrics;
+
+
+/**
+ * JMX interface for exporting RegionServer statistics.  
+ */
+public interface RegionServerStatisticsMBean {
+
+  /**
+   * Count of regions carried by this regionserver
+   */
+  public int getRegionsNum();
+
+  /*
+   * Count of requests to the regionservers since last call to metrics update
+   */
+  public float getRequestsRate();
+
+  /**
+   * Count of stores open on the regionserver.
+   */
+  public int getStoresNum();
+
+  /**
+   * Count of storefiles open on the regionserver.
+   */
+  public int getStoreFilesNum();
+
+  /**
+   * Sum of all the storefile index sizes in this regionserver in MB
+   */
+  public int getStoreFileIndexSizeMB();
+
+  /**
+   * Sum of all the memcache sizes in this regionserver in MB
+   */
+  public int getMemcacheSizeMB();
+
+}