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

[1/2] hbase git commit: HBASE-14997 Move compareOp and Comparators out of filter to client package

Repository: hbase
Updated Branches:
  refs/heads/master 5ff04c5e7 -> 6752eba68


http://git-wip-us.apache.org/repos/asf/hbase/blob/6752eba6/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 1fcf7ae..63d79d3 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
@@ -37,6 +37,7 @@ 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.Coprocessor;
 import org.apache.hadoop.hbase.HBaseConfiguration;
@@ -64,7 +65,6 @@ import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.RegionObserver;
 import org.apache.hadoop.hbase.coprocessor.RegionObserver.MutationType;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
 import org.apache.hadoop.hbase.io.Reference;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
@@ -1007,7 +1007,7 @@ public class RegionCoprocessorHost
    * @param row row to check
    * @param family column family
    * @param qualifier column qualifier
-   * @param compareOp the comparison operation
+   * @param op the comparison operation
    * @param comparator the comparator
    * @param put data to put if check succeeds
    * @return true or false to return to client if default processing should
@@ -1015,7 +1015,7 @@ public class RegionCoprocessorHost
    * @throws IOException e
    */
   public Boolean preCheckAndPut(final byte [] row, final byte [] family,
-      final byte [] qualifier, final CompareOp compareOp,
+      final byte [] qualifier, final CompareOperator op,
       final ByteArrayComparable comparator, final Put put)
       throws IOException {
     return execOperationWithResult(true, false,
@@ -1024,7 +1024,7 @@ public class RegionCoprocessorHost
       public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
           throws IOException {
         setResult(oserver.preCheckAndPut(ctx, row, family, qualifier,
-          compareOp, comparator, put, getResult()));
+          op, comparator, put, getResult()));
       }
     });
   }
@@ -1033,7 +1033,7 @@ public class RegionCoprocessorHost
    * @param row row to check
    * @param family column family
    * @param qualifier column qualifier
-   * @param compareOp the comparison operation
+   * @param op the comparison operation
    * @param comparator the comparator
    * @param put data to put if check succeeds
    * @return true or false to return to client if default processing should
@@ -1041,15 +1041,15 @@ public class RegionCoprocessorHost
    * @throws IOException e
    */
   public Boolean preCheckAndPutAfterRowLock(final byte[] row, final byte[] family,
-      final byte[] qualifier, final CompareOp compareOp, final ByteArrayComparable comparator,
-      final Put put) throws IOException {
+                                            final byte[] qualifier, final CompareOperator op, final ByteArrayComparable comparator,
+                                            final Put put) throws IOException {
     return execOperationWithResult(true, false,
         coprocessors.isEmpty() ? null : new RegionOperationWithResult<Boolean>() {
       @Override
       public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
           throws IOException {
         setResult(oserver.preCheckAndPutAfterRowLock(ctx, row, family, qualifier,
-          compareOp, comparator, put, getResult()));
+          op, comparator, put, getResult()));
       }
     });
   }
@@ -1058,13 +1058,13 @@ public class RegionCoprocessorHost
    * @param row row to check
    * @param family column family
    * @param qualifier column qualifier
-   * @param compareOp the comparison operation
+   * @param op the comparison operation
    * @param comparator the comparator
    * @param put data to put if check succeeds
    * @throws IOException e
    */
   public boolean postCheckAndPut(final byte [] row, final byte [] family,
-      final byte [] qualifier, final CompareOp compareOp,
+      final byte [] qualifier, final CompareOperator op,
       final ByteArrayComparable comparator, final Put put,
       boolean result) throws IOException {
     return execOperationWithResult(result,
@@ -1073,7 +1073,7 @@ public class RegionCoprocessorHost
       public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
           throws IOException {
         setResult(oserver.postCheckAndPut(ctx, row, family, qualifier,
-          compareOp, comparator, put, getResult()));
+          op, comparator, put, getResult()));
       }
     });
   }
@@ -1082,7 +1082,7 @@ public class RegionCoprocessorHost
    * @param row row to check
    * @param family column family
    * @param qualifier column qualifier
-   * @param compareOp the comparison operation
+   * @param op the comparison operation
    * @param comparator the comparator
    * @param delete delete to commit if check succeeds
    * @return true or false to return to client if default processing should
@@ -1090,7 +1090,7 @@ public class RegionCoprocessorHost
    * @throws IOException e
    */
   public Boolean preCheckAndDelete(final byte [] row, final byte [] family,
-      final byte [] qualifier, final CompareOp compareOp,
+      final byte [] qualifier, final CompareOperator op,
       final ByteArrayComparable comparator, final Delete delete)
       throws IOException {
     return execOperationWithResult(true, false,
@@ -1099,7 +1099,7 @@ public class RegionCoprocessorHost
       public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
           throws IOException {
         setResult(oserver.preCheckAndDelete(ctx, row, family,
-            qualifier, compareOp, comparator, delete, getResult()));
+            qualifier, op, comparator, delete, getResult()));
       }
     });
   }
@@ -1108,7 +1108,7 @@ public class RegionCoprocessorHost
    * @param row row to check
    * @param family column family
    * @param qualifier column qualifier
-   * @param compareOp the comparison operation
+   * @param op the comparison operation
    * @param comparator the comparator
    * @param delete delete to commit if check succeeds
    * @return true or false to return to client if default processing should
@@ -1116,15 +1116,15 @@ public class RegionCoprocessorHost
    * @throws IOException e
    */
   public Boolean preCheckAndDeleteAfterRowLock(final byte[] row, final byte[] family,
-      final byte[] qualifier, final CompareOp compareOp, final ByteArrayComparable comparator,
-      final Delete delete) throws IOException {
+                                               final byte[] qualifier, final CompareOperator op, final ByteArrayComparable comparator,
+                                               final Delete delete) throws IOException {
     return execOperationWithResult(true, false,
         coprocessors.isEmpty() ? null : new RegionOperationWithResult<Boolean>() {
       @Override
       public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
           throws IOException {
         setResult(oserver.preCheckAndDeleteAfterRowLock(ctx, row,
-              family, qualifier, compareOp, comparator, delete, getResult()));
+              family, qualifier, op, comparator, delete, getResult()));
       }
     });
   }
@@ -1133,13 +1133,13 @@ public class RegionCoprocessorHost
    * @param row row to check
    * @param family column family
    * @param qualifier column qualifier
-   * @param compareOp the comparison operation
+   * @param op the comparison operation
    * @param comparator the comparator
    * @param delete delete to commit if check succeeds
    * @throws IOException e
    */
   public boolean postCheckAndDelete(final byte [] row, final byte [] family,
-      final byte [] qualifier, final CompareOp compareOp,
+      final byte [] qualifier, final CompareOperator op,
       final ByteArrayComparable comparator, final Delete delete,
       boolean result) throws IOException {
     return execOperationWithResult(result,
@@ -1148,7 +1148,7 @@ public class RegionCoprocessorHost
       public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
           throws IOException {
         setResult(oserver.postCheckAndDelete(ctx, row, family,
-            qualifier, compareOp, comparator, delete, getResult()));
+            qualifier, op, comparator, delete, getResult()));
       }
     });
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6752eba6/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 3b7988e..1e63b13 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
@@ -18,20 +18,10 @@
  */
 package org.apache.hadoop.hbase.security.access;
 
-import java.io.IOException;
-import java.net.InetAddress;
-import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.TreeSet;
-
+import com.google.protobuf.Message;
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.Service;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -39,6 +29,7 @@ import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.CompoundConfiguration;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
@@ -80,7 +71,6 @@ import org.apache.hadoop.hbase.coprocessor.RegionObserver;
 import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.RegionServerObserver;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
-import org.apache.hadoop.hbase.filter.CompareFilter;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.io.hfile.HFile;
@@ -110,6 +100,13 @@ import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.security.access.Permission.Action;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ArrayListMultimap;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableSet;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ListMultimap;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.MapMaker;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest;
@@ -123,17 +120,19 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.SimpleMutableByteRange;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ArrayListMultimap;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableSet;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ListMultimap;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.MapMaker;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
-import com.google.protobuf.Message;
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.Service;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
 
 /**
  * Provides basic authorization checks for data access and administrative
@@ -1763,7 +1762,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
   @Override
   public boolean preCheckAndPut(final ObserverContext<RegionCoprocessorEnvironment> c,
       final byte [] row, final byte [] family, final byte [] qualifier,
-      final CompareFilter.CompareOp compareOp,
+      final CompareOperator op,
       final ByteArrayComparable comparator, final Put put,
       final boolean result) throws IOException {
     User user = getActiveUser(c);
@@ -1797,9 +1796,10 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
 
   @Override
   public boolean preCheckAndPutAfterRowLock(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final byte[] row, final byte[] family, final byte[] qualifier,
-      final CompareFilter.CompareOp compareOp, final ByteArrayComparable comparator, final Put put,
-      final boolean result) throws IOException {
+                                            final byte[] row, final byte[] family, final byte[] qualifier,
+                                            final CompareOperator opp, final ByteArrayComparable comparator, final Put put,
+                                            final boolean result)
+  throws IOException {
     if (put.getAttribute(CHECK_COVERING_PERM) != null) {
       // We had failure with table, cf and q perm checks and now giving a chance for cell
       // perm check
@@ -1826,7 +1826,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
   @Override
   public boolean preCheckAndDelete(final ObserverContext<RegionCoprocessorEnvironment> c,
       final byte [] row, final byte [] family, final byte [] qualifier,
-      final CompareFilter.CompareOp compareOp,
+      final CompareOperator op,
       final ByteArrayComparable comparator, final Delete delete,
       final boolean result) throws IOException {
     // An ACL on a delete is useless, we shouldn't allow it
@@ -1856,7 +1856,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
   @Override
   public boolean preCheckAndDeleteAfterRowLock(
       final ObserverContext<RegionCoprocessorEnvironment> c, final byte[] row, final byte[] family,
-      final byte[] qualifier, final CompareFilter.CompareOp compareOp,
+      final byte[] qualifier, final CompareOperator op,
       final ByteArrayComparable comparator, final Delete delete, final boolean result)
       throws IOException {
     if (delete.getAttribute(CHECK_COVERING_PERM) != null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6752eba6/hbase-server/src/main/java/org/apache/hadoop/hbase/util/EncryptionTest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/EncryptionTest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/EncryptionTest.java
index 1efab58..f68dd29 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/EncryptionTest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/EncryptionTest.java
@@ -26,6 +26,7 @@ import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
@@ -34,7 +35,7 @@ import org.apache.hadoop.hbase.io.crypto.Encryption;
 import org.apache.hadoop.hbase.io.crypto.KeyStoreKeyProvider;
 import org.apache.hadoop.hbase.security.EncryptionUtil;
 
-@InterfaceAudience.Public
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
 public class EncryptionTest {
   private static final Log LOG = LogFactory.getLog(EncryptionTest.class);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6752eba6/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCheckAndMutate.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCheckAndMutate.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCheckAndMutate.java
index afc4fb1..5c3114e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCheckAndMutate.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCheckAndMutate.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.filter.CompareFilter;
@@ -31,12 +32,16 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
+import java.io.IOException;
+
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 @Category(MediumTests.class)
 public class TestCheckAndMutate {
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final byte[] ROWKEY = Bytes.toBytes("12345");
+  private static final byte[] FAMILY = Bytes.toBytes("cf");
 
   @Rule
   public TestName name = new TestName();
@@ -57,61 +62,86 @@ public class TestCheckAndMutate {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  @Test
-  public void testCheckAndMutate() throws Throwable {
+  private Table createTable()
+  throws IOException, InterruptedException {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    final byte[] rowKey = Bytes.toBytes("12345");
-    final byte[] family = Bytes.toBytes("cf");
-    Table table = TEST_UTIL.createTable(tableName, family);
+    Table table = TEST_UTIL.createTable(tableName, FAMILY);
     TEST_UTIL.waitTableAvailable(tableName.getName(), 5000);
-    try {
+    return table;
+  }
+
+  private void putOneRow(Table table) throws IOException {
+    Put put = new Put(ROWKEY);
+    put.addColumn(FAMILY, Bytes.toBytes("A"), Bytes.toBytes("a"));
+    put.addColumn(FAMILY, Bytes.toBytes("B"), Bytes.toBytes("b"));
+    put.addColumn(FAMILY, Bytes.toBytes("C"), Bytes.toBytes("c"));
+    table.put(put);
+  }
+
+  private void getOneRowAndAssertAllExist(final Table table) throws IOException {
+    Get get = new Get(ROWKEY);
+    Result result = table.get(get);
+    assertTrue("Column A value should be a",
+      Bytes.toString(result.getValue(FAMILY, Bytes.toBytes("A"))).equals("a"));
+    assertTrue("Column B value should be b",
+      Bytes.toString(result.getValue(FAMILY, Bytes.toBytes("B"))).equals("b"));
+    assertTrue("Column C value should be c",
+      Bytes.toString(result.getValue(FAMILY, Bytes.toBytes("C"))).equals("c"));
+  }
+
+  private void getOneRowAndAssertAllButCExist(final Table table) throws IOException {
+    Get get = new Get(ROWKEY);
+    Result result = table.get(get);
+    assertTrue("Column A value should be a",
+      Bytes.toString(result.getValue(FAMILY, Bytes.toBytes("A"))).equals("a"));
+    assertTrue("Column B value should be b",
+      Bytes.toString(result.getValue(FAMILY, Bytes.toBytes("B"))).equals("b"));
+    assertTrue("Column C should not exist",
+    result.getValue(FAMILY, Bytes.toBytes("C")) == null);
+  }
+
+  private RowMutations makeRowMutationsWithColumnCDeleted() throws IOException {
+    RowMutations rm = new RowMutations(ROWKEY, 2);
+    Put put = new Put(ROWKEY);
+    put.addColumn(FAMILY, Bytes.toBytes("A"), Bytes.toBytes("a"));
+    put.addColumn(FAMILY, Bytes.toBytes("B"), Bytes.toBytes("b"));
+    rm.add(put);
+    Delete del = new Delete(ROWKEY);
+    del.addColumn(FAMILY, Bytes.toBytes("C"));
+    rm.add(del);
+    return rm;
+  }
+
+  private RowMutations getBogusRowMutations() throws IOException {
+    Put p = new Put(ROWKEY);
+    byte[] value = new byte[0];
+    p.addColumn(new byte[]{'b', 'o', 'g', 'u', 's'}, new byte[]{'A'}, value);
+    RowMutations rm = new RowMutations(ROWKEY);
+    rm.add(p);
+    return rm;
+  }
+
+  @Test
+  public void testCheckAndMutate() throws Throwable {
+    try (Table table = createTable()) {
       // put one row
-      Put put = new Put(rowKey);
-      put.addColumn(family, Bytes.toBytes("A"), Bytes.toBytes("a"));
-      put.addColumn(family, Bytes.toBytes("B"), Bytes.toBytes("b"));
-      put.addColumn(family, Bytes.toBytes("C"), Bytes.toBytes("c"));
-      table.put(put);
+      putOneRow(table);
       // get row back and assert the values
-      Get get = new Get(rowKey);
-      Result result = table.get(get);
-      assertTrue("Column A value should be a",
-          Bytes.toString(result.getValue(family, Bytes.toBytes("A"))).equals("a"));
-      assertTrue("Column B value should be b",
-          Bytes.toString(result.getValue(family, Bytes.toBytes("B"))).equals("b"));
-      assertTrue("Column C value should be c",
-          Bytes.toString(result.getValue(family, Bytes.toBytes("C"))).equals("c"));
+      getOneRowAndAssertAllExist(table);
 
       // put the same row again with C column deleted
-      RowMutations rm = new RowMutations(rowKey, 2);
-      put = new Put(rowKey);
-      put.addColumn(family, Bytes.toBytes("A"), Bytes.toBytes("a"));
-      put.addColumn(family, Bytes.toBytes("B"), Bytes.toBytes("b"));
-      rm.add(put);
-      Delete del = new Delete(rowKey);
-      del.addColumn(family, Bytes.toBytes("C"));
-      rm.add(del);
-      boolean res = table.checkAndMutate(rowKey, family, Bytes.toBytes("A"), CompareFilter.CompareOp.EQUAL,
-          Bytes.toBytes("a"), rm);
+      RowMutations rm = makeRowMutationsWithColumnCDeleted();
+      boolean res = table.checkAndMutate(ROWKEY, FAMILY, Bytes.toBytes("A"),
+        CompareFilter.CompareOp.EQUAL, Bytes.toBytes("a"), rm);
       assertTrue(res);
 
       // get row back and assert the values
-      get = new Get(rowKey);
-      result = table.get(get);
-      assertTrue("Column A value should be a",
-          Bytes.toString(result.getValue(family, Bytes.toBytes("A"))).equals("a"));
-      assertTrue("Column B value should be b",
-          Bytes.toString(result.getValue(family, Bytes.toBytes("B"))).equals("b"));
-      assertTrue("Column C should not exist",
-          result.getValue(family, Bytes.toBytes("C")) == null);
+      getOneRowAndAssertAllButCExist(table);
 
       //Test that we get a region level exception
       try {
-        Put p = new Put(rowKey);
-        byte[] value = new byte[0];
-        p.addColumn(new byte[]{'b', 'o', 'g', 'u', 's'}, new byte[]{'A'}, value);
-        rm = new RowMutations(rowKey);
-        rm.add(p);
-        table.checkAndMutate(rowKey, family, Bytes.toBytes("A"), CompareFilter.CompareOp.EQUAL,
+        rm = getBogusRowMutations();
+        table.checkAndMutate(ROWKEY, FAMILY, Bytes.toBytes("A"), CompareFilter.CompareOp.EQUAL,
             Bytes.toBytes("a"), rm);
         fail("Expected NoSuchColumnFamilyException");
       } catch (RetriesExhaustedWithDetailsException e) {
@@ -121,8 +151,39 @@ public class TestCheckAndMutate {
           // expected
         }
       }
-    } finally {
-      table.close();
+    }
+  }
+
+  @Test
+  public void testCheckAndMutateUsingNewComparisonOperatorInstead() throws Throwable {
+    try (Table table = createTable()) {
+      // put one row
+      putOneRow(table);
+      // get row back and assert the values
+      getOneRowAndAssertAllExist(table);
+
+      // put the same row again with C column deleted
+      RowMutations rm = makeRowMutationsWithColumnCDeleted();
+      boolean res = table.checkAndMutate(ROWKEY, FAMILY, Bytes.toBytes("A"),
+        CompareOperator.EQUAL, Bytes.toBytes("a"), rm);
+      assertTrue(res);
+
+      // get row back and assert the values
+      getOneRowAndAssertAllButCExist(table);
+
+      //Test that we get a region level exception
+      try {
+        rm = getBogusRowMutations();
+        table.checkAndMutate(ROWKEY, FAMILY, Bytes.toBytes("A"), CompareOperator.EQUAL,
+          Bytes.toBytes("a"), rm);
+        fail("Expected NoSuchColumnFamilyException");
+      } catch (RetriesExhaustedWithDetailsException e) {
+        try {
+          throw e.getCause(0);
+        } catch (NoSuchColumnFamilyException e1) {
+          // expected
+        }
+      }
     }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/6752eba6/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 5da445a..c03eb54 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,24 +19,11 @@
 
 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 org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
-
-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;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.client.Append;
@@ -49,11 +36,9 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
 import org.apache.hadoop.hbase.io.Reference;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
-import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
 import org.apache.hadoop.hbase.regionserver.Leases;
@@ -66,10 +51,23 @@ 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.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 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;
+
 /**
  * A sample region observer that tests the RegionObserver interface.
  * It works with TestRegionObserverInterface to provide the test case.
@@ -542,15 +540,15 @@ public class SimpleRegionObserver implements RegionObserver {
 
   @Override
   public boolean preCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> e, byte[] row,
-      byte[] family, byte[] qualifier, CompareOp compareOp, ByteArrayComparable comparator,
-      Put put, boolean result) throws IOException {
+                                byte[] family, byte[] qualifier, CompareOperator compareOp, ByteArrayComparable comparator,
+                                Put put, boolean result) throws IOException {
     ctPreCheckAndPut.incrementAndGet();
     return true;
   }
 
   @Override
   public boolean preCheckAndPutAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> e,
-      byte[] row, byte[] family, byte[] qualifier, CompareOp compareOp,
+      byte[] row, byte[] family, byte[] qualifier, CompareOperator compareOp,
       ByteArrayComparable comparator, Put put, boolean result) throws IOException {
     ctPreCheckAndPutAfterRowLock.incrementAndGet();
     return true;
@@ -558,23 +556,23 @@ public class SimpleRegionObserver implements RegionObserver {
 
   @Override
   public boolean postCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> e, byte[] row,
-      byte[] family, byte[] qualifier, CompareOp compareOp, ByteArrayComparable comparator,
-      Put put, boolean result) throws IOException {
+                                 byte[] family, byte[] qualifier, CompareOperator compareOp, ByteArrayComparable comparator,
+                                 Put put, boolean result) throws IOException {
     ctPostCheckAndPut.incrementAndGet();
     return true;
   }
 
   @Override
   public boolean preCheckAndDelete(ObserverContext<RegionCoprocessorEnvironment> e, byte[] row,
-      byte[] family, byte[] qualifier, CompareOp compareOp, ByteArrayComparable comparator,
-      Delete delete, boolean result) throws IOException {
+                                   byte[] family, byte[] qualifier, CompareOperator compareOp, ByteArrayComparable comparator,
+                                   Delete delete, boolean result) throws IOException {
     ctPreCheckAndDelete.incrementAndGet();
     return true;
   }
 
   @Override
   public boolean preCheckAndDeleteAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> e,
-      byte[] row, byte[] family, byte[] qualifier, CompareOp compareOp,
+      byte[] row, byte[] family, byte[] qualifier, CompareOperator compareOp,
       ByteArrayComparable comparator, Delete delete, boolean result) throws IOException {
     ctPreCheckAndDeleteAfterRowLock.incrementAndGet();
     return true;
@@ -582,8 +580,8 @@ public class SimpleRegionObserver implements RegionObserver {
 
   @Override
   public boolean postCheckAndDelete(ObserverContext<RegionCoprocessorEnvironment> e, byte[] row,
-      byte[] family, byte[] qualifier, CompareOp compareOp, ByteArrayComparable comparator,
-      Delete delete, boolean result) throws IOException {
+                                    byte[] family, byte[] qualifier, CompareOperator compareOp, ByteArrayComparable comparator,
+                                    Delete delete, boolean result) throws IOException {
     ctPostCheckAndDelete.incrementAndGet();
     return true;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6752eba6/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/RegionAsTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/RegionAsTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/RegionAsTable.java
index 89885b6..8f34b19 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/RegionAsTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/RegionAsTable.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
@@ -225,6 +226,13 @@ public class RegionAsTable implements Table {
   }
 
   @Override
+  public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
+                             CompareOperator compareOp, byte[] value, Put put)
+  throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
   public void delete(Delete delete) throws IOException {
     this.region.delete(delete);
   }
@@ -249,6 +257,13 @@ public class RegionAsTable implements Table {
   }
 
   @Override
+  public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
+                                CompareOperator compareOp, byte[] value, Delete delete)
+  throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
   public void mutateRow(RowMutations rm) throws IOException {
     throw new UnsupportedOperationException();
   }
@@ -325,6 +340,14 @@ public class RegionAsTable implements Table {
   }
 
   @Override
+  public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier,
+                                CompareOperator compareOp,
+                                byte[] value, RowMutations mutation)
+  throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
   public void setOperationTimeout(int operationTimeout) {
     throw new UnsupportedOperationException();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6752eba6/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
index 1936c98..2427cad 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
@@ -16,21 +16,6 @@
  * limitations under the License.
  */
 package org.apache.hadoop.hbase.regionserver;
-import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
-import static org.apache.hadoop.hbase.HBaseTestingUtility.fam2;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Random;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -39,6 +24,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -61,7 +47,6 @@ import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -75,6 +60,21 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
+import static org.apache.hadoop.hbase.HBaseTestingUtility.fam2;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 /**
  * Testing of HRegion.incrementColumnValue, HRegion.increment,
  * and HRegion.append
@@ -653,7 +653,7 @@ public class TestAtomicOperation {
       }
       testStep = TestStep.CHECKANDPUT_STARTED;
       region.checkAndMutate(Bytes.toBytes("r1"), Bytes.toBytes(family), Bytes.toBytes("q1"),
-        CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("10")), put, true);
+        CompareOperator.EQUAL, new BinaryComparator(Bytes.toBytes("10")), put, true);
       testStep = TestStep.CHECKANDPUT_COMPLETED;
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6752eba6/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index d567513..b495ad8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -18,54 +18,6 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import static org.apache.hadoop.hbase.HBaseTestingUtility.COLUMNS;
-import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
-import static org.apache.hadoop.hbase.HBaseTestingUtility.fam2;
-import static org.apache.hadoop.hbase.HBaseTestingUtility.fam3;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyBoolean;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.TreeMap;
-import java.util.UUID;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -78,6 +30,7 @@ import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
 import org.apache.hadoop.hbase.DroppedSnapshotException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
@@ -140,6 +93,8 @@ import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
@@ -181,6 +136,51 @@ import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hbase.HBaseTestingUtility.COLUMNS;
+import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
+import static org.apache.hadoop.hbase.HBaseTestingUtility.fam2;
+import static org.apache.hadoop.hbase.HBaseTestingUtility.fam3;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
 /**
  * Basic stand-alone testing of HRegion.  No clusters!
  *
@@ -1713,7 +1713,7 @@ public class TestHRegion {
       put.addColumn(fam1, qf1, emptyVal);
 
       // checkAndPut with empty value
-      boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(
+      boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.EQUAL, new BinaryComparator(
           emptyVal), put, true);
       assertTrue(res);
 
@@ -1722,25 +1722,25 @@ public class TestHRegion {
       put.addColumn(fam1, qf1, val1);
 
       // checkAndPut with correct value
-      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(emptyVal),
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.EQUAL, new BinaryComparator(emptyVal),
           put, true);
       assertTrue(res);
 
       // not empty anymore
-      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(emptyVal),
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.EQUAL, new BinaryComparator(emptyVal),
           put, true);
       assertFalse(res);
 
       Delete delete = new Delete(row1);
       delete.addColumn(fam1, qf1);
-      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(emptyVal),
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.EQUAL, new BinaryComparator(emptyVal),
           delete, true);
       assertFalse(res);
 
       put = new Put(row1);
       put.addColumn(fam1, qf1, val2);
       // checkAndPut with correct value
-      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(val1),
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.EQUAL, new BinaryComparator(val1),
           put, true);
       assertTrue(res);
 
@@ -1748,12 +1748,12 @@ public class TestHRegion {
       delete = new Delete(row1);
       delete.addColumn(fam1, qf1);
       delete.addColumn(fam1, qf1);
-      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(val2),
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.EQUAL, new BinaryComparator(val2),
           delete, true);
       assertTrue(res);
 
       delete = new Delete(row1);
-      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(emptyVal),
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.EQUAL, new BinaryComparator(emptyVal),
           delete, true);
       assertTrue(res);
 
@@ -1762,7 +1762,7 @@ public class TestHRegion {
       put.addColumn(fam1, qf1, val1);
 
       res = region
-          .checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new NullComparator(), put, true);
+          .checkAndMutate(row1, fam1, qf1, CompareOperator.EQUAL, new NullComparator(), put, true);
       assertTrue(res);
     } finally {
       HBaseTestingUtility.closeRegionAndWAL(this.region);
@@ -1787,14 +1787,14 @@ public class TestHRegion {
       region.put(put);
 
       // checkAndPut with wrong value
-      boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(
+      boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.EQUAL, new BinaryComparator(
           val2), put, true);
       assertEquals(false, res);
 
       // checkAndDelete with wrong value
       Delete delete = new Delete(row1);
       delete.addFamily(fam1);
-      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(val2),
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.EQUAL, new BinaryComparator(val2),
           put, true);
       assertEquals(false, res);
     } finally {
@@ -1819,14 +1819,14 @@ public class TestHRegion {
       region.put(put);
 
       // checkAndPut with correct value
-      boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(
+      boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.EQUAL, new BinaryComparator(
           val1), put, true);
       assertEquals(true, res);
 
       // checkAndDelete with correct value
       Delete delete = new Delete(row1);
       delete.addColumn(fam1, qf1);
-      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(val1),
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.EQUAL, new BinaryComparator(val1),
           delete, true);
       assertEquals(true, res);
     } finally {
@@ -1854,12 +1854,12 @@ public class TestHRegion {
       region.put(put);
 
       // Test CompareOp.LESS: original = val3, compare with val3, fail
-      boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.LESS,
+      boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.LESS,
           new BinaryComparator(val3), put, true);
       assertEquals(false, res);
 
       // Test CompareOp.LESS: original = val3, compare with val4, fail
-      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.LESS,
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.LESS,
           new BinaryComparator(val4), put, true);
       assertEquals(false, res);
 
@@ -1867,18 +1867,18 @@ public class TestHRegion {
       // succeed (now value = val2)
       put = new Put(row1);
       put.addColumn(fam1, qf1, val2);
-      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.LESS,
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.LESS,
           new BinaryComparator(val2), put, true);
       assertEquals(true, res);
 
       // Test CompareOp.LESS_OR_EQUAL: original = val2, compare with val3, fail
-      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.LESS_OR_EQUAL,
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.LESS_OR_EQUAL,
           new BinaryComparator(val3), put, true);
       assertEquals(false, res);
 
       // Test CompareOp.LESS_OR_EQUAL: original = val2, compare with val2,
       // succeed (value still = val2)
-      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.LESS_OR_EQUAL,
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.LESS_OR_EQUAL,
           new BinaryComparator(val2), put, true);
       assertEquals(true, res);
 
@@ -1886,17 +1886,17 @@ public class TestHRegion {
       // succeed (now value = val3)
       put = new Put(row1);
       put.addColumn(fam1, qf1, val3);
-      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.LESS_OR_EQUAL,
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.LESS_OR_EQUAL,
           new BinaryComparator(val1), put, true);
       assertEquals(true, res);
 
       // Test CompareOp.GREATER: original = val3, compare with val3, fail
-      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.GREATER,
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.GREATER,
           new BinaryComparator(val3), put, true);
       assertEquals(false, res);
 
       // Test CompareOp.GREATER: original = val3, compare with val2, fail
-      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.GREATER,
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.GREATER,
           new BinaryComparator(val2), put, true);
       assertEquals(false, res);
 
@@ -1904,23 +1904,23 @@ public class TestHRegion {
       // succeed (now value = val2)
       put = new Put(row1);
       put.addColumn(fam1, qf1, val2);
-      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.GREATER,
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.GREATER,
           new BinaryComparator(val4), put, true);
       assertEquals(true, res);
 
       // Test CompareOp.GREATER_OR_EQUAL: original = val2, compare with val1, fail
-      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.GREATER_OR_EQUAL,
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.GREATER_OR_EQUAL,
           new BinaryComparator(val1), put, true);
       assertEquals(false, res);
 
       // Test CompareOp.GREATER_OR_EQUAL: original = val2, compare with val2,
       // succeed (value still = val2)
-      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.GREATER_OR_EQUAL,
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.GREATER_OR_EQUAL,
           new BinaryComparator(val2), put, true);
       assertEquals(true, res);
 
       // Test CompareOp.GREATER_OR_EQUAL: original = val2, compare with val3, succeed
-      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.GREATER_OR_EQUAL,
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.GREATER_OR_EQUAL,
           new BinaryComparator(val3), put, true);
       assertEquals(true, res);
     } finally {
@@ -1955,7 +1955,7 @@ public class TestHRegion {
       put.add(kv);
 
       // checkAndPut with wrong value
-      boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(
+      boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.EQUAL, new BinaryComparator(
           val1), put, true);
       assertEquals(true, res);
 
@@ -1982,7 +1982,7 @@ public class TestHRegion {
       Put put = new Put(row2);
       put.addColumn(fam1, qual1, value1);
       try {
-        region.checkAndMutate(row, fam1, qual1, CompareOp.EQUAL,
+        region.checkAndMutate(row, fam1, qual1, CompareOperator.EQUAL,
             new BinaryComparator(value2), put, false);
         fail();
       } catch (org.apache.hadoop.hbase.DoNotRetryIOException expected) {
@@ -2031,7 +2031,7 @@ public class TestHRegion {
       delete.addColumn(fam1, qf1);
       delete.addColumn(fam2, qf1);
       delete.addColumn(fam1, qf3);
-      boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(
+      boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.EQUAL, new BinaryComparator(
           val2), delete, true);
       assertEquals(true, res);
 
@@ -2047,7 +2047,7 @@ public class TestHRegion {
       // Family delete
       delete = new Delete(row1);
       delete.addFamily(fam2);
-      res = region.checkAndMutate(row1, fam2, qf1, CompareOp.EQUAL, new BinaryComparator(emptyVal),
+      res = region.checkAndMutate(row1, fam2, qf1, CompareOperator.EQUAL, new BinaryComparator(emptyVal),
           delete, true);
       assertEquals(true, res);
 
@@ -2058,7 +2058,7 @@ public class TestHRegion {
 
       // Row delete
       delete = new Delete(row1);
-      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(val1),
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOperator.EQUAL, new BinaryComparator(val1),
           delete, true);
       assertEquals(true, res);
       get = new Get(row1);
@@ -6279,7 +6279,7 @@ public class TestHRegion {
     p = new Put(row);
     p.setDurability(Durability.SKIP_WAL);
     p.addColumn(fam1, qual1, qual2);
-    region.checkAndMutate(row, fam1, qual1, CompareOp.EQUAL, new BinaryComparator(qual1), p, false);
+    region.checkAndMutate(row, fam1, qual1, CompareOperator.EQUAL, new BinaryComparator(qual1), p, false);
     result = region.get(new Get(row));
     c = result.getColumnLatestCell(fam1, qual1);
     assertEquals(c.getTimestamp(), 10L);
@@ -6373,7 +6373,7 @@ public class TestHRegion {
     p.addColumn(fam1, qual1, qual2);
     RowMutations rm = new RowMutations(row);
     rm.add(p);
-    assertTrue(region.checkAndRowMutate(row, fam1, qual1, CompareOp.EQUAL,
+    assertTrue(region.checkAndRowMutate(row, fam1, qual1, CompareOperator.EQUAL,
         new BinaryComparator(qual1), rm, false));
     result = region.get(new Get(row));
     c = result.getColumnLatestCell(fam1, qual1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/6752eba6/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java
index 180faec..0556af7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Append;
@@ -55,7 +56,6 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
-import org.apache.hadoop.hbase.filter.CompareFilter;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
 import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
@@ -941,7 +941,7 @@ public class TestWithDisabledAuthorization extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preCheckAndPut(ObserverContext.createAndPrepare(RCP_ENV, null),
-          TEST_ROW, TEST_FAMILY, TEST_Q1, CompareFilter.CompareOp.EQUAL,
+          TEST_ROW, TEST_FAMILY, TEST_Q1, CompareOperator.EQUAL,
           new BinaryComparator("foo".getBytes()), new Put(TEST_ROW), true);
         return null;
       }
@@ -952,7 +952,7 @@ public class TestWithDisabledAuthorization extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preCheckAndDelete(ObserverContext.createAndPrepare(RCP_ENV, null),
-          TEST_ROW, TEST_FAMILY, TEST_Q1, CompareFilter.CompareOp.EQUAL,
+          TEST_ROW, TEST_FAMILY, TEST_Q1, CompareOperator.EQUAL,
           new BinaryComparator("foo".getBytes()), new Delete(TEST_ROW), true);
         return null;
       }


[2/2] hbase git commit: HBASE-14997 Move compareOp and Comparators out of filter to client package

Posted by st...@apache.org.
HBASE-14997 Move compareOp and Comparators out of filter to client package

Add a new generic CompareOperator enum and use it instead of the filter
CompareOp enum everywhere BUT inside CompareFilter.


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

Branch: refs/heads/master
Commit: 6752eba68f69e5d2d279b2b9433e584e1cc9b58b
Parents: 5ff04c5
Author: Apekshit Sharma <ap...@apache.org>
Authored: Tue Aug 29 10:27:21 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Thu Sep 7 10:03:45 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/client/AsyncTableBase.java     |  14 +-
 .../hadoop/hbase/client/AsyncTableImpl.java     |  14 +-
 .../org/apache/hadoop/hbase/client/HTable.java  | 227 ++++++++++++-------
 .../org/apache/hadoop/hbase/client/Query.java   |   3 +
 .../hadoop/hbase/client/RawAsyncTableImpl.java  |  14 +-
 .../org/apache/hadoop/hbase/client/Table.java   |  76 +++++++
 .../hadoop/hbase/filter/CompareFilter.java      |  11 +-
 .../apache/hadoop/hbase/CompareOperator.java    |  44 ++++
 .../TableBasedReplicationQueuesImpl.java        |   5 +-
 .../hadoop/hbase/rest/client/RemoteHTable.java  |  49 ++--
 .../hadoop/hbase/client/HTableWrapper.java      |  18 ++
 .../hbase/coprocessor/RegionObserver.java       |  58 ++---
 .../hadoop/hbase/regionserver/HRegion.java      | 150 ++++++------
 .../hbase/regionserver/RSRpcServices.java       |  99 ++++----
 .../hadoop/hbase/regionserver/Region.java       |  17 +-
 .../regionserver/RegionCoprocessorHost.java     |  42 ++--
 .../hbase/security/access/AccessController.java |  64 +++---
 .../hadoop/hbase/util/EncryptionTest.java       |   3 +-
 .../hadoop/hbase/client/TestCheckAndMutate.java | 151 ++++++++----
 .../hbase/coprocessor/SimpleRegionObserver.java |  50 ++--
 .../hbase/regionserver/RegionAsTable.java       |  23 ++
 .../hbase/regionserver/TestAtomicOperation.java |  34 +--
 .../hadoop/hbase/regionserver/TestHRegion.java  | 158 ++++++-------
 .../access/TestWithDisabledAuthorization.java   |   6 +-
 24 files changed, 813 insertions(+), 517 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6752eba6/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBase.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBase.java
index d525086..5c83cd7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBase.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBase.java
@@ -21,6 +21,7 @@ import static java.util.stream.Collectors.toList;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.allOf;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.toCheckExistenceOnly;
 
+import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
 
 import java.util.List;
@@ -30,7 +31,6 @@ import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -204,7 +204,7 @@ public interface AsyncTableBase {
    */
   default CompletableFuture<Boolean> checkAndPut(byte[] row, byte[] family, byte[] qualifier,
       byte[] value, Put put) {
-    return checkAndPut(row, family, qualifier, CompareOp.EQUAL, value, put);
+    return checkAndPut(row, family, qualifier, CompareOperator.EQUAL, value, put);
   }
 
   /**
@@ -221,7 +221,7 @@ public interface AsyncTableBase {
    *         a {@link CompletableFuture}.
    */
   CompletableFuture<Boolean> checkAndPut(byte[] row, byte[] family, byte[] qualifier,
-      CompareOp compareOp, byte[] value, Put put);
+                                         CompareOperator compareOp, byte[] value, Put put);
 
   /**
    * Atomically checks if a row/family/qualifier value equals to the expected value. If it does, it
@@ -237,7 +237,7 @@ public interface AsyncTableBase {
    */
   default CompletableFuture<Boolean> checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
       byte[] value, Delete delete) {
-    return checkAndDelete(row, family, qualifier, CompareOp.EQUAL, value, delete);
+    return checkAndDelete(row, family, qualifier, CompareOperator.EQUAL, value, delete);
   }
 
   /**
@@ -254,7 +254,7 @@ public interface AsyncTableBase {
    *         by a {@link CompletableFuture}.
    */
   CompletableFuture<Boolean> checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
-      CompareOp compareOp, byte[] value, Delete delete);
+                                            CompareOperator compareOp, byte[] value, Delete delete);
 
   /**
    * Performs multiple mutations atomically on a single row. Currently {@link Put} and
@@ -278,7 +278,7 @@ public interface AsyncTableBase {
    */
   default CompletableFuture<Boolean> checkAndMutate(byte[] row, byte[] family, byte[] qualifier,
       byte[] value, RowMutations mutation) {
-    return checkAndMutate(row, family, qualifier, CompareOp.EQUAL, value, mutation);
+    return checkAndMutate(row, family, qualifier, CompareOperator.EQUAL, value, mutation);
   }
 
   /**
@@ -295,7 +295,7 @@ public interface AsyncTableBase {
    *         a {@link CompletableFuture}.
    */
   CompletableFuture<Boolean> checkAndMutate(byte[] row, byte[] family, byte[] qualifier,
-      CompareOp compareOp, byte[] value, RowMutations mutation);
+                                            CompareOperator compareOp, byte[] value, RowMutations mutation);
 
   /**
    * Return all the results that match the given scan object.

http://git-wip-us.apache.org/repos/asf/hbase/blob/6752eba6/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableImpl.java
index 29c0698..addcbc7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableImpl.java
@@ -25,9 +25,9 @@ import java.util.concurrent.TimeUnit;
 import static java.util.stream.Collectors.*;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 
 /**
@@ -122,14 +122,14 @@ class AsyncTableImpl implements AsyncTable {
 
   @Override
   public CompletableFuture<Boolean> checkAndPut(byte[] row, byte[] family, byte[] qualifier,
-      CompareOp compareOp, byte[] value, Put put) {
-    return wrap(rawTable.checkAndPut(row, family, qualifier, compareOp, value, put));
+                                                CompareOperator op, byte[] value, Put put) {
+    return wrap(rawTable.checkAndPut(row, family, qualifier, op, value, put));
   }
 
   @Override
   public CompletableFuture<Boolean> checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
-      CompareOp compareOp, byte[] value, Delete delete) {
-    return wrap(rawTable.checkAndDelete(row, family, qualifier, compareOp, value, delete));
+                                                   CompareOperator op, byte[] value, Delete delete) {
+    return wrap(rawTable.checkAndDelete(row, family, qualifier, op, value, delete));
   }
 
   @Override
@@ -139,8 +139,8 @@ class AsyncTableImpl implements AsyncTable {
 
   @Override
   public CompletableFuture<Boolean> checkAndMutate(byte[] row, byte[] family, byte[] qualifier,
-      CompareOp compareOp, byte[] value, RowMutations mutation) {
-    return wrap(rawTable.checkAndMutate(row, family, qualifier, compareOp, value, mutation));
+                                                   CompareOperator op, byte[] value, RowMutations mutation) {
+    return wrap(rawTable.checkAndMutate(row, family, qualifier, op, value, mutation));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/6752eba6/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
index b5c2f92..0ca26f0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
@@ -18,35 +18,19 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import static org.apache.hadoop.hbase.client.ConnectionUtils.checkHasFamilies;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 // DO NOT MAKE USE OF THESE IMPORTS! THEY ARE HERE FOR COPROCESSOR ENDPOINTS ONLY.
 // Internally, we use shaded protobuf. This below are part of our public API.
+//SEE ABOVE NOTE!
 import com.google.protobuf.Descriptors;
 import com.google.protobuf.Message;
 import com.google.protobuf.Service;
 import com.google.protobuf.ServiceException;
 
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -60,7 +44,7 @@ import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-//SEE ABOVE NOTE!
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
@@ -75,6 +59,23 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.hadoop.hbase.util.Threads;
 
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.hbase.client.ConnectionUtils.checkHasFamilies;
+
 /**
  * An implementation of {@link Table}. Used to communicate with a single HBase table.
  * Lightweight. Get as needed and just close when done.
@@ -746,58 +747,74 @@ public class HTable implements Table {
     return checkAndPut(row, family, qualifier, CompareOp.EQUAL, value, put);
   }
 
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public boolean checkAndPut(final byte [] row, final byte [] family,
-      final byte [] qualifier, final CompareOp compareOp, final byte [] value,
-      final Put put)
+  private boolean doCheckAndPut(final byte [] row, final byte [] family,
+                                final byte [] qualifier, final String opName,
+                                final byte [] value, final Put put)
   throws IOException {
     ClientServiceCallable<Boolean> callable =
-        new ClientServiceCallable<Boolean>(this.connection, getName(), row,
-            this.rpcControllerFactory.newController(), put.getPriority()) {
+    new ClientServiceCallable<Boolean>(this.connection, getName(), row,
+    this.rpcControllerFactory.newController(), put.getPriority()) {
       @Override
       protected Boolean rpcCall() throws Exception {
-        CompareType compareType = CompareType.valueOf(compareOp.name());
+        CompareType compareType = CompareType.valueOf(opName);
         MutateRequest request = RequestConverter.buildMutateRequest(
-          getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
-          new BinaryComparator(value), compareType, put);
+        getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
+        new BinaryComparator(value), compareType, put);
         MutateResponse response = doMutate(request);
         return Boolean.valueOf(response.getProcessed());
       }
     };
     return rpcCallerFactory.<Boolean> newCaller(this.writeRpcTimeout).
-        callWithRetries(callable, this.operationTimeout);
+    callWithRetries(callable, this.operationTimeout);
   }
 
   /**
    * {@inheritDoc}
    */
   @Override
-  public boolean checkAndDelete(final byte [] row, final byte [] family, final byte [] qualifier,
-      final byte [] value, final Delete delete) throws IOException {
-    return checkAndDelete(row, family, qualifier, CompareOp.EQUAL, value, delete);
+  public boolean checkAndPut(final byte [] row, final byte [] family,
+      final byte [] qualifier, final CompareOp compareOp, final byte [] value,
+      final Put put)
+  throws IOException {
+    return doCheckAndPut(row, family, qualifier, compareOp.name(), value, put);
   }
 
   /**
    * {@inheritDoc}
    */
   @Override
-  public boolean checkAndDelete(final byte [] row, final byte [] family,
-      final byte [] qualifier, final CompareOp compareOp, final byte [] value,
-      final Delete delete)
+  public boolean checkAndPut(final byte [] row, final byte [] family,
+                             final byte [] qualifier, final CompareOperator op,
+                             final byte [] value, final Put put)
+  throws IOException {
+    // The name of the operators in CompareOperator are intentionally those of the
+    // operators in the filter's CompareOp enum.
+    return doCheckAndPut(row, family, qualifier, op.name(), value, put);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public boolean checkAndDelete(final byte [] row, final byte [] family, final byte [] qualifier,
+      final byte [] value, final Delete delete) throws IOException {
+    return checkAndDelete(row, family, qualifier, CompareOp.EQUAL, value, delete);
+  }
+
+  private boolean doCheckAndDelete(final byte [] row, final byte [] family,
+                                   final byte [] qualifier, final String opName,
+                                   final byte [] value, final Delete delete)
   throws IOException {
     CancellableRegionServerCallable<SingleResponse> callable =
-        new CancellableRegionServerCallable<SingleResponse>(
-            this.connection, getName(), row, this.rpcControllerFactory.newController(),
-            writeRpcTimeout, new RetryingTimeTracker().start(), delete.getPriority()) {
+    new CancellableRegionServerCallable<SingleResponse>(
+    this.connection, getName(), row, this.rpcControllerFactory.newController(),
+    writeRpcTimeout, new RetryingTimeTracker().start(), delete.getPriority()) {
       @Override
       protected SingleResponse rpcCall() throws Exception {
-        CompareType compareType = CompareType.valueOf(compareOp.name());
+        CompareType compareType = CompareType.valueOf(opName);
         MutateRequest request = RequestConverter.buildMutateRequest(
-          getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
-          new BinaryComparator(value), compareType, delete);
+        getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
+        new BinaryComparator(value), compareType, delete);
         MutateResponse response = doMutate(request);
         return ResponseConverter.getResult(request, response, getRpcControllerCellScanner());
       }
@@ -805,16 +822,16 @@ public class HTable implements Table {
     List<Delete> rows = Collections.singletonList(delete);
     Object[] results = new Object[1];
     AsyncProcessTask task = AsyncProcessTask.newBuilder()
-            .setPool(pool)
-            .setTableName(tableName)
-            .setRowAccess(rows)
-            .setCallable(callable)
-            // TODO any better timeout?
-            .setRpcTimeout(Math.max(readRpcTimeout, writeRpcTimeout))
-            .setOperationTimeout(operationTimeout)
-            .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL)
-            .setResults(results)
-            .build();
+    .setPool(pool)
+    .setTableName(tableName)
+    .setRowAccess(rows)
+    .setCallable(callable)
+    // TODO any better timeout?
+    .setRpcTimeout(Math.max(readRpcTimeout, writeRpcTimeout))
+    .setOperationTimeout(operationTimeout)
+    .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL)
+    .setResults(results)
+    .build();
     AsyncRequestFuture ars = multiAp.submit(task);
     ars.waitUntilDone();
     if (ars.hasError()) {
@@ -827,31 +844,49 @@ public class HTable implements Table {
    * {@inheritDoc}
    */
   @Override
-  public boolean checkAndMutate(final byte [] row, final byte [] family, final byte [] qualifier,
-    final CompareOp compareOp, final byte [] value, final RowMutations rm)
-    throws IOException {
+  public boolean checkAndDelete(final byte [] row, final byte [] family,
+      final byte [] qualifier, final CompareOp compareOp, final byte [] value,
+      final Delete delete)
+  throws IOException {
+    return doCheckAndDelete(row, family, qualifier, compareOp.name(), value, delete);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public boolean checkAndDelete(final byte [] row, final byte [] family,
+                                final byte [] qualifier, final CompareOperator op,
+                                final byte [] value, final Delete delete)
+  throws IOException {
+    return doCheckAndDelete(row, family, qualifier, op.name(), value, delete);
+  }
+
+  private boolean doCheckAndMutate(final byte [] row, final byte [] family, final byte [] qualifier,
+                                final String opName, final byte [] value, final RowMutations rm)
+  throws IOException {
     CancellableRegionServerCallable<MultiResponse> callable =
-      new CancellableRegionServerCallable<MultiResponse>(connection, getName(), rm.getRow(),
-        rpcControllerFactory.newController(), writeRpcTimeout, new RetryingTimeTracker().start(), rm.getMaxPriority()) {
-        @Override
-        protected MultiResponse rpcCall() throws Exception {
-          CompareType compareType = CompareType.valueOf(compareOp.name());
-          MultiRequest request = RequestConverter.buildMutateRequest(
-            getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
-            new BinaryComparator(value), compareType, rm);
-          ClientProtos.MultiResponse response = doMulti(request);
-          ClientProtos.RegionActionResult res = response.getRegionActionResultList().get(0);
-          if (res.hasException()) {
-            Throwable ex = ProtobufUtil.toException(res.getException());
-            if (ex instanceof IOException) {
-              throw (IOException)ex;
-            }
-            throw new IOException("Failed to checkAndMutate row: "+
-              Bytes.toStringBinary(rm.getRow()), ex);
+    new CancellableRegionServerCallable<MultiResponse>(connection, getName(), rm.getRow(),
+    rpcControllerFactory.newController(), writeRpcTimeout, new RetryingTimeTracker().start(), rm.getMaxPriority()) {
+      @Override
+      protected MultiResponse rpcCall() throws Exception {
+        CompareType compareType = CompareType.valueOf(opName);
+        MultiRequest request = RequestConverter.buildMutateRequest(
+        getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
+        new BinaryComparator(value), compareType, rm);
+        ClientProtos.MultiResponse response = doMulti(request);
+        ClientProtos.RegionActionResult res = response.getRegionActionResultList().get(0);
+        if (res.hasException()) {
+          Throwable ex = ProtobufUtil.toException(res.getException());
+          if (ex instanceof IOException) {
+            throw (IOException)ex;
           }
-          return ResponseConverter.getResults(request, response, getRpcControllerCellScanner());
+          throw new IOException("Failed to checkAndMutate row: "+
+          Bytes.toStringBinary(rm.getRow()), ex);
         }
-      };
+        return ResponseConverter.getResults(request, response, getRpcControllerCellScanner());
+      }
+    };
 
     /**
      *  Currently, we use one array to store 'processed' flag which is returned by server.
@@ -859,16 +894,16 @@ public class HTable implements Table {
      * */
     Object[] results = new Object[rm.getMutations().size()];
     AsyncProcessTask task = AsyncProcessTask.newBuilder()
-            .setPool(pool)
-            .setTableName(tableName)
-            .setRowAccess(rm.getMutations())
-            .setResults(results)
-            .setCallable(callable)
-            // TODO any better timeout?
-            .setRpcTimeout(Math.max(readRpcTimeout, writeRpcTimeout))
-            .setOperationTimeout(operationTimeout)
-            .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL)
-            .build();
+    .setPool(pool)
+    .setTableName(tableName)
+    .setRowAccess(rm.getMutations())
+    .setResults(results)
+    .setCallable(callable)
+    // TODO any better timeout?
+    .setRpcTimeout(Math.max(readRpcTimeout, writeRpcTimeout))
+    .setOperationTimeout(operationTimeout)
+    .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL)
+    .build();
     AsyncRequestFuture ars = multiAp.submit(task);
     ars.waitUntilDone();
     if (ars.hasError()) {
@@ -882,6 +917,26 @@ public class HTable implements Table {
    * {@inheritDoc}
    */
   @Override
+  public boolean checkAndMutate(final byte [] row, final byte [] family, final byte [] qualifier,
+    final CompareOp compareOp, final byte [] value, final RowMutations rm)
+  throws IOException {
+    return doCheckAndMutate(row, family, qualifier, compareOp.name(), value, rm);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public boolean checkAndMutate(final byte [] row, final byte [] family, final byte [] qualifier,
+                                final CompareOperator op, final byte [] value, final RowMutations rm)
+  throws IOException {
+    return doCheckAndMutate(row, family, qualifier, op.name(), value, rm);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
   public boolean exists(final Get get) throws IOException {
     Result r = get(get, true);
     assert r.getExists() != null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/6752eba6/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java
index a738c84..b2f0471 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java
@@ -35,6 +35,9 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.ArrayListMultima
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ListMultimap;
 import org.apache.hadoop.hbase.util.Bytes;
 
+/**
+ * Base class for HBase read operations; e.g. Scan and Get.
+ */
 @InterfaceAudience.Public
 public abstract class Query extends OperationWithAttributes {
   private static final String ISOLATION_LEVEL = "_isolationlevel_";

http://git-wip-us.apache.org/repos/asf/hbase/blob/6752eba6/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
index 5110b8a..85cc8be 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
@@ -37,13 +37,13 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Function;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.SingleRequestCallerBuilder;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -253,24 +253,24 @@ class RawAsyncTableImpl implements RawAsyncTable {
 
   @Override
   public CompletableFuture<Boolean> checkAndPut(byte[] row, byte[] family, byte[] qualifier,
-      CompareOp compareOp, byte[] value, Put put) {
+                                                CompareOperator op, byte[] value, Put put) {
     return this.<Boolean> newCaller(row, rpcTimeoutNs)
         .action((controller, loc, stub) -> RawAsyncTableImpl.<Put, Boolean> mutate(controller, loc,
           stub, put,
           (rn, p) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier,
-            new BinaryComparator(value), CompareType.valueOf(compareOp.name()), p),
+            new BinaryComparator(value), CompareType.valueOf(op.name()), p),
           (c, r) -> r.getProcessed()))
         .call();
   }
 
   @Override
   public CompletableFuture<Boolean> checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
-      CompareOp compareOp, byte[] value, Delete delete) {
+                                                   CompareOperator op, byte[] value, Delete delete) {
     return this.<Boolean> newCaller(row, rpcTimeoutNs)
         .action((controller, loc, stub) -> RawAsyncTableImpl.<Delete, Boolean> mutate(controller,
           loc, stub, delete,
           (rn, d) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier,
-            new BinaryComparator(value), CompareType.valueOf(compareOp.name()), d),
+            new BinaryComparator(value), CompareType.valueOf(op.name()), d),
           (c, r) -> r.getProcessed()))
         .call();
   }
@@ -330,12 +330,12 @@ class RawAsyncTableImpl implements RawAsyncTable {
 
   @Override
   public CompletableFuture<Boolean> checkAndMutate(byte[] row, byte[] family, byte[] qualifier,
-      CompareOp compareOp, byte[] value, RowMutations mutation) {
+                                                   CompareOperator op, byte[] value, RowMutations mutation) {
     return this.<Boolean> newCaller(mutation, rpcTimeoutNs)
         .action((controller, loc, stub) -> RawAsyncTableImpl.<Boolean> mutateRow(controller, loc,
           stub, mutation,
           (rn, rm) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier,
-            new BinaryComparator(value), CompareType.valueOf(compareOp.name()), rm),
+            new BinaryComparator(value), CompareType.valueOf(op.name()), rm),
           resp -> resp.getExists()))
         .call();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6752eba6/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
index 0aaf6dd..2409406 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -250,11 +251,36 @@ public interface Table extends Closeable {
    * @param put data to put if check succeeds
    * @throws IOException e
    * @return true if the new put was executed, false otherwise
+   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use
+   *  {@link #checkAndPut(byte[], byte[], byte[], CompareOperator, byte[], Put)}}
    */
+  @Deprecated
   boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
     CompareFilter.CompareOp compareOp, byte[] value, Put put) throws IOException;
 
   /**
+   * Atomically checks if a row/family/qualifier value matches the expected
+   * value. If it does, it adds the put.  If the passed value is null, the check
+   * is for the lack of column (ie: non-existence)
+   *
+   * The expected value argument of this call is on the left and the current
+   * value of the cell is on the right side of the comparison operator.
+   *
+   * Ie. eg. GREATER operator means expected value > existing <=> add the put.
+   *
+   * @param row to check
+   * @param family column family to check
+   * @param qualifier column qualifier to check
+   * @param op comparison operator to use
+   * @param value the expected value
+   * @param put data to put if check succeeds
+   * @throws IOException e
+   * @return true if the new put was executed, false otherwise
+   */
+  boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
+                      CompareOperator op, byte[] value, Put put) throws IOException;
+
+  /**
    * Deletes the specified cells/row.
    *
    * @param delete The object that specifies what to delete.
@@ -310,11 +336,36 @@ public interface Table extends Closeable {
    * @param delete data to delete if check succeeds
    * @throws IOException e
    * @return true if the new delete was executed, false otherwise
+   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use
+   *  {@link #checkAndDelete(byte[], byte[], byte[], byte[], Delete)}
    */
+  @Deprecated
   boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
     CompareFilter.CompareOp compareOp, byte[] value, Delete delete) throws IOException;
 
   /**
+   * Atomically checks if a row/family/qualifier value matches the expected
+   * value. If it does, it adds the delete.  If the passed value is null, the
+   * check is for the lack of column (ie: non-existence)
+   *
+   * The expected value argument of this call is on the left and the current
+   * value of the cell is on the right side of the comparison operator.
+   *
+   * Ie. eg. GREATER operator means expected value > existing <=> add the delete.
+   *
+   * @param row to check
+   * @param family column family to check
+   * @param qualifier column qualifier to check
+   * @param op comparison operator to use
+   * @param value the expected value
+   * @param delete data to delete if check succeeds
+   * @throws IOException e
+   * @return true if the new delete was executed, false otherwise
+   */
+  boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
+                         CompareOperator op, byte[] value, Delete delete) throws IOException;
+
+  /**
    * Performs multiple mutations atomically on a single row. Currently
    * {@link Put} and {@link Delete} are supported.
    *
@@ -556,11 +607,36 @@ public interface Table extends Closeable {
    * @param mutation  mutations to perform if check succeeds
    * @throws IOException e
    * @return true if the new put was executed, false otherwise
+   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use
+   * {@link #checkAndMutate(byte[], byte[], byte[], CompareOperator, byte[], RowMutations)}
    */
+  @Deprecated
   boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier,
       CompareFilter.CompareOp compareOp, byte[] value, RowMutations mutation) throws IOException;
 
   /**
+   * Atomically checks if a row/family/qualifier value matches the expected value.
+   * If it does, it performs the row mutations.  If the passed value is null, the check
+   * is for the lack of column (ie: non-existence)
+   *
+   * The expected value argument of this call is on the left and the current
+   * value of the cell is on the right side of the comparison operator.
+   *
+   * Ie. eg. GREATER operator means expected value > existing <=> perform row mutations.
+   *
+   * @param row to check
+   * @param family column family to check
+   * @param qualifier column qualifier to check
+   * @param op the comparison operator
+   * @param value the expected value
+   * @param mutation  mutations to perform if check succeeds
+   * @throws IOException e
+   * @return true if the new put was executed, false otherwise
+   */
+  boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
+                         byte[] value, RowMutations mutation) throws IOException;
+
+  /**
    * Set timeout (millisecond) of each operation in this Table instance, will override the value
    * of hbase.client.operation.timeout in configuration.
    * Operation timeout is a top-level restriction that makes sure a blocking method will not be

http://git-wip-us.apache.org/repos/asf/hbase/blob/6752eba6/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
index 56c633b..631fbe4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
@@ -24,6 +24,7 @@ import java.util.ArrayList;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
@@ -49,8 +50,12 @@ import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
  */
 @InterfaceAudience.Public
 public abstract class CompareFilter extends FilterBase {
-
-  /** Comparison operators. */
+  /**
+   * Comparison operators. For filters only!
+   * Use {@link CompareOperator} otherwise.
+   * It (intentionally) has at least the below enums with same names.
+   * TODO: Replace with generic {@link CompareOperator}
+   */
   @InterfaceAudience.Public
   public enum CompareOp {
     /** less than */
@@ -215,4 +220,4 @@ public abstract class CompareFilter extends FilterBase {
         this.compareOp.name(),
         Bytes.toStringBinary(this.comparator.getValue()));
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/6752eba6/hbase-common/src/main/java/org/apache/hadoop/hbase/CompareOperator.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CompareOperator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CompareOperator.java
new file mode 100644
index 0000000..46fa55d
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CompareOperator.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Generic set of comparison operators.
+ * @since 2.0.0
+ */
+@InterfaceAudience.Public
+public enum CompareOperator {
+  // Keeps same names as the enums over in filter's CompareOp intentionally.
+  // The convertion of operator to protobuf representation is via a name comparison.
+  /** less than */
+  LESS,
+  /** less than or equal to */
+  LESS_OR_EQUAL,
+  /** equals */
+  EQUAL,
+  /** not equal */
+  NOT_EQUAL,
+  /** greater than or equal to */
+  GREATER_OR_EQUAL,
+  /** greater than */
+  GREATER,
+  /** no operation */
+  NO_OP,
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/6752eba6/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
index e91f1ad..5b08129 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
@@ -364,7 +365,7 @@ public class TableBasedReplicationQueuesImpl extends ReplicationTableBase
   private void safeQueueUpdate(RowMutations mutate) throws ReplicationException, IOException{
     try (Table replicationTable = getOrBlockOnReplicationTable()) {
       boolean updateSuccess = replicationTable.checkAndMutate(mutate.getRow(),
-          CF_QUEUE, COL_QUEUE_OWNER, CompareFilter.CompareOp.EQUAL, serverNameBytes, mutate);
+          CF_QUEUE, COL_QUEUE_OWNER, CompareOperator.EQUAL, serverNameBytes, mutate);
       if (!updateSuccess) {
         throw new ReplicationException("Failed to update Replication Table because we lost queue " +
             " ownership");
@@ -409,7 +410,7 @@ public class TableBasedReplicationQueuesImpl extends ReplicationTableBase
     // new owner and update the queue's history
     try (Table replicationTable = getOrBlockOnReplicationTable()) {
       boolean success = replicationTable.checkAndMutate(queue.getRow(),
-          CF_QUEUE, COL_QUEUE_OWNER, CompareFilter.CompareOp.EQUAL, Bytes.toBytes(originalServer),
+          CF_QUEUE, COL_QUEUE_OWNER, CompareOperator.EQUAL, Bytes.toBytes(originalServer),
           claimAndRenameQueue);
       return success;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6752eba6/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
index aeee96e..6580381 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
@@ -19,23 +19,16 @@
 
 package org.apache.hadoop.hbase.rest.client;
 
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.io.UnsupportedEncodingException;
-import java.net.URLEncoder;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Message;
+import com.google.protobuf.Service;
+import com.google.protobuf.ServiceException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -70,10 +63,17 @@ import org.apache.hadoop.hbase.rest.model.TableSchemaModel;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.util.StringUtils;
 
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-import com.google.protobuf.Service;
-import com.google.protobuf.ServiceException;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URLEncoder;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
 
 /**
  * HTable interface to remote tables accessed via REST gateway
@@ -721,6 +721,12 @@ public class RemoteHTable implements Table {
   }
 
   @Override
+  public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
+                             CompareOperator compareOp, byte[] value, Put put) throws IOException {
+    throw new IOException("checkAndPut for non-equal comparison not implemented");
+  }
+
+  @Override
   public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
       byte[] value, Delete delete) throws IOException {
     Put put = new Put(row);
@@ -765,6 +771,12 @@ public class RemoteHTable implements Table {
   }
 
   @Override
+  public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
+                                CompareOperator compareOp, byte[] value, Delete delete) throws IOException {
+    throw new IOException("checkAndDelete for non-equal comparison not implemented");
+  }
+
+  @Override
   public Result increment(Increment increment) throws IOException {
     throw new IOException("Increment not supported");
   }
@@ -841,6 +853,11 @@ public class RemoteHTable implements Table {
     throw new UnsupportedOperationException("checkAndMutate not implemented");
   }
 
+  @Override public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier,
+                                          CompareOperator compareOp, byte[] value, RowMutations rm) throws IOException {
+    throw new UnsupportedOperationException("checkAndMutate not implemented");
+  }
+
   @Override
   public void setOperationTimeout(int operationTimeout) {
     throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/hbase/blob/6752eba6/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java
index 14e4271..4dfa7e6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java
@@ -30,6 +30,7 @@ import java.util.Map;
 import java.util.concurrent.ExecutorService;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
@@ -170,6 +171,11 @@ public final class HTableWrapper implements Table {
     return table.checkAndPut(row, family, qualifier, compareOp, value, put);
   }
 
+  public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
+                             CompareOperator op, byte[] value, Put put) throws IOException {
+    return table.checkAndPut(row, family, qualifier, op, value, put);
+  }
+
   public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
       byte[] value, Delete delete) throws IOException {
     return table.checkAndDelete(row, family, qualifier, value, delete);
@@ -180,6 +186,11 @@ public final class HTableWrapper implements Table {
     return table.checkAndDelete(row, family, qualifier, compareOp, value, delete);
   }
 
+  public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
+                                CompareOperator op, byte[] value, Delete delete) throws IOException {
+    return table.checkAndDelete(row, family, qualifier, op, value, delete);
+  }
+
   public long incrementColumnValue(byte[] row, byte[] family,
       byte[] qualifier, long amount) throws IOException {
     return table.incrementColumnValue(row, family, qualifier, amount);
@@ -293,6 +304,13 @@ public final class HTableWrapper implements Table {
   }
 
   @Override
+  public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier,
+                                CompareOperator op, byte[] value, RowMutations rm)
+  throws IOException {
+    return table.checkAndMutate(row, family, qualifier, op, value, rm);
+  }
+
+  @Override
   public void setOperationTimeout(int operationTimeout) {
     table.setOperationTimeout(operationTimeout);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6752eba6/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 4f997c2..4fd0e21 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,17 +19,10 @@
 
 package org.apache.hadoop.hbase.coprocessor;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
-
-import java.io.IOException;
-import java.util.Collections;
-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;
+import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -45,7 +38,6 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
 import org.apache.hadoop.hbase.io.Reference;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
@@ -63,9 +55,16 @@ 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.regionserver.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.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.
@@ -789,7 +788,7 @@ public interface RegionObserver extends Coprocessor {
    * @param row row to check
    * @param family column family
    * @param qualifier column qualifier
-   * @param compareOp the comparison operation
+   * @param op the comparison operation
    * @param comparator the comparator
    * @param put data to put if check succeeds
    * @param result
@@ -797,9 +796,9 @@ public interface RegionObserver extends Coprocessor {
    * processing
    */
   default boolean preCheckAndPut(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final byte [] row, final byte [] family, final byte [] qualifier,
-      final CompareOp compareOp, final ByteArrayComparable comparator,
-      final Put put, final boolean result)
+                                 final byte [] row, final byte [] family, final byte [] qualifier,
+                                 final CompareOperator op, final ByteArrayComparable comparator,
+                                 final Put put, final boolean result)
     throws IOException {
     return result;
   }
@@ -822,7 +821,7 @@ public interface RegionObserver extends Coprocessor {
    * @param row row to check
    * @param family column family
    * @param qualifier column qualifier
-   * @param compareOp the comparison operation
+   * @param op the comparison operation
    * @param comparator the comparator
    * @param put data to put if check succeeds
    * @param result
@@ -830,7 +829,7 @@ public interface RegionObserver extends Coprocessor {
    * processing
    */
   default boolean preCheckAndPutAfterRowLock(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final byte[] row, final byte[] family, final byte[] qualifier, final CompareOp compareOp,
+      final byte[] row, final byte[] family, final byte[] qualifier, final CompareOperator op,
       final ByteArrayComparable comparator, final Put put,
       final boolean result) throws IOException {
     return result;
@@ -848,16 +847,16 @@ public interface RegionObserver extends Coprocessor {
    * @param row row to check
    * @param family column family
    * @param qualifier column qualifier
-   * @param compareOp the comparison operation
+   * @param op the comparison operation
    * @param comparator the comparator
    * @param put data to put if check succeeds
    * @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 CompareOp compareOp, final ByteArrayComparable comparator,
-      final Put put, final boolean result)
+                                  final byte [] row, final byte [] family, final byte [] qualifier,
+                                  final CompareOperator op, final ByteArrayComparable comparator,
+                                  final Put put, final boolean result)
     throws IOException {
     return result;
   }
@@ -876,16 +875,16 @@ public interface RegionObserver extends Coprocessor {
    * @param row row to check
    * @param family column family
    * @param qualifier column qualifier
-   * @param compareOp the comparison operation
+   * @param op the comparison operation
    * @param comparator the comparator
    * @param delete delete to commit if check succeeds
    * @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 CompareOp compareOp, final ByteArrayComparable comparator,
-      final Delete delete, final boolean result)
+                                    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;
   }
@@ -908,7 +907,7 @@ public interface RegionObserver extends Coprocessor {
    * @param row row to check
    * @param family column family
    * @param qualifier column qualifier
-   * @param compareOp the comparison operation
+   * @param op the comparison operation
    * @param comparator the comparator
    * @param delete delete to commit if check succeeds
    * @param result
@@ -916,7 +915,8 @@ public interface RegionObserver extends Coprocessor {
    */
   default boolean preCheckAndDeleteAfterRowLock(
       final ObserverContext<RegionCoprocessorEnvironment> c,
-      final byte[] row, final byte[] family, final byte[] qualifier, final CompareOp compareOp,
+      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;
@@ -934,16 +934,16 @@ public interface RegionObserver extends Coprocessor {
    * @param row row to check
    * @param family column family
    * @param qualifier column qualifier
-   * @param compareOp the comparison operation
+   * @param op the comparison operation
    * @param comparator the comparator
    * @param delete delete to commit if check succeeds
    * @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 CompareOp compareOp, final ByteArrayComparable comparator,
-      final Delete delete, final boolean result)
+                                     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;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6752eba6/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 823ef39..59b2990 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -17,65 +17,6 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL;
-import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Optional;
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
-import org.apache.hadoop.hbase.shaded.com.google.common.io.Closeables;
-
-import java.io.EOFException;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.lang.reflect.Constructor;
-import java.nio.ByteBuffer;
-import java.text.ParseException;
-import java.util.AbstractList;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.NavigableMap;
-import java.util.NavigableSet;
-import java.util.RandomAccess;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.UUID;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CompletionService;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorCompletionService;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.FutureTask;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.LongAdder;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.function.Function;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -84,12 +25,15 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellBuilderType;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.CompoundConfiguration;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.DroppedSnapshotException;
+import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
@@ -123,15 +67,12 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.conf.ConfigurationManager;
 import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
 import org.apache.hadoop.hbase.coprocessor.RegionObserver.MutationType;
-import org.apache.hadoop.hbase.CellBuilderType;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
 import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
 import org.apache.hadoop.hbase.exceptions.RegionInRecoveryException;
 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
 import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
-import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.FilterWrapper;
 import org.apache.hadoop.hbase.filter.IncompatibleFilterException;
 import org.apache.hadoop.hbase.io.HFileLink;
@@ -155,6 +96,12 @@ import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Optional;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
+import org.apache.hadoop.hbase.shaded.com.google.common.io.Closeables;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -196,6 +143,58 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
 
+import java.io.EOFException;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.lang.reflect.Constructor;
+import java.nio.ByteBuffer;
+import java.text.ParseException;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.NavigableMap;
+import java.util.NavigableSet;
+import java.util.RandomAccess;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.LongAdder;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+
+import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL;
+import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
+
 @SuppressWarnings("deprecation")
 @InterfaceAudience.Private
 public class HRegion implements HeapSize, PropagatingConfigurationObserver, Region {
@@ -3685,20 +3684,19 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
   @Override
   public boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier,
-      CompareOp compareOp, ByteArrayComparable comparator, Mutation mutation,
-      boolean writeToWAL)
+                                CompareOperator op, ByteArrayComparable comparator, Mutation mutation, boolean writeToWAL)
   throws IOException{
     checkMutationType(mutation, row);
-    return doCheckAndRowMutate(row, family, qualifier, compareOp, comparator, null,
+    return doCheckAndRowMutate(row, family, qualifier, op, comparator, null,
       mutation, writeToWAL);
   }
 
   @Override
   public boolean checkAndRowMutate(byte [] row, byte [] family, byte [] qualifier,
-      CompareOp compareOp, ByteArrayComparable comparator, RowMutations rm,
-      boolean writeToWAL)
+                                   CompareOperator op, ByteArrayComparable comparator, RowMutations rm,
+                                   boolean writeToWAL)
   throws IOException {
-    return doCheckAndRowMutate(row, family, qualifier, compareOp, comparator, rm, null,
+    return doCheckAndRowMutate(row, family, qualifier, op, comparator, rm, null,
       writeToWAL);
   }
 
@@ -3707,8 +3705,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * switches in the few places where there is deviation.
    */
   private boolean doCheckAndRowMutate(byte [] row, byte [] family, byte [] qualifier,
-      CompareOp compareOp, ByteArrayComparable comparator, RowMutations rowMutations,
-      Mutation mutation, boolean writeToWAL)
+                                      CompareOperator op, ByteArrayComparable comparator, RowMutations rowMutations,
+                                      Mutation mutation, boolean writeToWAL)
   throws IOException {
     // Could do the below checks but seems wacky with two callers only. Just comment out for now.
     // One caller passes a Mutation, the other passes RowMutation. Presume all good so we don't
@@ -3732,10 +3730,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           Boolean processed = null;
           if (mutation instanceof Put) {
             processed = this.getCoprocessorHost().preCheckAndPutAfterRowLock(row, family,
-                qualifier, compareOp, comparator, (Put)mutation);
+                qualifier, op, comparator, (Put)mutation);
           } else if (mutation instanceof Delete) {
             processed = this.getCoprocessorHost().preCheckAndDeleteAfterRowLock(row, family,
-                qualifier, compareOp, comparator, (Delete)mutation);
+                qualifier, op, comparator, (Delete)mutation);
           }
           if (processed != null) {
             return processed;
@@ -3757,7 +3755,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           Cell kv = result.get(0);
           cellTs = kv.getTimestamp();
           int compareResult = CellComparator.compareValue(kv, comparator);
-          matches = matches(compareOp, compareResult);
+          matches = matches(op, compareResult);
         }
         // If matches put the new put or delete the new delete
         if (matches) {
@@ -3813,9 +3811,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
   }
 
-  private boolean matches(final CompareOp compareOp, final int compareResult) {
+  private boolean matches(final CompareOperator op, final int compareResult) {
     boolean matches = false;
-    switch (compareOp) {
+    switch (op) {
       case LESS:
         matches = compareResult < 0;
         break;
@@ -3835,7 +3833,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         matches = compareResult > 0;
         break;
       default:
-        throw new RuntimeException("Unknown Compare op " + compareOp.name());
+        throw new RuntimeException("Unknown Compare op " + op.name());
     }
     return matches;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6752eba6/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 4efa01c..1b944dc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -18,31 +18,6 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.net.BindException;
-import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.NavigableMap;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.LongAdder;
-
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -53,6 +28,7 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.DroppedSnapshotException;
 import org.apache.hadoop.hbase.HBaseIOException;
@@ -83,7 +59,6 @@ import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
 import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
 import org.apache.hadoop.hbase.exceptions.ScannerResetException;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.ipc.HBaseRPCErrorHandler;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.PriorityFunction;
@@ -115,18 +90,6 @@ import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.DNS;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
-import org.apache.hadoop.hbase.util.Strings;
-import org.apache.hadoop.hbase.wal.WAL;
-import org.apache.hadoop.hbase.wal.WALKey;
-import org.apache.hadoop.hbase.wal.WALSplitter;
-import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
-import org.apache.zookeeper.KeeperException;
-
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.shaded.com.google.common.cache.Cache;
 import org.apache.hadoop.hbase.shaded.com.google.common.cache.CacheBuilder;
@@ -220,6 +183,42 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescr
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.DNS;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
+import org.apache.hadoop.hbase.util.Strings;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALSplitter;
+import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
+import org.apache.zookeeper.KeeperException;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.net.BindException;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.NavigableMap;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.LongAdder;
 
 /**
  * Implements the regionserver RPC services.
@@ -601,9 +600,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
    * @param comparator @throws IOException
    */
   private boolean checkAndRowMutate(final Region region, final List<ClientProtos.Action> actions,
-      final CellScanner cellScanner, byte[] row, byte[] family, byte[] qualifier,
-      CompareOp compareOp, ByteArrayComparable comparator, RegionActionResult.Builder builder,
-      ActivePolicyEnforcement spaceQuotaEnforcement) throws IOException {
+                                    final CellScanner cellScanner, byte[] row, byte[] family, byte[] qualifier,
+                                    CompareOperator op, ByteArrayComparable comparator, RegionActionResult.Builder builder,
+                                    ActivePolicyEnforcement spaceQuotaEnforcement) throws IOException {
     if (!region.getRegionInfo().isMetaTable()) {
       regionServer.cacheFlusher.reclaimMemStoreMemory();
     }
@@ -642,7 +641,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       builder.addResultOrException(
           resultOrExceptionOrBuilder.build());
     }
-    return region.checkAndRowMutate(row, family, qualifier, compareOp,
+    return region.checkAndRowMutate(row, family, qualifier, op,
         comparator, rm, Boolean.TRUE);
   }
 
@@ -2597,11 +2596,12 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
             byte[] row = condition.getRow().toByteArray();
             byte[] family = condition.getFamily().toByteArray();
             byte[] qualifier = condition.getQualifier().toByteArray();
-            CompareOp compareOp = CompareOp.valueOf(condition.getCompareType().name());
+            CompareOperator op =
+              CompareOperator.valueOf(condition.getCompareType().name());
             ByteArrayComparable comparator =
                 ProtobufUtil.toComparator(condition.getComparator());
             processed = checkAndRowMutate(region, regionAction.getActionList(),
-                  cellScanner, row, family, qualifier, compareOp,
+                  cellScanner, row, family, qualifier, op,
                   comparator, regionActionResultBuilder, spaceQuotaEnforcement);
           } else {
             mutateRows(region, regionAction.getActionList(), cellScanner,
@@ -2737,7 +2737,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           byte[] row = condition.getRow().toByteArray();
           byte[] family = condition.getFamily().toByteArray();
           byte[] qualifier = condition.getQualifier().toByteArray();
-          CompareOp compareOp = CompareOp.valueOf(condition.getCompareType().name());
+          CompareOperator compareOp =
+            CompareOperator.valueOf(condition.getCompareType().name());
           ByteArrayComparable comparator =
             ProtobufUtil.toComparator(condition.getComparator());
           if (region.getCoprocessorHost() != null) {
@@ -2768,19 +2769,19 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           byte[] row = condition.getRow().toByteArray();
           byte[] family = condition.getFamily().toByteArray();
           byte[] qualifier = condition.getQualifier().toByteArray();
-          CompareOp compareOp = CompareOp.valueOf(condition.getCompareType().name());
+          CompareOperator op = CompareOperator.valueOf(condition.getCompareType().name());
           ByteArrayComparable comparator =
             ProtobufUtil.toComparator(condition.getComparator());
           if (region.getCoprocessorHost() != null) {
             processed = region.getCoprocessorHost().preCheckAndDelete(
-              row, family, qualifier, compareOp, comparator, delete);
+              row, family, qualifier, op, comparator, delete);
           }
           if (processed == null) {
             boolean result = region.checkAndMutate(row, family,
-              qualifier, compareOp, comparator, delete, true);
+              qualifier, op, comparator, delete, true);
             if (region.getCoprocessorHost() != null) {
               result = region.getCoprocessorHost().postCheckAndDelete(row, family,
-                qualifier, compareOp, comparator, delete, result);
+                qualifier, op, comparator, delete, result);
             }
             processed = result;
           }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6752eba6/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
index fe17cb2..dbb7a64 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
@@ -23,12 +23,8 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellComparator;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HDFSBlocksDistribution;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Append;
@@ -45,7 +41,6 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.Service;
@@ -370,14 +365,14 @@ public interface Region extends ConfigurationObserver {
    * @param row to check
    * @param family column family to check
    * @param qualifier column qualifier to check
-   * @param compareOp the comparison operator
+   * @param op the comparison operator
    * @param comparator
    * @param mutation
    * @param writeToWAL
    * @return true if mutation was applied, false otherwise
    * @throws IOException
    */
-  boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier, CompareOp compareOp,
+  boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier, CompareOperator op,
       ByteArrayComparable comparator, Mutation mutation, boolean writeToWAL) throws IOException;
 
   /**
@@ -388,14 +383,14 @@ public interface Region extends ConfigurationObserver {
    * @param row to check
    * @param family column family to check
    * @param qualifier column qualifier to check
-   * @param compareOp the comparison operator
+   * @param op the comparison operator
    * @param comparator
    * @param mutations
    * @param writeToWAL
    * @return true if mutations were applied, false otherwise
    * @throws IOException
    */
-  boolean checkAndRowMutate(byte [] row, byte [] family, byte [] qualifier, CompareOp compareOp,
+  boolean checkAndRowMutate(byte [] row, byte [] family, byte [] qualifier, CompareOperator op,
       ByteArrayComparable comparator, RowMutations mutations, boolean writeToWAL)
       throws IOException;