@InterfaceAudience.LimitedPrivate(value="Coprocesssor") @InterfaceStability.Evolving public class RegionCoprocessorHost extends CoprocessorHost<org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.RegionEnvironment>
Region
.CoprocessorHost.Environment
ABORT_ON_ERROR_KEY, abortable, conf, coprocessors, COPROCESSORS_ENABLED_CONF_KEY, DEFAULT_ABORT_ON_ERROR, DEFAULT_COPROCESSORS_ENABLED, DEFAULT_USER_COPROCESSORS_ENABLED, loadSequence, MASTER_COPROCESSOR_CONF_KEY, pathPrefix, REGION_COPROCESSOR_CONF_KEY, REGIONSERVER_COPROCESSOR_CONF_KEY, USER_COPROCESSORS_ENABLED_CONF_KEY, USER_REGION_COPROCESSOR_CONF_KEY, WAL_COPROCESSOR_CONF_KEY
Constructor and Description |
---|
RegionCoprocessorHost(Region region,
RegionServerServices rsServices,
org.apache.hadoop.conf.Configuration conf)
Constructor
|
Modifier and Type | Method and Description |
---|---|
org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.RegionEnvironment |
createEnvironment(Class<?> implClass,
Coprocessor instance,
int priority,
int seq,
org.apache.hadoop.conf.Configuration conf)
Called when a new Coprocessor class is loaded
|
Result |
postAppend(Append append,
Result result) |
void |
postBatchMutate(MiniBatchOperationInProgress<Mutation> miniBatchOp) |
void |
postBatchMutateIndispensably(MiniBatchOperationInProgress<Mutation> miniBatchOp,
boolean success) |
boolean |
postBulkLoadHFile(List<Pair<byte[],String>> familyPaths,
boolean hasLoaded) |
boolean |
postCheckAndDelete(byte[] row,
byte[] family,
byte[] qualifier,
CompareFilter.CompareOp compareOp,
ByteArrayComparable comparator,
Delete delete,
boolean result) |
boolean |
postCheckAndPut(byte[] row,
byte[] family,
byte[] qualifier,
CompareFilter.CompareOp compareOp,
ByteArrayComparable comparator,
Put put,
boolean result) |
void |
postClose(boolean abortRequested)
Invoked after a region is closed
|
void |
postCloseRegionOperation(Region.Operation op) |
void |
postCommitStoreFile(byte[] family,
org.apache.hadoop.fs.Path srcPath,
org.apache.hadoop.fs.Path dstPath) |
void |
postCompact(Store store,
StoreFile resultFile,
CompactionRequest request)
Called after the store compaction has completed.
|
void |
postCompactSelection(Store store,
com.google.common.collect.ImmutableList<StoreFile> selected,
CompactionRequest request)
Called after the
StoreFile s to be compacted have been selected from the available
candidates. |
void |
postCompleteSplit()
Invoked after a split is completed irrespective of a failure or success.
|
void |
postDelete(Delete delete,
WALEdit edit,
Durability durability) |
void |
postEndpointInvocation(com.google.protobuf.Service service,
String methodName,
com.google.protobuf.Message request,
com.google.protobuf.Message.Builder responseBuilder) |
boolean |
postExists(Get get,
boolean exists) |
void |
postFlush()
Invoked after a memstore flush
|
void |
postFlush(Store store,
StoreFile storeFile)
Invoked after a memstore flush
|
void |
postGet(Get get,
List<Cell> results) |
void |
postGetClosestRowBefore(byte[] row,
byte[] family,
Result result) |
Result |
postIncrement(Increment increment,
Result result) |
DeleteTracker |
postInstantiateDeleteTracker(DeleteTracker tracker) |
void |
postLogReplay()
Invoked after log replay on region
|
Cell |
postMutationBeforeWAL(RegionObserver.MutationType opType,
Mutation mutation,
Cell oldCell,
Cell newCell) |
void |
postOpen()
Invoked after a region open
|
void |
postPut(Put put,
WALEdit edit,
Durability durability) |
void |
postRollBackSplit()
Invoked just after the rollback of a failed split is done
|
void |
postScannerClose(InternalScanner s) |
boolean |
postScannerFilterRow(InternalScanner s,
byte[] currentRow,
int offset,
short length)
This will be called by the scan flow when the current scanned row is being filtered out by the
filter.
|
boolean |
postScannerNext(InternalScanner s,
List<Result> results,
int limit,
boolean hasMore) |
RegionScanner |
postScannerOpen(Scan scan,
RegionScanner s) |
void |
postSplit(Region l,
Region r)
Invoked just after a split
|
void |
postStartRegionOperation(Region.Operation op) |
StoreFile.Reader |
postStoreFileReaderOpen(org.apache.hadoop.fs.FileSystem fs,
org.apache.hadoop.fs.Path p,
FSDataInputStreamWrapper in,
long size,
CacheConfig cacheConf,
Reference r,
StoreFile.Reader reader) |
void |
postWALRestore(HRegionInfo info,
HLogKey logKey,
WALEdit logEdit)
Deprecated.
|
void |
postWALRestore(HRegionInfo info,
WALKey logKey,
WALEdit logEdit) |
Result |
preAppend(Append append) |
Result |
preAppendAfterRowLock(Append append) |
boolean |
preBatchMutate(MiniBatchOperationInProgress<Mutation> miniBatchOp) |
boolean |
preBulkLoadHFile(List<Pair<byte[],String>> familyPaths) |
Boolean |
preCheckAndDelete(byte[] row,
byte[] family,
byte[] qualifier,
CompareFilter.CompareOp compareOp,
ByteArrayComparable comparator,
Delete delete) |
Boolean |
preCheckAndDeleteAfterRowLock(byte[] row,
byte[] family,
byte[] qualifier,
CompareFilter.CompareOp compareOp,
ByteArrayComparable comparator,
Delete delete) |
Boolean |
preCheckAndPut(byte[] row,
byte[] family,
byte[] qualifier,
CompareFilter.CompareOp compareOp,
ByteArrayComparable comparator,
Put put) |
Boolean |
preCheckAndPutAfterRowLock(byte[] row,
byte[] family,
byte[] qualifier,
CompareFilter.CompareOp compareOp,
ByteArrayComparable comparator,
Put put) |
void |
preClose(boolean abortRequested)
Invoked before a region is closed
|
boolean |
preCommitStoreFile(byte[] family,
List<Pair<org.apache.hadoop.fs.Path,org.apache.hadoop.fs.Path>> pairs) |
InternalScanner |
preCompact(Store store,
InternalScanner scanner,
ScanType scanType,
CompactionRequest request)
Called prior to rewriting the store files selected for compaction
|
InternalScanner |
preCompactScannerOpen(Store store,
List<StoreFileScanner> scanners,
ScanType scanType,
long earliestPutTs,
CompactionRequest request,
long readPoint)
|
boolean |
preCompactSelection(Store store,
List<StoreFile> candidates,
CompactionRequest request)
Called prior to selecting the
StoreFile s for compaction from the list of currently
available candidates. |
boolean |
preDelete(Delete delete,
WALEdit edit,
Durability durability) |
com.google.protobuf.Message |
preEndpointInvocation(com.google.protobuf.Service service,
String methodName,
com.google.protobuf.Message request) |
Boolean |
preExists(Get get) |
void |
preFlush()
Invoked before a memstore flush
|
InternalScanner |
preFlush(Store store,
InternalScanner scanner)
Invoked before a memstore flush
|
InternalScanner |
preFlushScannerOpen(Store store,
KeyValueScanner memstoreScanner,
long readPoint)
|
boolean |
preGet(Get get,
List<Cell> results) |
boolean |
preGetClosestRowBefore(byte[] row,
byte[] family,
Result result) |
Result |
preIncrement(Increment increment) |
Result |
preIncrementAfterRowLock(Increment increment) |
void |
preOpen()
Invoked before a region open.
|
boolean |
prePrepareTimeStampForDeleteVersion(Mutation mutation,
Cell kv,
byte[] byteNow,
Get get) |
boolean |
prePut(Put put,
WALEdit edit,
Durability durability) |
void |
preRollBackSplit()
Invoked just before the rollback of a failed split is started
|
boolean |
preScannerClose(InternalScanner s) |
Boolean |
preScannerNext(InternalScanner s,
List<Result> results,
int limit) |
RegionScanner |
preScannerOpen(Scan scan) |
void |
preSplit()
Invoked just before a split
|
void |
preSplit(byte[] splitRow)
Invoked just before a split
|
void |
preSplitAfterPONR() |
boolean |
preSplitBeforePONR(byte[] splitKey,
List<Mutation> metaEntries) |
StoreFile.Reader |
preStoreFileReaderOpen(org.apache.hadoop.fs.FileSystem fs,
org.apache.hadoop.fs.Path p,
FSDataInputStreamWrapper in,
long size,
CacheConfig cacheConf,
Reference r) |
KeyValueScanner |
preStoreScannerOpen(Store store,
Scan scan,
NavigableSet<byte[]> targetCols)
|
boolean |
preWALRestore(HRegionInfo info,
HLogKey logKey,
WALEdit logEdit)
Deprecated.
|
boolean |
preWALRestore(HRegionInfo info,
WALKey logKey,
WALEdit logEdit) |
static void |
testTableCoprocessorAttrs(org.apache.hadoop.conf.Configuration conf,
HTableDescriptor htd)
Sanity check the table coprocessor attributes of the supplied schema.
|
abortServer, abortServer, findCoprocessor, findCoprocessorEnvironment, findCoprocessors, getCoprocessors, getLoadedCoprocessors, handleCoprocessorThrowable, legacyWarning, load, load, load, loadInstance, loadSystemCoprocessors, shutdown, useLegacyMethod
public RegionCoprocessorHost(Region region, RegionServerServices rsServices, org.apache.hadoop.conf.Configuration conf)
region
- the regionrsServices
- interface to available region server functionalityconf
- the configurationpublic static void testTableCoprocessorAttrs(org.apache.hadoop.conf.Configuration conf, HTableDescriptor htd) throws IOException
conf
- htd
- IOException
public org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.RegionEnvironment createEnvironment(Class<?> implClass, Coprocessor instance, int priority, int seq, org.apache.hadoop.conf.Configuration conf)
CoprocessorHost
createEnvironment
in class CoprocessorHost<org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.RegionEnvironment>
public void preOpen() throws IOException
IOException
- Signals that an I/O exception has occurred.public void postOpen()
public void postLogReplay()
public void preClose(boolean abortRequested) throws IOException
abortRequested
- true if the server is abortingIOException
public void postClose(boolean abortRequested)
abortRequested
- true if the server is abortingpublic InternalScanner preCompactScannerOpen(Store store, List<StoreFileScanner> scanners, ScanType scanType, long earliestPutTs, CompactionRequest request, long readPoint) throws IOException
IOException
public boolean preCompactSelection(Store store, List<StoreFile> candidates, CompactionRequest request) throws IOException
StoreFile
s for compaction from the list of currently
available candidates.store
- The store where compaction is being requestedcandidates
- The currently available store filesrequest
- custom compaction requesttrue
, skip the normal selection process and use the current listIOException
public void postCompactSelection(Store store, com.google.common.collect.ImmutableList<StoreFile> selected, CompactionRequest request)
StoreFile
s to be compacted have been selected from the available
candidates.store
- The store where compaction is being requestedselected
- The store files selected to compactrequest
- custom compactionpublic InternalScanner preCompact(Store store, InternalScanner scanner, ScanType scanType, CompactionRequest request) throws IOException
store
- the store being compactedscanner
- the scanner used to read store data during compactionscanType
- type of Scanrequest
- the compaction that will be executedIOException
public void postCompact(Store store, StoreFile resultFile, CompactionRequest request) throws IOException
store
- the store being compactedresultFile
- the new store file written during compactionrequest
- the compaction that is being executedIOException
public InternalScanner preFlush(Store store, InternalScanner scanner) throws IOException
IOException
public void preFlush() throws IOException
IOException
public InternalScanner preFlushScannerOpen(Store store, KeyValueScanner memstoreScanner, long readPoint) throws IOException
RegionObserver.preFlushScannerOpen(ObserverContext,
Store, KeyValueScanner, InternalScanner, long)
IOException
public void postFlush() throws IOException
IOException
public void postFlush(Store store, StoreFile storeFile) throws IOException
IOException
public void preSplit() throws IOException
IOException
public void preSplit(byte[] splitRow) throws IOException
IOException
public void postSplit(Region l, Region r) throws IOException
l
- the new left-hand daughter regionr
- the new right-hand daughter regionIOException
public boolean preSplitBeforePONR(byte[] splitKey, List<Mutation> metaEntries) throws IOException
IOException
public void preSplitAfterPONR() throws IOException
IOException
public void preRollBackSplit() throws IOException
IOException
public void postRollBackSplit() throws IOException
IOException
public void postCompleteSplit() throws IOException
IOException
public boolean preGetClosestRowBefore(byte[] row, byte[] family, Result result) throws IOException
row
- the row keyfamily
- the familyresult
- the result set from the regionIOException
- Exceptionpublic void postGetClosestRowBefore(byte[] row, byte[] family, Result result) throws IOException
row
- the row keyfamily
- the familyresult
- the result set from the regionIOException
- Exceptionpublic boolean preGet(Get get, List<Cell> results) throws IOException
get
- the Get requestIOException
- Exceptionpublic void postGet(Get get, List<Cell> results) throws IOException
get
- the Get requestresults
- the result settIOException
- Exceptionpublic Boolean preExists(Get get) throws IOException
get
- the Get requestIOException
- Exceptionpublic boolean postExists(Get get, boolean exists) throws IOException
get
- the Get requestexists
- the result returned by the region serverIOException
- Exceptionpublic boolean prePut(Put put, WALEdit edit, Durability durability) throws IOException
put
- The Put objectedit
- The WALEdit object.durability
- The durability usedIOException
- Exceptionpublic boolean prePrepareTimeStampForDeleteVersion(Mutation mutation, Cell kv, byte[] byteNow, Get get) throws IOException
mutation
- - the current mutationkv
- - the current cellbyteNow
- - current timestamp in bytesget
- - the get that could be used
Note that the get only does not specify the family and qualifier that should be usedIOException
- Exceptionpublic void postPut(Put put, WALEdit edit, Durability durability) throws IOException
put
- The Put objectedit
- The WALEdit object.durability
- The durability usedIOException
- Exceptionpublic boolean preDelete(Delete delete, WALEdit edit, Durability durability) throws IOException
delete
- The Delete objectedit
- The WALEdit object.durability
- The durability usedIOException
- Exceptionpublic void postDelete(Delete delete, WALEdit edit, Durability durability) throws IOException
delete
- The Delete objectedit
- The WALEdit object.durability
- The durability usedIOException
- Exceptionpublic boolean preBatchMutate(MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException
miniBatchOp
- IOException
public void postBatchMutate(MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException
miniBatchOp
- IOException
public void postBatchMutateIndispensably(MiniBatchOperationInProgress<Mutation> miniBatchOp, boolean success) throws IOException
IOException
public Boolean preCheckAndPut(byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, ByteArrayComparable comparator, Put put) throws IOException
row
- row to checkfamily
- column familyqualifier
- column qualifiercompareOp
- the comparison operationcomparator
- the comparatorput
- data to put if check succeedsIOException
- epublic Boolean preCheckAndPutAfterRowLock(byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, ByteArrayComparable comparator, Put put) throws IOException
row
- row to checkfamily
- column familyqualifier
- column qualifiercompareOp
- the comparison operationcomparator
- the comparatorput
- data to put if check succeedsIOException
- epublic boolean postCheckAndPut(byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, ByteArrayComparable comparator, Put put, boolean result) throws IOException
row
- row to checkfamily
- column familyqualifier
- column qualifiercompareOp
- the comparison operationcomparator
- the comparatorput
- data to put if check succeedsIOException
- epublic Boolean preCheckAndDelete(byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, ByteArrayComparable comparator, Delete delete) throws IOException
row
- row to checkfamily
- column familyqualifier
- column qualifiercompareOp
- the comparison operationcomparator
- the comparatordelete
- delete to commit if check succeedsIOException
- epublic Boolean preCheckAndDeleteAfterRowLock(byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, ByteArrayComparable comparator, Delete delete) throws IOException
row
- row to checkfamily
- column familyqualifier
- column qualifiercompareOp
- the comparison operationcomparator
- the comparatordelete
- delete to commit if check succeedsIOException
- epublic boolean postCheckAndDelete(byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, ByteArrayComparable comparator, Delete delete, boolean result) throws IOException
row
- row to checkfamily
- column familyqualifier
- column qualifiercompareOp
- the comparison operationcomparator
- the comparatordelete
- delete to commit if check succeedsIOException
- epublic Result preAppend(Append append) throws IOException
append
- append objectIOException
- if an error occurred on the coprocessorpublic Result preAppendAfterRowLock(Append append) throws IOException
append
- append objectIOException
- if an error occurred on the coprocessorpublic Result preIncrement(Increment increment) throws IOException
increment
- increment objectIOException
- if an error occurred on the coprocessorpublic Result preIncrementAfterRowLock(Increment increment) throws IOException
increment
- increment objectIOException
- if an error occurred on the coprocessorpublic Result postAppend(Append append, Result result) throws IOException
append
- Append objectresult
- the result returned by the appendIOException
- if an error occurred on the coprocessorpublic Result postIncrement(Increment increment, Result result) throws IOException
increment
- increment objectresult
- the result returned by postIncrementIOException
- if an error occurred on the coprocessorpublic RegionScanner preScannerOpen(Scan scan) throws IOException
scan
- the Scan specificationIOException
- Exceptionpublic KeyValueScanner preStoreScannerOpen(Store store, Scan scan, NavigableSet<byte[]> targetCols) throws IOException
IOException
public RegionScanner postScannerOpen(Scan scan, RegionScanner s) throws IOException
scan
- the Scan specifications
- the scannerIOException
- Exceptionpublic Boolean preScannerNext(InternalScanner s, List<Result> results, int limit) throws IOException
s
- the scannerresults
- the result set returned by the region serverlimit
- the maximum number of results to returnIOException
- Exceptionpublic boolean postScannerNext(InternalScanner s, List<Result> results, int limit, boolean hasMore) throws IOException
s
- the scannerresults
- the result set returned by the region serverlimit
- the maximum number of results to returnhasMore
- IOException
- Exceptionpublic boolean postScannerFilterRow(InternalScanner s, byte[] currentRow, int offset, short length) throws IOException
s
- the scannercurrentRow
- The current rowkey which got filtered outoffset
- offset to rowkeylength
- length of rowkeyIOException
public boolean preScannerClose(InternalScanner s) throws IOException
s
- the scannerIOException
- Exceptionpublic void postScannerClose(InternalScanner s) throws IOException
IOException
- Exceptionpublic boolean preWALRestore(HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException
info
- logKey
- logEdit
- IOException
@Deprecated public boolean preWALRestore(HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException
preWALRestore(HRegionInfo, WALKey, WALEdit)
IOException
public void postWALRestore(HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException
info
- logKey
- logEdit
- IOException
@Deprecated public void postWALRestore(HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException
postWALRestore(HRegionInfo, WALKey, WALEdit)
IOException
public boolean preCommitStoreFile(byte[] family, List<Pair<org.apache.hadoop.fs.Path,org.apache.hadoop.fs.Path>> pairs) throws IOException
IOException
public void postCommitStoreFile(byte[] family, org.apache.hadoop.fs.Path srcPath, org.apache.hadoop.fs.Path dstPath) throws IOException
IOException
public boolean preBulkLoadHFile(List<Pair<byte[],String>> familyPaths) throws IOException
familyPaths
- pairs of { CF, file path } submitted for bulk loadIOException
public boolean postBulkLoadHFile(List<Pair<byte[],String>> familyPaths, boolean hasLoaded) throws IOException
familyPaths
- pairs of { CF, file path } submitted for bulk loadhasLoaded
- whether load was successful or notIOException
public void postStartRegionOperation(Region.Operation op) throws IOException
IOException
public void postCloseRegionOperation(Region.Operation op) throws IOException
IOException
public StoreFile.Reader preStoreFileReaderOpen(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf, Reference r) throws IOException
fs
- fileystem to read fromp
- path to the filein
- FSDataInputStreamWrapper
size
- Full size of the filecacheConf
- r
- original reference file. This will be not null only when reading a split file.IOException
public StoreFile.Reader postStoreFileReaderOpen(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf, Reference r, StoreFile.Reader reader) throws IOException
fs
- fileystem to read fromp
- path to the filein
- FSDataInputStreamWrapper
size
- Full size of the filecacheConf
- r
- original reference file. This will be not null only when reading a split file.reader
- the base reader instanceIOException
public Cell postMutationBeforeWAL(RegionObserver.MutationType opType, Mutation mutation, Cell oldCell, Cell newCell) throws IOException
IOException
public com.google.protobuf.Message preEndpointInvocation(com.google.protobuf.Service service, String methodName, com.google.protobuf.Message request) throws IOException
IOException
public void postEndpointInvocation(com.google.protobuf.Service service, String methodName, com.google.protobuf.Message request, com.google.protobuf.Message.Builder responseBuilder) throws IOException
IOException
public DeleteTracker postInstantiateDeleteTracker(DeleteTracker tracker) throws IOException
IOException
Copyright © 2007–2019 The Apache Software Foundation. All rights reserved.