hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ecl...@apache.org
Subject svn commit: r1526314 - in /hbase/trunk: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/ hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/ hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/regionse...
Date Wed, 25 Sep 2013 21:44:34 GMT
Author: eclark
Date: Wed Sep 25 21:44:33 2013
New Revision: 1526314

URL: http://svn.apache.org/r1526314
Log:
HBASE-9653 Add compaction metrics to trunk

Modified:
    hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
    hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
    hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java
    hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java
    hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
    hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
    hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
    hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
    hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java

Modified: hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java?rev=1526314&r1=1526313&r2=1526314&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
(original)
+++ hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
Wed Sep 25 21:44:33 2013
@@ -155,6 +155,8 @@ public interface MetricsRegionServerSour
   String PERCENT_FILES_LOCAL_DESC =
       "The percent of HFiles that are stored on the local hdfs data node.";
   String COMPACTION_QUEUE_LENGTH = "compactionQueueLength";
+  String LARGE_COMPACTION_QUEUE_LENGTH = "largeCompactionQueueLength";
+  String SMALL_COMPACTION_QUEUE_LENGTH = "smallCompactionQueueLength";
   String COMPACTION_QUEUE_LENGTH_DESC = "Length of the queue for compactions.";
   String FLUSH_QUEUE_LENGTH = "flushQueueLength";
   String FLUSH_QUEUE_LENGTH_DESC = "Length of the queue for region flushes";

Modified: hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java?rev=1526314&r1=1526313&r2=1526314&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
(original)
+++ hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
Wed Sep 25 21:44:33 2013
@@ -148,6 +148,10 @@ public interface MetricsRegionServerWrap
    */
   int getCompactionQueueSize();
 
+  int getSmallCompactionQueueSize();
+
+  int getLargeCompactionQueueSize();
+
   /**
    * Get the size of the flush queue.
    */

Modified: hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java?rev=1526314&r1=1526313&r2=1526314&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java
(original)
+++ hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java
Wed Sep 25 21:44:33 2013
@@ -27,6 +27,14 @@ public interface MetricsRegionSource ext
 
   String OPS_SAMPLE_NAME = "ops";
   String SIZE_VALUE_NAME = "size";
+  String COMPACTIONS_COMPLETED_COUNT = "compactionsCompletedCount";
+  String NUM_BYTES_COMPACTED_COUNT = "numBytesCompactedCount";
+  String NUM_FILES_COMPACTED_COUNT = "numFilesCompactedCount";
+  String COMPACTIONS_COMPLETED_DESC = "Number of compactions that have completed.";
+  String  NUM_BYTES_COMPACTED_DESC =
+      "Sum of filesize on all files entering a finished, successful or aborted, compaction";
+  String NUM_FILES_COMPACTED_DESC =
+      "Number of files that were input for finished, successful or aborted, compactions";
 
   /**
    * Close the region's metrics as this region is closing.

Modified: hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java?rev=1526314&r1=1526313&r2=1526314&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java
(original)
+++ hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java
Wed Sep 25 21:44:33 2013
@@ -74,4 +74,9 @@ public interface MetricsRegionWrapper {
    */
   long getWriteRequestCount();
 
+  long getNumFilesCompacted();
+
+  long getNumBytesCompacted();
+
+  long getNumCompactionsCompleted();
 }

Modified: hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java?rev=1526314&r1=1526313&r2=1526314&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
(original)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
Wed Sep 25 21:44:33 2013
@@ -174,6 +174,12 @@ public class MetricsRegionServerSourceIm
           .addGauge(COMPACTION_QUEUE_LENGTH,
               COMPACTION_QUEUE_LENGTH_DESC,
               rsWrap.getCompactionQueueSize())
+          .addGauge(LARGE_COMPACTION_QUEUE_LENGTH,
+              COMPACTION_QUEUE_LENGTH_DESC,
+              rsWrap.getLargeCompactionQueueSize())
+          .addGauge(SMALL_COMPACTION_QUEUE_LENGTH,
+              COMPACTION_QUEUE_LENGTH_DESC,
+              rsWrap.getSmallCompactionQueueSize())
           .addGauge(FLUSH_QUEUE_LENGTH, FLUSH_QUEUE_LENGTH_DESC, rsWrap.getFlushQueueSize())
           .addGauge(BLOCK_CACHE_FREE_SIZE, BLOCK_CACHE_FREE_DESC, rsWrap.getBlockCacheFreeSize())
           .addGauge(BLOCK_CACHE_COUNT, BLOCK_CACHE_COUNT_DESC, rsWrap.getBlockCacheCount())

Modified: hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java?rev=1526314&r1=1526313&r2=1526314&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
(original)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
Wed Sep 25 21:44:33 2013
@@ -179,6 +179,16 @@ public class MetricsRegionSourceImpl imp
     mrb.addCounter(regionNamePrefix + MetricsRegionServerSource.WRITE_REQUEST_COUNT,
         MetricsRegionServerSource.WRITE_REQUEST_COUNT_DESC,
         this.regionWrapper.getWriteRequestCount());
+    mrb.addCounter(regionNamePrefix + MetricsRegionSource.COMPACTIONS_COMPLETED_COUNT,
+        MetricsRegionSource.COMPACTIONS_COMPLETED_DESC,
+        this.regionWrapper.getNumCompactionsCompleted());
+    mrb.addCounter(regionNamePrefix + MetricsRegionSource.NUM_BYTES_COMPACTED_COUNT,
+        MetricsRegionSource.NUM_BYTES_COMPACTED_DESC,
+        this.regionWrapper.getNumBytesCompacted());
+    mrb.addCounter(regionNamePrefix + MetricsRegionSource.NUM_FILES_COMPACTED_COUNT,
+        MetricsRegionSource.NUM_FILES_COMPACTED_DESC,
+        this.regionWrapper.getNumFilesCompacted());
+
 
   }
 }

Modified: hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java?rev=1526314&r1=1526313&r2=1526314&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
(original)
+++ hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
Wed Sep 25 21:44:33 2013
@@ -102,5 +102,20 @@ public class TestMetricsRegionSourceImpl
     public long getWriteRequestCount() {
       return 0;
     }
+
+    @Override
+    public long getNumFilesCompacted() {
+      return 0;
+    }
+
+    @Override
+    public long getNumBytesCompacted() {
+      return 0;
+    }
+
+    @Override
+    public long getNumCompactionsCompleted() {
+      return 0;
+    }
   }
 }

Modified: hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java?rev=1526314&r1=1526313&r2=1526314&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
(original)
+++ hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
Wed Sep 25 21:44:33 2013
@@ -180,5 +180,15 @@ public class MetricsRegionSourceImpl imp
     mrb.addGauge(Interns.info(regionNamePrefix + MetricsRegionServerSource.STOREFILE_SIZE,
         MetricsRegionServerSource.STOREFILE_SIZE_DESC),
         this.regionWrapper.getStoreFileSize());
+    mrb.addCounter(Interns.info(regionNamePrefix + MetricsRegionSource.COMPACTIONS_COMPLETED_COUNT,
+        MetricsRegionSource.COMPACTIONS_COMPLETED_DESC),
+        this.regionWrapper.getNumCompactionsCompleted());
+    mrb.addCounter(Interns.info(regionNamePrefix + MetricsRegionSource.NUM_BYTES_COMPACTED_COUNT,
+        MetricsRegionSource.NUM_BYTES_COMPACTED_DESC),
+        this.regionWrapper.getNumBytesCompacted());
+    mrb.addCounter(Interns.info(regionNamePrefix + MetricsRegionSource.NUM_FILES_COMPACTED_COUNT,
+        MetricsRegionSource.NUM_FILES_COMPACTED_DESC),
+        this.regionWrapper.getNumFilesCompacted());
+
   }
 }

Modified: hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java?rev=1526314&r1=1526313&r2=1526314&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
(original)
+++ hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
Wed Sep 25 21:44:33 2013
@@ -102,5 +102,20 @@ public class TestMetricsRegionSourceImpl
     public long getWriteRequestCount() {
       return 0;
     }
+
+    @Override
+    public long getNumFilesCompacted() {
+      return 0;
+    }
+
+    @Override
+    public long getNumBytesCompacted() {
+      return 0;
+    }
+
+    @Override
+    public long getNumCompactionsCompleted() {
+      return 0;
+    }
   }
 }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java?rev=1526314&r1=1526313&r2=1526314&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
(original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
Wed Sep 25 21:44:33 2013
@@ -381,6 +381,16 @@ public class CompactSplitThread implemen
     return largeCompactions.getQueue().size() + smallCompactions.getQueue().size();
   }
 
+  public int getLargeCompactionQueueSize() {
+    return largeCompactions.getQueue().size();
+  }
+
+
+  public int getSmallCompactionQueueSize() {
+    return smallCompactions.getQueue().size();
+  }
+
+
   private boolean shouldSplitRegion() {
     return (regionSplitLimit > server.getNumberOfOnlineRegions());
   }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1526314&r1=1526313&r2=1526314&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
(original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
Wed Sep 25 21:44:33 2013
@@ -250,8 +250,11 @@ public class HRegion implements HeapSize
   final Counter readRequestsCount = new Counter();
   final Counter writeRequestsCount = new Counter();
 
-  //How long operations were blocked by a memstore over highwater.
-  final Counter updatesBlockedMs = new Counter();
+  // Compaction counters
+  final AtomicLong compactionsFinished = new AtomicLong(0L);
+  final AtomicLong compactionNumFilesCompacted = new AtomicLong(0L);
+  final AtomicLong compactionNumBytesCompacted = new AtomicLong(0L);
+
 
   private final HLog log;
   private final HRegionFileSystem fs;
@@ -4927,7 +4930,7 @@ public class HRegion implements HeapSize
   public static final long FIXED_OVERHEAD = ClassSize.align(
       ClassSize.OBJECT +
       ClassSize.ARRAY +
-      38 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT +
+      40 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT +
       (11 * Bytes.SIZEOF_LONG) +
       4 * Bytes.SIZEOF_BOOLEAN);
 
@@ -5496,8 +5499,14 @@ public class HRegion implements HeapSize
     (isMajor ? majorInProgress : minorInProgress).incrementAndGet();
   }
 
-  public void reportCompactionRequestEnd(boolean isMajor){
+  public void reportCompactionRequestEnd(boolean isMajor, int numFiles, long filesSizeCompacted){
     int newValue = (isMajor ? majorInProgress : minorInProgress).decrementAndGet();
+
+    // metrics
+    compactionsFinished.incrementAndGet();
+    compactionNumFilesCompacted.addAndGet(numFiles);
+    compactionNumBytesCompacted.addAndGet(filesSizeCompacted);
+
     assert newValue >= 0;
   }
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java?rev=1526314&r1=1526313&r2=1526314&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
(original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
Wed Sep 25 21:44:33 2013
@@ -985,6 +985,7 @@ public class HStore implements Store {
     try {
       // Commence the compaction.
       List<Path> newFiles = compaction.compact();
+
       // TODO: get rid of this!
       if (!this.conf.getBoolean("hbase.hstore.compaction.complete", true)) {
         LOG.warn("hbase.hstore.compaction.complete is set to false");
@@ -1333,7 +1334,7 @@ public class HStore implements Store {
   }
 
   private void finishCompactionRequest(CompactionRequest cr) {
-    this.region.reportCompactionRequestEnd(cr.isMajor());
+    this.region.reportCompactionRequestEnd(cr.isMajor(), cr.getFiles().size(), cr.getSize());
     if (cr.isOffPeak()) {
       offPeakCompactionTracker.set(false);
       cr.setOffPeak(false);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java?rev=1526314&r1=1526313&r2=1526314&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
(original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
Wed Sep 25 21:44:33 2013
@@ -156,6 +156,24 @@ class MetricsRegionServerWrapperImpl
   }
 
   @Override
+  public int getSmallCompactionQueueSize() {
+    //The thread could be zero.  if so assume there is no queue.
+    if (this.regionServer.compactSplitThread == null) {
+      return 0;
+    }
+    return this.regionServer.compactSplitThread.getSmallCompactionQueueSize();
+  }
+
+  @Override
+  public int getLargeCompactionQueueSize() {
+    //The thread could be zero.  if so assume there is no queue.
+    if (this.regionServer.compactSplitThread == null) {
+      return 0;
+    }
+    return this.regionServer.compactSplitThread.getLargeCompactionQueueSize();
+  }
+
+  @Override
   public int getFlushQueueSize() {
     //If there is no flusher there should be no queue.
     if (this.regionServer.cacheFlusher == null) {

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java?rev=1526314&r1=1526313&r2=1526314&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
(original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
Wed Sep 25 21:44:33 2013
@@ -114,6 +114,21 @@ public class MetricsRegionWrapperImpl im
     return this.region.getWriteRequestsCount();
   }
 
+  @Override
+  public long getNumFilesCompacted() {
+    return this.region.compactionNumFilesCompacted.get();
+  }
+
+  @Override
+  public long getNumBytesCompacted() {
+    return this.region.compactionNumBytesCompacted.get();
+  }
+
+  @Override
+  public long getNumCompactionsCompleted() {
+    return this.region.compactionsFinished.get();
+  }
+
   public class HRegionMetricsWrapperRunnable implements Runnable {
 
     @Override

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java?rev=1526314&r1=1526313&r2=1526314&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
(original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
Wed Sep 25 21:44:33 2013
@@ -136,6 +136,16 @@ public class MetricsRegionServerWrapperS
   }
 
   @Override
+  public int getSmallCompactionQueueSize() {
+    return 0;
+  }
+
+  @Override
+  public int getLargeCompactionQueueSize() {
+    return 0;
+  }
+
+  @Override
   public int getFlushQueueSize() {
     return 412;
   }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java?rev=1526314&r1=1526313&r2=1526314&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java
(original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java
Wed Sep 25 21:44:33 2013
@@ -64,4 +64,19 @@ public class MetricsRegionWrapperStub im
   public long getWriteRequestCount() {
     return 106;
   }
+
+  @Override
+  public long getNumFilesCompacted() {
+    return 0;
+  }
+
+  @Override
+  public long getNumBytesCompacted() {
+    return 0;
+  }
+
+  @Override
+  public long getNumCompactionsCompleted() {
+    return 0;
+  }
 }



Mime
View raw message