You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2014/08/01 18:39:48 UTC

[3/9] git commit: HBASE-11516 Track time spent in executing coprocessors in each region (Srikanth Srungarapu and Andrew Purtell)

HBASE-11516 Track time spent in executing coprocessors in each region (Srikanth Srungarapu and Andrew Purtell)


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/19e9b8aa
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/19e9b8aa
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/19e9b8aa

Branch: refs/heads/master
Commit: 19e9b8aa5206a7954d0c02455bf39cfe45fdf6f1
Parents: 33e1418
Author: Andrew Purtell <ap...@apache.org>
Authored: Fri Aug 1 09:37:56 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Aug 1 09:37:56 2014 -0700

----------------------------------------------------------------------
 hbase-hadoop-compat/pom.xml                     |   4 +
 .../hbase/regionserver/MetricsRegionSource.java |   2 +
 .../regionserver/MetricsRegionWrapper.java      |   9 +
 .../regionserver/MetricsRegionSourceImpl.java   |  29 +++
 .../TestMetricsRegionSourceImpl.java            |  14 +-
 .../tmpl/regionserver/RegionListTmpl.jamon      |  69 +++++++
 .../hadoop/hbase/regionserver/HRegion.java      |   2 +-
 .../hbase/regionserver/MetricsRegion.java       |   8 +-
 .../regionserver/MetricsRegionWrapperImpl.java  |  21 ++-
 .../regionserver/RegionCoprocessorHost.java     | 187 ++++++++++++++++++-
 .../regionserver/MetricsRegionWrapperStub.java  |  10 +
 11 files changed, 344 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/19e9b8aa/hbase-hadoop-compat/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/pom.xml b/hbase-hadoop-compat/pom.xml
index 1e211bb..ce59c68 100644
--- a/hbase-hadoop-compat/pom.xml
+++ b/hbase-hadoop-compat/pom.xml
@@ -82,6 +82,10 @@
         <groupId>commons-logging</groupId>
         <artifactId>commons-logging</artifactId>
       </dependency>
+      <dependency>
+        <groupId>org.apache.commons</groupId>
+        <artifactId>commons-math</artifactId>
+      </dependency>
     </dependencies>
 
     <profiles>

http://git-wip-us.apache.org/repos/asf/hbase/blob/19e9b8aa/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java
index 60d6e8b..8abbb1f 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java
@@ -35,6 +35,8 @@ public interface MetricsRegionSource extends Comparable<MetricsRegionSource> {
       "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";
+  String COPROCESSOR_EXECUTION_STATISTICS = "coprocessorExecutionStatistics";
+  String COPROCESSOR_EXECUTION_STATISTICS_DESC = "Statistics for coprocessor execution times";
 
   /**
    * Close the region's metrics as this region is closing.

http://git-wip-us.apache.org/repos/asf/hbase/blob/19e9b8aa/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java
index 27be767..7d61f81 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java
@@ -18,6 +18,10 @@
 
 package org.apache.hadoop.hbase.regionserver;
 
+import java.util.Map;
+
+import org.apache.commons.math.stat.descriptive.DescriptiveStatistics;
+
 /**
  * Interface of class that will wrap an HRegion and export numbers so they can be
  * used in MetricsRegionSource
@@ -79,4 +83,9 @@ public interface MetricsRegionWrapper {
   long getNumBytesCompacted();
 
   long getNumCompactionsCompleted();
+
+  /**
+   * Get the time spent by coprocessors in this region.
+   */
+  Map<String, DescriptiveStatistics> getCoprocessorExecutionStatistics();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/19e9b8aa/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
index 4ca792e..1a0c4c1 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
@@ -18,8 +18,11 @@
 
 package org.apache.hadoop.hbase.regionserver;
 
+import java.util.Map;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.commons.math.stat.descriptive.DescriptiveStatistics;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.impl.JmxCacheBuster;
@@ -194,6 +197,32 @@ public class MetricsRegionSourceImpl implements MetricsRegionSource {
     mrb.addCounter(Interns.info(regionNamePrefix + MetricsRegionSource.NUM_FILES_COMPACTED_COUNT,
         MetricsRegionSource.NUM_FILES_COMPACTED_DESC),
         this.regionWrapper.getNumFilesCompacted());
+    for (Map.Entry<String, DescriptiveStatistics> entry : this.regionWrapper
+        .getCoprocessorExecutionStatistics()
+        .entrySet()) {
+      DescriptiveStatistics ds = entry.getValue();
+      mrb.addGauge(Interns.info(regionNamePrefix + " " + entry.getKey() + " "
+          + MetricsRegionSource.COPROCESSOR_EXECUTION_STATISTICS,
+        MetricsRegionSource.COPROCESSOR_EXECUTION_STATISTICS_DESC + "Min: "), ds.getMin() / 1000);
+      mrb.addGauge(Interns.info(regionNamePrefix + " " + entry.getKey() + " "
+          + MetricsRegionSource.COPROCESSOR_EXECUTION_STATISTICS,
+        MetricsRegionSource.COPROCESSOR_EXECUTION_STATISTICS_DESC + "Mean: "), ds.getMean() / 1000);
+      mrb.addGauge(Interns.info(regionNamePrefix + " " + entry.getKey() + " "
+          + MetricsRegionSource.COPROCESSOR_EXECUTION_STATISTICS,
+        MetricsRegionSource.COPROCESSOR_EXECUTION_STATISTICS_DESC + "Max: "), ds.getMax() / 1000);
+      mrb.addGauge(Interns.info(regionNamePrefix + " " + entry.getKey() + " "
+          + MetricsRegionSource.COPROCESSOR_EXECUTION_STATISTICS,
+        MetricsRegionSource.COPROCESSOR_EXECUTION_STATISTICS_DESC + "90th percentile: "), ds
+          .getPercentile(90d) / 1000);
+      mrb.addGauge(Interns.info(regionNamePrefix + " " + entry.getKey() + " "
+          + MetricsRegionSource.COPROCESSOR_EXECUTION_STATISTICS,
+        MetricsRegionSource.COPROCESSOR_EXECUTION_STATISTICS_DESC + "95th percentile: "), ds
+          .getPercentile(95d) / 1000);
+      mrb.addGauge(Interns.info(regionNamePrefix + " " + entry.getKey() + " "
+          + MetricsRegionSource.COPROCESSOR_EXECUTION_STATISTICS,
+        MetricsRegionSource.COPROCESSOR_EXECUTION_STATISTICS_DESC + "99th percentile: "), ds
+          .getPercentile(99d) / 1000);
+    }
 
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/19e9b8aa/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
index eb5c814..4be8905 100644
--- a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
+++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
@@ -18,13 +18,16 @@
 
 package org.apache.hadoop.hbase.regionserver;
 
-import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
-import org.junit.Test;
-
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertTrue;
 
+import java.util.Map;
+
+import org.apache.commons.math.stat.descriptive.DescriptiveStatistics;
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.junit.Test;
+
 public class TestMetricsRegionSourceImpl {
 
   @Test
@@ -120,5 +123,10 @@ public class TestMetricsRegionSourceImpl {
     public long getNumCompactionsCompleted() {
       return 0;
     }
+
+    @Override
+    public Map<String, DescriptiveStatistics> getCoprocessorExecutionStatistics() {
+      return null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/19e9b8aa/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon
index 7c4f1f1..3ff4cb6 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon
@@ -22,15 +22,18 @@
 </%args>
 <%import>
         java.util.*;
+        org.apache.commons.math.stat.descriptive.DescriptiveStatistics;
         org.apache.hadoop.hbase.regionserver.HRegionServer;
         org.apache.hadoop.hbase.util.Bytes;
         org.apache.hadoop.hbase.HRegionInfo;
+        org.apache.hadoop.hbase.regionserver.HRegion;
         org.apache.hadoop.hbase.ServerName;
         org.apache.hadoop.hbase.HBaseConfiguration;
         org.apache.hadoop.hbase.protobuf.ProtobufUtil;
         org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
         org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad;
         org.apache.hadoop.hbase.client.RegionReplicaUtil;
+        org.apache.hadoop.hbase.regionserver.MetricsRegionWrapper;
 </%import>
 <%if (onlineRegions != null && onlineRegions.size() > 0) %>
 
@@ -45,6 +48,7 @@
             <li class=""><a href="#tab_regionStoreStats" data-toggle="tab">Storefile Metrics</a></li>
             <li class=""><a href="#tab_regionMemstoreStats" data-toggle="tab">Memstore Metrics</a></li>
             <li class=""><a href="#tab_regionCompactStats" data-toggle="tab">Compaction Metrics</a></li>
+            <li class=""><a href="#tab_coprocessorStats" data-toggle="tab">Coprocessor Metrics</a></li>
         </ul>
         <div class="tab-content" style="padding-bottom: 9px; border-bottom: 1px solid #ddd;">
             <div class="tab-pane active" id="tab_regionBaseInfo">
@@ -62,6 +66,9 @@
             <div class="tab-pane" id="tab_regionCompactStats">
                 <& compactStats; onlineRegions = onlineRegions; &>
             </div>
+            <div class="tab-pane" id="tab_coprocessorStats">
+                <& coprocessorStats; onlineRegions = onlineRegions; &>
+            </div>
         </div>
     </div>
     <p>Region names are made of the containing table's name, a comma,
@@ -231,3 +238,65 @@
         </%for>
     </table>
 </%def>
+
+<%def coprocessorStats>
+<%args>
+    List<HRegionInfo> onlineRegions;
+</%args>
+    <table class="table table-striped">
+        <tr>
+            <th>Region Name</th>
+            <th>Coprocessor</th>
+            <th>Execution Time Statistics</th>
+        </tr>
+
+        <%for HRegionInfo r: onlineRegions %>
+        <%java>    
+            HRegion region = regionServer.getFromOnlineRegions(r.getEncodedName());
+            MetricsRegionWrapper mWrap = region == null ? null: region.getMetrics().getRegionWrapper();
+        </%java>
+        
+        <%if mWrap != null %>
+
+            <%for Map.Entry<String, DescriptiveStatistics> entry: mWrap.getCoprocessorExecutionStatistics().entrySet() %>
+            <tr>
+                <%java>
+                    String coprocessorName = entry.getKey();
+                    DescriptiveStatistics ds = entry.getValue();
+                </%java>
+                <td><% r.getRegionNameAsString() %></td>
+                <td><% coprocessorName %></td>
+                <td>
+                <table class="table-condensed">
+                <tr>
+                    <td>Min Time </td>
+                    <td><% String.format("%.3f%n", ds.getMin()/1000/1000) %>ms</td>
+                </tr>
+                <tr>
+                    <td>Avg Time </td>
+                    <td><% String.format("%.3f%n", ds.getMean()/1000/1000) %>ms</td>
+                </tr>
+                <tr>
+                    <td>Max Time </td>
+                    <td><% String.format("%.3f%n", ds.getMax()/1000/1000) %>ms</td>
+                </tr>
+                <tr>
+                    <td>90th percentile </td>
+                    <td><% String.format("%.3f%n", ds.getPercentile(90d)/1000/1000) %>ms</td>
+                </tr>
+                <tr>
+                    <td>95th percentile </td>
+                    <td><% String.format("%.3f%n", ds.getPercentile(95d)/1000/1000) %>ms</td>
+                </tr>
+                <tr>
+                    <td>99th percentile </td>
+                    <td><% String.format("%.3f%n", ds.getPercentile(99d)/1000/1000) %>ms</td>
+                </tr>
+                </table>
+                </td>
+            </tr>
+            </%for>
+        </%if>
+        </%for>
+    </table>
+</%def>

http://git-wip-us.apache.org/repos/asf/hbase/blob/19e9b8aa/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 891c698..c0fd07c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -960,7 +960,7 @@ public class HRegion implements HeapSize { // , Writable{
     return this.writeRequestsCount.get();
   }
 
-  MetricsRegion getMetrics() {
+  public MetricsRegion getMetrics() {
     return metricsRegion;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/19e9b8aa/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegion.java
index 1d6e9e2..bc2eaef 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegion.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.CompatibilityFactory;
 import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
 
 
@@ -31,10 +30,12 @@ import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
 public class MetricsRegion {
 
   private final MetricsRegionSource source;
+  private MetricsRegionWrapper regionWrapper;
 
   public MetricsRegion(final MetricsRegionWrapper wrapper) {
     source = CompatibilitySingletonFactory.getInstance(MetricsRegionServerSourceFactory.class)
                                              .createRegion(wrapper);
+    this.regionWrapper = wrapper;
   }
 
   public void close() {
@@ -68,4 +69,9 @@ public class MetricsRegion {
   MetricsRegionSource getSource() {
     return source;
   }
+
+  public MetricsRegionWrapper getRegionWrapper() {
+    return regionWrapper;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/19e9b8aa/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
index 3a0377f..ac858f2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
@@ -18,18 +18,20 @@
 
 package org.apache.hadoop.hbase.regionserver;
 
-import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.metrics2.MetricsExecutor;
-
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.commons.math.stat.descriptive.DescriptiveStatistics;
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.metrics2.MetricsExecutor;
+
 public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable {
 
   public static final int PERIOD = 45;
@@ -41,6 +43,7 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
   private long numStoreFiles;
   private long memstoreSize;
   private long storeFileSize;
+  private Map<String, DescriptiveStatistics> coprocessorTimes;
 
   private ScheduledFuture<?> regionMetricsUpdateTask;
 
@@ -50,6 +53,7 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
     this.runnable = new HRegionMetricsWrapperRunnable();
     this.regionMetricsUpdateTask = this.executor.scheduleWithFixedDelay(this.runnable, PERIOD,
       PERIOD, TimeUnit.SECONDS);
+    this.coprocessorTimes = new HashMap<String, DescriptiveStatistics>();
   }
 
   @Override
@@ -148,6 +152,8 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
       numStoreFiles = tempNumStoreFiles;
       memstoreSize = tempMemstoreSize;
       storeFileSize = tempStoreFileSize;
+      coprocessorTimes = region.getCoprocessorHost().getCoprocessorExecutionStatistics();
+
     }
   }
 
@@ -156,4 +162,9 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
     regionMetricsUpdateTask.cancel(true);
   }
 
+  @Override
+  public Map<String, DescriptiveStatistics> getCoprocessorExecutionStatistics() {
+    return coprocessorTimes;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/19e9b8aa/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
index e476ea2..6329d47 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
@@ -21,9 +21,13 @@ package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableSet;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.regex.Matcher;
@@ -32,6 +36,7 @@ import org.apache.commons.collections.map.AbstractReferenceMap;
 import org.apache.commons.collections.map.ReferenceMap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.commons.math.stat.descriptive.DescriptiveStatistics;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -71,6 +76,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
 import com.google.protobuf.Message;
 import com.google.protobuf.Service;
 
@@ -95,6 +101,9 @@ public class RegionCoprocessorHost
     private HRegion region;
     private RegionServerServices rsServices;
     ConcurrentMap<String, Object> sharedData;
+    private static final int LATENCY_BUFFER_SIZE = 100;
+    private final BlockingQueue<Long> coprocessorTimeNanos = new ArrayBlockingQueue<Long>(
+        LATENCY_BUFFER_SIZE);
 
     /**
      * Constructor
@@ -130,6 +139,17 @@ public class RegionCoprocessorHost
     public ConcurrentMap<String, Object> getSharedData() {
       return sharedData;
     }
+
+    public void offerExecutionLatency(long latencyNanos) {
+      coprocessorTimeNanos.offer(latencyNanos);
+    }
+
+    public Collection<Long> getExecutionLatenciesNanos() {
+      final List<Long> latencies = Lists.newArrayListWithCapacity(coprocessorTimeNanos.size());
+      coprocessorTimeNanos.drainTo(latencies);
+      return latencies;
+    }
+
   }
 
   /** The region server services */
@@ -289,9 +309,11 @@ public class RegionCoprocessorHost
    * @throws IOException Signals that an I/O exception has occurred.
    */
   public void preOpen() throws IOException {
+
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -303,11 +325,13 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
       }
     }
+
   }
 
   /**
@@ -317,6 +341,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -328,11 +353,13 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
       }
     }
+
   }
 
   /**
@@ -342,6 +369,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -353,6 +381,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -368,6 +397,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -379,8 +409,10 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
       }
     }
+
   }
 
   /**
@@ -391,6 +423,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -402,9 +435,11 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
       }
       shutdown(env);
     }
+
   }
 
   /**
@@ -418,6 +453,7 @@ public class RegionCoprocessorHost
     InternalScanner s = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -430,6 +466,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -453,6 +490,7 @@ public class RegionCoprocessorHost
     boolean bypass = false;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -466,6 +504,7 @@ public class RegionCoprocessorHost
           currentThread.setContextClassLoader(cl);
         }
         bypass |= ctx.shouldBypass();
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -486,6 +525,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -498,11 +538,13 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
       }
     }
+
   }
 
   /**
@@ -520,6 +562,7 @@ public class RegionCoprocessorHost
     InternalScanner s = scanner;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -533,6 +576,7 @@ public class RegionCoprocessorHost
           currentThread.setContextClassLoader(cl);
         }
         bypass |= ctx.shouldBypass();
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -553,6 +597,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -581,6 +626,7 @@ public class RegionCoprocessorHost
     InternalScanner s = scanner;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -593,6 +639,7 @@ public class RegionCoprocessorHost
           currentThread.setContextClassLoader(cl);
         }
         bypass |= ctx.shouldBypass();
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -609,6 +656,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -620,6 +668,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -638,6 +687,7 @@ public class RegionCoprocessorHost
     InternalScanner s = null;
     for (RegionEnvironment env : coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -650,6 +700,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -666,6 +717,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -677,6 +729,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -692,6 +745,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -703,6 +757,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -718,6 +773,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -729,11 +785,13 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
       }
     }
+
   }
 
   /**
@@ -744,6 +802,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -755,11 +814,13 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
       }
     }
+
   }
 
   /**
@@ -772,6 +833,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -783,6 +845,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -796,6 +859,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env : coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -808,6 +872,7 @@ public class RegionCoprocessorHost
           currentThread.setContextClassLoader(cl);
         }
         bypass |= ctx.shouldBypass();
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -820,6 +885,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env : coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -831,6 +897,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -846,6 +913,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env : coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -857,6 +925,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -872,6 +941,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env : coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -883,6 +953,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -898,6 +969,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env : coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -909,6 +981,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -931,6 +1004,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -943,6 +1017,7 @@ public class RegionCoprocessorHost
           currentThread.setContextClassLoader(cl);
         }
         bypass |= ctx.shouldBypass();
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -962,6 +1037,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -973,6 +1049,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -991,6 +1068,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1003,6 +1081,7 @@ public class RegionCoprocessorHost
           currentThread.setContextClassLoader(cl);
         }
         bypass |= ctx.shouldBypass();
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1021,6 +1100,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1032,6 +1112,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1051,6 +1132,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1063,6 +1145,7 @@ public class RegionCoprocessorHost
           currentThread.setContextClassLoader(cl);
         }
         bypass |= ctx.shouldBypass();
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1082,6 +1165,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1093,6 +1177,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1114,6 +1199,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1126,6 +1212,7 @@ public class RegionCoprocessorHost
           currentThread.setContextClassLoader(cl);
         }
         bypass |= ctx.shouldBypass();
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1150,6 +1237,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env : coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1164,6 +1252,7 @@ public class RegionCoprocessorHost
           currentThread.setContextClassLoader(cl);
         }
         bypass |= ctx.shouldBypass();
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1183,6 +1272,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1194,6 +1284,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1214,6 +1305,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1226,6 +1318,7 @@ public class RegionCoprocessorHost
           currentThread.setContextClassLoader(cl);
         }
         bypass |= ctx.shouldBypass();
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1245,6 +1338,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1256,6 +1350,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1274,6 +1369,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env : coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1286,11 +1382,13 @@ public class RegionCoprocessorHost
           currentThread.setContextClassLoader(cl);
         }
         bypass |= ctx.shouldBypass();
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
       }
     }
+
     return bypass;
   }
 
@@ -1303,6 +1401,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env : coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1314,6 +1413,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1327,6 +1427,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env : coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1339,6 +1440,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1366,6 +1468,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1379,6 +1482,7 @@ public class RegionCoprocessorHost
           currentThread.setContextClassLoader(cl);
         }
         bypass |= ctx.shouldBypass();
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1406,6 +1510,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env : coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1419,6 +1524,7 @@ public class RegionCoprocessorHost
           currentThread.setContextClassLoader(cl);
         }
         bypass |= ctx.shouldBypass();
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1444,6 +1550,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1456,6 +1563,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1484,6 +1592,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1497,6 +1606,7 @@ public class RegionCoprocessorHost
           currentThread.setContextClassLoader(cl);
         }
         bypass |= ctx.shouldBypass();
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1524,6 +1634,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env : coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1537,6 +1648,7 @@ public class RegionCoprocessorHost
           currentThread.setContextClassLoader(cl);
         }
         bypass |= ctx.shouldBypass();
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1561,6 +1673,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1573,6 +1686,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1593,6 +1707,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1605,6 +1720,7 @@ public class RegionCoprocessorHost
           currentThread.setContextClassLoader(cl);
         }
         bypass |= ctx.shouldBypass();
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1625,6 +1741,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env : coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1637,6 +1754,7 @@ public class RegionCoprocessorHost
           currentThread.setContextClassLoader(cl);
         }
         bypass |= ctx.shouldBypass();
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1657,6 +1775,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1669,6 +1788,7 @@ public class RegionCoprocessorHost
           currentThread.setContextClassLoader(cl);
         }
         bypass |= ctx.shouldBypass();
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1689,6 +1809,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env : coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1701,6 +1822,7 @@ public class RegionCoprocessorHost
           currentThread.setContextClassLoader(cl);
         }
         bypass |= ctx.shouldBypass();
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1718,6 +1840,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1729,6 +1852,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1745,6 +1869,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1756,6 +1881,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1776,6 +1902,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1788,6 +1915,7 @@ public class RegionCoprocessorHost
           currentThread.setContextClassLoader(cl);
         }
         bypass |= ctx.shouldBypass();
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1807,6 +1935,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1819,6 +1948,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1837,6 +1967,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1848,6 +1979,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1871,6 +2003,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1889,6 +2022,7 @@ public class RegionCoprocessorHost
         }
       }
     }
+
     return bypass ? hasNext : null;
   }
 
@@ -1906,6 +2040,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1918,6 +2053,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1942,6 +2078,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env : coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1954,6 +2091,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -1972,6 +2110,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -1984,22 +2123,24 @@ public class RegionCoprocessorHost
           currentThread.setContextClassLoader(cl);
         }
         bypass |= ctx.shouldBypass();
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
       }
     }
+
     return bypass;
   }
 
   /**
-   * @param s the scanner
    * @exception IOException Exception
    */
   public void postScannerClose(final InternalScanner s) throws IOException {
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -2011,6 +2152,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -2031,6 +2173,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -2043,6 +2186,7 @@ public class RegionCoprocessorHost
           currentThread.setContextClassLoader(cl);
         }
         bypass |= ctx.shouldBypass();
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -2062,6 +2206,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -2073,6 +2218,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -2090,6 +2236,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -2102,6 +2249,7 @@ public class RegionCoprocessorHost
           currentThread.setContextClassLoader(cl);
         }
         bypass |= ctx.shouldBypass();
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -2121,6 +2269,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -2133,6 +2282,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -2145,6 +2295,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env : coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -2156,6 +2307,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -2167,6 +2319,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env : coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -2178,6 +2331,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -2203,6 +2357,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env : coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -2215,6 +2370,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -2240,6 +2396,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env : coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -2252,6 +2409,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -2265,6 +2423,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env : coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -2277,6 +2436,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -2290,6 +2450,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env : coprocessors) {
       if (env.getInstance() instanceof EndpointObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -2302,6 +2463,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -2315,6 +2477,7 @@ public class RegionCoprocessorHost
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env : coprocessors) {
       if (env.getInstance() instanceof EndpointObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -2327,17 +2490,20 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
       }
     }
+
   }
 
   public DeleteTracker postInstantiateDeleteTracker(DeleteTracker tracker) throws IOException {
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env : coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
+        long startTime = System.nanoTime();
         ctx = ObserverContext.createAndPrepare(env, ctx);
         Thread currentThread = Thread.currentThread();
         ClassLoader cl = currentThread.getContextClassLoader();
@@ -2350,6 +2516,7 @@ public class RegionCoprocessorHost
         } finally {
           currentThread.setContextClassLoader(cl);
         }
+        env.offerExecutionLatency(System.nanoTime() - startTime);
         if (ctx.shouldComplete()) {
           break;
         }
@@ -2357,4 +2524,22 @@ public class RegionCoprocessorHost
     }
     return tracker;
   }
+
+  public Map<String, DescriptiveStatistics> getCoprocessorExecutionStatistics() {
+    Map<String, DescriptiveStatistics> results = new HashMap<String, DescriptiveStatistics>();
+    for (RegionEnvironment env : coprocessors) {
+      DescriptiveStatistics ds = new DescriptiveStatistics();
+      if (env.getInstance() instanceof RegionObserver) {
+        for (Long time : env.getExecutionLatenciesNanos()) {
+          ds.addValue(time);
+        }
+        // Ensures that web ui circumvents the display of NaN values when there are zero samples.
+        if (ds.getN() == 0) {
+          ds.addValue(0);
+        }
+        results.put(env.getInstance().getClass().getSimpleName(), ds);
+      }
+    }
+    return results;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/19e9b8aa/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java
index c8ad44a..2658c0a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java
@@ -18,6 +18,11 @@
 
 package org.apache.hadoop.hbase.regionserver;
 
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.math.stat.descriptive.DescriptiveStatistics;
+
 public class MetricsRegionWrapperStub implements MetricsRegionWrapper {
 
   @Override
@@ -79,4 +84,9 @@ public class MetricsRegionWrapperStub implements MetricsRegionWrapper {
   public long getNumCompactionsCompleted() {
     return 0;
   }
+
+  @Override
+  public Map<String, DescriptiveStatistics> getCoprocessorExecutionStatistics() {
+    return new HashMap<String, DescriptiveStatistics>();
+  }
 }