hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From mbau...@apache.org
Subject svn commit: r1241589 - in /hbase/branches/0.89-fb/src: main/java/org/apache/hadoop/hbase/regionserver/ test/java/org/apache/hadoop/hbase/regionserver/
Date Tue, 07 Feb 2012 20:09:41 GMT
Author: mbautin
Date: Tue Feb  7 20:09:41 2012
New Revision: 1241589

URL: http://svn.apache.org/viewvc?rev=1241589&view=rev
Log:
[jira] [HBASE-5292] [89-fb] Prevent counting getSize on compactions

Summary:

Added two separate metrics for both get() and next(). This is done by
refactoring on internal next() API. To be more specific, only Get.get()
and ResultScanner.next() passes the metric name ("getsize" and
"nextsize" repectively) to
  HRegion::RegionScanner::next(List<KeyValue>, String)

This will eventually hit StoreScanner()::next((List<KeyValue>,
int, String) where the metrics are counted.

And their call paths are:

1) Get

  HTable::get(final Get get)
  => HRegionServer::get(byte [] regionName, Get get)
  => HRegion::get(final Get get, final Integer lockid)
  => HRegion::get(final Get get)      [pass METRIC_GETSIZE to the
  callee]

  => HRegion::RegionScanner::next(List<KeyValue> outResults, String
  metric)
  => HRegion::RegionScanner::next(List<KeyValue> outResults, int limit,
  String metric)
  => HRegion::RegionScanner::nextInternal(int limit, String metric)
  => KeyValueHeap::next(List<KeyValue> result, int limit, String metric)
  => StoreScanner::next(List<KeyValue> outResult, int limit, String
  metric)

2) Next

  HTable::ClientScanner::next()
  => ScannerCallable::call()
  => HRegionServer::next(long scannerId)
  => HRegionServer::next(final long scannerId, int nbRows)  [pass
  METRIC_NEXTSIZE to the callee]

  => HRegion::RegionScanner::next(List<KeyValue> outResults, String
  metric)
  => HRegion::RegionScanner::next(List<KeyValue> outResults, int limit,
  String metric)
  => HRegion::RegionScanner::nextInternal(int limit, String metric)
  => KeyValueHeap::next(List<KeyValue> result, int limit, String metric)
  => StoreScanner::next(List<KeyValue> outResult, int limit, String
  metric)

Task ID: #898948

Blame Rev:

Reviewers: kanan, liyintang, mbautin

Reviewed By: kannan

CC: kanan, liyintang, mbautin, zhiqiu

Test Plan:

  1. Passed unit tests.
  2. Created a testcase TestRegionServerMetrics::testGetNextSize to
  guarantee:
    * Get/Next contributes to getsize/nextsize metrics
    * Both getsize/nextsize are per Column Family
    * Flush/compaction won't affect these two metrics

Revert Plan:

Tags:

Modified:
    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/InternalScanner.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
    hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java

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=1241589&r1=1241588&r2=1241589&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
Tue Feb  7 20:09:41 2012
@@ -31,6 +31,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableSet;
@@ -291,6 +292,9 @@ public class HRegion implements HeapSize
   public static final ConcurrentMap<String, AtomicLong>
     numericMetrics = new ConcurrentHashMap<String, AtomicLong>();
 
+  public static final String METRIC_GETSIZE = "getsize";
+  public static final String METRIC_NEXTSIZE = "nextsize";
+
   // for simple numeric metrics (current block cache size)
   // These ones are not reset to zero when queried, unlike the previous.
   public static final ConcurrentMap<String, AtomicLong>
@@ -2930,6 +2934,12 @@ public class HRegion implements HeapSize
     @Override
     public synchronized boolean next(List<KeyValue> outResults, int limit)
         throws IOException {
+      return next(outResults, limit, null);
+    }
+
+    @Override
+    public synchronized boolean next(List<KeyValue> outResults, int limit,
+        String metric) throws IOException {
       if (this.filterClosed) {
         throw new UnknownScannerException("Scanner was closed (timed out?) " +
             "after we renewed it. Could be caused by a very slow scanner " +
@@ -2945,7 +2955,7 @@ public class HRegion implements HeapSize
       ReadWriteConsistencyControl.setThreadReadPoint(this.readPt);
 
       results.clear();
-      boolean returnResult = nextInternal(limit);
+      boolean returnResult = nextInternal(limit, metric);
 
       outResults.addAll(results);
       resetFilters();
@@ -2959,7 +2969,14 @@ public class HRegion implements HeapSize
     public synchronized boolean next(List<KeyValue> outResults)
         throws IOException {
       // apply the batching limit by default
-      return next(outResults, batch);
+      return next(outResults, batch, null);
+    }
+
+    @Override
+    public synchronized boolean next(List<KeyValue> outResults, String metric)
+        throws IOException {
+      // apply the batching limit by default
+      return next(outResults, batch, metric);
     }
 
     /*
@@ -2969,7 +2986,7 @@ public class HRegion implements HeapSize
       return this.filter != null && this.filter.filterAllRemaining();
     }
 
-    private boolean nextInternal(int limit) throws IOException {
+    private boolean nextInternal(int limit, String metric) throws IOException {
       while (true) {
         byte [] currentRow = peekRow();
         if (isStopRow(currentRow)) {
@@ -2986,7 +3003,7 @@ public class HRegion implements HeapSize
         } else {
           byte [] nextRow;
           do {
-            this.storeHeap.next(results, limit - results.size());
+            this.storeHeap.next(results, limit - results.size(), metric);
             if (limit > 0 && results.size() == limit) {
               if (this.filter != null && filter.hasFilterRow()) throw new IncompatibleFilterException(
                   "Filter with filterRow(List<KeyValue>) incompatible with scan with
limit!");
@@ -3586,7 +3603,7 @@ public class HRegion implements HeapSize
     InternalScanner scanner = null;
     try {
       scanner = getScanner(scan);
-      scanner.next(results);
+      scanner.next(results, HRegion.METRIC_GETSIZE);
     } finally {
       if (scanner != null)
         scanner.close();

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=1241589&r1=1241588&r2=1241589&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
Tue Feb  7 20:09:41 2012
@@ -2174,7 +2174,7 @@ public class HRegionServer implements HR
       for (int i = 0; i < nbRows && currentScanResultSize < maxScannerResultSize;
i++) {
         requestCount.incrementAndGet();
         // Collect values to be returned here
-        boolean moreRows = s.next(values);
+        boolean moreRows = s.next(values, HRegion.METRIC_NEXTSIZE);
         if (!values.isEmpty()) {
           for (KeyValue kv : values) {
             currentScanResultSize += kv.heapSize();

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/InternalScanner.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/InternalScanner.java?rev=1241589&r1=1241588&r2=1241589&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/InternalScanner.java
(original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/InternalScanner.java
Tue Feb  7 20:09:41 2012
@@ -49,6 +49,15 @@ public interface InternalScanner extends
   public boolean next(List<KeyValue> results) throws IOException;
 
   /**
+   * Grab the next row's worth of values.
+   * @param results return output array
+   * @param metric the metric name
+   * @return true if more rows exist after this one, false if scanner is done
+   * @throws IOException e
+   */
+  public boolean next(List<KeyValue> results, String metric) throws IOException;
+
+  /**
    * Grab the next row's worth of values with a limit on the number of values
    * to return.
    * @param result return output array
@@ -59,6 +68,17 @@ public interface InternalScanner extends
   public boolean next(List<KeyValue> result, int limit) throws IOException;
 
   /**
+   * Grab the next row's worth of values with a limit on the number of values
+   * to return.
+   * @param result return output array
+   * @param limit limit on row count to get
+   * @param metric the metric name
+   * @return true if more rows exist after this one, false if scanner is done
+   * @throws IOException e
+   */
+  public boolean next(List<KeyValue> result, int limit, String metric) throws IOException;
+
+  /**
    * Closes the scanner and releases any resources it has allocated
    * @throws IOException
    */

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java?rev=1241589&r1=1241588&r2=1241589&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java
(original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java
Tue Feb  7 20:09:41 2012
@@ -115,16 +115,32 @@ public class KeyValueHeap extends NonLaz
    * <p>
    * This can ONLY be called when you are using Scanners that implement
    * InternalScanner as well as KeyValueScanner (a {@link StoreScanner}).
-   * @param result
-   * @param limit
+   * @param result output result list
+   * @param limit limit on row count to get
    * @return true if there are more keys, false if all scanners are done
    */
   public boolean next(List<KeyValue> result, int limit) throws IOException {
+    return next(result, limit, null);
+  }
+
+  /**
+   * Gets the next row of keys from the top-most scanner.
+   * <p>
+   * This method takes care of updating the heap.
+   * <p>
+   * This can ONLY be called when you are using Scanners that implement
+   * InternalScanner as well as KeyValueScanner (a {@link StoreScanner}).
+   * @param result output result list
+   * @param limit limit on row count to get
+   * @param metric the metric name
+   * @return true if there are more keys, false if all scanners are done
+   */
+  public boolean next(List<KeyValue> result, int limit, String metric) throws IOException
{
     if (this.current == null) {
       return false;
     }
     InternalScanner currentAsInternal = (InternalScanner)this.current;
-    boolean mayContainsMoreRows = currentAsInternal.next(result, limit);
+    boolean mayContainsMoreRows = currentAsInternal.next(result, limit, metric);
     KeyValue pee = this.current.peek();
     /*
      * By definition, any InternalScanner must return false only when it has no
@@ -156,6 +172,11 @@ public class KeyValueHeap extends NonLaz
     return next(result, -1);
   }
 
+  @Override
+  public boolean next(List<KeyValue> result, String metric) throws IOException {
+    return next(result, -1, metric);
+  }
+
   private static class KVScannerComparator implements Comparator<KeyValueScanner> {
     private KVComparator kvComparator;
     /**

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java?rev=1241589&r1=1241588&r2=1241589&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
(original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
Tue Feb  7 20:09:41 2012
@@ -46,7 +46,7 @@ class StoreScanner extends NonLazyKeyVal
   private boolean cacheBlocks;
   private int countPerRow = 0;
   private int storeLimit;
-  private String metricNameGetsize;
+  private String metricNamePrefix;
   // Used to indicate that the scanner has closed (see HBASE-1107)
   // Doesnt need to be volatile because it's always accessed via synchronized methods
   private boolean closing = false;
@@ -196,9 +196,8 @@ class StoreScanner extends NonLazyKeyVal
       family = Bytes.toString(store.getFamily().getName());
     }
 
-    String mutationSignature = SchemaMetrics.generateSchemaMetricsPrefix(
-        tableName, family);
-    this.metricNameGetsize = mutationSignature + "getsize";
+    this.metricNamePrefix =
+        SchemaMetrics.generateSchemaMetricsPrefix(tableName, family);
   }
 
   /**
@@ -276,6 +275,18 @@ class StoreScanner extends NonLazyKeyVal
    */
   @Override
   public synchronized boolean next(List<KeyValue> outResult, int limit) throws IOException
{
+    return next(outResult, limit, null);
+  }
+
+  /**
+   * Get the next row of values from this Store.
+   * @param outResult
+   * @param limit
+   * @return true if there are more rows, false if scanner is done
+   */
+  @Override
+  public synchronized boolean next(List<KeyValue> outResult, int limit,
+      String metric) throws IOException {
 
     checkReseek();
 
@@ -331,7 +342,11 @@ class StoreScanner extends NonLazyKeyVal
             break LOOP;
           }
 
-          HRegion.incrNumericMetric(this.metricNameGetsize, copyKv.getLength());
+          if (metric != null) {
+            HRegion.incrNumericMetric(this.metricNamePrefix + metric,
+                copyKv.getLength());
+          }
+
           results.add(copyKv);
 
           if (qcode == ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_ROW) {
@@ -410,7 +425,13 @@ class StoreScanner extends NonLazyKeyVal
 
   @Override
   public synchronized boolean next(List<KeyValue> outResult) throws IOException {
-    return next(outResult, -1);
+    return next(outResult, -1, null);
+  }
+
+  @Override
+  public synchronized boolean next(List<KeyValue> outResult, String metric)
+      throws IOException {
+    return next(outResult, -1, metric);
   }
 
   // Implementation of ChangedReadersObserver

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=1241589&r1=1241588&r2=1241589&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
Tue Feb  7 20:09:41 2012
@@ -23,10 +23,19 @@ import static org.junit.Assert.assertEqu
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
 import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics.
     StoreMetricType;
@@ -54,7 +63,7 @@ public class TestRegionServerMetrics {
   private static final SchemaMetrics ALL_METRICS =
       SchemaMetrics.ALL_SCHEMA_METRICS;
 
-  private static final HBaseTestingUtility TEST_UTIL =
+  private final HBaseTestingUtility TEST_UTIL =
       new HBaseTestingUtility();
 
   private Map<String, Long> startingMetrics;
@@ -123,4 +132,79 @@ public class TestRegionServerMetrics {
         NUM_FLUSHES, HRegion.getNumericMetric(storeMetricName));
   }
 
+
+  private void assertSizeMetric(String table, String[] cfs, int[] metrics) {
+    // we have getsize & nextsize for each column family
+    assertEquals(cfs.length * 2, metrics.length);
+
+    for (int i =0; i < cfs.length; ++i) {
+      String prefix = SchemaMetrics.generateSchemaMetricsPrefix(table, cfs[i]);
+      String getMetric = prefix + HRegion.METRIC_GETSIZE;
+      String nextMetric = prefix + HRegion.METRIC_NEXTSIZE;
+
+      // verify getsize and nextsize matches
+      int getSize = HRegion.numericMetrics.containsKey(getMetric) ?
+          HRegion.numericMetrics.get(getMetric).intValue() : 0;
+      int nextSize = HRegion.numericMetrics.containsKey(nextMetric) ?
+          HRegion.numericMetrics.get(nextMetric).intValue() : 0;
+
+      assertEquals(metrics[i], getSize);
+      assertEquals(metrics[cfs.length + i], nextSize);
+    }
+  }
+
+  @Test
+  public void testGetNextSize() throws IOException, InterruptedException {
+    String rowName = "row1";
+    byte[] ROW = Bytes.toBytes(rowName);
+    String tableName = "SizeMetricTest";
+    byte[] TABLE = Bytes.toBytes(tableName);
+    String cf1Name = "cf1";
+    String cf2Name = "cf2";
+    String[] cfs = new String[] {cf1Name, cf2Name};
+    byte[] CF1 = Bytes.toBytes(cf1Name);
+    byte[] CF2 = Bytes.toBytes(cf2Name);
+
+    long ts = 1234;
+    HTable hTable = TEST_UTIL.createTable(TABLE, new byte[][]{CF1, CF2});
+    HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
+
+    Put p = new Put(ROW);
+    p.add(CF1, CF1, ts, CF1);
+    p.add(CF2, CF2, ts, CF2);
+    hTable.put(p);
+
+    KeyValue kv1 = new KeyValue(ROW, CF1, CF1, ts, CF1);
+    KeyValue kv2 = new KeyValue(ROW, CF2, CF2, ts, CF2);
+    int kvLength = kv1.getLength();
+    assertEquals(kvLength, kv2.getLength());
+
+    // only cf1.getsize is set on Get
+    hTable.get(new Get(ROW).addFamily(CF1));
+    assertSizeMetric(tableName, cfs, new int[] {kvLength, 0, 0, 0});
+
+    // only cf2.getsize is set on Get
+    hTable.get(new Get(ROW).addFamily(CF2));
+    assertSizeMetric(tableName, cfs, new int[] {kvLength, kvLength, 0, 0});
+
+    // only cf2.nextsize is set
+    for (Result res : hTable.getScanner(CF2)) {
+    }
+    assertSizeMetric(tableName, cfs,
+        new int[] {kvLength, kvLength, 0, kvLength});
+
+    // only cf2.nextsize is set
+    for (Result res : hTable.getScanner(CF1)) {
+    }
+    assertSizeMetric(tableName, cfs,
+        new int[] {kvLength, kvLength, kvLength, kvLength});
+
+    // getsize/nextsize should not be set on flush or compaction
+    for (HRegion hr : TEST_UTIL.getMiniHBaseCluster().getRegions(TABLE)) {
+      hr.flushcache();
+      hr.compactStores();
+    }
+    assertSizeMetric(tableName, cfs,
+        new int[] {kvLength, kvLength, kvLength, kvLength});
+  }
 }



Mime
View raw message