You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2017/09/13 03:46:46 UTC

[1/2] hbase git commit: HBASE-18793 Remove deprecated methods in RegionObserver

Repository: hbase
Updated Branches:
  refs/heads/master 4b124913f -> e6e52cd80


http://git-wip-us.apache.org/repos/asf/hbase/blob/e6e52cd8/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
index 3b7f1f8..07fc179 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
@@ -71,6 +71,7 @@ import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
@@ -416,7 +417,7 @@ public class TestRegionObserverInterface {
 
     @Override
     public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> e, Store store,
-        final InternalScanner scanner, final ScanType scanType) {
+        final InternalScanner scanner, final ScanType scanType, CompactionRequest request) {
       return new InternalScanner() {
         @Override
         public boolean next(List<Cell> results) throws IOException {
@@ -455,7 +456,7 @@ public class TestRegionObserverInterface {
 
     @Override
     public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e, Store store,
-        StoreFile resultFile) {
+        StoreFile resultFile, CompactionRequest request) {
       lastCompaction = EnvironmentEdgeManager.currentTime();
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6e52cd8/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
index e9bf09b..8ab0175 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
@@ -61,6 +61,7 @@ import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreScanner;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
@@ -139,7 +140,8 @@ public class TestRegionObserverScannerOpenHook {
 
     @Override
     public InternalScanner preFlushScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
-        Store store, List<KeyValueScanner> scanners, InternalScanner s) throws IOException {
+        Store store, List<KeyValueScanner> scanners, InternalScanner s, long readPoint)
+        throws IOException {
       scanners.forEach(KeyValueScanner::close);
       return NO_DATA;
     }
@@ -153,7 +155,8 @@ public class TestRegionObserverScannerOpenHook {
     @Override
     public InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
         Store store, List<? extends KeyValueScanner> scanners, ScanType scanType,
-        long earliestPutTs, InternalScanner s) throws IOException {
+        long earliestPutTs, InternalScanner s, CompactionRequest request, long readPoint)
+        throws IOException {
       scanners.forEach(KeyValueScanner::close);
       return NO_DATA;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6e52cd8/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
index defac7b..5179b84 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
@@ -74,6 +74,7 @@ import org.apache.hadoop.hbase.quotas.QuotaUtil;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -459,8 +460,8 @@ public class TestNamespaceAuditor {
     volatile CountDownLatch postCompact;
 
     @Override
-    public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e,
-                            Store store, StoreFile resultFile) throws IOException {
+    public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e, Store store,
+        StoreFile resultFile, CompactionRequest request) throws IOException {
       postCompact.countDown();
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6e52cd8/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java
index 5423578..fcc3d4a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.client.TestFromClientSideWithCoprocessor;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 
 /**
  * RegionObserver that just reimplements the default behavior,
@@ -44,7 +45,8 @@ public class NoOpScanPolicyObserver implements RegionObserver {
    */
   @Override
   public InternalScanner preFlushScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
-      Store store, List<KeyValueScanner> scanners, InternalScanner s) throws IOException {
+      Store store, List<KeyValueScanner> scanners, InternalScanner s, long readPoint)
+      throws IOException {
     ScanInfo oldSI = store.getScanInfo();
     ScanInfo scanInfo = new ScanInfo(oldSI.getConfiguration(), store.getColumnFamilyDescriptor(),
         oldSI.getTtl(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator());
@@ -59,7 +61,7 @@ public class NoOpScanPolicyObserver implements RegionObserver {
   public InternalScanner preCompactScannerOpen(
       final ObserverContext<RegionCoprocessorEnvironment> c, Store store,
       List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs,
-      InternalScanner s) throws IOException {
+      InternalScanner s, CompactionRequest request, long readPoint) throws IOException {
     // this demonstrates how to override the scanners default behavior
     ScanInfo oldSI = store.getScanInfo();
     ScanInfo scanInfo = new ScanInfo(oldSI.getConfiguration(), store.getColumnFamilyDescriptor(),
@@ -69,8 +71,8 @@ public class NoOpScanPolicyObserver implements RegionObserver {
   }
 
   @Override
-  public KeyValueScanner preStoreScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
-      Store store, final Scan scan, final NavigableSet<byte[]> targetCols, KeyValueScanner s)
+  public KeyValueScanner preStoreScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
+      Store store, Scan scan, NavigableSet<byte[]> targetCols, KeyValueScanner s, long readPoint)
       throws IOException {
     Region r = c.getEnvironment().getRegion();
     return scan.isReversed() ? new ReversedStoreScanner(store,

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6e52cd8/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
index 49b0a72..ea14962 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
@@ -67,6 +67,7 @@ import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.regionserver.wal.TestWALActionsListener;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
@@ -252,9 +253,8 @@ public class TestHRegionServerBulkLoad {
   public static class MyObserver implements RegionObserver {
     static int sleepDuration;
     @Override
-    public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> e,
-        final Store store, final InternalScanner scanner, final ScanType scanType)
-            throws IOException {
+    public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> e, Store store,
+        InternalScanner scanner, ScanType scanType, CompactionRequest request) throws IOException {
       try {
         Thread.sleep(sleepDuration);
       } catch (InterruptedException ie) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6e52cd8/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index 97b1633..bf36821 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -902,7 +902,7 @@ public class TestAccessController extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preCompact(ObserverContext.createAndPrepare(RCP_ENV, null), null, null,
-          ScanType.COMPACT_RETAIN_DELETES);
+          ScanType.COMPACT_RETAIN_DELETES, null);
         return null;
       }
     };

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6e52cd8/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
index 9ea8686..656a0c7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.hbase.regionserver.ScanInfo;
 import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreScanner;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
@@ -239,8 +240,8 @@ public class TestCoprocessorScanPolicy {
 
     @Override
     public InternalScanner preFlushScannerOpen(
-        final ObserverContext<RegionCoprocessorEnvironment> c,
-        Store store, List<KeyValueScanner> scanners, InternalScanner s) throws IOException {
+        final ObserverContext<RegionCoprocessorEnvironment> c, Store store,
+        List<KeyValueScanner> scanners, InternalScanner s, long readPoint) throws IOException {
       Long newTtl = ttls.get(store.getTableName());
       if (newTtl != null) {
         System.out.println("PreFlush:" + newTtl);
@@ -262,7 +263,7 @@ public class TestCoprocessorScanPolicy {
     public InternalScanner preCompactScannerOpen(
         final ObserverContext<RegionCoprocessorEnvironment> c, Store store,
         List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs,
-        InternalScanner s) throws IOException {
+        InternalScanner s, CompactionRequest request, long readPoint) throws IOException {
       Long newTtl = ttls.get(store.getTableName());
       Integer newVersions = versions.get(store.getTableName());
       ScanInfo oldSI = store.getScanInfo();


[2/2] hbase git commit: HBASE-18793 Remove deprecated methods in RegionObserver

Posted by zh...@apache.org.
HBASE-18793 Remove deprecated methods in RegionObserver


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

Branch: refs/heads/master
Commit: e6e52cd80f4ba26b196e2d20cd84ba167b303475
Parents: 4b12491
Author: zhangduo <zh...@apache.org>
Authored: Tue Sep 12 20:44:03 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Sep 13 11:33:03 2017 +0800

----------------------------------------------------------------------
 .../example/ZooKeeperScanPolicyObserver.java    |  17 +-
 .../hbase/coprocessor/RegionObserver.java       | 758 ++++---------------
 .../regionserver/RegionCoprocessorHost.java     | 154 +---
 .../hbase/security/access/AccessController.java |  32 +-
 ...estAvoidCellReferencesIntoShippedBlocks.java |  11 +-
 .../hbase/coprocessor/SimpleRegionObserver.java | 115 +--
 .../coprocessor/TestCoprocessorInterface.java   |   5 +-
 .../TestRegionObserverInterface.java            |   5 +-
 .../TestRegionObserverScannerOpenHook.java      |   7 +-
 .../hbase/namespace/TestNamespaceAuditor.java   |   5 +-
 .../regionserver/NoOpScanPolicyObserver.java    |  10 +-
 .../regionserver/TestHRegionServerBulkLoad.java |   6 +-
 .../security/access/TestAccessController.java   |   2 +-
 .../hbase/util/TestCoprocessorScanPolicy.java   |   7 +-
 14 files changed, 253 insertions(+), 881 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e6e52cd8/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java
----------------------------------------------------------------------
diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java
index b489fe4..344d188 100644
--- a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java
+++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.regionserver.ScanInfo;
 import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreScanner;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.zookeeper.KeeperException;
@@ -187,8 +188,9 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver {
   }
 
   @Override
-  public InternalScanner preFlushScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
-      Store store, List<KeyValueScanner> scanners, InternalScanner s) throws IOException {
+  public InternalScanner preFlushScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
+      Store store, List<KeyValueScanner> scanners, InternalScanner s, long readPoint)
+      throws IOException {
     ScanInfo scanInfo = getScanInfo(store, c.getEnvironment());
     if (scanInfo == null) {
       // take default action
@@ -199,10 +201,9 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver {
   }
 
   @Override
-  public InternalScanner preCompactScannerOpen(
-      final ObserverContext<RegionCoprocessorEnvironment> c,
+  public InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
       Store store, List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs,
-      InternalScanner s) throws IOException {
+      InternalScanner s, CompactionRequest request, long readPoint) throws IOException {
     ScanInfo scanInfo = getScanInfo(store, c.getEnvironment());
     if (scanInfo == null) {
       // take default action
@@ -213,9 +214,9 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver {
   }
 
   @Override
-  public KeyValueScanner preStoreScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Store store, final Scan scan, final NavigableSet<byte[]> targetCols,
-      final KeyValueScanner s) throws IOException {
+  public KeyValueScanner preStoreScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
+      Store store, Scan scan, NavigableSet<byte[]> targetCols, KeyValueScanner s, long readPoint)
+      throws IOException {
     ScanInfo scanInfo = getScanInfo(store, c.getEnvironment());
     if (scanInfo == null) {
       // take default action

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6e52cd8/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
index 224a4f5..b036608 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
@@ -19,6 +19,11 @@
 
 package org.apache.hadoop.hbase.coprocessor;
 
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableSet;
+
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
@@ -54,41 +59,40 @@ import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileReader;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker;
-import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
 
-import java.io.IOException;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableSet;
-
 /**
- * Coprocessors implement this interface to observe and mediate client actions
- * on the region.
- *
+ * Coprocessors implement this interface to observe and mediate client actions on the region.
+ * <p>
  * Since most implementations will be interested in only a subset of hooks, this class uses
  * 'default' functions to avoid having to add unnecessary overrides. When the functions are
- * non-empty, it's simply to satisfy the compiler by returning value of expected (non-void) type.
- * It is done in a way that these default definitions act as no-op. So our suggestion to
- * implementation would be to not call these 'default' methods from overrides.
- * <br><br>
- *
- * <h3>Exception Handling</h3>
+ * non-empty, it's simply to satisfy the compiler by returning value of expected (non-void) type. It
+ * is done in a way that these default definitions act as no-op. So our suggestion to implementation
+ * would be to not call these 'default' methods from overrides.
+ * <p>
+ * <h3>Exception Handling</h3><br>
  * For all functions, exception handling is done as follows:
  * <ul>
- *   <li>Exceptions of type {@link IOException} are reported back to client.</li>
- *   <li>For any other kind of exception:
- *     <ul>
- *       <li>If the configuration {@link CoprocessorHost#ABORT_ON_ERROR_KEY} is set to true, then
- *         the server aborts.</li>
- *       <li>Otherwise, coprocessor is removed from the server and
- *         {@link org.apache.hadoop.hbase.DoNotRetryIOException} is returned to the client.</li>
- *     </ul>
- *   </li>
+ * <li>Exceptions of type {@link IOException} are reported back to client.</li>
+ * <li>For any other kind of exception:
+ * <ul>
+ * <li>If the configuration {@link CoprocessorHost#ABORT_ON_ERROR_KEY} is set to true, then the
+ * server aborts.</li>
+ * <li>Otherwise, coprocessor is removed from the server and
+ * {@link org.apache.hadoop.hbase.DoNotRetryIOException} is returned to the client.</li>
+ * </ul>
+ * </li>
  * </ul>
+ * <p>
+ * <h3>For Split Related Hooks</h3> <br>
+ * In hbase2/AMv2, master runs splits, so the split related hooks are moved to
+ * {@link MasterObserver}.
+ * <p>
+ * <h3>Increment Column Value</h3><br>
+ * We do not call this hook anymore.
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
 @InterfaceStability.Evolving
@@ -105,19 +109,19 @@ public interface RegionObserver extends Coprocessor {
    * Called before the region is reported as open to the master.
    * @param c the environment provided by the region server
    */
-  default void preOpen(final ObserverContext<RegionCoprocessorEnvironment> c) throws IOException {}
+  default void preOpen(ObserverContext<RegionCoprocessorEnvironment> c) throws IOException {}
 
   /**
    * Called after the region is reported as open to the master.
    * @param c the environment provided by the region server
    */
-  default void postOpen(final ObserverContext<RegionCoprocessorEnvironment> c) {}
+  default void postOpen(ObserverContext<RegionCoprocessorEnvironment> c) {}
 
   /**
    * Called after the log replay on the region is over.
    * @param c the environment provided by the region server
    */
-  default void postLogReplay(final ObserverContext<RegionCoprocessorEnvironment> c) {}
+  default void postLogReplay(ObserverContext<RegionCoprocessorEnvironment> c) {}
 
   /**
    * Called before a memstore is flushed to disk and prior to creating the scanner to read from
@@ -130,61 +134,20 @@ public interface RegionObserver extends Coprocessor {
    * @param store the store being flushed
    * @param scanners the scanners for the memstore that is flushed
    * @param s the base scanner, if not {@code null}, from previous RegionObserver in the chain
+   * @param readPoint the readpoint to create scanner
    * @return the scanner to use during the flush.  {@code null} if the default implementation
    * is to be used.
-   * @deprecated Use {@link #preFlushScannerOpen(ObserverContext, Store, List,
-   *             InternalScanner, long)}
    */
-  @Deprecated
-  default InternalScanner preFlushScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Store store, final List<KeyValueScanner> scanners, final InternalScanner s)
+  default InternalScanner preFlushScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
+      Store store, List<KeyValueScanner> scanners, InternalScanner s, long readPoint)
       throws IOException {
     return s;
   }
 
   /**
-   * Called before a memstore is flushed to disk and prior to creating the scanner to read from
-   * the memstore.  To override or modify how a memstore is flushed,
-   * implementing classes can return a new scanner to provide the KeyValues to be
-   * stored into the new {@code StoreFile} or null to perform the default processing.
-   * Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no
-   * effect in this hook.
-   * @param c the environment provided by the region server
-   * @param store the store being flushed
-   * @param scanners the scanners for the memstore that is flushed
-   * @param s the base scanner, if not {@code null}, from previous RegionObserver in the chain
-   * @param readPoint the readpoint to create scanner
-   * @return the scanner to use during the flush.  {@code null} if the default implementation
-   * is to be used.
-   */
-  default InternalScanner preFlushScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Store store, final List<KeyValueScanner> scanners, final InternalScanner s,
-      final long readPoint) throws IOException {
-    return preFlushScannerOpen(c, store, scanners, s);
-  }
-
-  /**
-   * Maintain backward compatibility.
-   * @param c the environment provided by the region server
-   * @param store the store being flushed
-   * @param scanner the scanner for the memstore that is flushed
-   * @param s the base scanner, if not {@code null}, from previous RegionObserver in the chain
-   * @param readPoint the readpoint to create scanner
-   * @return the scanner to use during the flush.  {@code null} if the default implementation
-   * is to be used.
-   */
-  default InternalScanner preFlushScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Store store, final KeyValueScanner scanner, final InternalScanner s,
-      final long readPoint) throws IOException {
-    return preFlushScannerOpen(c, store, Collections.singletonList(scanner), s, readPoint);
-  }
-
-  /**
    * Called before the memstore is flushed to disk.
    * @param c the environment provided by the region server
-   * @deprecated use {@link #preFlush(ObserverContext, Store, InternalScanner)} instead
    */
-  @Deprecated
   default void preFlush(final ObserverContext<RegionCoprocessorEnvironment> c) throws IOException {}
 
   /**
@@ -195,19 +158,17 @@ public interface RegionObserver extends Coprocessor {
    * @return the scanner to use during compaction.  Should not be {@code null}
    * unless the implementation is writing new store files on its own.
    */
-  default InternalScanner preFlush(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Store store, final InternalScanner scanner) throws IOException {
+  default InternalScanner preFlush(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
+      InternalScanner scanner) throws IOException {
     return scanner;
   }
 
   /**
    * Called after the memstore is flushed to disk.
    * @param c the environment provided by the region server
-   * @deprecated use {@link #preFlush(ObserverContext, Store, InternalScanner)} instead.
+   * @throws IOException if an error occurred on the coprocessor
    */
-  @Deprecated
-  default void postFlush(final ObserverContext<RegionCoprocessorEnvironment> c)
-      throws IOException {}
+  default void postFlush(ObserverContext<RegionCoprocessorEnvironment> c) throws IOException {}
 
   /**
    * Called after a Store's memstore is flushed to disk.
@@ -215,8 +176,8 @@ public interface RegionObserver extends Coprocessor {
    * @param store the store being flushed
    * @param resultFile the new store file written out during compaction
    */
-  default void postFlush(final ObserverContext<RegionCoprocessorEnvironment> c, final Store store,
-      final StoreFile resultFile) throws IOException {}
+  default void postFlush(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
+      StoreFile resultFile) throws IOException {}
 
   /**
    * Called prior to selecting the {@link StoreFile StoreFiles} to compact from the list of
@@ -227,25 +188,8 @@ public interface RegionObserver extends Coprocessor {
    * @param candidates the store files currently available for compaction
    * @param request custom compaction request
    */
-  default void preCompactSelection(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Store store, final List<StoreFile> candidates, final CompactionRequest request)
-      throws IOException {
-    preCompactSelection(c, store, candidates);
-  }
-
-  /**
-   * Called prior to selecting the {@link StoreFile}s to compact from the list of available
-   * candidates. To alter the files used for compaction, you may mutate the passed in list of
-   * candidates.
-   * @param c the environment provided by the region server
-   * @param store the store where compaction is being requested
-   * @param candidates the store files currently available for compaction
-   * @deprecated Use {@link #preCompactSelection(ObserverContext, Store, List, CompactionRequest)}
-   *             instead
-   */
-  @Deprecated
-  default void preCompactSelection(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Store store, final List<StoreFile> candidates) throws IOException {}
+  default void preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
+      List<StoreFile> candidates, CompactionRequest request) throws IOException {}
 
   /**
    * Called after the {@link StoreFile}s to compact have been selected from the available
@@ -255,23 +199,8 @@ public interface RegionObserver extends Coprocessor {
    * @param selected the store files selected to compact
    * @param request custom compaction request
    */
-  default void postCompactSelection(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Store store, final ImmutableList<StoreFile> selected, CompactionRequest request) {
-    postCompactSelection(c, store, selected);
-  }
-
-  /**
-   * Called after the {@link StoreFile}s to compact have been selected from the available
-   * candidates.
-   * @param c the environment provided by the region server
-   * @param store the store being compacted
-   * @param selected the store files selected to compact
-   * @deprecated use {@link #postCompactSelection(ObserverContext, Store, ImmutableList,
-   *             CompactionRequest)} instead.
-   */
-  @Deprecated
-  default void postCompactSelection(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Store store, final ImmutableList<StoreFile> selected) {}
+  default void postCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
+      ImmutableList<StoreFile> selected, CompactionRequest request) {}
 
   /**
    * Called prior to writing the {@link StoreFile}s selected for compaction into a new
@@ -295,40 +224,9 @@ public interface RegionObserver extends Coprocessor {
    * @return the scanner to use during compaction. Should not be {@code null} unless the
    *         implementation is writing new store files on its own.
    */
-  default InternalScanner preCompact(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Store store, final InternalScanner scanner, final ScanType scanType,
+  default InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> c,
+      Store store, InternalScanner scanner, ScanType scanType,
       CompactionRequest request) throws IOException {
-    return preCompact(c, store, scanner, scanType);
-  }
-
-  /**
-   * Called prior to writing the {@link StoreFile}s selected for compaction into a new
-   * {@code StoreFile}. To override or modify the compaction process, implementing classes have two
-   * options:
-   * <ul>
-   * <li>Wrap the provided {@link InternalScanner} with a custom implementation that is returned
-   * from this method. The custom scanner can then inspect
-   *  {@link org.apache.hadoop.hbase.KeyValue}s from the wrapped scanner, applying its own
-   *   policy to what gets written.</li>
-   * <li>Call {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} and provide a
-   * custom implementation for writing of new {@link StoreFile}s. <strong>Note: any implementations
-   * bypassing core compaction using this approach must write out new store files themselves or the
-   * existing data will no longer be available after compaction.</strong></li>
-   * </ul>
-   * @param c the environment provided by the region server
-   * @param store the store being compacted
-   * @param scanner the scanner over existing data used in the store file rewriting
-   * @param scanType type of Scan
-   * @return the scanner to use during compaction. Should not be {@code null} unless the
-   *         implementation is writing new store files on its own.
-   * @deprecated use
-   *             {@link #preCompact(ObserverContext, Store, InternalScanner,
-   *             ScanType, CompactionRequest)} instead
-   */
-  @Deprecated
-  default InternalScanner preCompact(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Store store, final InternalScanner scanner, final ScanType scanType)
-      throws IOException {
     return scanner;
   }
 
@@ -347,75 +245,14 @@ public interface RegionObserver extends Coprocessor {
    * @param earliestPutTs timestamp of the earliest put that was found in any of the involved store
    *          files
    * @param s the base scanner, if not {@code null}, from previous RegionObserver in the chain
-   * @param request the requested compaction
-   * @return the scanner to use during compaction. {@code null} if the default implementation is to
-   *         be used.
-   * @deprecated Use {@link #preCompactScannerOpen(ObserverContext, Store, List, ScanType, long,
-   *             InternalScanner, CompactionRequest, long)} instead.
-   */
-  @Deprecated
-  default InternalScanner preCompactScannerOpen(
-      final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Store store, List<? extends KeyValueScanner> scanners, final ScanType scanType,
-      final long earliestPutTs, final InternalScanner s, CompactionRequest request)
-      throws IOException {
-    return preCompactScannerOpen(c, store, scanners, scanType, earliestPutTs, s);
-  }
-
-  /**
-   * Called prior to writing the {@link StoreFile}s selected for compaction into a new
-   * {@code StoreFile} and prior to creating the scanner used to read the input files. To override
-   * or modify the compaction process, implementing classes can return a new scanner to provide the
-   * KeyValues to be stored into the new {@code StoreFile} or null to perform the default
-   * processing. Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no
-   * effect in this hook.
-   * @param c the environment provided by the region server
-   * @param store the store being compacted
-   * @param scanners the list {@link org.apache.hadoop.hbase.regionserver.StoreFileScanner}s
-   *  to be read from
-   * @param scanType the {@link ScanType} indicating whether this is a major or minor compaction
-   * @param earliestPutTs timestamp of the earliest put that was found in any of the involved store
-   *          files
-   * @param s the base scanner, if not {@code null}, from previous RegionObserver in the chain
    * @param request compaction request
    * @param readPoint the readpoint to create scanner
    * @return the scanner to use during compaction. {@code null} if the default implementation is to
    *          be used.
    */
-  default InternalScanner preCompactScannerOpen(
-      final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Store store, List<? extends KeyValueScanner> scanners, final ScanType scanType,
-      final long earliestPutTs, final InternalScanner s, final CompactionRequest request,
-      final long readPoint) throws IOException {
-    return preCompactScannerOpen(c, store, scanners, scanType, earliestPutTs, s, request);
-  }
-
-  /**
-   * Called prior to writing the {@link StoreFile}s selected for compaction into a new
-   * {@code StoreFile} and prior to creating the scanner used to read the input files. To override
-   * or modify the compaction process, implementing classes can return a new scanner to provide the
-   * KeyValues to be stored into the new {@code StoreFile} or null to perform the default
-   * processing. Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no
-   * effect in this hook.
-   * @param c the environment provided by the region server
-   * @param store the store being compacted
-   * @param scanners the list {@link org.apache.hadoop.hbase.regionserver.StoreFileScanner}s
-   *  to be read from
-   * @param scanType the {@link ScanType} indicating whether this is a major or minor compaction
-   * @param earliestPutTs timestamp of the earliest put that was found in any of the involved store
-   *          files
-   * @param s the base scanner, if not {@code null}, from previous RegionObserver in the chain
-   * @return the scanner to use during compaction. {@code null} if the default implementation is to
-   *         be used.
-   * @deprecated Use
-   *             {@link #preCompactScannerOpen(ObserverContext, Store, List, ScanType, long,
-   *             InternalScanner, CompactionRequest, long)} instead.
-   */
-  @Deprecated
-  default InternalScanner preCompactScannerOpen(
-      final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Store store, List<? extends KeyValueScanner> scanners, final ScanType scanType,
-      final long earliestPutTs, final InternalScanner s) throws IOException {
+  default InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
+      Store store, List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs,
+      InternalScanner s, CompactionRequest request, long readPoint) throws IOException {
     return s;
   }
 
@@ -426,137 +263,23 @@ public interface RegionObserver extends Coprocessor {
    * @param resultFile the new store file written out during compaction
    * @param request the requested compaction
    */
-  default void postCompact(final ObserverContext<RegionCoprocessorEnvironment> c, final Store store,
-      StoreFile resultFile, CompactionRequest request) throws IOException {
-    postCompact(c, store, resultFile);
-  }
+  default void postCompact(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
+      StoreFile resultFile, CompactionRequest request) throws IOException {}
 
   /**
-   * Called after compaction has completed and the new store file has been moved in to place.
-   * @param c the environment provided by the region server
-   * @param store the store being compacted
-   * @param resultFile the new store file written out during compaction
-   * @deprecated Use {@link #postCompact(ObserverContext, Store, StoreFile, CompactionRequest)}
-   *             instead
-   */
-  @Deprecated
-  default void postCompact(final ObserverContext<RegionCoprocessorEnvironment> c, final Store store,
-      StoreFile resultFile) throws IOException {}
-
-  /**
-   * Called before the region is split.
-   * @param c the environment provided by the region server
-   * (e.getRegion() returns the parent region)
-   * @deprecated No longer called in hbase2/AMv2 given the master runs splits now;
-   * @see MasterObserver
-   */
-  @Deprecated
-  default void preSplit(final ObserverContext<RegionCoprocessorEnvironment> c) throws IOException {}
-
-  /**
-   * Called before the region is split.
-   * @param c the environment provided by the region server
-   * (e.getRegion() returns the parent region)
-   *
-   * Note: the logic moves to Master; it is unused in RS
-   * @deprecated No longer called in hbase2/AMv2 given the master runs splits now;
-   * @see MasterObserver
-   */
-  @Deprecated
-  default void preSplit(final ObserverContext<RegionCoprocessorEnvironment> c, byte[] splitRow)
-      throws IOException {}
-
-  /**
-   * Called after the region is split.
-   * @param c the environment provided by the region server
-   * (e.getRegion() returns the parent region)
-   * @param l the left daughter region
-   * @param r the right daughter region
-   * @deprecated No longer called in hbase2/AMv2 given the master runs splits now;
-   * @see MasterObserver
-   */
-  @Deprecated
-  default void postSplit(final ObserverContext<RegionCoprocessorEnvironment> c, final Region l,
-      final Region r) throws IOException {}
-
-  /**
-   * This will be called before PONR step as part of split transaction. Calling
-   * {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} rollback the split
-   * @param ctx
-   * @param splitKey
-   * @param metaEntries
-   *
-   * Note: the logic moves to Master; it is unused in RS
-   * @deprecated No longer called in hbase2/AMv2 given the master runs splits now;
-   * @see MasterObserver
-  */
-  @Deprecated
-  default void preSplitBeforePONR(final ObserverContext<RegionCoprocessorEnvironment> ctx,
-      byte[] splitKey, List<Mutation> metaEntries) throws IOException {}
-
-  /**
-   * This will be called after PONR step as part of split transaction
-   * Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no
-   * effect in this hook.
-   * @param ctx
-   * Note: the logic moves to Master; it is unused in RS
-   * @deprecated No longer called in hbase2/AMv2 given the master runs splits now;
-   * @see MasterObserver
-  */
-  @Deprecated
-  default void preSplitAfterPONR(final ObserverContext<RegionCoprocessorEnvironment> ctx)
-      throws IOException {}
-
-  /**
-   * This will be called before the roll back of the split region is completed
-   * @param ctx
-   *
-   * Note: the logic moves to Master; it is unused in RS
-   * @deprecated No longer called in hbase2/AMv2 given the master runs splits now;
-   * @see MasterObserver
-  */
-  @Deprecated
-  default void preRollBackSplit(final ObserverContext<RegionCoprocessorEnvironment> ctx)
-      throws IOException {}
-
-  /**
-   * This will be called after the roll back of the split region is completed
-   * @param ctx
-   *
-   * Note: the logic moves to Master; it is unused in RS
-   * @deprecated No longer called in hbase2/AMv2 given the master runs splits now;
-   * @see MasterObserver
-  */
-  @Deprecated
-  default void postRollBackSplit(final ObserverContext<RegionCoprocessorEnvironment> ctx)
-    throws IOException {}
-
-  /**
-   * Called after any split request is processed.  This will be called irrespective of success or
-   * failure of the split.
-   * @param ctx
-   * @deprecated No longer called in hbase2/AMv2 given the master runs splits now;
-   * implement {@link MasterObserver#postCompletedSplitRegionAction(ObserverContext, HRegionInfo, HRegionInfo)}
-   * instead.
-   */
-  @Deprecated
-  default void postCompleteSplit(final ObserverContext<RegionCoprocessorEnvironment> ctx)
-    throws IOException {}
-  /**
    * Called before the region is reported as closed to the master.
    * @param c the environment provided by the region server
    * @param abortRequested true if the region server is aborting
    */
-  default void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
-      boolean abortRequested) throws IOException {}
+  default void preClose(ObserverContext<RegionCoprocessorEnvironment> c, boolean abortRequested)
+      throws IOException {}
 
   /**
    * Called after the region is reported as closed to the master.
    * @param c the environment provided by the region server
    * @param abortRequested true if the region server is aborting
    */
-  default void postClose(final ObserverContext<RegionCoprocessorEnvironment> c,
-      boolean abortRequested) {}
+  default void postClose(ObserverContext<RegionCoprocessorEnvironment> c, boolean abortRequested) {}
 
   /**
    * Called before the client performs a Get
@@ -571,9 +294,8 @@ public interface RegionObserver extends Coprocessor {
    * is bypassed. Can be modified. Will not be used if default processing
    * is not bypassed.
    */
-  default void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
-      final List<Cell> result)
-    throws IOException {}
+  default void preGetOp(ObserverContext<RegionCoprocessorEnvironment> c, Get get, List<Cell> result)
+      throws IOException {}
 
   /**
    * Called after the client performs a Get
@@ -587,9 +309,8 @@ public interface RegionObserver extends Coprocessor {
    * @param get the Get request
    * @param result the result to return to the client, modify as necessary
    */
-  default void postGetOp(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
-      final List<Cell> result)
-    throws IOException {}
+  default void postGetOp(ObserverContext<RegionCoprocessorEnvironment> c, Get get,
+      List<Cell> result) throws IOException {}
 
   /**
    * Called before the client tests for existence using a Get.
@@ -603,8 +324,8 @@ public interface RegionObserver extends Coprocessor {
    * @param exists
    * @return the value to return to the client if bypassing default processing
    */
-  default boolean preExists(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
-      final boolean exists) throws IOException {
+  default boolean preExists(ObserverContext<RegionCoprocessorEnvironment> c, Get get,
+      boolean exists) throws IOException {
     return exists;
   }
 
@@ -618,8 +339,8 @@ public interface RegionObserver extends Coprocessor {
    * @param exists the result returned by the region server
    * @return the result to return to the client
    */
-  default boolean postExists(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
-      final boolean exists) throws IOException {
+  default boolean postExists(ObserverContext<RegionCoprocessorEnvironment> c, Get get,
+      boolean exists) throws IOException {
     return exists;
   }
 
@@ -638,9 +359,8 @@ public interface RegionObserver extends Coprocessor {
    * @param edit The WALEdit object that will be written to the wal
    * @param durability Persistence guarantee for this Put
    */
-  default void prePut(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Put put, final WALEdit edit, final Durability durability)
-    throws IOException {}
+  default void prePut(ObserverContext<RegionCoprocessorEnvironment> c, Put put, WALEdit edit,
+      Durability durability) throws IOException {}
 
   /**
    * Called after the client stores a value.
@@ -655,9 +375,8 @@ public interface RegionObserver extends Coprocessor {
    * @param edit The WALEdit object for the wal
    * @param durability Persistence guarantee for this Put
    */
-  default void postPut(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Put put, final WALEdit edit, final Durability durability)
-    throws IOException {}
+  default void postPut(ObserverContext<RegionCoprocessorEnvironment> c, Put put, WALEdit edit,
+      Durability durability) throws IOException {}
 
   /**
    * Called before the client deletes a value.
@@ -674,27 +393,24 @@ public interface RegionObserver extends Coprocessor {
    * @param edit The WALEdit object for the wal
    * @param durability Persistence guarantee for this Delete
    */
-  default void preDelete(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Delete delete, final WALEdit edit, final Durability durability)
-    throws IOException {}
-/**
- * Called before the server updates the timestamp for version delete with latest timestamp.
- * <p>
- * Call CoprocessorEnvironment#bypass to skip default actions
- * <p>
- * Call CoprocessorEnvironment#complete to skip any subsequent chained
- * coprocessors
- * @param c the environment provided by the region server
- * @param mutation - the parent mutation associated with this delete cell
- * @param cell - The deleteColumn with latest version cell
- * @param byteNow - timestamp bytes
- * @param get - the get formed using the current cell's row.
- * Note that the get does not specify the family and qualifier
- */
-  default void prePrepareTimeStampForDeleteVersion(
-      final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Mutation mutation, final Cell cell, final byte[] byteNow,
-      final Get get) throws IOException {}
+  default void preDelete(ObserverContext<RegionCoprocessorEnvironment> c, Delete delete,
+      WALEdit edit, Durability durability) throws IOException {}
+
+  /**
+   * Called before the server updates the timestamp for version delete with latest timestamp.
+   * <p>
+   * Call CoprocessorEnvironment#bypass to skip default actions
+   * <p>
+   * Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
+   * @param c the environment provided by the region server
+   * @param mutation - the parent mutation associated with this delete cell
+   * @param cell - The deleteColumn with latest version cell
+   * @param byteNow - timestamp bytes
+   * @param get - the get formed using the current cell's row. Note that the get does not specify
+   *          the family and qualifier
+   */
+  default void prePrepareTimeStampForDeleteVersion(ObserverContext<RegionCoprocessorEnvironment> c,
+      Mutation mutation, Cell cell, byte[] byteNow, Get get) throws IOException {}
 
   /**
    * Called after the client deletes a value.
@@ -709,9 +425,8 @@ public interface RegionObserver extends Coprocessor {
    * @param edit The WALEdit object for the wal
    * @param durability Persistence guarantee for this Delete
    */
-  default void postDelete(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Delete delete, final WALEdit edit, final Durability durability)
-    throws IOException {}
+  default void postDelete(ObserverContext<RegionCoprocessorEnvironment> c, Delete delete,
+      WALEdit edit, Durability durability) throws IOException {}
 
   /**
    * This will be called for every batch mutation operation happening at the server. This will be
@@ -725,8 +440,8 @@ public interface RegionObserver extends Coprocessor {
    * @param c the environment provided by the region server
    * @param miniBatchOp batch of Mutations getting applied to region.
    */
-  default void preBatchMutate(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {}
+  default void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+      MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {}
 
   /**
    * This will be called after applying a batch of Mutations on a region. The Mutations are added to
@@ -740,8 +455,8 @@ public interface RegionObserver extends Coprocessor {
    * @param c the environment provided by the region server
    * @param miniBatchOp batch of Mutations applied to region.
    */
-  default void postBatchMutate(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {}
+  default void postBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+      MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {}
 
   /**
    * This will be called for region operations where read lock is acquired in
@@ -749,7 +464,7 @@ public interface RegionObserver extends Coprocessor {
    * @param ctx
    * @param operation The operation is about to be taken on the region
    */
-  default void postStartRegionOperation(final ObserverContext<RegionCoprocessorEnvironment> ctx,
+  default void postStartRegionOperation(ObserverContext<RegionCoprocessorEnvironment> ctx,
       Operation operation) throws IOException {}
 
   /**
@@ -757,7 +472,7 @@ public interface RegionObserver extends Coprocessor {
    * @param ctx
    * @param operation
    */
-  default void postCloseRegionOperation(final ObserverContext<RegionCoprocessorEnvironment> ctx,
+  default void postCloseRegionOperation(ObserverContext<RegionCoprocessorEnvironment> ctx,
       Operation operation) throws IOException {}
 
   /**
@@ -770,9 +485,8 @@ public interface RegionObserver extends Coprocessor {
    * @param miniBatchOp
    * @param success true if batch operation is successful otherwise false.
    */
-  default void postBatchMutateIndispensably(final ObserverContext<RegionCoprocessorEnvironment> ctx,
-      MiniBatchOperationInProgress<Mutation> miniBatchOp, final boolean success)
-      throws IOException {}
+  default void postBatchMutateIndispensably(ObserverContext<RegionCoprocessorEnvironment> ctx,
+      MiniBatchOperationInProgress<Mutation> miniBatchOp, boolean success) throws IOException {}
 
   /**
    * Called before checkAndPut.
@@ -795,11 +509,9 @@ public interface RegionObserver extends Coprocessor {
    * @return the return value to return to client if bypassing default
    * processing
    */
-  default boolean preCheckAndPut(final ObserverContext<RegionCoprocessorEnvironment> c,
-                                 final byte [] row, final byte [] family, final byte [] qualifier,
-                                 final CompareOperator op, final ByteArrayComparable comparator,
-                                 final Put put, final boolean result)
-    throws IOException {
+  default boolean preCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row,
+      byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Put put,
+      boolean result) throws IOException {
     return result;
   }
 
@@ -828,10 +540,9 @@ public interface RegionObserver extends Coprocessor {
    * @return the return value to return to client if bypassing default
    * processing
    */
-  default boolean preCheckAndPutAfterRowLock(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final byte[] row, final byte[] family, final byte[] qualifier, final CompareOperator op,
-      final ByteArrayComparable comparator, final Put put,
-      final boolean result) throws IOException {
+  default boolean preCheckAndPutAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c,
+      byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
+      ByteArrayComparable comparator, Put put, boolean result) throws IOException {
     return result;
   }
 
@@ -853,11 +564,9 @@ public interface RegionObserver extends Coprocessor {
    * @param result from the checkAndPut
    * @return the possibly transformed return value to return to client
    */
-  default boolean postCheckAndPut(final ObserverContext<RegionCoprocessorEnvironment> c,
-                                  final byte [] row, final byte [] family, final byte [] qualifier,
-                                  final CompareOperator op, final ByteArrayComparable comparator,
-                                  final Put put, final boolean result)
-    throws IOException {
+  default boolean postCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row,
+      byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Put put,
+      boolean result) throws IOException {
     return result;
   }
 
@@ -881,11 +590,9 @@ public interface RegionObserver extends Coprocessor {
    * @param result
    * @return the value to return to client if bypassing default processing
    */
-  default boolean preCheckAndDelete(final ObserverContext<RegionCoprocessorEnvironment> c,
-                                    final byte [] row, final byte [] family, final byte [] qualifier,
-                                    final CompareOperator op, final ByteArrayComparable comparator,
-                                    final Delete delete, final boolean result)
-    throws IOException {
+  default boolean preCheckAndDelete(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row,
+      byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator,
+      Delete delete, boolean result) throws IOException {
     return result;
   }
 
@@ -913,12 +620,9 @@ public interface RegionObserver extends Coprocessor {
    * @param result
    * @return the value to return to client if bypassing default processing
    */
-  default boolean preCheckAndDeleteAfterRowLock(
-      final ObserverContext<RegionCoprocessorEnvironment> c,
-      final byte[] row, final byte[] family, final byte[] qualifier,
-      final CompareOperator op,
-      final ByteArrayComparable comparator, final Delete delete,
-      final boolean result) throws IOException {
+  default boolean preCheckAndDeleteAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c,
+      byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
+      ByteArrayComparable comparator, Delete delete, boolean result) throws IOException {
     return result;
   }
 
@@ -940,56 +644,9 @@ public interface RegionObserver extends Coprocessor {
    * @param result from the CheckAndDelete
    * @return the possibly transformed returned value to return to client
    */
-  default boolean postCheckAndDelete(final ObserverContext<RegionCoprocessorEnvironment> c,
-                                     final byte [] row, final byte [] family, final byte [] qualifier,
-                                     final CompareOperator op, final ByteArrayComparable comparator,
-                                     final Delete delete, final boolean result)
-    throws IOException {
-    return result;
-  }
-
-  /**
-   * Called before incrementColumnValue
-   * <p>
-   * Call CoprocessorEnvironment#bypass to skip default actions
-   * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
-   * @param c the environment provided by the region server
-   * @param row row to check
-   * @param family column family
-   * @param qualifier column qualifier
-   * @param amount long amount to increment
-   * @param writeToWAL true if the change should be written to the WAL
-   * @return value to return to the client if bypassing default processing
-   * @deprecated This hook is no longer called by the RegionServer
-   */
-  @Deprecated
-  default long preIncrementColumnValue(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final byte [] row, final byte [] family, final byte [] qualifier,
-      final long amount, final boolean writeToWAL) throws IOException {
-    return amount;
-  }
-
-  /**
-   * Called after incrementColumnValue
-   * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
-   * @param c the environment provided by the region server
-   * @param row row to check
-   * @param family column family
-   * @param qualifier column qualifier
-   * @param amount long amount to increment
-   * @param writeToWAL true if the change should be written to the WAL
-   * @param result the result returned by incrementColumnValue
-   * @return the result to return to the client
-   * @deprecated This hook is no longer called by the RegionServer
-   */
-  @Deprecated
-  default long postIncrementColumnValue(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final byte [] row, final byte [] family, final byte [] qualifier,
-      final long amount, final boolean writeToWAL, final long result) throws IOException {
+  default boolean postCheckAndDelete(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row,
+      byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator,
+      Delete delete, boolean result) throws IOException {
     return result;
   }
 
@@ -1007,9 +664,8 @@ public interface RegionObserver extends Coprocessor {
    * @param append Append object
    * @return result to return to the client if bypassing default processing
    */
-  default Result preAppend(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Append append)
-    throws IOException {
+  default Result preAppend(ObserverContext<RegionCoprocessorEnvironment> c, Append append)
+      throws IOException {
     return null;
   }
 
@@ -1031,8 +687,8 @@ public interface RegionObserver extends Coprocessor {
    * @param append Append object
    * @return result to return to the client if bypassing default processing
    */
-  default Result preAppendAfterRowLock(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Append append) throws IOException {
+  default Result preAppendAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c,
+      Append append) throws IOException {
     return null;
   }
 
@@ -1049,9 +705,8 @@ public interface RegionObserver extends Coprocessor {
    * @param result the result returned by increment
    * @return the result to return to the client
    */
-  default Result postAppend(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Append append, final Result result)
-    throws IOException {
+  default Result postAppend(ObserverContext<RegionCoprocessorEnvironment> c, Append append,
+      Result result) throws IOException {
     return result;
   }
 
@@ -1069,9 +724,8 @@ public interface RegionObserver extends Coprocessor {
    * @param increment increment object
    * @return result to return to the client if bypassing default processing
    */
-  default Result preIncrement(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Increment increment)
-    throws IOException {
+  default Result preIncrement(ObserverContext<RegionCoprocessorEnvironment> c, Increment increment)
+      throws IOException {
     return null;
   }
 
@@ -1096,8 +750,8 @@ public interface RegionObserver extends Coprocessor {
    * @return result to return to the client if bypassing default processing
    *           if an error occurred on the coprocessor
    */
-  default Result preIncrementAfterRowLock(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Increment increment) throws IOException {
+  default Result preIncrementAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c,
+      Increment increment) throws IOException {
     return null;
   }
 
@@ -1114,8 +768,8 @@ public interface RegionObserver extends Coprocessor {
    * @param result the result returned by increment
    * @return the result to return to the client
    */
-  default Result postIncrement(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Increment increment, final Result result) throws IOException {
+  default Result postIncrement(ObserverContext<RegionCoprocessorEnvironment> c, Increment increment,
+      Result result) throws IOException {
     return result;
   }
 
@@ -1135,39 +789,8 @@ public interface RegionObserver extends Coprocessor {
    * @return an RegionScanner instance to use instead of the base scanner if
    * overriding default behavior, null otherwise
    */
-  default RegionScanner preScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Scan scan, final RegionScanner s) throws IOException {
-    return s;
-  }
-
-  /**
-   * Called before a store opens a new scanner.
-   * This hook is called when a "user" scanner is opened.
-   * <p>
-   * See {@link #preFlushScannerOpen(ObserverContext, Store, List, InternalScanner, long)} and {@link #preCompactScannerOpen(ObserverContext,
-   *  Store, List, ScanType, long, InternalScanner, CompactionRequest, long)}
-   * to override scanners created for flushes or compactions, resp.
-   * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors.
-   * Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no
-   * effect in this hook.
-   * <p>
-   * Note: Do not retain references to any Cells returned by scanner, beyond the life of this
-   * invocation. If need a Cell reference for later use, copy the cell and use that.
-   * @param c the environment provided by the region server
-   * @param store the store being scanned
-   * @param scan the Scan specification
-   * @param targetCols columns to be used in the scanner
-   * @param s the base scanner, if not {@code null}, from previous RegionObserver in the chain
-   * @return a KeyValueScanner instance to use or {@code null} to use the default implementation
-   * @deprecated use {@link #preStoreScannerOpen(ObserverContext, Store, Scan, NavigableSet,
-   *   KeyValueScanner, long)} instead
-   */
-  @Deprecated
-  default KeyValueScanner preStoreScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Store store, final Scan scan, final NavigableSet<byte[]> targetCols,
-      final KeyValueScanner s) throws IOException {
+  default RegionScanner preScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Scan scan,
+      RegionScanner s) throws IOException {
     return s;
   }
 
@@ -1194,10 +817,10 @@ public interface RegionObserver extends Coprocessor {
    * @param readPt the read point
    * @return a KeyValueScanner instance to use or {@code null} to use the default implementation
    */
-  default KeyValueScanner preStoreScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Store store, final Scan scan, final NavigableSet<byte[]> targetCols,
-      final KeyValueScanner s, final long readPt) throws IOException {
-    return preStoreScannerOpen(c, store, scan, targetCols, s);
+  default KeyValueScanner preStoreScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
+      Store store, Scan scan, NavigableSet<byte[]> targetCols, KeyValueScanner s, long readPt)
+      throws IOException {
+    return s;
   }
 
   /**
@@ -1213,9 +836,8 @@ public interface RegionObserver extends Coprocessor {
    * @param s if not null, the base scanner
    * @return the scanner instance to use
    */
-  default RegionScanner postScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Scan scan, final RegionScanner s)
-    throws IOException {
+  default RegionScanner postScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Scan scan,
+      RegionScanner s) throws IOException {
     return s;
   }
 
@@ -1238,10 +860,8 @@ public interface RegionObserver extends Coprocessor {
    * @param hasNext the 'has more' indication
    * @return 'has more' indication that should be sent to client
    */
-  default boolean preScannerNext(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final InternalScanner s, final List<Result> result,
-      final int limit, final boolean hasNext)
-    throws IOException {
+  default boolean preScannerNext(ObserverContext<RegionCoprocessorEnvironment> c, InternalScanner s,
+      List<Result> result, int limit, boolean hasNext) throws IOException {
     return hasNext;
   }
 
@@ -1260,9 +880,8 @@ public interface RegionObserver extends Coprocessor {
    * @param hasNext the 'has more' indication
    * @return 'has more' indication that should be sent to client
    */
-  default boolean postScannerNext(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final InternalScanner s, final List<Result> result, final int limit,
-      final boolean hasNext) throws IOException {
+  default boolean postScannerNext(ObserverContext<RegionCoprocessorEnvironment> c,
+      InternalScanner s, List<Result> result, int limit, boolean hasNext) throws IOException {
     return hasNext;
   }
 
@@ -1275,35 +894,6 @@ public interface RegionObserver extends Coprocessor {
    * <li>
    * <code>boolean filterRow()</code> returning true</li>
    * <li>
-   * <code>default void filterRow(List&lt;KeyValue&gt; kvs)</code> removing all the kvs
-   * from the passed List</li>
-   * </ol>
-   * @param c the environment provided by the region server
-   * @param s the scanner
-   * @param currentRow The current rowkey which got filtered out
-   * @param offset offset to rowkey
-   * @param length length of rowkey
-   * @param hasMore the 'has more' indication
-   * @return whether more rows are available for the scanner or not
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
-   * Instead use {@link #postScannerFilterRow(ObserverContext, InternalScanner, Cell, boolean)}
-   */
-  @Deprecated
-  default boolean postScannerFilterRow(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final InternalScanner s, final byte[] currentRow, final int offset, final short length,
-      final boolean hasMore) throws IOException {
-    return hasMore;
-  }
-
-  /**
-   * This will be called by the scan flow when the current scanned row is being filtered out by the
-   * filter. The filter may be filtering out the row via any of the below scenarios
-   * <ol>
-   * <li>
-   * <code>boolean filterRowKey(byte [] buffer, int offset, int length)</code> returning true</li>
-   * <li>
-   * <code>boolean filterRow()</code> returning true</li>
-   * <li>
    * <code>default void filterRow(List&lt;KeyValue&gt; kvs)</code> removing all the kvs from
    * the passed List</li>
    * </ol>
@@ -1316,8 +906,8 @@ public interface RegionObserver extends Coprocessor {
    * @param hasMore the 'has more' indication
    * @return whether more rows are available for the scanner or not
    */
-  default boolean postScannerFilterRow(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final InternalScanner s, Cell curRowCell, final boolean hasMore) throws IOException {
+  default boolean postScannerFilterRow(ObserverContext<RegionCoprocessorEnvironment> c,
+      InternalScanner s, Cell curRowCell, boolean hasMore) throws IOException {
     return hasMore;
   }
 
@@ -1331,8 +921,8 @@ public interface RegionObserver extends Coprocessor {
    * @param c the environment provided by the region server
    * @param s the scanner
    */
-  default void preScannerClose(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final InternalScanner s) throws IOException {}
+  default void preScannerClose(ObserverContext<RegionCoprocessorEnvironment> c, InternalScanner s)
+      throws IOException {}
 
   /**
    * Called after the client closes a scanner.
@@ -1342,8 +932,8 @@ public interface RegionObserver extends Coprocessor {
    * @param ctx the environment provided by the region server
    * @param s the scanner
    */
-  default void postScannerClose(final ObserverContext<RegionCoprocessorEnvironment> ctx,
-      final InternalScanner s) throws IOException {}
+  default void postScannerClose(ObserverContext<RegionCoprocessorEnvironment> ctx,
+      InternalScanner s) throws IOException {}
 
   /**
    * Called before replaying WALs for this region.
@@ -1353,7 +943,7 @@ public interface RegionObserver extends Coprocessor {
    * @param info the RegionInfo for this region
    * @param edits the file of recovered edits
    */
-  default void preReplayWALs(final ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
+  default void preReplayWALs(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
       HRegionInfo info, Path edits) throws IOException {}
 
   /**
@@ -1362,7 +952,7 @@ public interface RegionObserver extends Coprocessor {
    * @param info the RegionInfo for this region
    * @param edits the file of recovered edits
    */
-  default void postReplayWALs(final ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
+  default void postReplayWALs(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
       HRegionInfo info, Path edits) throws IOException {}
 
   /**
@@ -1370,7 +960,7 @@ public interface RegionObserver extends Coprocessor {
    * replayed for this region.
    * @param ctx the environment provided by the region server
    */
-  default void preWALRestore(final ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
+  default void preWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
       HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {}
 
   /**
@@ -1378,7 +968,7 @@ public interface RegionObserver extends Coprocessor {
    * replayed for this region.
    * @param ctx the environment provided by the region server
    */
-  default void postWALRestore(final ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
+  default void postWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
       HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {}
 
   /**
@@ -1389,7 +979,7 @@ public interface RegionObserver extends Coprocessor {
    * @param familyPaths pairs of { CF, HFile path } submitted for bulk load. Adding
    * or removing from this list will add or remove HFiles to be bulk loaded.
    */
-  default void preBulkLoadHFile(final ObserverContext<RegionCoprocessorEnvironment> ctx,
+  default void preBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
     List<Pair<byte[], String>> familyPaths) throws IOException {}
 
   /**
@@ -1400,9 +990,8 @@ public interface RegionObserver extends Coprocessor {
    * @param pairs List of pairs of { HFile location in staging dir, HFile path in region dir }
    * Each pair are for the same hfile.
    */
-  default void preCommitStoreFile(final ObserverContext<RegionCoprocessorEnvironment> ctx,
-      final byte[] family, final List<Pair<Path, Path>> pairs) throws IOException {
-  }
+  default void preCommitStoreFile(ObserverContext<RegionCoprocessorEnvironment> ctx, byte[] family,
+      List<Pair<Path, Path>> pairs) throws IOException {}
 
   /**
    * Called after moving bulk loaded hfile to region directory.
@@ -1412,38 +1001,21 @@ public interface RegionObserver extends Coprocessor {
    * @param srcPath Path to file before the move
    * @param dstPath Path to file after the move
    */
-  default void postCommitStoreFile(final ObserverContext<RegionCoprocessorEnvironment> ctx,
-      final byte[] family, Path srcPath, Path dstPath) throws IOException {}
-
-  /**
-   * Called after bulkLoadHFile.
-   *
-   * @param ctx the environment provided by the region server
-   * @param stagingFamilyPaths pairs of { CF, HFile path } submitted for bulk load
-   * @param finalPaths Map of CF to List of file paths for the final loaded files
-   * @param hasLoaded whether the bulkLoad was successful
-   * @return the new value of hasLoaded
-   */
-  default boolean postBulkLoadHFile(final ObserverContext<RegionCoprocessorEnvironment> ctx,
-    List<Pair<byte[], String>> stagingFamilyPaths, Map<byte[], List<Path>> finalPaths,
-    boolean hasLoaded) throws IOException {
-    return postBulkLoadHFile(ctx, stagingFamilyPaths, hasLoaded);
-  }
+  default void postCommitStoreFile(ObserverContext<RegionCoprocessorEnvironment> ctx, byte[] family,
+      Path srcPath, Path dstPath) throws IOException {}
 
   /**
    * Called after bulkLoadHFile.
    *
    * @param ctx the environment provided by the region server
    * @param stagingFamilyPaths pairs of { CF, HFile path } submitted for bulk load
+   * @param finalPaths Map of CF to List of file paths for the loaded files
    * @param hasLoaded whether the bulkLoad was successful
    * @return the new value of hasLoaded
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-17123">HBASE-17123</a>).
-   *             Use {@link #postBulkLoadHFile(ObserverContext, List, Map, boolean)}.
    */
-  @Deprecated
-  default boolean postBulkLoadHFile(final ObserverContext<RegionCoprocessorEnvironment> ctx,
-    List<Pair<byte[], String>> stagingFamilyPaths, boolean hasLoaded) throws IOException {
+  default boolean postBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
+      List<Pair<byte[], String>> stagingFamilyPaths, Map<byte[], List<Path>> finalPaths,
+      boolean hasLoaded) throws IOException {
     return hasLoaded;
   }
 
@@ -1463,10 +1035,9 @@ public interface RegionObserver extends Coprocessor {
    * @return a Reader instance to use instead of the base reader if overriding
    * default behavior, null otherwise
    */
-  default StoreFileReader preStoreFileReaderOpen(
-      final ObserverContext<RegionCoprocessorEnvironment> ctx,
-      final FileSystem fs, final Path p, final FSDataInputStreamWrapper in, long size,
-      final CacheConfig cacheConf, final Reference r, StoreFileReader reader) throws IOException {
+  default StoreFileReader preStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx,
+      FileSystem fs, Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf,
+      Reference r, StoreFileReader reader) throws IOException {
     return reader;
   }
 
@@ -1483,10 +1054,9 @@ public interface RegionObserver extends Coprocessor {
    * @param reader the base reader instance
    * @return The reader to use
    */
-  default StoreFileReader postStoreFileReaderOpen(
-      final ObserverContext<RegionCoprocessorEnvironment> ctx,
-      final FileSystem fs, final Path p, final FSDataInputStreamWrapper in, long size,
-      final CacheConfig cacheConf, final Reference r, StoreFileReader reader) throws IOException {
+  default StoreFileReader postStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx,
+      FileSystem fs, Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf,
+      Reference r, StoreFileReader reader) throws IOException {
     return reader;
   }
 
@@ -1517,8 +1087,8 @@ public interface RegionObserver extends Coprocessor {
    * @return the Delete Tracker
    */
   default DeleteTracker postInstantiateDeleteTracker(
-      final ObserverContext<RegionCoprocessorEnvironment> ctx, DeleteTracker delTracker)
+      ObserverContext<RegionCoprocessorEnvironment> ctx, DeleteTracker delTracker)
       throws IOException {
-        return delTracker;
+    return delTracker;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6e52cd8/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 009d65c..bed2a7a 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
@@ -19,6 +19,9 @@
 
 package org.apache.hadoop.hbase.regionserver;
 
+import com.google.protobuf.Message;
+import com.google.protobuf.Service;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -37,8 +40,8 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
@@ -73,17 +76,15 @@ import org.apache.hadoop.hbase.metrics.MetricRegistry;
 import org.apache.hadoop.hbase.regionserver.Region.Operation;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker;
-import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CoprocessorClassLoader;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-import com.google.protobuf.Message;
-import com.google.protobuf.Service;
 
 /**
  * Implements the coprocessor environment and runtime support for coprocessors
@@ -96,8 +97,8 @@ public class RegionCoprocessorHost
 
   private static final Log LOG = LogFactory.getLog(RegionCoprocessorHost.class);
   // The shared data map
-  private static ReferenceMap sharedDataMap =
-      new ReferenceMap(AbstractReferenceMap.ReferenceStrength.HARD,
+  private static final ReferenceMap<String, ConcurrentMap<String, Object>> SHARED_DATA_MAP =
+      new ReferenceMap<>(AbstractReferenceMap.ReferenceStrength.HARD,
           AbstractReferenceMap.ReferenceStrength.WEAK);
 
   // optimization: no need to call postScannerFilterRow, if no coprocessor implements it
@@ -401,14 +402,11 @@ public class RegionCoprocessorHost
     }
     ConcurrentMap<String, Object> classData;
     // make sure only one thread can add maps
-    synchronized (sharedDataMap) {
+    synchronized (SHARED_DATA_MAP) {
       // as long as at least one RegionEnvironment holds on to its classData it will
       // remain in this map
-      classData = (ConcurrentMap<String, Object>)sharedDataMap.get(implClass.getName());
-      if (classData == null) {
-        classData = new ConcurrentHashMap<>();
-        sharedDataMap.put(implClass.getName(), classData);
-      }
+      classData =
+          SHARED_DATA_MAP.computeIfAbsent(implClass.getName(), k -> new ConcurrentHashMap<>());
     }
     return new RegionEnvironment(instance, priority, seq, conf, region,
         rsServices, classData);
@@ -672,136 +670,7 @@ public class RegionCoprocessorHost
     });
   }
 
-  /**
-   * Invoked just before a split
-   * @throws IOException
-   */
-  @Deprecated
-  public void preSplit(final User user) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
-      @Override
-      public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preSplit(ctx);
-      }
-    });
-  }
-
-  /**
-   * Invoked just before a split
-   * @throws IOException
-   *
-   * Note: the logic moves to Master; it is unused in RS
-   */
-  @Deprecated
-  public void preSplit(final byte[] splitRow, final User user) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
-      @Override
-      public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preSplit(ctx, splitRow);
-      }
-    });
-  }
-
-  /**
-   * Invoked just after a split
-   * @param l the new left-hand daughter region
-   * @param r the new right-hand daughter region
-   * @throws IOException
-   *
-   * Note: the logic moves to Master; it is unused in RS
-   */
-  @Deprecated
-  public void postSplit(final Region l, final Region r, final User user) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
-      @Override
-      public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postSplit(ctx, l, r);
-      }
-    });
-  }
-
-  /**
-  * Note: the logic moves to Master; it is unused in RS
-  */
- @Deprecated
-  public boolean preSplitBeforePONR(final byte[] splitKey,
-      final List<Mutation> metaEntries, final User user) throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
-      @Override
-      public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preSplitBeforePONR(ctx, splitKey, metaEntries);
-      }
-    });
-  }
-
-  /**
-  * Note: the logic moves to Master; it is unused in RS
-  */
-  @Deprecated
-  public void preSplitAfterPONR(final User user) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
-      @Override
-      public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preSplitAfterPONR(ctx);
-      }
-    });
-  }
-
-  /**
-   * Invoked just before the rollback of a failed split is started
-   * @throws IOException
-   *
-  * Note: the logic moves to Master; it is unused in RS
-  */
-  @Deprecated
-  public void preRollBackSplit(final User user) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
-      @Override
-      public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.preRollBackSplit(ctx);
-      }
-    });
-  }
-
-  /**
-   * Invoked just after the rollback of a failed split is done
-   * @throws IOException
-   *
-  * Note: the logic moves to Master; it is unused in RS
-  */
-  @Deprecated
-  public void postRollBackSplit(final User user) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
-      @Override
-      public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postRollBackSplit(ctx);
-      }
-    });
-  }
-
-  /**
-   * Invoked after a split is completed irrespective of a failure or success.
-   * @throws IOException
-   */
-  public void postCompleteSplit() throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new RegionOperation() {
-      @Override
-      public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
-          throws IOException {
-        oserver.postCompleteSplit(ctx);
-      }
-    });
-  }
-
   // RegionObserver support
-
   /**
    * @param get the Get request
    * @return true if default processing should be bypassed
@@ -1272,8 +1141,7 @@ public class RegionCoprocessorHost
 
   /**
    * See
-   * {@link RegionObserver#preStoreScannerOpen(ObserverContext,
-   *    Store, Scan, NavigableSet, KeyValueScanner)}
+   * {@link RegionObserver#preStoreScannerOpen(ObserverContext, Store, Scan, NavigableSet, KeyValueScanner, long)}
    */
   public KeyValueScanner preStoreScannerOpen(final Store store, final Scan scan,
       final NavigableSet<byte[]> targetCols, final long readPt) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6e52cd8/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index ca9accd..931386e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -92,6 +92,7 @@ import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.hadoop.hbase.regionserver.Store;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
@@ -1530,11 +1531,10 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
   }
 
   @Override
-  public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> c,
-      final Store store, final InternalScanner scanner, final ScanType scanType)
-          throws IOException {
+  public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
+      InternalScanner scanner, ScanType scanType, CompactionRequest request) throws IOException {
     requirePermission(getActiveUser(c), "compact", getTableName(c.getEnvironment()), null, null,
-        Action.ADMIN, Action.CREATE);
+      Action.ADMIN, Action.CREATE);
     return scanner;
   }
 
@@ -1896,30 +1896,6 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
   }
 
   @Override
-  public long preIncrementColumnValue(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final byte [] row, final byte [] family, final byte [] qualifier,
-      final long amount, final boolean writeToWAL)
-      throws IOException {
-    // Require WRITE permission to the table, CF, and the KV to be replaced by the
-    // incremented value
-    RegionCoprocessorEnvironment env = c.getEnvironment();
-    Map<byte[],? extends Collection<byte[]>> families = makeFamilyMap(family, qualifier);
-    User user = getActiveUser(c);
-    AuthResult authResult = permissionGranted(OpType.INCREMENT_COLUMN_VALUE, user, env, families,
-        Action.WRITE);
-    if (!authResult.isAllowed() && cellFeaturesEnabled && !compatibleEarlyTermination) {
-      authResult.setAllowed(checkCoveringPermission(user, OpType.INCREMENT_COLUMN_VALUE, env, row,
-        families, HConstants.LATEST_TIMESTAMP, Action.WRITE));
-      authResult.setReason("Covering cell set");
-    }
-    logResult(authResult);
-    if (authorizationEnabled && !authResult.isAllowed()) {
-      throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
-    }
-    return -1;
-  }
-
-  @Override
   public Result preAppend(ObserverContext<RegionCoprocessorEnvironment> c, Append append)
       throws IOException {
     User user = getActiveUser(c);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6e52cd8/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java
index 8dfc8aa..aff3d99 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java
@@ -255,15 +255,8 @@ public class TestAvoidCellReferencesIntoShippedBlocks {
   public static class CompactorRegionObserver implements RegionObserver {
     @Override
     public InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
-        Store store, List<? extends KeyValueScanner> scanners, ScanType scanType,
-        long earliestPutTs, InternalScanner s) throws IOException {
-      return createCompactorScanner(store, scanners, scanType, earliestPutTs);
-    }
-
-    @Override
-    public InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
-        Store store, List<? extends KeyValueScanner> scanners, ScanType scanType,
-        long earliestPutTs, InternalScanner s, CompactionRequest request) throws IOException {
+        Store store, List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs,
+        InternalScanner s, CompactionRequest request, long readPoint) throws IOException {
       return createCompactorScanner(store, scanners, scanType, earliestPutTs);
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6e52cd8/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
index 630c640..bef6f6b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
@@ -19,6 +19,18 @@
 
 package org.apache.hadoop.hbase.coprocessor;
 
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
@@ -43,30 +55,19 @@ import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
 import org.apache.hadoop.hbase.regionserver.Leases;
 import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
-import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.Region.Operation;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileReader;
-import org.apache.hadoop.hbase.wal.WALEdit;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
 
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableSet;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
 
 /**
  * A sample region observer that tests the RegionObserver interface.
@@ -82,8 +83,6 @@ public class SimpleRegionObserver implements RegionObserver {
   final AtomicInteger ctPreFlush = new AtomicInteger(0);
   final AtomicInteger ctPreFlushScannerOpen = new AtomicInteger(0);
   final AtomicInteger ctPostFlush = new AtomicInteger(0);
-  final AtomicInteger ctPreSplit = new AtomicInteger(0);
-  final AtomicInteger ctPostSplit = new AtomicInteger(0);
   final AtomicInteger ctPreCompactSelect = new AtomicInteger(0);
   final AtomicInteger ctPostCompactSelect = new AtomicInteger(0);
   final AtomicInteger ctPreCompactScanner = new AtomicInteger(0);
@@ -124,8 +123,6 @@ public class SimpleRegionObserver implements RegionObserver {
   final AtomicInteger ctPostReplayWALs = new AtomicInteger(0);
   final AtomicInteger ctPreWALRestore = new AtomicInteger(0);
   final AtomicInteger ctPostWALRestore = new AtomicInteger(0);
-  final AtomicInteger ctPreSplitBeforePONR = new AtomicInteger(0);
-  final AtomicInteger ctPreSplitAfterPONR = new AtomicInteger(0);
   final AtomicInteger ctPreStoreFileReaderOpen = new AtomicInteger(0);
   final AtomicInteger ctPostStoreFileReaderOpen = new AtomicInteger(0);
   final AtomicInteger ctPostBatchMutateIndispensably = new AtomicInteger(0);
@@ -184,10 +181,11 @@ public class SimpleRegionObserver implements RegionObserver {
   }
 
   @Override
-  public InternalScanner preFlushScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
-      Store store, List<KeyValueScanner> scanners, InternalScanner s) throws IOException {
+  public InternalScanner preFlushScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
+      Store store, List<KeyValueScanner> scanners, InternalScanner s, long readPoint)
+      throws IOException {
     ctPreFlushScannerOpen.incrementAndGet();
-    return null;
+    return s;
   }
 
   @Override
@@ -204,63 +202,36 @@ public class SimpleRegionObserver implements RegionObserver {
   }
 
   @Override
-  public void preSplit(ObserverContext<RegionCoprocessorEnvironment> c) {
-    ctPreSplit.incrementAndGet();
-  }
-
-  @Override
-  public void preSplitBeforePONR(
-      ObserverContext<RegionCoprocessorEnvironment> ctx, byte[] splitKey,
-      List<Mutation> metaEntries) throws IOException {
-    ctPreSplitBeforePONR.incrementAndGet();
-  }
-
-  @Override
-  public void preSplitAfterPONR(
-      ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException {
-    ctPreSplitAfterPONR.incrementAndGet();
-  }
-
-  @Override
-  public void postSplit(ObserverContext<RegionCoprocessorEnvironment> c, Region l, Region r) {
-    ctPostSplit.incrementAndGet();
-  }
-
-  public boolean wasSplit() {
-    return ctPreSplit.get() > 0 && ctPostSplit.get() > 0;
-  }
-
-  @Override
-  public void preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c,
-      Store store, List<StoreFile> candidates) {
+  public void preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
+      List<StoreFile> candidates, CompactionRequest request) throws IOException {
     ctPreCompactSelect.incrementAndGet();
   }
 
   @Override
-  public void postCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c,
-      Store store, ImmutableList<StoreFile> selected) {
+  public void postCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
+      ImmutableList<StoreFile> selected, CompactionRequest request) {
     ctPostCompactSelect.incrementAndGet();
   }
 
+
   @Override
-  public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> e,
-      Store store, InternalScanner scanner, ScanType scanType) {
+  public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
+      InternalScanner scanner, ScanType scanType, CompactionRequest request) throws IOException {
     ctPreCompact.incrementAndGet();
     return scanner;
   }
 
   @Override
-  public InternalScanner preCompactScannerOpen(
-      final ObserverContext<RegionCoprocessorEnvironment> c,
+  public InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
       Store store, List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs,
-      InternalScanner s) throws IOException {
+      InternalScanner s, CompactionRequest request, long readPoint) throws IOException {
     ctPreCompactScanner.incrementAndGet();
-    return null;
+    return s;
   }
 
   @Override
-  public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e,
-      Store store, StoreFile resultFile) {
+  public void postCompact(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
+      StoreFile resultFile, CompactionRequest request) throws IOException {
     ctPostCompact.incrementAndGet();
   }
 
@@ -277,11 +248,11 @@ public class SimpleRegionObserver implements RegionObserver {
   }
 
   @Override
-  public KeyValueScanner preStoreScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Store store, final Scan scan, final NavigableSet<byte[]> targetCols,
-      final KeyValueScanner s) throws IOException {
+  public KeyValueScanner preStoreScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
+      Store store, Scan scan, NavigableSet<byte[]> targetCols, KeyValueScanner s, long readPt)
+      throws IOException {
     ctPreStoreScannerOpen.incrementAndGet();
-    return null;
+    return s;
   }
 
   @Override
@@ -864,22 +835,6 @@ public class SimpleRegionObserver implements RegionObserver {
     return ctPostFlush.get();
   }
 
-  public int getCtPreSplit() {
-    return ctPreSplit.get();
-  }
-
-  public int getCtPreSplitBeforePONR() {
-    return ctPreSplitBeforePONR.get();
-  }
-
-  public int getCtPreSplitAfterPONR() {
-    return ctPreSplitAfterPONR.get();
-  }
-
-  public int getCtPostSplit() {
-    return ctPostSplit.get();
-  }
-
   public int getCtPreCompactSelect() {
     return ctPreCompactSelect.get();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e6e52cd8/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
index 7f29e0b..97c45ab 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
@@ -61,6 +61,7 @@ import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.Rule;
@@ -193,13 +194,13 @@ public class TestCoprocessorInterface {
     }
     @Override
     public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> e,
-        Store store, InternalScanner scanner, ScanType scanType) {
+        Store store, InternalScanner scanner, ScanType scanType, CompactionRequest request) {
       preCompactCalled = true;
       return scanner;
     }
     @Override
     public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e,
-        Store store, StoreFile resultFile) {
+        Store store, StoreFile resultFile, CompactionRequest request) {
       postCompactCalled = true;
     }
     @Override