You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by li...@apache.org on 2012/12/28 20:19:20 UTC

svn commit: r1426612 - in /hbase/branches/0.89-fb/src: main/java/org/apache/hadoop/hbase/ main/java/org/apache/hadoop/hbase/regionserver/ main/java/org/apache/hadoop/hbase/regionserver/metrics/ main/resources/hbase-webapps/regionserver/ test/java/org/a...

Author: liyin
Date: Fri Dec 28 19:19:20 2012
New Revision: 1426612

URL: http://svn.apache.org/viewvc?rev=1426612&view=rev
Log:
[HBASE-7276] Keep track of the num of rows being read and updated in RegionServer

Author: liyintang

Summary:
There was some confusions before whether RS shall maintain the num of read/write rpc call ops or the num of rows being read/updated. After some discussions with Adela, we felt like there is useless to maintain the rpc call ops since region server had already kept track of the multiput, next and get ops anyway.

So this diff is to keep track of the num of rows being read and updated in RS. In addition, I have removed the per-region level read/write cnt since it is not cose-effective. (Expensive to maintain it but not much value gained)

Test Plan: going to test it on the dev cluster and shadow cluster

Reviewers: adela

Reviewed By: adela

CC: hbase-eng@

Differential Revision: https://phabricator.fb.com/D667725

Modified:
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HServerLoad.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java
    hbase/branches/0.89-fb/src/main/resources/hbase-webapps/regionserver/regionserver.jsp
    hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
    hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
    hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HServerLoad.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HServerLoad.java?rev=1426612&r1=1426611&r2=1426612&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HServerLoad.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HServerLoad.java Fri Dec 28 19:19:20 2012
@@ -94,15 +94,6 @@ public class HServerLoad implements Writ
      */
     private int totalStaticBloomSizeKB = -1;
 
-    /** The total read request count */
-    private long totalReadRequest = 0;
-    /** The total write request count */
-    private long totalWriteRequest = 0;
-    /** The total read request per sec*/
-    private int readRequestPerSec = 0;
-    /** The total write request per sec*/
-    private int writeRequestPerSec = 0;
-
     /**
      * Constructor, for Writable
      */
@@ -117,24 +108,6 @@ public class HServerLoad implements Writ
      * @param storefileSizeMB
      * @param memstoreSizeMB
      * @param storefileIndexSizeMB
-     */
-    public RegionLoad(final byte[] name, final int stores,
-        final int storefiles, final int storefileSizeMB,
-        final int memstoreSizeMB, final int storefileIndexSizeMB,
-        final int rootIndexSizeKB, final int totalStaticIndexSizeKB,
-        final int totalStaticBloomSizeKB) {
-      this(name, stores, storefiles, storefileSizeMB, memstoreSizeMB,
-          storefileIndexSizeMB, rootIndexSizeKB, totalStaticIndexSizeKB,
-          totalStaticBloomSizeKB, null, null);
-    }
-
-    /**
-     * @param name
-     * @param stores
-     * @param storefiles
-     * @param storefileSizeMB
-     * @param memstoreSizeMB
-     * @param storefileIndexSizeMB
      * @param totalReadRequest
      * @param totalReadRequest
      * @param readRequestPerSec
@@ -144,8 +117,7 @@ public class HServerLoad implements Writ
         final int storefiles, final int storefileSizeMB,
         final int memstoreSizeMB, final int storefileIndexSizeMB,
         final int rootIndexSizeKB, final int totalStaticIndexSizeKB,
-        final int totalStaticBloomSizeKB, RequestMetrics readRequest,
-        RequestMetrics writeRequest) {
+        final int totalStaticBloomSizeKB) {
       this.name = name;
       this.stores = stores;
       this.storefiles = storefiles;
@@ -155,14 +127,6 @@ public class HServerLoad implements Writ
       this.rootIndexSizeKB = rootIndexSizeKB;
       this.totalStaticIndexSizeKB = totalStaticIndexSizeKB;
       this.totalStaticBloomSizeKB = totalStaticBloomSizeKB;
-      if (readRequest != null) {
-        this.totalReadRequest = readRequest.getTotalRequestCount();
-        this.readRequestPerSec = readRequest.getRequestPerSecond();
-      }
-      if (writeRequest != null) {
-        this.totalWriteRequest = writeRequest.getTotalRequestCount();
-        this.writeRequestPerSec = writeRequest.getRequestPerSecond();
-      }
     }
 
     // Getters
@@ -216,22 +180,6 @@ public class HServerLoad implements Writ
       return storefileIndexSizeMB;
     }
 
-    public long getTotalReadRequest() {
-      return totalReadRequest;
-    }
-
-    public long getTotalWriteRequest() {
-      return totalWriteRequest;
-    }
-
-    public int getReadRequestPerSec() {
-      return readRequestPerSec;
-    }
-
-    public int getWriteRequestPerSec() {
-      return writeRequestPerSec;
-    }
-
     // Setters
 
     /**
@@ -322,16 +270,6 @@ public class HServerLoad implements Writ
         sb = Strings.appendKeyValue(sb, "totalStaticBloomSizeKB",
           Integer.valueOf(this.totalStaticBloomSizeKB));
       }
-
-      if (this.totalReadRequest != 0) {
-        sb = Strings.appendKeyValue(sb, "totalReadRequest",
-          Long.valueOf(this.totalReadRequest));
-      }
-
-      if (this.totalWriteRequest != 0) {
-        sb = Strings.appendKeyValue(sb, "totalWriteRequest",
-          Long.valueOf(this.totalWriteRequest));
-      }
       return sb.toString();
     }
   }

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1426612&r1=1426611&r2=1426612&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Fri Dec 28 19:19:20 2012
@@ -184,8 +184,12 @@ public class HRegion implements HeapSize
   // private byte [] name = null;
 
   protected final AtomicLong memstoreSize = new AtomicLong(0);
-  protected final RequestMetrics readRequests = new RequestMetrics();
-  protected final RequestMetrics writeRequests = new RequestMetrics();
+  
+  // The number of rows are read
+  protected final AtomicInteger rowReadCnt = new AtomicInteger(0);
+  
+  // The numbe of rows are updated
+  protected final AtomicInteger rowUpdateCnt = new AtomicInteger(0);
 
   private HRegionServer regionServer = null;
   /**
@@ -737,13 +741,6 @@ public class HRegion implements HeapSize
 
   /** @return a HRegionInfo object for this region */
   public HRegionInfo getRegionInfo() {
-    return getRegionInfo(false);
-  }
-  
-  protected HRegionInfo getRegionInfo(boolean callFromRPC) {
-    if (callFromRPC) {
-      readRequests.incrTotalRequestCount();
-    }
     return this.regionInfo;
   }
 
@@ -1743,7 +1740,6 @@ public class HRegion implements HeapSize
    */
   public void delete(Delete delete, Integer lockid, boolean writeToWAL)
   throws IOException {
-    writeRequests.incrTotalRequestCount();
     checkReadOnly();
     checkResources();
     Integer lid = null;
@@ -1758,7 +1754,7 @@ public class HRegion implements HeapSize
       delete(delete.getFamilyMap(), writeToWAL);
 
     } finally {
-      if(lockid == null) internalReleaseRowLock(lid);
+      if(lockid == null) releaseRowLock(lid);
       splitsAndClosesLock.readLock().unlock();
     }
   }
@@ -1826,7 +1822,7 @@ public class HRegion implements HeapSize
   public void delete(Map<byte[], List<KeyValue>> familyMap, boolean writeToWAL)
   throws IOException {
     long now = EnvironmentEdgeManager.currentTimeMillis();
-
+    
     byte [] byteNow = Bytes.toBytes(now);
     boolean flush = false;
 
@@ -1914,7 +1910,6 @@ public class HRegion implements HeapSize
     // read lock, resources may run out.  For now, the thought is that this
     // will be extremely rare; we'll deal with it when it happens.
     checkResources();
-    writeRequests.incrTotalRequestCount();
     splitsAndClosesLock.readLock().lock();
 
     try {
@@ -1931,7 +1926,7 @@ public class HRegion implements HeapSize
         // All edits for the given row (across all column families) must happen atomically.
         put(put.getFamilyMap(), writeToWAL);
       } finally {
-        if(lockid == null) internalReleaseRowLock(lid);
+        if (lockid == null) releaseRowLock(lid);
       }
     } finally {
       splitsAndClosesLock.readLock().unlock();
@@ -1982,7 +1977,6 @@ public class HRegion implements HeapSize
    */
   public OperationStatusCode[] batchMutateWithLocks(Pair<Mutation, Integer>[] putsAndLocks,
       String methodName) throws IOException {
-    writeRequests.incrTotalRequestCount();
     BatchOperationInProgress<Pair<Mutation, Integer>> batchOp =
       new BatchOperationInProgress<Pair<Mutation,Integer>>(putsAndLocks);
 
@@ -2013,6 +2007,7 @@ public class HRegion implements HeapSize
     boolean isSignatureClear = true;
 
     long now = EnvironmentEdgeManager.currentTimeMillis();
+
     byte[] byteNow = Bytes.toBytes(now);
     boolean locked = false;
 
@@ -2183,7 +2178,6 @@ public class HRegion implements HeapSize
   public boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier,
       byte [] expectedValue, Writable w, Integer lockId, boolean writeToWAL)
   throws IOException{
-    writeRequests.incrTotalRequestCount();
     checkReadOnly();
     //TODO, add check for value length or maybe even better move this to the
     //client if this becomes a global setting
@@ -2233,7 +2227,7 @@ public class HRegion implements HeapSize
         }
         return false;
       } finally {
-        if(lockId == null) internalReleaseRowLock(lid);
+        if(lockId == null) releaseRowLock(lid);
       }
     } finally {
       splitsAndClosesLock.readLock().unlock();
@@ -2255,28 +2249,6 @@ public class HRegion implements HeapSize
     }
   }
 
-//  /*
-//   * Utility method to verify values length.
-//   * @param batchUpdate The update to verify
-//   * @throws IOException Thrown if a value is too long
-//   */
-//  private void validateValuesLength(Put put)
-//  throws IOException {
-//    Map<byte[], List<KeyValue>> families = put.getFamilyMap();
-//    for(Map.Entry<byte[], List<KeyValue>> entry : families.entrySet()) {
-//      HColumnDescriptor hcd =
-//        this.regionInfo.getTableDesc().getFamily(entry.getKey());
-//      int maxLen = hcd.getMaxValueLength();
-//      for(KeyValue kv : entry.getValue()) {
-//        if(kv.getValueLength() > maxLen) {
-//          throw new ValueOverMaxLengthException("Value in column "
-//            + Bytes.toString(kv.getColumn()) + " is too long. "
-//            + kv.getValueLength() + " > " + maxLen);
-//        }
-//      }
-//    }
-//  }
-
   /*
    * Check if resources to support an update.
    *
@@ -2406,7 +2378,11 @@ public class HRegion implements HeapSize
 
   private long applyFamilyMapToMemstore(Map<byte[], List<KeyValue>> familyMap,
                  MultiVersionConsistencyControl.WriteEntry writeEntryToUse) {
+    // Increment the rowUpdatedCnt
+    this.rowUpdateCnt.incrementAndGet();
+    
     long start = EnvironmentEdgeManager.currentTimeMillis();
+    
     MultiVersionConsistencyControl.WriteEntry w = null;
     long size = 0;
     try {
@@ -2819,7 +2795,6 @@ public class HRegion implements HeapSize
    * @return The id of the held lock.
    */
   public Integer obtainRowLock(final byte [] row) throws IOException {
-    readRequests.incrTotalRequestCount();
     return internalObtainRowLock(row, true);
   }
 
@@ -2900,11 +2875,6 @@ public class HRegion implements HeapSize
    * @param lockid  The lock ID to release.
    */
   public void releaseRowLock(final Integer lockid) {
-    readRequests.incrTotalRequestCount();
-    internalReleaseRowLock(lockid);
-  }
-
-  private void internalReleaseRowLock(final Integer lockid) {
     synchronized (lockedRows) {
       byte[] row = lockIds.remove(lockid);
       lockedRows.remove(row);
@@ -3218,7 +3188,6 @@ public class HRegion implements HeapSize
      */
     public synchronized Result[] nextRows(int nbRows, String metric) 
     throws IOException {
-      readRequests.incrTotalRequestCount();
       preCondition();
       boolean prefetchingEnabled = getOriginalScan().getServerPrefetching();
       int limit = this.getOriginalScan().getBatch();
@@ -3251,12 +3220,12 @@ public class HRegion implements HeapSize
         ScanPrefetcher callable = new ScanPrefetcher(nbRows, limit, metric);
         prefetchScanFuture = HRegionServer.scanPrefetchThreadPool.submit(callable);
       }
-      
+      rowReadCnt.addAndGet(scanResult.outResults.length);
       return scanResult.outResults;
     }
     
     /**
-     * This is used by Gets & Compactions & unit tests, whereas nextRows() is
+     * This is used by Gets & unit tests, whereas nextRows() is
      * used by Scans
      */
     @Override
@@ -3273,7 +3242,7 @@ public class HRegion implements HeapSize
         returnResult = nextInternal(tmpList, limit, metric);
         outResults.addAll(tmpList);
       }
-
+      rowReadCnt.incrementAndGet();
       resetFilters();
       if (isFilterDone()) {
         return false;
@@ -3547,7 +3516,7 @@ public class HRegion implements HeapSize
           Writables.getBytes(r.getRegionInfo())));
       meta.put(HConstants.CATALOG_FAMILY, edits);
     } finally {
-      meta.internalReleaseRowLock(lid);
+      meta.releaseRowLock(lid);
     }
   }
 
@@ -3914,7 +3883,6 @@ public class HRegion implements HeapSize
    * @throws IOException read exceptions
    */
   public Result get(final Get get, final Integer lockid) throws IOException {
-    readRequests.incrTotalRequestCount();
     // Verify families are all valid
     if (get.hasFamilies()) {
       for (byte [] family: get.familySet()) {
@@ -3960,11 +3928,7 @@ public class HRegion implements HeapSize
 
   public void mutateRow(RowMutations rm) throws IOException {
     boolean flush = false;
-
     Integer lid = null;
-
-    writeRequests.incrTotalRequestCount();
-
     splitsAndClosesLock.readLock().lock();
     try {
       // 1. run all pre-hooks before the atomic operation
@@ -4036,7 +4000,7 @@ public class HRegion implements HeapSize
     } finally {
       if (lid != null) {
         // 11. release the row lock
-        internalReleaseRowLock(lid);
+        releaseRowLock(lid);
       }
       if (flush) {
         // 12. Flush cache if needed. Do it outside update lock.
@@ -4060,7 +4024,7 @@ public class HRegion implements HeapSize
   throws IOException {
     // to be used for metrics
     long before = EnvironmentEdgeManager.currentTimeMillis();
-    writeRequests.incrTotalRequestCount();
+    this.rowUpdateCnt.incrementAndGet();
 
     checkRow(row);
     boolean flush = false;
@@ -4107,7 +4071,7 @@ public class HRegion implements HeapSize
       flush = isFlushSize(size);
     } finally {
       this.updatesLock.readLock().unlock();
-      internalReleaseRowLock(lid);
+      releaseRowLock(lid);
       HRegion.writeOps.incrementAndGet();
     }
 
@@ -4296,12 +4260,16 @@ public class HRegion implements HeapSize
     return false;
   }
 
-  public RequestMetrics getReadRequest() {
-    return this.readRequests;
+  public int getAndResetRowReadCnt() {
+    int readCnt = this.rowReadCnt.get();
+    this.rowReadCnt.addAndGet(-readCnt);
+    return readCnt;
   }
 
-  public RequestMetrics getWriteRequest() {
-    return this.writeRequests;
+  public int getAndResetRowUpdateCnt() {
+    int updateCnt = this.rowUpdateCnt.get();
+    this.rowUpdateCnt.addAndGet(-updateCnt);
+    return updateCnt;
   }
 
   /**

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1426612&r1=1426611&r2=1426612&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Fri Dec 28 19:19:20 2012
@@ -241,9 +241,6 @@ public class HRegionServer implements HR
   // Leases
   private Leases leases;
 
-  private volatile AtomicInteger numReads = new AtomicInteger();
-  private volatile AtomicInteger numWrites = new AtomicInteger();
-
   // Info server.  Default access so can be used by unit tests.  REGIONSERVER
   // is name of the webapp and the attribute name used stuffing this instance
   // into web context.
@@ -654,7 +651,8 @@ public class HRegionServer implements HR
             MemoryUsage memory =
               ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
             doMetrics();
-            HServerLoad hsl = new HServerLoad((numReads.get() + numWrites.get()),
+            HServerLoad hsl = new HServerLoad(
+              (int)this.metrics.requests.getPreviousIntervalValue(),
               (int)(memory.getUsed()/1024/1024),
               (int)(memory.getMax()/1024/1024));
             for (HRegion r: onlineRegions.values()) {
@@ -1017,8 +1015,6 @@ public class HRegionServer implements HR
       this.dynamicMetrics = RegionServerDynamicMetrics.newInstance(this);
       startServiceThreads();
       isOnline = true;
-      this.numReads.set(0);
-      this.numWrites.set(0);
 
       // Create the thread for the ThriftServer.
       // NOTE this defaults to FALSE so you have to enable it in conf
@@ -1077,8 +1073,7 @@ public class HRegionServer implements HR
     }
     return new HServerLoad.RegionLoad(name, stores, storefiles,
       storefileSizeMB, memstoreSizeMB, storefileIndexSizeMB, rootIndexSizeKB,
-      totalStaticIndexSizeKB, totalStaticBloomSizeKB, r.getReadRequest(),
-      r.getWriteRequest());
+      totalStaticIndexSizeKB, totalStaticBloomSizeKB);
   }
 
   /**
@@ -1339,9 +1334,6 @@ public class HRegionServer implements HR
   }
 
   protected void metrics() {
-    int numReads = 0;
-    int numWrites = 0;
-
     this.metrics.regions.set(this.onlineRegions.size());
     // Is this too expensive every three seconds getting a lock on onlineRegions
     // and then per store carried?  Can I make metrics be sloppier and avoid
@@ -1352,6 +1344,8 @@ public class HRegionServer implements HR
     long storefileIndexSize = 0;
     long totalStaticIndexSize = 0;
     long totalStaticBloomSize = 0;
+    int rowReadCnt = 0;
+    int rowUpdateCnt = 0;
 
     // Note that this is a map of Doubles instead of Longs. This is because we
     // do effective integer division, which would perhaps truncate more than it
@@ -1365,8 +1359,8 @@ public class HRegionServer implements HR
       for (Map.Entry<Integer, HRegion> e: this.onlineRegions.entrySet()) {
         HRegion r = e.getValue();
         memstoreSize += r.memstoreSize.get();
-        numReads += r.readRequests.getTotalRequestCount();
-        numWrites += r.writeRequests.getTotalRequestCount();
+        rowReadCnt += r.getAndResetRowReadCnt();
+        rowUpdateCnt += r.getAndResetRowUpdateCnt();
         synchronized (r.stores) {
           stores += r.stores.size();
           for(Map.Entry<byte [], Store> ee: r.stores.entrySet()) {
@@ -1380,7 +1374,6 @@ public class HRegionServer implements HR
               storefiles += tmpStorefiles;
             }
 
-
             {
               long tmpStorefileIndexSize = store.getStorefilesIndexSize();
               schemaMetrics.accumulateStoreMetric(tempVals,
@@ -1423,13 +1416,12 @@ public class HRegionServer implements HR
     for (Entry<String, MutableDouble> e : tempVals.entrySet()) {
       HRegion.setNumericMetric(e.getKey(), e.getValue().longValue());
     }
-
-    this.numReads.set(numReads);
-    this.numWrites.set(numWrites);
-
-    this.metrics.requests.set(numReads + numWrites);
-    this.metrics.numReads.set(numReads);
-    this.metrics.numWrites.set(numWrites);
+    
+    
+    this.metrics.rowReadCnt.inc(rowReadCnt);
+    this.metrics.rowUpdatedCnt.inc(rowUpdateCnt);
+    this.metrics.requests.inc(rowReadCnt + rowUpdateCnt);
+    
     this.metrics.stores.set(stores);
     this.metrics.storefiles.set(storefiles);
     this.metrics.memstoreSizeMB.set((int)(memstoreSize/(1024*1024)));
@@ -2352,7 +2344,7 @@ public class HRegionServer implements HR
   @Override
   public HRegionInfo getRegionInfo(final byte [] regionName)
   throws NotServingRegionException {
-    return getRegion(regionName).getRegionInfo(true);
+    return getRegion(regionName).getRegionInfo();
   }
 
 
@@ -2573,7 +2565,6 @@ public class HRegionServer implements HR
     }
     try {
       HRegion r = getRegion(regionName);
-      r.getReadRequest().incrTotalRequestCount();
       return addScanner(r.getScanner(scan));
     } catch (Throwable t) {
       throw convertThrowableToIOE(cleanup(t, "Failed openScanner"));
@@ -3077,17 +3068,7 @@ public class HRegionServer implements HR
   public HRegion getOnlineRegion(final byte [] regionName) {
     return onlineRegions.get(Bytes.mapKey(regionName));
   }
-
-  /** @return the number of reads */
-  public AtomicInteger getNumReads() {
-    return this.numReads;
-  }
-
-  /** @return the number of writes */
-  public AtomicInteger getNumWrites() {
-    return this.numWrites;
-  }
-
+  
   /** @return reference to FlushRequester */
   public FlushRequester getFlushRequester() {
     return this.cacheFlusher;

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java?rev=1426612&r1=1426611&r2=1426612&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java Fri Dec 28 19:19:20 2012
@@ -111,10 +111,20 @@ public class RegionServerMetrics impleme
   public final MetricsIntValue blockCacheHitRatio = new MetricsIntValue("blockCacheHitRatio", registry);
 
   /*
-   * Count of requests to the regionservers since last call to metrics update
+   * Count of rows read or updated to the regionservers since last call to metrics update
    */
   public final MetricsRate requests = new MetricsRate("requests", registry);
+  
+  /*
+   * Count of rows read from the regionservers since last call to metrics update
+   */
+  public final MetricsRate rowReadCnt = new MetricsRate("rowReadCnt", registry);
 
+  /*
+   * Count of row updated to the regionservers since last call to metrics update
+   */
+  public final MetricsRate rowUpdatedCnt =  new MetricsRate("rowUpdatedCnt", registry);
+  
   /**
    * Count of stores open on the regionserver.
    */
@@ -205,12 +215,6 @@ public class RegionServerMetrics impleme
   public final MetricsTimeVaryingRate mvccWaitTime =
     new MetricsTimeVaryingRate("mvccWait", registry);
 
-  public final MetricsRate numReads =
-    new MetricsRate("numReads", registry);
-
-  public final MetricsRate numWrites =
-    new MetricsRate("numWrites", registry);
-
   /**
    * time each scheduled compaction takes
    */
@@ -317,8 +321,8 @@ public class RegionServerMetrics impleme
       this.blockCacheEvictedMultiCount.pushMetric(this.metricsRecord);
       this.blockCacheEvictedMemoryCount.pushMetric(this.metricsRecord);
       this.blockCacheHitRatio.pushMetric(this.metricsRecord);
-      this.numReads.pushMetric(this.metricsRecord);
-      this.numWrites.pushMetric(this.metricsRecord);
+      this.rowReadCnt.pushMetric(this.metricsRecord);
+      this.rowUpdatedCnt.pushMetric(this.metricsRecord);
 
       // Be careful. Here is code for MTVR from up in hadoop:
       // public synchronized void inc(final int numOps, final long time) {
@@ -476,9 +480,9 @@ public class RegionServerMetrics impleme
     sb = Strings.appendKeyValue(sb, "compactionQueueSize",
       Integer.valueOf(this.compactionQueueSize.get()));
     sb = Strings.appendKeyValue(sb, "numWrites",
-      Float.valueOf(this.numWrites.getPreviousIntervalValue()));
+      Float.valueOf(this.rowUpdatedCnt.getPreviousIntervalValue()));
     sb = Strings.appendKeyValue(sb, "numReads",
-      Float.valueOf(this.numReads.getPreviousIntervalValue()));
+      Float.valueOf(this.rowReadCnt.getPreviousIntervalValue()));
 
     // Duplicate from jvmmetrics because metrics are private there so
     // inaccessible.

Modified: hbase/branches/0.89-fb/src/main/resources/hbase-webapps/regionserver/regionserver.jsp
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/resources/hbase-webapps/regionserver/regionserver.jsp?rev=1426612&r1=1426611&r2=1426612&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/resources/hbase-webapps/regionserver/regionserver.jsp (original)
+++ hbase/branches/0.89-fb/src/main/resources/hbase-webapps/regionserver/regionserver.jsp Fri Dec 28 19:19:20 2012
@@ -53,8 +53,6 @@
 <% if (onlineRegionInfoAndOpenDate != null && onlineRegionInfoAndOpenDate.size() > 0) { %>
 <table>
 <tr><th>Region Name</th>
-	<th>Read Requests/sec</th>
-	<th>Write Requests/sec</th>
 	<th>Start Key</th>
 	<th>End Key</th>
   <th>Metrics</th>
@@ -65,12 +63,10 @@
  %>
 
 <tr><td><%= r.getRegionNameAsString() %></td>
-	<td><%= load.getReadRequestPerSec() %></td>
-	<td><%= load.getWriteRequestPerSec() %></td>
     <td><%= Bytes.toStringBinary(r.getStartKey()) %></td>
-	<td><%= Bytes.toStringBinary(r.getEndKey()) %></td>
+    <td><%= Bytes.toStringBinary(r.getEndKey()) %></td>
     <td><%= load.toString() %></td>
-  <td><%= onlineRegionInfoAndOpenDate.get(r) %></td>
+    <td><%= onlineRegionInfoAndOpenDate.get(r) %></td>
 </tr>
 <%
      }

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java?rev=1426612&r1=1426611&r2=1426612&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java Fri Dec 28 19:19:20 2012
@@ -110,6 +110,7 @@ public class HBaseTestingUtility {
   private final Configuration conf;
   private final CacheConfig cacheConf;
   private MiniZooKeeperCluster zkCluster = null;
+  private static int USERNAME_SUFFIX = 0;
 
   /**
    * The default number of regions per regionserver when creating a pre-split
@@ -1129,16 +1130,14 @@ public class HBaseTestingUtility {
    * @return A new configuration instance with a different user set into it.
    * @throws IOException
    */
-  public static Configuration setDifferentUser(final Configuration c,
-    final String differentiatingSuffix)
+  public static Configuration setDifferentUser(final Configuration c)
   throws IOException {
     FileSystem currentfs = FileSystem.get(c);
     Preconditions.checkArgument(currentfs instanceof DistributedFileSystem);
     // Else distributed filesystem.  Make a new instance per daemon.  Below
     // code is taken from the AppendTestUtil over in hdfs.
     Configuration c2 = new Configuration(c);
-    String username = UserGroupInformation.getCurrentUGI().getUserName() +
-      differentiatingSuffix;
+    String username = UserGroupInformation.getCurrentUGI().getUserName() + (USERNAME_SUFFIX++);
     UnixUserGroupInformation.saveToConf(c2,
       UnixUserGroupInformation.UGI_PROPERTY_NAME,
       new UnixUserGroupInformation(username, new String[]{"supergroup"}));

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java?rev=1426612&r1=1426611&r2=1426612&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java Fri Dec 28 19:19:20 2012
@@ -227,16 +227,27 @@ public class TestRegionServerMetrics {
     final HRegionServer rs =
         testUtil.getMiniHBaseCluster().getRegionServer(0);
     
+    long preNumRead = 0;
+    long preNumWrite = 0;
+    for (HRegion region: rs.getOnlineRegions()) {
+      preNumRead += region.rowReadCnt.get();
+      preNumRead += region.rowUpdateCnt.get();
+    }
+    
     HRegion[] regions = rs.getOnlineRegionsAsArray();
-    int reads = rs.getNumReads().get();
-    int writes = rs.getNumWrites().get();
     for (int i=0; i<regions.length;  i++) {
       Get g = new Get(Bytes.toBytes("row" + i));
       regions[i].get(g, null);
     }
-    rs.doMetrics();
-    assertEquals(regions.length, (rs.getNumReads().get() - reads));
-    assertEquals(0, (rs.getNumWrites().get() - writes));
+    
+    long numRead = 0;
+    long numWrite = 0;
+    for (HRegion region: rs.getOnlineRegions()) {
+      numRead += region.rowReadCnt.get();
+      numWrite += region.rowUpdateCnt.get();
+    }
+    assertEquals(regions.length, numRead - preNumRead);
+    assertEquals(0, numWrite - preNumWrite);
   }
 
   @Test

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java?rev=1426612&r1=1426611&r2=1426612&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java Fri Dec 28 19:19:20 2012
@@ -207,8 +207,7 @@ public class TestWALReplay {
     wal.sync();
 
     // Now 'crash' the region by stealing its wal
-    Configuration newConf = HBaseTestingUtility.setDifferentUser(this.conf,
-        tableNameStr);
+    Configuration newConf = HBaseTestingUtility.setDifferentUser(this.conf);
     runWALSplit(newConf);
     HLog wal2 = createWAL(newConf);
     HRegion region2 = new HRegion(basedir, wal2, FileSystem.get(newConf),
@@ -291,8 +290,7 @@ public class TestWALReplay {
     // Set down maximum recovery so we dfsclient doesn't linger retrying something
     // long gone.
     HBaseTestingUtility.setMaxRecoveryErrorCount(wal2.getOutputStream(), 1);
-    Configuration newConf = HBaseTestingUtility.setDifferentUser(this.conf,
-      tableNameStr);
+    Configuration newConf = HBaseTestingUtility.setDifferentUser(this.conf);
     runWALSplit(newConf);
     FileSystem newFS = FileSystem.get(newConf);
     // Make a new wal for new region open.
@@ -447,8 +445,7 @@ public class TestWALReplay {
 
     // Make a new conf and a new fs for the splitter to run on so we can take
     // over old wal.
-    Configuration newConf = HBaseTestingUtility.setDifferentUser(this.conf,
-      ".replay.wal.secondtime");
+    Configuration newConf = HBaseTestingUtility.setDifferentUser(this.conf);
     runWALSplit(newConf);
     FileSystem newFS = FileSystem.get(newConf);
     // 100k seems to make for about 4 flushes during HRegion#initialize.