HBASE-13642 Deprecate RegionObserver#postScannerFilterRow CP hook with byte[],int,int args in favor of taking Cell arg.
This commit is contained in:
parent
5e06ede3f7
commit
d69807663c
@ -24,6 +24,7 @@ import java.util.List;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
@ -31,6 +32,7 @@ import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
|
||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
|
||||
/***
|
||||
@ -89,4 +91,11 @@ public class ConstraintProcessor extends BaseRegionObserver {
|
||||
}
|
||||
// if we made it here, then the Put is valid
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean postScannerFilterRow(final ObserverContext<RegionCoprocessorEnvironment> e,
|
||||
final InternalScanner s, final Cell curRowCell, final boolean hasMore) throws IOException {
|
||||
// Impl in BaseRegionObserver might do unnecessary copy for Off heap backed Cells.
|
||||
return hasMore;
|
||||
}
|
||||
}
|
||||
|
@ -429,12 +429,22 @@ public abstract class BaseRegionObserver implements RegionObserver {
|
||||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public boolean postScannerFilterRow(final ObserverContext<RegionCoprocessorEnvironment> e,
|
||||
final InternalScanner s, final byte[] currentRow, final int offset, final short length,
|
||||
final boolean hasMore) throws IOException {
|
||||
return hasMore;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean postScannerFilterRow(final ObserverContext<RegionCoprocessorEnvironment> e,
|
||||
final InternalScanner s, final Cell curRowCell, final boolean hasMore) throws IOException {
|
||||
// TODO when cell is backed by DirectByteBuffer, we would need to copy row bytes to temp byte[]
|
||||
// and call old method for BC.
|
||||
return postScannerFilterRow(e, s, curRowCell.getRowArray(), curRowCell.getRowOffset(),
|
||||
curRowCell.getRowLength(), hasMore);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void preScannerClose(final ObserverContext<RegionCoprocessorEnvironment> e,
|
||||
final InternalScanner s) throws IOException {
|
||||
|
@ -1078,11 +1078,35 @@ public interface RegionObserver extends Coprocessor {
|
||||
* @param hasMore the 'has more' indication
|
||||
* @return whether more rows are available for the scanner or not
|
||||
* @throws IOException
|
||||
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
|
||||
* Instead use {@link #postScannerFilterRow(ObserverContext, InternalScanner, Cell, boolean)}
|
||||
*/
|
||||
@Deprecated
|
||||
boolean postScannerFilterRow(final ObserverContext<RegionCoprocessorEnvironment> c,
|
||||
final InternalScanner s, final byte[] currentRow, final int offset, final short length,
|
||||
final boolean hasMore) throws IOException;
|
||||
|
||||
|
||||
/**
|
||||
* This will be called by the scan flow when the current scanned row is being filtered out by the
|
||||
* filter. The filter may be filtering out the row via any of the below scenarios
|
||||
* <ol>
|
||||
* <li>
|
||||
* <code>boolean filterRowKey(byte [] buffer, int offset, int length)</code> returning true</li>
|
||||
* <li>
|
||||
* <code>boolean filterRow()</code> returning true</li>
|
||||
* <li>
|
||||
* <code>void filterRow(List<KeyValue> kvs)</code> removing all the kvs from the passed List</li>
|
||||
* </ol>
|
||||
* @param c the environment provided by the region server
|
||||
* @param s the scanner
|
||||
* @param curRowCell The cell in the current row which got filtered out
|
||||
* @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, Cell curRowCell, final boolean hasMore) throws IOException;
|
||||
|
||||
/**
|
||||
* Called before the client closes a scanner.
|
||||
* <p>
|
||||
|
@ -5556,7 +5556,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||
// Check if rowkey filter wants to exclude this row. If so, loop to next.
|
||||
// Technically, if we hit limits before on this row, we don't need this call.
|
||||
if (filterRowKey(currentRow, offset, length)) {
|
||||
boolean moreRows = nextRow(currentRow, offset, length);
|
||||
boolean moreRows = nextRow(current);
|
||||
if (!moreRows) {
|
||||
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
|
||||
}
|
||||
@ -5607,7 +5607,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||
|
||||
if ((isEmptyRow || ret == FilterWrapper.FilterRowRetCode.EXCLUDE) || filterRow()) {
|
||||
results.clear();
|
||||
boolean moreRows = nextRow(currentRow, offset, length);
|
||||
boolean moreRows = nextRow(current);
|
||||
if (!moreRows) {
|
||||
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
|
||||
}
|
||||
@ -5650,7 +5650,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||
// Double check to prevent empty rows from appearing in result. It could be
|
||||
// the case when SingleColumnValueExcludeFilter is used.
|
||||
if (results.isEmpty()) {
|
||||
boolean moreRows = nextRow(currentRow, offset, length);
|
||||
boolean moreRows = nextRow(current);
|
||||
if (!moreRows) {
|
||||
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
|
||||
}
|
||||
@ -5712,18 +5712,18 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||
&& filter.filterRowKey(row, offset, length);
|
||||
}
|
||||
|
||||
protected boolean nextRow(byte [] currentRow, int offset, short length) throws IOException {
|
||||
protected boolean nextRow(Cell curRowCell) throws IOException {
|
||||
assert this.joinedContinuationRow == null: "Trying to go to next row during joinedHeap read.";
|
||||
Cell next;
|
||||
while ((next = this.storeHeap.peek()) != null &&
|
||||
CellUtil.matchingRow(next, currentRow, offset, length)) {
|
||||
CellUtil.matchingRow(next, curRowCell)) {
|
||||
this.storeHeap.next(MOCKED_LIST);
|
||||
}
|
||||
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, offset, length);
|
||||
.postScannerFilterRow(this, curRowCell);
|
||||
}
|
||||
|
||||
protected boolean isStopRow(byte[] currentRow, int offset, short length) {
|
||||
|
@ -1363,20 +1363,18 @@ public class RegionCoprocessorHost
|
||||
* This will be called by the scan flow when the current scanned row is being filtered out by the
|
||||
* filter.
|
||||
* @param s the scanner
|
||||
* @param currentRow The current rowkey which got filtered out
|
||||
* @param offset offset to rowkey
|
||||
* @param length length of rowkey
|
||||
* @param curRowCell The cell in the current row which got filtered out
|
||||
* @return whether more rows are available for the scanner or not
|
||||
* @throws IOException
|
||||
*/
|
||||
public boolean postScannerFilterRow(final InternalScanner s, final byte[] currentRow,
|
||||
final int offset, final short length) throws IOException {
|
||||
public boolean postScannerFilterRow(final InternalScanner s, final Cell curRowCell)
|
||||
throws IOException {
|
||||
return execOperationWithResult(true,
|
||||
coprocessors.isEmpty() ? null : new RegionOperationWithResult<Boolean>() {
|
||||
@Override
|
||||
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
|
||||
throws IOException {
|
||||
setResult(oserver.postScannerFilterRow(ctx, s, currentRow, offset,length, getResult()));
|
||||
setResult(oserver.postScannerFilterRow(ctx, s, curRowCell, getResult()));
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -22,6 +22,8 @@ import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
|
||||
@ -63,17 +65,16 @@ class ReversedRegionScannerImpl extends RegionScannerImpl {
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean nextRow(byte[] currentRow, int offset, short length)
|
||||
protected boolean nextRow(Cell curRowCell)
|
||||
throws IOException {
|
||||
assert super.joinedContinuationRow == null : "Trying to go to next row during joinedHeap read.";
|
||||
byte row[] = new byte[length];
|
||||
System.arraycopy(currentRow, offset, row, 0, length);
|
||||
byte[] row = new byte[curRowCell.getRowLength()];
|
||||
CellUtil.copyRowTo(curRowCell, row, 0);
|
||||
this.storeHeap.seekToPreviousRow(KeyValueUtil.createFirstOnRow(row));
|
||||
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, offset, length);
|
||||
return this.region.getCoprocessorHost().postScannerFilterRow(this, curRowCell);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
@ -2047,6 +2047,13 @@ public class AccessController extends BaseMasterAndRegionObserver
|
||||
scannerOwners.remove(s);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean postScannerFilterRow(final ObserverContext<RegionCoprocessorEnvironment> e,
|
||||
final InternalScanner s, final Cell curRowCell, final boolean hasMore) throws IOException {
|
||||
// Impl in BaseRegionObserver might do unnecessary copy for Off heap backed Cells.
|
||||
return hasMore;
|
||||
}
|
||||
|
||||
/**
|
||||
* Verify, when servicing an RPC, that the caller is the scanner owner.
|
||||
* If so, we assume that access control is correctly enforced based on
|
||||
|
@ -772,6 +772,13 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
|
||||
return VisibilityLabelsProtos.VisibilityLabelsService.newReflectiveService(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean postScannerFilterRow(final ObserverContext<RegionCoprocessorEnvironment> e,
|
||||
final InternalScanner s, final Cell curRowCell, final boolean hasMore) throws IOException {
|
||||
// Impl in BaseRegionObserver might do unnecessary copy for Off heap backed Cells.
|
||||
return hasMore;
|
||||
}
|
||||
|
||||
/****************************** VisibilityEndpoint service related methods ******************************/
|
||||
@Override
|
||||
public synchronized void addLabels(RpcController controller, VisibilityLabelsRequest request,
|
||||
|
Loading…
x
Reference in New Issue
Block a user