hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From anoopsamj...@apache.org
Subject [1/2] hbase git commit: HBASE-13346: Clean up Filter package for post 1.0 s/KeyValue/Cell/g
Date Fri, 27 Oct 2017 05:47:20 GMT
Repository: hbase
Updated Branches:
  refs/heads/master 660ead011 -> 82b1c320f


http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java
index 0a287ce..fdd7e77 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java
@@ -146,7 +146,7 @@ public class TestFilterFromRegionSide {
     }
 
     @Override
-    public ReturnCode filterKeyValue(Cell v) {
+    public ReturnCode filterCell(final Cell v) {
       if (count++ < NUM_COLS) {
         return ReturnCode.INCLUDE;
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
index 8c83cf6..4774767 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
@@ -137,7 +137,7 @@ public class TestFilterList {
      * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
      * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
      * if false, we will also call</li>
-     * <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
+     * <li>{@link #filterCell(org.apache.hadoop.hbase.KeyValue)} -> true to drop this cell</li>
      * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
      * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
      * </li>
@@ -152,7 +152,7 @@ public class TestFilterList {
       assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
         Bytes.toBytes(i));
-      assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
+      assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterCell(kv));
       assertFalse(filterMPONE.filterRow());
     }
 
@@ -161,7 +161,7 @@ public class TestFilterList {
     assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
     KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
         Bytes.toBytes(0));
-    assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
+    assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterCell(kv));
     assertFalse(filterMPONE.filterRow());
 
     /* reach MAX_PAGES already, should filter any rows */
@@ -169,7 +169,7 @@ public class TestFilterList {
     assertTrue(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
     kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
         Bytes.toBytes(0));
-    assertFalse(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
+    assertFalse(Filter.ReturnCode.INCLUDE == filterMPONE.filterCell(kv));
     assertFalse(filterMPONE.filterRow());
 
     /* We should filter any row */
@@ -203,7 +203,7 @@ public class TestFilterList {
      * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
      * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
      * if false, we will also call</li>
-     * <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
+     * <li>{@link #filterCell(org.apache.hadoop.hbase.KeyValue)} -> true to drop this cell</li>
      * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
      * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
      * </li>
@@ -216,14 +216,14 @@ public class TestFilterList {
       assertFalse(filterMPALL.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
         Bytes.toBytes(i));
-      assertTrue(Filter.ReturnCode.INCLUDE == filterMPALL.filterKeyValue(kv));
+      assertTrue(Filter.ReturnCode.INCLUDE == filterMPALL.filterCell(kv));
     }
     filterMPALL.reset();
     rowkey = Bytes.toBytes("z");
     assertTrue(filterMPALL.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
     // Should fail here; row should be filtered out.
     KeyValue kv = new KeyValue(rowkey, rowkey, rowkey, rowkey);
-    assertTrue(Filter.ReturnCode.NEXT_ROW == filterMPALL.filterKeyValue(kv));
+    assertTrue(Filter.ReturnCode.NEXT_ROW == filterMPALL.filterCell(kv));
   }
 
   /**
@@ -251,7 +251,7 @@ public class TestFilterList {
      * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
      * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
      * if false, we will also call</li>
-     * <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
+     * <li>{@link #filterCell(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
      * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
      * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
      * </li>
@@ -266,7 +266,7 @@ public class TestFilterList {
       assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
           Bytes.toBytes(i));
-        assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
+        assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterCell(kv));
       assertFalse(filterMPONE.filterRow());
     }
 
@@ -276,7 +276,7 @@ public class TestFilterList {
       assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
           Bytes.toBytes(i));
-        assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
+        assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterCell(kv));
       assertFalse(filterMPONE.filterRow());
     }
 
@@ -286,7 +286,7 @@ public class TestFilterList {
       assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
           Bytes.toBytes(i));
-        assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
+        assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterCell(kv));
       assertFalse(filterMPONE.filterRow());
     }
   }
@@ -305,23 +305,23 @@ public class TestFilterList {
     FilterList flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
     flist.addFilter(new PrefixFilter(r1));
     flist.filterRowKey(KeyValueUtil.createFirstOnRow(r1));
-    assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r1, r1, r1)));
-    assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r11, r11, r11)));
+    assertEquals(ReturnCode.INCLUDE, flist.filterCell(new KeyValue(r1, r1, r1)));
+    assertEquals(ReturnCode.INCLUDE, flist.filterCell(new KeyValue(r11, r11, r11)));
 
     flist.reset();
     flist.filterRowKey(KeyValueUtil.createFirstOnRow(r2));
-    assertEquals(ReturnCode.SKIP, flist.filterKeyValue(new KeyValue(r2, r2, r2)));
+    assertEquals(ReturnCode.SKIP, flist.filterCell(new KeyValue(r2, r2, r2)));
 
     flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
     flist.addFilter(new AlwaysNextColFilter());
     flist.addFilter(new PrefixFilter(r1));
     flist.filterRowKey(KeyValueUtil.createFirstOnRow(r1));
-    assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r1, r1, r1)));
-    assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r11, r11, r11)));
+    assertEquals(ReturnCode.INCLUDE, flist.filterCell(new KeyValue(r1, r1, r1)));
+    assertEquals(ReturnCode.INCLUDE, flist.filterCell(new KeyValue(r11, r11, r11)));
 
     flist.reset();
     flist.filterRowKey(KeyValueUtil.createFirstOnRow(r2));
-    assertEquals(ReturnCode.NEXT_COL, flist.filterKeyValue(new KeyValue(r2, r2, r2)));
+    assertEquals(ReturnCode.NEXT_COL, flist.filterCell(new KeyValue(r2, r2, r2)));
   }
 
   /**
@@ -339,12 +339,12 @@ public class TestFilterList {
     flist.addFilter(new AlwaysNextColFilter());
     flist.addFilter(new InclusiveStopFilter(r1));
     flist.filterRowKey(KeyValueUtil.createFirstOnRow(r1));
-    assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r1, r1, r1)));
-    assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r11, r11, r11)));
+    assertEquals(ReturnCode.INCLUDE, flist.filterCell(new KeyValue(r1, r1, r1)));
+    assertEquals(ReturnCode.INCLUDE, flist.filterCell(new KeyValue(r11, r11, r11)));
 
     flist.reset();
     flist.filterRowKey(KeyValueUtil.createFirstOnRow(r2));
-    assertEquals(ReturnCode.NEXT_COL, flist.filterKeyValue(new KeyValue(r2, r2, r2)));
+    assertEquals(ReturnCode.NEXT_COL, flist.filterCell(new KeyValue(r2, r2, r2)));
   }
 
   public static class AlwaysNextColFilter extends FilterBase {
@@ -353,7 +353,7 @@ public class TestFilterList {
     }
 
     @Override
-    public ReturnCode filterKeyValue(Cell v) {
+    public ReturnCode filterCell(final Cell v) {
       return ReturnCode.NEXT_COL;
     }
 
@@ -388,14 +388,14 @@ public class TestFilterList {
   }
 
   /**
-   * Test filterKeyValue logic.
+   * Test filterCell logic.
    * @throws Exception
    */
   @Test
-  public void testFilterKeyValue() throws Exception {
+  public void testFilterCell() throws Exception {
     Filter includeFilter = new FilterBase() {
       @Override
-      public Filter.ReturnCode filterKeyValue(Cell v) {
+      public Filter.ReturnCode filterCell(final Cell v) {
         return Filter.ReturnCode.INCLUDE;
       }
     };
@@ -404,7 +404,7 @@ public class TestFilterList {
       boolean returnInclude = true;
 
       @Override
-      public Filter.ReturnCode filterKeyValue(Cell v) {
+      public Filter.ReturnCode filterCell(final Cell v) {
         Filter.ReturnCode returnCode = returnInclude ? Filter.ReturnCode.INCLUDE :
                                                        Filter.ReturnCode.SKIP;
         returnInclude = !returnInclude;
@@ -416,7 +416,7 @@ public class TestFilterList {
       boolean returnIncludeOnly = false;
 
       @Override
-      public Filter.ReturnCode filterKeyValue(Cell v) {
+      public Filter.ReturnCode filterCell(final Cell v) {
         Filter.ReturnCode returnCode = returnIncludeOnly ? Filter.ReturnCode.INCLUDE :
                                                            Filter.ReturnCode.INCLUDE_AND_NEXT_COL;
         returnIncludeOnly = !returnIncludeOnly;
@@ -428,17 +428,17 @@ public class TestFilterList {
     FilterList mpOnefilterList = new FilterList(Operator.MUST_PASS_ONE,
         Arrays.asList(new Filter[] { includeFilter, alternateIncludeFilter, alternateFilter }));
     // INCLUDE, INCLUDE, INCLUDE_AND_NEXT_COL.
-    assertEquals(ReturnCode.INCLUDE, mpOnefilterList.filterKeyValue(null));
+    assertEquals(ReturnCode.INCLUDE, mpOnefilterList.filterCell(null));
     // INCLUDE, SKIP, INCLUDE.
-    assertEquals(Filter.ReturnCode.INCLUDE, mpOnefilterList.filterKeyValue(null));
+    assertEquals(Filter.ReturnCode.INCLUDE, mpOnefilterList.filterCell(null));
 
     // Check must pass all filter.
     FilterList mpAllfilterList = new FilterList(Operator.MUST_PASS_ALL,
         Arrays.asList(new Filter[] { includeFilter, alternateIncludeFilter, alternateFilter }));
     // INCLUDE, INCLUDE, INCLUDE_AND_NEXT_COL.
-    assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL, mpAllfilterList.filterKeyValue(null));
+    assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL, mpAllfilterList.filterCell(null));
     // INCLUDE, SKIP, INCLUDE.
-    assertEquals(Filter.ReturnCode.SKIP, mpAllfilterList.filterKeyValue(null));
+    assertEquals(Filter.ReturnCode.SKIP, mpAllfilterList.filterCell(null));
   }
 
   /**
@@ -458,14 +458,14 @@ public class TestFilterList {
       }
 
       @Override
-      public ReturnCode filterKeyValue(Cell ignored) throws IOException {
+      public ReturnCode filterCell(final Cell ignored) throws IOException {
         return ReturnCode.INCLUDE;
       }
     };
 
     Filter filterMinHint = new FilterBase() {
       @Override
-      public ReturnCode filterKeyValue(Cell ignored) {
+      public ReturnCode filterCell(final Cell ignored) {
         return ReturnCode.SEEK_NEXT_USING_HINT;
       }
 
@@ -480,7 +480,7 @@ public class TestFilterList {
 
     Filter filterMaxHint = new FilterBase() {
       @Override
-      public ReturnCode filterKeyValue(Cell ignored) {
+      public ReturnCode filterCell(final Cell ignored) {
         return ReturnCode.SEEK_NEXT_USING_HINT;
       }
 
@@ -521,30 +521,30 @@ public class TestFilterList {
     // Should take the first hint
     filterList = new FilterList(Operator.MUST_PASS_ALL,
         Arrays.asList(new Filter [] { filterMinHint, filterMaxHint } ));
-    filterList.filterKeyValue(null);
+    filterList.filterCell(null);
     assertEquals(0,
       CellComparatorImpl.COMPARATOR.compare(filterList.getNextCellHint(null), maxKeyValue));
 
     filterList = new FilterList(Operator.MUST_PASS_ALL,
         Arrays.asList(new Filter [] { filterMaxHint, filterMinHint } ));
-    filterList.filterKeyValue(null);
+    filterList.filterCell(null);
     assertEquals(0,
       CellComparatorImpl.COMPARATOR.compare(filterList.getNextCellHint(null), maxKeyValue));
 
     // Should have first hint even if a filter has no hint
     filterList = new FilterList(Operator.MUST_PASS_ALL,
         Arrays.asList(new Filter[] { filterNoHint, filterMinHint, filterMaxHint }));
-    filterList.filterKeyValue(null);
+    filterList.filterCell(null);
     assertEquals(0,
       CellComparatorImpl.COMPARATOR.compare(filterList.getNextCellHint(null), maxKeyValue));
     filterList = new FilterList(Operator.MUST_PASS_ALL,
         Arrays.asList(new Filter[] { filterNoHint, filterMaxHint }));
-    filterList.filterKeyValue(null);
+    filterList.filterCell(null);
     assertEquals(0,
       CellComparatorImpl.COMPARATOR.compare(filterList.getNextCellHint(null), maxKeyValue));
     filterList = new FilterList(Operator.MUST_PASS_ALL,
         Arrays.asList(new Filter[] { filterNoHint, filterMinHint }));
-    filterList.filterKeyValue(null);
+    filterList.filterCell(null);
     assertEquals(0,
       CellComparatorImpl.COMPARATOR.compare(filterList.getNextCellHint(null), minKeyValue));
   }
@@ -552,7 +552,7 @@ public class TestFilterList {
   /**
    * Tests the behavior of transform() in a hierarchical filter.
    *
-   * transform() only applies after a filterKeyValue() whose return-code includes the KeyValue.
+   * transform() only applies after a filterCell() whose return-code includes the KeyValue.
    * Lazy evaluation of AND
    */
   @Test
@@ -577,18 +577,18 @@ public class TestFilterList {
         Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual3"), Bytes.toBytes("value"));
 
     // Value for fam:qual1 should be stripped:
-    assertEquals(Filter.ReturnCode.INCLUDE, flist.filterKeyValue(kvQual1));
+    assertEquals(Filter.ReturnCode.INCLUDE, flist.filterCell(kvQual1));
     final KeyValue transformedQual1 = KeyValueUtil.ensureKeyValue(flist.transformCell(kvQual1));
     assertEquals(0, transformedQual1.getValueLength());
 
     // Value for fam:qual2 should not be stripped:
-    assertEquals(Filter.ReturnCode.INCLUDE, flist.filterKeyValue(kvQual2));
+    assertEquals(Filter.ReturnCode.INCLUDE, flist.filterCell(kvQual2));
     final KeyValue transformedQual2 = KeyValueUtil.ensureKeyValue(flist.transformCell(kvQual2));
     assertEquals("value", Bytes.toString(transformedQual2.getValueArray(),
       transformedQual2.getValueOffset(), transformedQual2.getValueLength()));
 
     // Other keys should be skipped:
-    assertEquals(Filter.ReturnCode.SKIP, flist.filterKeyValue(kvQual3));
+    assertEquals(Filter.ReturnCode.SKIP, flist.filterCell(kvQual3));
   }
 
   @Test
@@ -603,16 +603,16 @@ public class TestFilterList {
     KeyValue kv3 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual"),
         3, Bytes.toBytes("value"));
 
-    assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList01.filterKeyValue(kv1));
-    assertEquals(ReturnCode.SKIP, filterList01.filterKeyValue(kv2));
-    assertEquals(ReturnCode.SKIP, filterList01.filterKeyValue(kv3));
+    assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList01.filterCell(kv1));
+    assertEquals(ReturnCode.SKIP, filterList01.filterCell(kv2));
+    assertEquals(ReturnCode.SKIP, filterList01.filterCell(kv3));
 
     FilterList filterList11 =
         new FilterList(Operator.MUST_PASS_ONE, new ColumnPaginationFilter(1, 1));
 
-    assertEquals(ReturnCode.NEXT_COL, filterList11.filterKeyValue(kv1));
-    assertEquals(ReturnCode.SKIP, filterList11.filterKeyValue(kv2));
-    assertEquals(ReturnCode.SKIP, filterList11.filterKeyValue(kv3));
+    assertEquals(ReturnCode.NEXT_COL, filterList11.filterCell(kv1));
+    assertEquals(ReturnCode.SKIP, filterList11.filterCell(kv2));
+    assertEquals(ReturnCode.SKIP, filterList11.filterCell(kv3));
   }
 
   @Test
@@ -630,10 +630,10 @@ public class TestFilterList {
     KeyValue kv4 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("c"), 4,
         Bytes.toBytes("value"));
 
-    assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1));
-    assertEquals(ReturnCode.SKIP, filterList.filterKeyValue(kv2));
-    assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterKeyValue(kv3));
-    assertEquals(ReturnCode.SKIP, filterList.filterKeyValue(kv4));
+    assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1));
+    assertEquals(ReturnCode.SKIP, filterList.filterCell(kv2));
+    assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv3));
+    assertEquals(ReturnCode.SKIP, filterList.filterCell(kv4));
   }
 
   private static class MockFilter extends FilterBase {
@@ -645,7 +645,7 @@ public class TestFilterList {
     }
 
     @Override
-    public ReturnCode filterKeyValue(Cell v) throws IOException {
+    public ReturnCode filterCell(final Cell v) throws IOException {
       this.didCellPassToTheFilter = true;
       return targetRetCode;
     }
@@ -665,65 +665,65 @@ public class TestFilterList {
     MockFilter mockFilter = new MockFilter(ReturnCode.NEXT_COL);
     FilterList filter = new FilterList(Operator.MUST_PASS_ONE, mockFilter);
 
-    filter.filterKeyValue(kv1);
+    filter.filterCell(kv1);
     assertTrue(mockFilter.didCellPassToTheFilter);
 
     mockFilter.didCellPassToTheFilter = false;
-    filter.filterKeyValue(kv2);
+    filter.filterCell(kv2);
     assertFalse(mockFilter.didCellPassToTheFilter);
 
     mockFilter.didCellPassToTheFilter = false;
-    filter.filterKeyValue(kv3);
+    filter.filterCell(kv3);
     assertTrue(mockFilter.didCellPassToTheFilter);
 
     mockFilter = new MockFilter(ReturnCode.INCLUDE_AND_NEXT_COL);
     filter = new FilterList(Operator.MUST_PASS_ONE, mockFilter);
 
-    filter.filterKeyValue(kv1);
+    filter.filterCell(kv1);
     assertTrue(mockFilter.didCellPassToTheFilter);
 
     mockFilter.didCellPassToTheFilter = false;
-    filter.filterKeyValue(kv2);
+    filter.filterCell(kv2);
     assertFalse(mockFilter.didCellPassToTheFilter);
 
     mockFilter.didCellPassToTheFilter = false;
-    filter.filterKeyValue(kv3);
+    filter.filterCell(kv3);
     assertTrue(mockFilter.didCellPassToTheFilter);
 
     mockFilter = new MockFilter(ReturnCode.NEXT_ROW);
     filter = new FilterList(Operator.MUST_PASS_ONE, mockFilter);
-    filter.filterKeyValue(kv1);
+    filter.filterCell(kv1);
     assertTrue(mockFilter.didCellPassToTheFilter);
 
     mockFilter.didCellPassToTheFilter = false;
-    filter.filterKeyValue(kv2);
+    filter.filterCell(kv2);
     assertFalse(mockFilter.didCellPassToTheFilter);
 
     mockFilter.didCellPassToTheFilter = false;
-    filter.filterKeyValue(kv3);
+    filter.filterCell(kv3);
     assertFalse(mockFilter.didCellPassToTheFilter);
 
     filter.reset();
     mockFilter.didCellPassToTheFilter = false;
-    filter.filterKeyValue(kv4);
+    filter.filterCell(kv4);
     assertTrue(mockFilter.didCellPassToTheFilter);
 
     mockFilter = new MockFilter(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW);
     filter = new FilterList(Operator.MUST_PASS_ONE, mockFilter);
-    filter.filterKeyValue(kv1);
+    filter.filterCell(kv1);
     assertTrue(mockFilter.didCellPassToTheFilter);
 
     mockFilter.didCellPassToTheFilter = false;
-    filter.filterKeyValue(kv2);
+    filter.filterCell(kv2);
     assertFalse(mockFilter.didCellPassToTheFilter);
 
     mockFilter.didCellPassToTheFilter = false;
-    filter.filterKeyValue(kv3);
+    filter.filterCell(kv3);
     assertFalse(mockFilter.didCellPassToTheFilter);
 
     filter.reset();
     mockFilter.didCellPassToTheFilter = false;
-    filter.filterKeyValue(kv4);
+    filter.filterCell(kv4);
     assertTrue(mockFilter.didCellPassToTheFilter);
   }
 
@@ -740,29 +740,29 @@ public class TestFilterList {
     MockFilter filter7 = new MockFilter(ReturnCode.NEXT_ROW);
 
     FilterList filterList = new FilterList(Operator.MUST_PASS_ALL, filter1, filter2);
-    assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterKeyValue(kv1));
+    assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv1));
 
     filterList = new FilterList(Operator.MUST_PASS_ALL, filter2, filter3);
-    assertEquals(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, filterList.filterKeyValue(kv1));
+    assertEquals(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, filterList.filterCell(kv1));
 
     filterList = new FilterList(Operator.MUST_PASS_ALL, filter4, filter5, filter6);
-    assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1));
+    assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1));
 
     filterList = new FilterList(Operator.MUST_PASS_ALL, filter4, filter6);
-    assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1));
+    assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1));
 
     filterList = new FilterList(Operator.MUST_PASS_ALL, filter3, filter1);
-    assertEquals(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, filterList.filterKeyValue(kv1));
+    assertEquals(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, filterList.filterCell(kv1));
 
     filterList = new FilterList(Operator.MUST_PASS_ALL, filter3, filter2, filter1, filter5);
-    assertEquals(ReturnCode.NEXT_ROW, filterList.filterKeyValue(kv1));
+    assertEquals(ReturnCode.NEXT_ROW, filterList.filterCell(kv1));
 
     filterList = new FilterList(Operator.MUST_PASS_ALL, filter2,
         new FilterList(Operator.MUST_PASS_ALL, filter3, filter4));
-    assertEquals(ReturnCode.NEXT_ROW, filterList.filterKeyValue(kv1));
+    assertEquals(ReturnCode.NEXT_ROW, filterList.filterCell(kv1));
 
     filterList = new FilterList(Operator.MUST_PASS_ALL, filter3, filter7);
-    assertEquals(ReturnCode.NEXT_ROW, filterList.filterKeyValue(kv1));
+    assertEquals(ReturnCode.NEXT_ROW, filterList.filterCell(kv1));
   }
 
   @Test
@@ -776,33 +776,33 @@ public class TestFilterList {
     MockFilter filter5 = new MockFilter(ReturnCode.SKIP);
     MockFilter filter6 = new MockFilter(ReturnCode.SEEK_NEXT_USING_HINT);
     FilterList filterList = new FilterList(Operator.MUST_PASS_ONE, filter1, filter2);
-    assertEquals(filterList.filterKeyValue(kv1), ReturnCode.INCLUDE);
+    assertEquals(filterList.filterCell(kv1), ReturnCode.INCLUDE);
 
     filterList = new FilterList(Operator.MUST_PASS_ONE, filter2, filter3);
-    assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterKeyValue(kv1));
+    assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv1));
 
     filterList = new FilterList(Operator.MUST_PASS_ONE, filter4, filter5, filter6);
-    assertEquals(ReturnCode.SKIP, filterList.filterKeyValue(kv1));
+    assertEquals(ReturnCode.SKIP, filterList.filterCell(kv1));
 
     filterList = new FilterList(Operator.MUST_PASS_ONE, filter4, filter6);
-    assertEquals(ReturnCode.SKIP, filterList.filterKeyValue(kv1));
+    assertEquals(ReturnCode.SKIP, filterList.filterCell(kv1));
 
     filterList = new FilterList(Operator.MUST_PASS_ONE, filter3, filter1);
-    assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(kv1));
+    assertEquals(ReturnCode.INCLUDE, filterList.filterCell(kv1));
 
     filterList = new FilterList(Operator.MUST_PASS_ONE, filter3, filter2, filter1, filter5);
-    assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(kv1));
+    assertEquals(ReturnCode.INCLUDE, filterList.filterCell(kv1));
 
     filterList = new FilterList(Operator.MUST_PASS_ONE, filter2,
         new FilterList(Operator.MUST_PASS_ONE, filter3, filter4));
-    assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterKeyValue(kv1));
+    assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv1));
 
     filterList = new FilterList(Operator.MUST_PASS_ONE, filter2,
         new FilterList(Operator.MUST_PASS_ONE, filter3, filter4));
-    assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterKeyValue(kv1));
+    assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv1));
 
     filterList = new FilterList(Operator.MUST_PASS_ONE, filter6, filter6);
-    assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1));
+    assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1));
   }
 
   static class MockSeekHintFilter extends FilterBase {
@@ -813,7 +813,7 @@ public class TestFilterList {
     }
 
     @Override
-    public ReturnCode filterKeyValue(Cell v) throws IOException {
+    public ReturnCode filterCell(final Cell v) throws IOException {
       return ReturnCode.SEEK_NEXT_USING_HINT;
     }
 
@@ -844,7 +844,7 @@ public class TestFilterList {
     filterList.addFilter(filter2);
     filterList.addFilter(filter3);
 
-    Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1));
+    Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1));
     Assert.assertEquals(kv3, filterList.getNextCellHint(kv1));
 
     filterList = new FilterList(Operator.MUST_PASS_ALL);
@@ -853,7 +853,7 @@ public class TestFilterList {
     filterList.addFilter(filter2);
     filterList.addFilter(filter3);
 
-    Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1));
+    Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1));
     Assert.assertEquals(kv1, filterList.getNextCellHint(kv1));
   }
 
@@ -869,12 +869,12 @@ public class TestFilterList {
     prefixFilter.setReversed(true);
     filterList.addFilter(prefixFilter);
     filterList.filterRowKey(KeyValueUtil.createFirstOnRow(r22));
-    assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(new KeyValue(r22, r22, r22)));
-    assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(new KeyValue(r2, r2, r2)));
+    assertEquals(ReturnCode.INCLUDE, filterList.filterCell(new KeyValue(r22, r22, r22)));
+    assertEquals(ReturnCode.INCLUDE, filterList.filterCell(new KeyValue(r2, r2, r2)));
 
     filterList.reset();
     filterList.filterRowKey(KeyValueUtil.createFirstOnRow(r1));
-    assertEquals(ReturnCode.SKIP, filterList.filterKeyValue(new KeyValue(r1, r1, r1)));
+    assertEquals(ReturnCode.SKIP, filterList.filterCell(new KeyValue(r1, r1, r1)));
 
     filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE);
     filterList.setReversed(true);
@@ -885,12 +885,12 @@ public class TestFilterList {
     filterList.addFilter(alwaysNextColFilter);
     filterList.addFilter(prefixFilter);
     filterList.filterRowKey(KeyValueUtil.createFirstOnRow(r22));
-    assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(new KeyValue(r22, r22, r22)));
-    assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(new KeyValue(r2, r2, r2)));
+    assertEquals(ReturnCode.INCLUDE, filterList.filterCell(new KeyValue(r22, r22, r22)));
+    assertEquals(ReturnCode.INCLUDE, filterList.filterCell(new KeyValue(r2, r2, r2)));
 
     filterList.reset();
     filterList.filterRowKey(KeyValueUtil.createFirstOnRow(r1));
-    assertEquals(ReturnCode.NEXT_COL, filterList.filterKeyValue(new KeyValue(r1, r1, r1)));
+    assertEquals(ReturnCode.NEXT_COL, filterList.filterCell(new KeyValue(r1, r1, r1)));
   }
 
   @Test
@@ -910,20 +910,20 @@ public class TestFilterList {
     FilterList keyOnlyFilterFirst =
         new FilterList(Operator.MUST_PASS_ALL, new KeyOnlyFilter(), internalFilterList);
 
-    assertEquals(ReturnCode.INCLUDE, keyOnlyFilterFirst.filterKeyValue(kv1));
+    assertEquals(ReturnCode.INCLUDE, keyOnlyFilterFirst.filterCell(kv1));
     c = keyOnlyFilterFirst.transformCell(kv1);
     assertEquals(0, c.getValueLength());
-    assertEquals(ReturnCode.INCLUDE, keyOnlyFilterFirst.filterKeyValue(kv2));
+    assertEquals(ReturnCode.INCLUDE, keyOnlyFilterFirst.filterCell(kv2));
     c = keyOnlyFilterFirst.transformCell(kv2);
     assertEquals(0, c.getValueLength());
 
     internalFilterList.reset();
     FilterList keyOnlyFilterLast =
         new FilterList(Operator.MUST_PASS_ALL, new KeyOnlyFilter(), internalFilterList);
-    assertEquals(ReturnCode.INCLUDE, keyOnlyFilterLast.filterKeyValue(kv1));
+    assertEquals(ReturnCode.INCLUDE, keyOnlyFilterLast.filterCell(kv1));
     c = keyOnlyFilterLast.transformCell(kv1);
     assertEquals(0, c.getValueLength());
-    assertEquals(ReturnCode.INCLUDE, keyOnlyFilterLast.filterKeyValue(kv2));
+    assertEquals(ReturnCode.INCLUDE, keyOnlyFilterLast.filterCell(kv2));
     c = keyOnlyFilterLast.transformCell(kv2);
     assertEquals(0, c.getValueLength());
   }
@@ -933,18 +933,18 @@ public class TestFilterList {
     KeyValue kv = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("cf"), Bytes.toBytes("column1"),
         1, Bytes.toBytes("value"));
     FilterList filterList = new FilterList(Operator.MUST_PASS_ALL);
-    assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(kv));
+    assertEquals(ReturnCode.INCLUDE, filterList.filterCell(kv));
     assertEquals(kv, filterList.transformCell(kv));
 
     filterList = new FilterList(Operator.MUST_PASS_ONE);
-    assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(kv));
+    assertEquals(ReturnCode.INCLUDE, filterList.filterCell(kv));
     assertEquals(kv, filterList.transformCell(kv));
   }
 
   private static class MockNextRowFilter extends FilterBase {
     private int hitCount = 0;
 
-    public ReturnCode filterKeyValue(Cell v) throws IOException {
+    public ReturnCode filterCell(final Cell v) throws IOException {
       hitCount++;
       return ReturnCode.NEXT_ROW;
     }
@@ -962,8 +962,8 @@ public class TestFilterList {
         Bytes.toBytes("value"));
     MockNextRowFilter mockNextRowFilter = new MockNextRowFilter();
     FilterList filter = new FilterList(Operator.MUST_PASS_ONE, mockNextRowFilter);
-    filter.filterKeyValue(kv1);
-    filter.filterKeyValue(kv2);
+    filter.filterCell(kv1);
+    filter.filterCell(kv2);
     assertEquals(2, mockNextRowFilter.getHitCount());
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFirstKeyValueMatchingQualifiersFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFirstKeyValueMatchingQualifiersFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFirstKeyValueMatchingQualifiersFilter.java
index 60e3514..dfee414 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFirstKeyValueMatchingQualifiersFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFirstKeyValueMatchingQualifiersFilter.java
@@ -40,7 +40,7 @@ public class TestFirstKeyValueMatchingQualifiersFilter extends TestCase {
 
   /**
    * Test the functionality of
-   * {@link FirstKeyValueMatchingQualifiersFilter#filterKeyValue(org.apache.hadoop.hbase.Cell)}
+   * {@link FirstKeyValueMatchingQualifiersFilter#filterCell(org.apache.hadoop.hbase.Cell)}
    * 
    * @throws Exception
    */
@@ -51,26 +51,26 @@ public class TestFirstKeyValueMatchingQualifiersFilter extends TestCase {
     Filter filter = new FirstKeyValueMatchingQualifiersFilter(quals);
 
     // Match in first attempt
-    KeyValue kv;
-    kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_1, VAL_1);
+    KeyValue cell;
+    cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_1, VAL_1);
     assertTrue("includeAndSetFlag",
-        filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
-    kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
+        filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
+    cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
     assertTrue("flagIsSetSkipToNextRow",
-        filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
+        filter.filterCell(cell) == Filter.ReturnCode.NEXT_ROW);
 
     // A mismatch in first attempt and match in second attempt.
     filter.reset();
-    kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_3, VAL_1);
-    System.out.println(filter.filterKeyValue(kv));
+    cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_3, VAL_1);
+    System.out.println(filter.filterCell(cell));
     assertTrue("includeFlagIsUnset",
-        filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
-    kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
+        filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
+    cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
     assertTrue("includeAndSetFlag",
-        filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
-    kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_1, VAL_1);
+        filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
+    cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_1, VAL_1);
     assertTrue("flagIsSetSkipToNextRow",
-        filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
+        filter.filterCell(cell) == Filter.ReturnCode.NEXT_ROW);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java
index b6bc2f1..ebccc34 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java
@@ -166,7 +166,7 @@ public class TestInvocationRecordFilter {
       visitedKeyValues.clear();
     }
 
-    public ReturnCode filterKeyValue(Cell ignored) {
+    public ReturnCode filterCell(final Cell ignored) {
       visitedKeyValues.add(ignored);
       return ReturnCode.INCLUDE;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java
index 0b1c368..a5d04d2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java
@@ -97,7 +97,7 @@ public class TestMultiRowRangeFilter {
      * Expected :SEEK_NEXT_USING_HINT
      * Actual   :INCLUDE
      * */
-    assertEquals(Filter.ReturnCode.SEEK_NEXT_USING_HINT, filter.filterKeyValue(null));
+    assertEquals(Filter.ReturnCode.SEEK_NEXT_USING_HINT, filter.filterCell(null));
   }
 
   @Test
@@ -107,15 +107,15 @@ public class TestMultiRowRangeFilter {
             new MultiRowRangeFilter.RowRange(Bytes.toBytes("d"), true, Bytes.toBytes("e"), true)
     ));
     filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("a")));
-    assertEquals(Filter.ReturnCode.SEEK_NEXT_USING_HINT, filter.filterKeyValue(null));
+    assertEquals(Filter.ReturnCode.SEEK_NEXT_USING_HINT, filter.filterCell(null));
     filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("b")));
-    assertEquals(Filter.ReturnCode.INCLUDE, filter.filterKeyValue(null));
+    assertEquals(Filter.ReturnCode.INCLUDE, filter.filterCell(null));
     filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("c")));
-    assertEquals(Filter.ReturnCode.INCLUDE, filter.filterKeyValue(null));
+    assertEquals(Filter.ReturnCode.INCLUDE, filter.filterCell(null));
     filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("d")));
-    assertEquals(Filter.ReturnCode.INCLUDE, filter.filterKeyValue(null));
+    assertEquals(Filter.ReturnCode.INCLUDE, filter.filterCell(null));
     filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("e")));
-    assertEquals(Filter.ReturnCode.INCLUDE, filter.filterKeyValue(null));
+    assertEquals(Filter.ReturnCode.INCLUDE, filter.filterCell(null));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueExcludeFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueExcludeFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueExcludeFilter.java
index 2d223dc..764d033 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueExcludeFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueExcludeFilter.java
@@ -37,7 +37,7 @@ import java.util.ArrayList;
 /**
  * Tests for {@link SingleColumnValueExcludeFilter}. Because this filter
  * extends {@link SingleColumnValueFilter}, only the added functionality is
- * tested. That is, method filterKeyValue(KeyValue).
+ * tested. That is, method filterCell(Cell).
  *
  */
 @Category({FilterTests.class, SmallTests.class})
@@ -50,17 +50,17 @@ public class TestSingleColumnValueExcludeFilter {
   private static final byte[] VAL_2 = Bytes.toBytes("ab");
 
   /**
-   * Test the overridden functionality of filterKeyValue(KeyValue)
+   * Test the overridden functionality of filterCell(Cell)
    * @throws Exception
    */
   @Test
-  public void testFilterKeyValue() throws Exception {
+  public void testFilterCell() throws Exception {
     Filter filter = new SingleColumnValueExcludeFilter(COLUMN_FAMILY, COLUMN_QUALIFIER,
     CompareOperator.EQUAL, VAL_1);
 
     // A 'match' situation
     List<Cell> kvs = new ArrayList<>();
-    KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
+    KeyValue c = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
 
     kvs.add (new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1));
     kvs.add (new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1));
@@ -69,21 +69,21 @@ public class TestSingleColumnValueExcludeFilter {
     filter.filterRowCells(kvs);
 
     assertEquals("resultSize", kvs.size(), 2);
-    assertTrue("leftKV1", CellComparatorImpl.COMPARATOR.compare(kvs.get(0), kv) == 0);
-    assertTrue("leftKV2", CellComparatorImpl.COMPARATOR.compare(kvs.get(1), kv) == 0);
+    assertTrue("leftKV1", CellComparatorImpl.COMPARATOR.compare(kvs.get(0), c) == 0);
+    assertTrue("leftKV2", CellComparatorImpl.COMPARATOR.compare(kvs.get(1), c) == 0);
     assertFalse("allRemainingWhenMatch", filter.filterAllRemaining());
 
     // A 'mismatch' situation
     filter.reset();
     // INCLUDE expected because test column has not yet passed
-    kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
-    assertTrue("otherColumn", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
+    c = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
+    assertTrue("otherColumn", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
     // Test column will pass (wont match), expect NEXT_ROW
-    kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
-    assertTrue("testedMismatch", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
+    c = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
+    assertTrue("testedMismatch", filter.filterCell(c) == Filter.ReturnCode.NEXT_ROW);
     // After a mismatch (at least with LatestVersionOnly), subsequent columns are EXCLUDE
-    kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
-    assertTrue("otherColumn", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
+    c = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
+    assertTrue("otherColumn", filter.filterCell(c) == Filter.ReturnCode.NEXT_ROW);
   }
 
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java
index a2167ce..c1ef1ee 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java
@@ -104,143 +104,143 @@ public class TestSingleColumnValueFilter {
   public void testLongComparator() throws IOException {
     Filter filter = new SingleColumnValueFilter(COLUMN_FAMILY,
         COLUMN_QUALIFIER, CompareOperator.GREATER, new LongComparator(100L));
-    KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
+    KeyValue cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
       Bytes.toBytes(1L));
-    assertTrue("less than", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
+    assertTrue("less than", filter.filterCell(cell) == Filter.ReturnCode.NEXT_ROW);
     filter.reset();
-    byte[] buffer = kv.getBuffer();
+    byte[] buffer = cell.getBuffer();
     Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
-    assertTrue("less than", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW);
+    assertTrue("less than", filter.filterCell(c) == Filter.ReturnCode.NEXT_ROW);
     filter.reset();
 
-    kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
+    cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
       Bytes.toBytes(100L));
-    assertTrue("Equals 100", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
+    assertTrue("Equals 100", filter.filterCell(cell) == Filter.ReturnCode.NEXT_ROW);
     filter.reset();
-    buffer = kv.getBuffer();
+    buffer = cell.getBuffer();
     c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
-    assertTrue("Equals 100", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW);
+    assertTrue("Equals 100", filter.filterCell(c) == Filter.ReturnCode.NEXT_ROW);
     filter.reset();
 
-    kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
+    cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
       Bytes.toBytes(120L));
-    assertTrue("include 120", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
+    assertTrue("include 120", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
     filter.reset();
-    buffer = kv.getBuffer();
+    buffer = cell.getBuffer();
     c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
-    assertTrue("include 120", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
+    assertTrue("include 120", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
   }
 
   private void basicFilterTests(SingleColumnValueFilter filter)
       throws Exception {
-    KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
-    assertTrue("basicFilter1", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
-    byte[] buffer = kv.getBuffer();
+    KeyValue cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
+    assertTrue("basicFilter1", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
+    byte[] buffer = cell.getBuffer();
     Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
-    assertTrue("basicFilter1", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
-    kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_3);
-    assertTrue("basicFilter2", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
-    buffer = kv.getBuffer();
+    assertTrue("basicFilter1", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
+    cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_3);
+    assertTrue("basicFilter2", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
+    buffer = cell.getBuffer();
     c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
-    assertTrue("basicFilter2", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
-    kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_4);
-    assertTrue("basicFilter3", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
-    buffer = kv.getBuffer();
+    assertTrue("basicFilter2", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
+    cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_4);
+    assertTrue("basicFilter3", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
+    buffer = cell.getBuffer();
     c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
-    assertTrue("basicFilter3", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
+    assertTrue("basicFilter3", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
     assertFalse("basicFilterNotNull", filter.filterRow());
     filter.reset();
-    kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1);
-    assertTrue("basicFilter4", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
-    buffer = kv.getBuffer();
+    cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1);
+    assertTrue("basicFilter4", filter.filterCell(cell) == Filter.ReturnCode.NEXT_ROW);
+    buffer = cell.getBuffer();
     c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
-    assertTrue("basicFilter4", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW);
-    kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
-    assertTrue("basicFilter4", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
-    buffer = kv.getBuffer();
+    assertTrue("basicFilter4", filter.filterCell(c) == Filter.ReturnCode.NEXT_ROW);
+    cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
+    assertTrue("basicFilter4", filter.filterCell(cell) == Filter.ReturnCode.NEXT_ROW);
+    buffer = cell.getBuffer();
     c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
-    assertTrue("basicFilter4", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW);
+    assertTrue("basicFilter4", filter.filterCell(c) == Filter.ReturnCode.NEXT_ROW);
     assertFalse("basicFilterAllRemaining", filter.filterAllRemaining());
     assertTrue("basicFilterNotNull", filter.filterRow());
     filter.reset();
     filter.setLatestVersionOnly(false);
-    kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1);
-    assertTrue("basicFilter5", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
-    buffer = kv.getBuffer();
+    cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1);
+    assertTrue("basicFilter5", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
+    buffer = cell.getBuffer();
     c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
-    assertTrue("basicFilter5", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
-    kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
-    assertTrue("basicFilter5", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
-    buffer = kv.getBuffer();
+    assertTrue("basicFilter5", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
+    cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
+    assertTrue("basicFilter5", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
+    buffer = cell.getBuffer();
     c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
-    assertTrue("basicFilter5", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
+    assertTrue("basicFilter5", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
     assertFalse("basicFilterNotNull", filter.filterRow());
   }
 
   private void nullFilterTests(Filter filter) throws Exception {
     ((SingleColumnValueFilter) filter).setFilterIfMissing(true);
-    KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, FULLSTRING_1);
-    assertTrue("null1", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
-    byte[] buffer = kv.getBuffer();
+    KeyValue cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, FULLSTRING_1);
+    assertTrue("null1", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
+    byte[] buffer = cell.getBuffer();
     Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
-    assertTrue("null1", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
+    assertTrue("null1", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
     assertFalse("null1FilterRow", filter.filterRow());
     filter.reset();
-    kv = new KeyValue(ROW, COLUMN_FAMILY, Bytes.toBytes("qual2"), FULLSTRING_2);
-    assertTrue("null2", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
-    buffer = kv.getBuffer();
+    cell = new KeyValue(ROW, COLUMN_FAMILY, Bytes.toBytes("qual2"), FULLSTRING_2);
+    assertTrue("null2", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
+    buffer = cell.getBuffer();
     c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
-    assertTrue("null2", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
+    assertTrue("null2", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
     assertTrue("null2FilterRow", filter.filterRow());
   }
 
   private void substrFilterTests(Filter filter)
       throws Exception {
-    KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
+    KeyValue cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
       FULLSTRING_1);
     assertTrue("substrTrue",
-      filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
-    byte[] buffer = kv.getBuffer();
+      filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
+    byte[] buffer = cell.getBuffer();
     Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
-    assertTrue("substrTrue", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
-    kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
+    assertTrue("substrTrue", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
+    cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
       FULLSTRING_2);
-    assertTrue("substrFalse", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
-    buffer = kv.getBuffer();
+    assertTrue("substrFalse", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
+    buffer = cell.getBuffer();
     c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
-    assertTrue("substrFalse", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
+    assertTrue("substrFalse", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
     assertFalse("substrFilterAllRemaining", filter.filterAllRemaining());
     assertFalse("substrFilterNotNull", filter.filterRow());
   }
 
   private void regexFilterTests(Filter filter)
       throws Exception {
-    KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
+    KeyValue cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
       FULLSTRING_1);
     assertTrue("regexTrue",
-      filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
-    byte[] buffer = kv.getBuffer();
+      filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
+    byte[] buffer = cell.getBuffer();
     Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
-    assertTrue("regexTrue", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
-    kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
+    assertTrue("regexTrue", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
+    cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
       FULLSTRING_2);
-    assertTrue("regexFalse", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
-    buffer = kv.getBuffer();
+    assertTrue("regexFalse", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
+    buffer = cell.getBuffer();
     c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
-    assertTrue("regexFalse", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
+    assertTrue("regexFalse", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
     assertFalse("regexFilterAllRemaining", filter.filterAllRemaining());
     assertFalse("regexFilterNotNull", filter.filterRow());
   }
 
   private void regexPatternFilterTests(Filter filter)
       throws Exception {
-    KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
+    KeyValue cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
       FULLSTRING_1);
     assertTrue("regexTrue",
-      filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
-    byte[] buffer = kv.getBuffer();
+      filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
+    byte[] buffer = cell.getBuffer();
     Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
-    assertTrue("regexTrue", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
+    assertTrue("regexTrue", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
     assertFalse("regexFilterAllRemaining", filter.filterAllRemaining());
     assertFalse("regexFilterNotNull", filter.filterRow());
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/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 41bd997..d75bec0 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
@@ -3451,7 +3451,7 @@ public class TestHRegion {
       scan.setLoadColumnFamiliesOnDemand(true);
       Filter bogusFilter = new FilterBase() {
         @Override
-        public ReturnCode filterKeyValue(Cell ignored) throws IOException {
+        public ReturnCode filterCell(final Cell ignored) throws IOException {
           return ReturnCode.INCLUDE;
         }
         @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
index 1382603..e90656d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
@@ -1140,7 +1140,7 @@ public class TestHStore {
       }
     }, new FilterBase() {
       @Override
-      public Filter.ReturnCode filterKeyValue(Cell v) throws IOException {
+      public Filter.ReturnCode filterCell(final Cell c) throws IOException {
         return ReturnCode.INCLUDE;
       }
     }, expectedSize);
@@ -1164,7 +1164,7 @@ public class TestHStore {
       }
     }, new FilterBase() {
       @Override
-      public Filter.ReturnCode filterKeyValue(Cell v) throws IOException {
+      public Filter.ReturnCode filterCell(final Cell c) throws IOException {
         if (timeToGoNextRow.get()) {
           timeToGoNextRow.set(false);
           return ReturnCode.NEXT_ROW;
@@ -1193,7 +1193,7 @@ public class TestHStore {
       }
     }, new FilterBase() {
       @Override
-      public Filter.ReturnCode filterKeyValue(Cell v) throws IOException {
+      public Filter.ReturnCode filterCell(final Cell c) throws IOException {
         if (timeToGetHint.get()) {
           timeToGetHint.set(false);
           return Filter.ReturnCode.SEEK_NEXT_USING_HINT;

http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java
index b9e77c6..d2d861e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java
@@ -273,7 +273,7 @@ public class TestScannerHeartbeatMessages {
   public static class SparseFilter extends FilterBase {
 
     @Override
-    public ReturnCode filterKeyValue(Cell v) throws IOException {
+    public ReturnCode filterCell(final Cell v) throws IOException {
       try {
         Thread.sleep(CLIENT_TIMEOUT / 2 + 100);
       } catch (InterruptedException e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java
index 6f3996b..a94c59c 100644
--- a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java
+++ b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java
@@ -107,7 +107,7 @@ public class SparkSQLPushDownFilter extends FilterBase{
   }
 
   @Override
-  public ReturnCode filterKeyValue(Cell c) throws IOException {
+  public ReturnCode filterCell(final Cell c) throws IOException {
 
     //If the map RowValueMap is empty then we need to populate
     // the row key


Mime
View raw message