Index: src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java (revision 1553074) +++ src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java (working copy) @@ -340,10 +340,11 @@ @Override public boolean postScannerFilterRow(final ObserverContext e, - final InternalScanner s, final byte[] currentRow, final boolean hasMore) throws IOException { + final InternalScanner s, final byte[] currentRow, final int offset, final short length, + final boolean hasMore) throws IOException { return hasMore; } - + @Override public void preScannerClose(final ObserverContext e, final InternalScanner s) throws IOException { Index: src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java (revision 1553074) +++ src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java (working copy) @@ -819,12 +819,15 @@ * @param c the environment provided by the region server * @param s the scanner * @param currentRow The current rowkey which got filtered out + * @param offset offset to rowkey + * @param length length of rowkey * @param hasMore the 'has more' indication * @return whether more rows are available for the scanner or not * @throws IOException */ boolean postScannerFilterRow(final ObserverContext c, - final InternalScanner s, final byte[] currentRow, final boolean hasMore) throws IOException; + final InternalScanner s, final byte[] currentRow, final int offset, final short length, + final boolean hasMore) throws IOException; /** * Called before the client closes a scanner. Index: src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (revision 1553074) +++ src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (working copy) @@ -4154,7 +4154,8 @@ resetFilters(); // Calling the hook in CP which allows it to do a fast forward if (this.region.getCoprocessorHost() != null) { - return this.region.getCoprocessorHost().postScannerFilterRow(this, currentRow); + return this.region.getCoprocessorHost().postScannerFilterRow(this, currentRow, offset, + length); } return true; } Index: src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java (revision 1553074) +++ src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java (working copy) @@ -1390,11 +1390,13 @@ * filter. * @param s the scanner * @param currentRow The current rowkey which got filtered out + * @param offset offset to rowkey + * @param length length of rowkey * @return whether more rows are available for the scanner or not * @throws IOException */ - public boolean postScannerFilterRow(final InternalScanner s, final byte[] currentRow) - throws IOException { + public boolean postScannerFilterRow(final InternalScanner s, final byte[] currentRow, int offset, + short length) throws IOException { boolean hasMore = true; // By default assume more rows there. ObserverContext ctx = null; for (RegionEnvironment env : coprocessors) { @@ -1402,7 +1404,7 @@ ctx = ObserverContext.createAndPrepare(env, ctx); try { hasMore = ((RegionObserver) env.getInstance()).postScannerFilterRow(ctx, s, currentRow, - hasMore); + offset, length, hasMore); } catch (Throwable e) { handleCoprocessorThrowable(env, e); }