HBASE-10225 Bug in calls to RegionObsever.postScannerFilterRow
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1553079 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
e6b2e7ff45
commit
9b2effdcc4
|
@ -436,10 +436,11 @@ public abstract class BaseRegionObserver implements RegionObserver {
|
|||
|
||||
@Override
|
||||
public boolean postScannerFilterRow(final ObserverContext<RegionCoprocessorEnvironment> 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<RegionCoprocessorEnvironment> e,
|
||||
final InternalScanner s) throws IOException {
|
||||
|
|
|
@ -967,12 +967,15 @@ public interface RegionObserver extends Coprocessor {
|
|||
* @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<RegionCoprocessorEnvironment> 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.
|
||||
|
|
|
@ -3944,8 +3944,8 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
resetFilters();
|
||||
// Calling the hook in CP which allows it to do a fast forward
|
||||
return this.region.getCoprocessorHost() == null
|
||||
|| this.region.getCoprocessorHost().postScannerFilterRow(this,
|
||||
currentRow);
|
||||
|| this.region.getCoprocessorHost()
|
||||
.postScannerFilterRow(this, currentRow, offset, length);
|
||||
}
|
||||
|
||||
protected boolean isStopRow(byte[] currentRow, int offset, short length) {
|
||||
|
|
|
@ -1503,11 +1503,13 @@ public class RegionCoprocessorHost
|
|||
* 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<RegionCoprocessorEnvironment> ctx = null;
|
||||
for (RegionEnvironment env : coprocessors) {
|
||||
|
@ -1515,7 +1517,7 @@ public class RegionCoprocessorHost
|
|||
ctx = ObserverContext.createAndPrepare(env, ctx);
|
||||
try {
|
||||
hasMore = ((RegionObserver) env.getInstance()).postScannerFilterRow(ctx, s, currentRow,
|
||||
hasMore);
|
||||
offset, length, hasMore);
|
||||
} catch (Throwable e) {
|
||||
handleCoprocessorThrowable(env, e);
|
||||
}
|
||||
|
|
|
@ -73,7 +73,7 @@ class ReversedRegionScannerImpl extends RegionScannerImpl {
|
|||
// 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);
|
||||
currentRow, offset, length);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue