@InterfaceAudience.LimitedPrivate(value="Coprocesssor") @InterfaceStability.Evolving public interface RegionObserver extends Coprocessor
Modifier and Type | Interface and Description |
---|---|
static class |
RegionObserver.MutationType
Mutation type for postMutationBeforeWAL hook
|
Coprocessor.State
PRIORITY_HIGHEST, PRIORITY_LOWEST, PRIORITY_SYSTEM, PRIORITY_USER, VERSION
Modifier and Type | Method and Description |
---|---|
Result |
postAppend(ObserverContext<RegionCoprocessorEnvironment> c,
Append append,
Result result)
Called after Append
|
void |
postBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
MiniBatchOperationInProgress<Mutation> miniBatchOp)
This will be called after applying a batch of Mutations on a region.
|
void |
postBatchMutateIndispensably(ObserverContext<RegionCoprocessorEnvironment> ctx,
MiniBatchOperationInProgress<Mutation> miniBatchOp,
boolean success)
Called after the completion of batch put/delete and will be called even if the batch operation
fails
|
boolean |
postBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
List<Pair<byte[],String>> familyPaths,
boolean hasLoaded)
Called after bulkLoadHFile.
|
boolean |
postCheckAndDelete(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] row,
byte[] family,
byte[] qualifier,
CompareFilter.CompareOp compareOp,
ByteArrayComparable comparator,
Delete delete,
boolean result)
Called after checkAndDelete
|
boolean |
postCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] row,
byte[] family,
byte[] qualifier,
CompareFilter.CompareOp compareOp,
ByteArrayComparable comparator,
Put put,
boolean result)
Called after checkAndPut
|
void |
postClose(ObserverContext<RegionCoprocessorEnvironment> c,
boolean abortRequested)
Called after the region is reported as closed to the master.
|
void |
postCloseRegionOperation(ObserverContext<RegionCoprocessorEnvironment> ctx,
Region.Operation operation)
Called after releasing read lock in
Region.closeRegionOperation() . |
void |
postCompact(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
StoreFile resultFile)
Deprecated.
|
void |
postCompact(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
StoreFile resultFile,
CompactionRequest request)
Called after compaction has completed and the new store file has been moved in to place.
|
void |
postCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
com.google.common.collect.ImmutableList<StoreFile> selected)
Deprecated.
|
void |
postCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
com.google.common.collect.ImmutableList<StoreFile> selected,
CompactionRequest request)
Called after the
StoreFile s to compact have been selected from the available
candidates. |
void |
postCompleteSplit(ObserverContext<RegionCoprocessorEnvironment> ctx)
Called after any split request is processed.
|
void |
postDelete(ObserverContext<RegionCoprocessorEnvironment> c,
Delete delete,
WALEdit edit,
Durability durability)
Called after the client deletes a value.
|
boolean |
postExists(ObserverContext<RegionCoprocessorEnvironment> c,
Get get,
boolean exists)
Called after the client tests for existence using a Get.
|
void |
postFlush(ObserverContext<RegionCoprocessorEnvironment> c)
Deprecated.
use
preFlush(ObserverContext, Store, InternalScanner) instead. |
void |
postFlush(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
StoreFile resultFile)
Called after a Store's memstore is flushed to disk.
|
void |
postGetClosestRowBefore(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] row,
byte[] family,
Result result)
Called after a client makes a GetClosestRowBefore request.
|
void |
postGetOp(ObserverContext<RegionCoprocessorEnvironment> c,
Get get,
List<Cell> result)
Called after the client performs a Get
|
Result |
postIncrement(ObserverContext<RegionCoprocessorEnvironment> c,
Increment increment,
Result result)
Called after increment
|
long |
postIncrementColumnValue(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] row,
byte[] family,
byte[] qualifier,
long amount,
boolean writeToWAL,
long result)
Deprecated.
This hook is no longer called by the RegionServer
|
DeleteTracker |
postInstantiateDeleteTracker(ObserverContext<RegionCoprocessorEnvironment> ctx,
DeleteTracker delTracker)
Called after the ScanQueryMatcher creates ScanDeleteTracker.
|
void |
postLogReplay(ObserverContext<RegionCoprocessorEnvironment> c)
Called after the log replay on the region is over.
|
Cell |
postMutationBeforeWAL(ObserverContext<RegionCoprocessorEnvironment> ctx,
RegionObserver.MutationType opType,
Mutation mutation,
Cell oldCell,
Cell newCell)
Called after a new cell has been created during an increment operation, but before
it is committed to the WAL or memstore.
|
void |
postOpen(ObserverContext<RegionCoprocessorEnvironment> c)
Called after the region is reported as open to the master.
|
void |
postPut(ObserverContext<RegionCoprocessorEnvironment> c,
Put put,
WALEdit edit,
Durability durability)
Called after the client stores a value.
|
void |
postRollBackSplit(ObserverContext<RegionCoprocessorEnvironment> ctx)
This will be called after the roll back of the split region is completed
|
void |
postScannerClose(ObserverContext<RegionCoprocessorEnvironment> c,
InternalScanner s)
Called after the client closes a scanner.
|
boolean |
postScannerFilterRow(ObserverContext<RegionCoprocessorEnvironment> c,
InternalScanner s,
byte[] currentRow,
int offset,
short length,
boolean hasMore)
This will be called by the scan flow when the current scanned row is being filtered out by the
filter.
|
boolean |
postScannerNext(ObserverContext<RegionCoprocessorEnvironment> c,
InternalScanner s,
List<Result> result,
int limit,
boolean hasNext)
Called after the client asks for the next row on a scanner.
|
RegionScanner |
postScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
Scan scan,
RegionScanner s)
Called after the client opens a new scanner.
|
void |
postSplit(ObserverContext<RegionCoprocessorEnvironment> c,
Region l,
Region r)
Deprecated.
Use postCompleteSplit() instead
|
void |
postStartRegionOperation(ObserverContext<RegionCoprocessorEnvironment> ctx,
Region.Operation operation)
This will be called for region operations where read lock is acquired in
Region.startRegionOperation() . |
StoreFile.Reader |
postStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx,
org.apache.hadoop.fs.FileSystem fs,
org.apache.hadoop.fs.Path p,
FSDataInputStreamWrapper in,
long size,
CacheConfig cacheConf,
Reference r,
StoreFile.Reader reader)
Called after the creation of Reader for a store file.
|
void |
postWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
HRegionInfo info,
WALKey logKey,
WALEdit logEdit)
Called after a
WALEdit
replayed for this region. |
void |
postWALRestore(ObserverContext<RegionCoprocessorEnvironment> ctx,
HRegionInfo info,
HLogKey logKey,
WALEdit logEdit)
Deprecated.
|
Result |
preAppend(ObserverContext<RegionCoprocessorEnvironment> c,
Append append)
Called before Append.
|
Result |
preAppendAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c,
Append append)
Called before Append but after acquiring rowlock.
|
void |
preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
MiniBatchOperationInProgress<Mutation> miniBatchOp)
This will be called for every batch mutation operation happening at the server.
|
void |
preBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
List<Pair<byte[],String>> familyPaths)
Called before bulkLoadHFile.
|
boolean |
preCheckAndDelete(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] row,
byte[] family,
byte[] qualifier,
CompareFilter.CompareOp compareOp,
ByteArrayComparable comparator,
Delete delete,
boolean result)
Called before checkAndDelete.
|
boolean |
preCheckAndDeleteAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] row,
byte[] family,
byte[] qualifier,
CompareFilter.CompareOp compareOp,
ByteArrayComparable comparator,
Delete delete,
boolean result)
Called before checkAndDelete but after acquiring rowock.
|
boolean |
preCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] row,
byte[] family,
byte[] qualifier,
CompareFilter.CompareOp compareOp,
ByteArrayComparable comparator,
Put put,
boolean result)
Called before checkAndPut.
|
boolean |
preCheckAndPutAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] row,
byte[] family,
byte[] qualifier,
CompareFilter.CompareOp compareOp,
ByteArrayComparable comparator,
Put put,
boolean result)
Called before checkAndPut but after acquiring rowlock.
|
void |
preClose(ObserverContext<RegionCoprocessorEnvironment> c,
boolean abortRequested)
Called before the region is reported as closed to the master.
|
InternalScanner |
preCompact(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
InternalScanner scanner,
ScanType scanType)
Deprecated.
|
InternalScanner |
preCompact(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
InternalScanner scanner,
ScanType scanType,
CompactionRequest request)
Called prior to writing the
StoreFile s selected for compaction into a new
StoreFile . |
InternalScanner |
preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
List<? extends KeyValueScanner> scanners,
ScanType scanType,
long earliestPutTs,
InternalScanner s)
|
InternalScanner |
preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
List<? extends KeyValueScanner> scanners,
ScanType scanType,
long earliestPutTs,
InternalScanner s,
CompactionRequest request)
Called prior to writing the
StoreFile s selected for compaction into a new
StoreFile and prior to creating the scanner used to read the input files. |
void |
preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
List<StoreFile> candidates)
Deprecated.
|
void |
preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
List<StoreFile> candidates,
CompactionRequest request)
Called prior to selecting the
StoreFiles to compact from the list of
available candidates. |
void |
preDelete(ObserverContext<RegionCoprocessorEnvironment> c,
Delete delete,
WALEdit edit,
Durability durability)
Called before the client deletes a value.
|
boolean |
preExists(ObserverContext<RegionCoprocessorEnvironment> c,
Get get,
boolean exists)
Called before the client tests for existence using a Get.
|
void |
preFlush(ObserverContext<RegionCoprocessorEnvironment> c)
Deprecated.
|
InternalScanner |
preFlush(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
InternalScanner scanner)
Called before a Store's memstore is flushed to disk.
|
InternalScanner |
preFlushScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
KeyValueScanner memstoreScanner,
InternalScanner s)
Called before a memstore is flushed to disk and prior to creating the scanner to read from
the memstore.
|
void |
preGetClosestRowBefore(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] row,
byte[] family,
Result result)
Called before a client makes a GetClosestRowBefore request.
|
void |
preGetOp(ObserverContext<RegionCoprocessorEnvironment> c,
Get get,
List<Cell> result)
Called before the client performs a Get
|
Result |
preIncrement(ObserverContext<RegionCoprocessorEnvironment> c,
Increment increment)
Called before Increment.
|
Result |
preIncrementAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c,
Increment increment)
Called before Increment but after acquiring rowlock.
|
long |
preIncrementColumnValue(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] row,
byte[] family,
byte[] qualifier,
long amount,
boolean writeToWAL)
Deprecated.
This hook is no longer called by the RegionServer
|
void |
preOpen(ObserverContext<RegionCoprocessorEnvironment> c)
Called before the region is reported as open to the master.
|
void |
prePrepareTimeStampForDeleteVersion(ObserverContext<RegionCoprocessorEnvironment> c,
Mutation mutation,
Cell cell,
byte[] byteNow,
Get get)
Called before the server updates the timestamp for version delete with latest timestamp.
|
void |
prePut(ObserverContext<RegionCoprocessorEnvironment> c,
Put put,
WALEdit edit,
Durability durability)
Called before the client stores a value.
|
void |
preRollBackSplit(ObserverContext<RegionCoprocessorEnvironment> ctx)
This will be called before the roll back of the split region is completed
|
void |
preScannerClose(ObserverContext<RegionCoprocessorEnvironment> c,
InternalScanner s)
Called before the client closes a scanner.
|
boolean |
preScannerNext(ObserverContext<RegionCoprocessorEnvironment> c,
InternalScanner s,
List<Result> result,
int limit,
boolean hasNext)
Called before the client asks for the next row on a scanner.
|
RegionScanner |
preScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
Scan scan,
RegionScanner s)
Called before the client opens a new scanner.
|
void |
preSplit(ObserverContext<RegionCoprocessorEnvironment> c)
Deprecated.
Use preSplit(
final ObserverContext
|
void |
preSplit(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] splitRow)
Called before the region is split.
|
void |
preSplitAfterPONR(ObserverContext<RegionCoprocessorEnvironment> ctx)
This will be called after PONR step as part of split transaction
Calling
ObserverContext.bypass() has no
effect in this hook. |
void |
preSplitBeforePONR(ObserverContext<RegionCoprocessorEnvironment> ctx,
byte[] splitKey,
List<Mutation> metaEntries)
This will be called before PONR step as part of split transaction.
|
StoreFile.Reader |
preStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx,
org.apache.hadoop.fs.FileSystem fs,
org.apache.hadoop.fs.Path p,
FSDataInputStreamWrapper in,
long size,
CacheConfig cacheConf,
Reference r,
StoreFile.Reader reader)
Called before creation of Reader for a store file.
|
KeyValueScanner |
preStoreScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
Scan scan,
NavigableSet<byte[]> targetCols,
KeyValueScanner s)
Called before a store opens a new scanner.
|
void |
preWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
HRegionInfo info,
WALKey logKey,
WALEdit logEdit)
Called before a
WALEdit
replayed for this region. |
void |
preWALRestore(ObserverContext<RegionCoprocessorEnvironment> ctx,
HRegionInfo info,
HLogKey logKey,
WALEdit logEdit)
Deprecated.
|
start, stop
void preOpen(ObserverContext<RegionCoprocessorEnvironment> c) throws IOException
c
- the environment provided by the region serverIOException
- if an error occurred on the coprocessorvoid postOpen(ObserverContext<RegionCoprocessorEnvironment> c)
c
- the environment provided by the region servervoid postLogReplay(ObserverContext<RegionCoprocessorEnvironment> c)
c
- the environment provided by the region serverInternalScanner preFlushScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Store store, KeyValueScanner memstoreScanner, InternalScanner s) throws IOException
StoreFile
or null to perform the default processing.
Calling ObserverContext.bypass()
has no
effect in this hook.c
- the environment provided by the region serverstore
- the store being flushedmemstoreScanner
- the scanner for the memstore that is flusheds
- the base scanner, if not null
, from previous RegionObserver in the chainnull
if the default implementation
is to be used.IOException
- if an error occurred on the coprocessorvoid preFlush(ObserverContext<RegionCoprocessorEnvironment> c) throws IOException
preFlush(ObserverContext, Store, InternalScanner)
insteadc
- the environment provided by the region serverIOException
- if an error occurred on the coprocessorInternalScanner preFlush(ObserverContext<RegionCoprocessorEnvironment> c, Store store, InternalScanner scanner) throws IOException
c
- the environment provided by the region serverstore
- the store where compaction is being requestedscanner
- the scanner over existing data used in the store filenull
unless the implementation is writing new store files on its own.IOException
- if an error occurred on the coprocessorvoid postFlush(ObserverContext<RegionCoprocessorEnvironment> c) throws IOException
preFlush(ObserverContext, Store, InternalScanner)
instead.c
- the environment provided by the region serverIOException
- if an error occurred on the coprocessorvoid postFlush(ObserverContext<RegionCoprocessorEnvironment> c, Store store, StoreFile resultFile) throws IOException
c
- the environment provided by the region serverstore
- the store being flushedresultFile
- the new store file written out during compactionIOException
- if an error occurred on the coprocessorvoid preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store, List<StoreFile> candidates, CompactionRequest request) throws IOException
StoreFiles
to compact from the list of
available candidates. To alter the files used for compaction, you may mutate the passed in list
of candidates.c
- the environment provided by the region serverstore
- the store where compaction is being requestedcandidates
- the store files currently available for compactionrequest
- custom compaction requestIOException
- if an error occurred on the coprocessor@Deprecated void preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store, List<StoreFile> candidates) throws IOException
preCompactSelection(ObserverContext, Store, List, CompactionRequest)
insteadStoreFile
s to compact from the list of available
candidates. To alter the files used for compaction, you may mutate the passed in list of
candidates.c
- the environment provided by the region serverstore
- the store where compaction is being requestedcandidates
- the store files currently available for compactionIOException
- if an error occurred on the coprocessorvoid postCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store, com.google.common.collect.ImmutableList<StoreFile> selected, CompactionRequest request)
StoreFile
s to compact have been selected from the available
candidates.c
- the environment provided by the region serverstore
- the store being compactedselected
- the store files selected to compactrequest
- custom compaction request@Deprecated void postCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store, com.google.common.collect.ImmutableList<StoreFile> selected)
postCompactSelection(ObserverContext, Store, ImmutableList,
CompactionRequest)
instead.StoreFile
s to compact have been selected from the available
candidates.c
- the environment provided by the region serverstore
- the store being compactedselected
- the store files selected to compactInternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> c, Store store, InternalScanner scanner, ScanType scanType, CompactionRequest request) throws IOException
StoreFile
s selected for compaction into a new
StoreFile
. To override or modify the compaction process, implementing classes have two
options:
InternalScanner
with a custom implementation that is returned
from this method. The custom scanner can then inspect
KeyValue
s from the wrapped
scanner, applying its own policy to what gets written.ObserverContext.bypass()
and provide a
custom implementation for writing of new StoreFile
s. Note: any implementations
bypassing core compaction using this approach must write out new store files themselves or the
existing data will no longer be available after compaction.c
- the environment provided by the region serverstore
- the store being compactedscanner
- the scanner over existing data used in the store file rewritingscanType
- type of Scanrequest
- the requested compactionnull
unless the
implementation is writing new store files on its own.IOException
- if an error occurred on the coprocessor@Deprecated InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> c, Store store, InternalScanner scanner, ScanType scanType) throws IOException
preCompact(ObserverContext, Store, InternalScanner,
ScanType, CompactionRequest)
insteadStoreFile
s selected for compaction into a new
StoreFile
. To override or modify the compaction process, implementing classes have two
options:
InternalScanner
with a custom implementation that is returned
from this method. The custom scanner can then inspect
KeyValue
s from the wrapped
scanner, applying its own policy to what gets written.ObserverContext.bypass()
and provide a
custom implementation for writing of new StoreFile
s. Note: any implementations
bypassing core compaction using this approach must write out new store files themselves or the
existing data will no longer be available after compaction.c
- the environment provided by the region serverstore
- the store being compactedscanner
- the scanner over existing data used in the store file rewritingscanType
- type of Scannull
unless the
implementation is writing new store files on its own.IOException
- if an error occurred on the coprocessorInternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Store store, List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs, InternalScanner s, CompactionRequest request) throws IOException
StoreFile
s selected for compaction into a new
StoreFile
and prior to creating the scanner used to read the input files. To override
or modify the compaction process, implementing classes can return a new scanner to provide the
KeyValues to be stored into the new StoreFile
or null to perform the default
processing. Calling ObserverContext.bypass()
has no
effect in this hook.c
- the environment provided by the region serverstore
- the store being compactedscanners
- the list StoreFileScanner
s
to be read fromscanType
- the ScanType
indicating whether this is a major or minor compactionearliestPutTs
- timestamp of the earliest put that was found in any of the involved store
filess
- the base scanner, if not null
, from previous RegionObserver in the chainrequest
- the requested compactionnull
if the default implementation is to
be used.IOException
- if an error occurred on the coprocessor@Deprecated InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Store store, List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs, InternalScanner s) throws IOException
preCompactScannerOpen(ObserverContext, Store, List, ScanType, long,
InternalScanner, CompactionRequest)
instead.StoreFile
s selected for compaction into a new
StoreFile
and prior to creating the scanner used to read the input files. To override
or modify the compaction process, implementing classes can return a new scanner to provide the
KeyValues to be stored into the new StoreFile
or null to perform the default
processing. Calling ObserverContext.bypass()
has no
effect in this hook.c
- the environment provided by the region serverstore
- the store being compactedscanners
- the list StoreFileScanner
s
to be read fromscanType
- the ScanType
indicating whether this is a major or minor compactionearliestPutTs
- timestamp of the earliest put that was found in any of the involved store
filess
- the base scanner, if not null
, from previous RegionObserver in the chainnull
if the default implementation is to
be used.IOException
- if an error occurred on the coprocessorvoid postCompact(ObserverContext<RegionCoprocessorEnvironment> c, Store store, StoreFile resultFile, CompactionRequest request) throws IOException
c
- the environment provided by the region serverstore
- the store being compactedresultFile
- the new store file written out during compactionrequest
- the requested compactionIOException
- if an error occurred on the coprocessor@Deprecated void postCompact(ObserverContext<RegionCoprocessorEnvironment> c, Store store, StoreFile resultFile) throws IOException
postCompact(ObserverContext, Store, StoreFile, CompactionRequest)
insteadc
- the environment provided by the region serverstore
- the store being compactedresultFile
- the new store file written out during compactionIOException
- if an error occurred on the coprocessorvoid preSplit(ObserverContext<RegionCoprocessorEnvironment> c) throws IOException
c
- the environment provided by the region server
(e.getRegion() returns the parent region)IOException
- if an error occurred on the coprocessorvoid preSplit(ObserverContext<RegionCoprocessorEnvironment> c, byte[] splitRow) throws IOException
c
- the environment provided by the region server
(e.getRegion() returns the parent region)IOException
- if an error occurred on the coprocessorvoid postSplit(ObserverContext<RegionCoprocessorEnvironment> c, Region l, Region r) throws IOException
c
- the environment provided by the region server
(e.getRegion() returns the parent region)l
- the left daughter regionr
- the right daughter regionIOException
- if an error occurred on the coprocessorvoid preSplitBeforePONR(ObserverContext<RegionCoprocessorEnvironment> ctx, byte[] splitKey, List<Mutation> metaEntries) throws IOException
ObserverContext.bypass()
rollback the splitctx
- splitKey
- metaEntries
- IOException
void preSplitAfterPONR(ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException
ObserverContext.bypass()
has no
effect in this hook.ctx
- IOException
void preRollBackSplit(ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException
ctx
- IOException
void postRollBackSplit(ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException
ctx
- IOException
void postCompleteSplit(ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException
ctx
- IOException
void preClose(ObserverContext<RegionCoprocessorEnvironment> c, boolean abortRequested) throws IOException
c
- the environment provided by the region serverabortRequested
- true if the region server is abortingIOException
void postClose(ObserverContext<RegionCoprocessorEnvironment> c, boolean abortRequested)
c
- the environment provided by the region serverabortRequested
- true if the region server is abortingvoid preGetClosestRowBefore(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, byte[] family, Result result) throws IOException
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverrow
- the rowfamily
- the familyresult
- The result to return to the client if default processing
is bypassed. Can be modified. Will not be used if default processing
is not bypassed.IOException
- if an error occurred on the coprocessorvoid postGetClosestRowBefore(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, byte[] family, Result result) throws IOException
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverrow
- the rowfamily
- the desired familyresult
- the result to return to the client, modify as necessaryIOException
- if an error occurred on the coprocessorvoid preGetOp(ObserverContext<RegionCoprocessorEnvironment> c, Get get, List<Cell> result) throws IOException
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverget
- the Get requestresult
- The result to return to the client if default processing
is bypassed. Can be modified. Will not be used if default processing
is not bypassed.IOException
- if an error occurred on the coprocessorvoid postGetOp(ObserverContext<RegionCoprocessorEnvironment> c, Get get, List<Cell> result) throws IOException
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverget
- the Get requestresult
- the result to return to the client, modify as necessaryIOException
- if an error occurred on the coprocessorboolean preExists(ObserverContext<RegionCoprocessorEnvironment> c, Get get, boolean exists) throws IOException
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverget
- the Get requestexists
- IOException
- if an error occurred on the coprocessorboolean postExists(ObserverContext<RegionCoprocessorEnvironment> c, Get get, boolean exists) throws IOException
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverget
- the Get requestexists
- the result returned by the region serverIOException
- if an error occurred on the coprocessorvoid prePut(ObserverContext<RegionCoprocessorEnvironment> c, Put put, WALEdit edit, Durability durability) throws IOException
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverput
- The Put objectedit
- The WALEdit object that will be written to the waldurability
- Persistence guarantee for this PutIOException
- if an error occurred on the coprocessorvoid postPut(ObserverContext<RegionCoprocessorEnvironment> c, Put put, WALEdit edit, Durability durability) throws IOException
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverput
- The Put objectedit
- The WALEdit object for the waldurability
- Persistence guarantee for this PutIOException
- if an error occurred on the coprocessorvoid preDelete(ObserverContext<RegionCoprocessorEnvironment> c, Delete delete, WALEdit edit, Durability durability) throws IOException
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverdelete
- The Delete objectedit
- The WALEdit object for the waldurability
- Persistence guarantee for this DeleteIOException
- if an error occurred on the coprocessorvoid prePrepareTimeStampForDeleteVersion(ObserverContext<RegionCoprocessorEnvironment> c, Mutation mutation, Cell cell, byte[] byteNow, Get get) throws IOException
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region servermutation
- - the parent mutation associated with this delete cellcell
- - The deleteColumn with latest version cellbyteNow
- - timestamp bytesget
- - the get formed using the current cell's row.
Note that the get does not specify the family and qualifierIOException
void postDelete(ObserverContext<RegionCoprocessorEnvironment> c, Delete delete, WALEdit edit, Durability durability) throws IOException
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverdelete
- The Delete objectedit
- The WALEdit object for the waldurability
- Persistence guarantee for this DeleteIOException
- if an error occurred on the coprocessorvoid preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c, MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException
MiniBatchOperationInProgress.setOperationStatus(int, OperationStatus)
),
RegionObserver
can make Region to skip these Mutations.c
- the environment provided by the region serverminiBatchOp
- batch of Mutations getting applied to region.IOException
- if an error occurred on the coprocessorvoid postBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c, MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException
c
- the environment provided by the region serverminiBatchOp
- batch of Mutations applied to region.IOException
- if an error occurred on the coprocessorvoid postStartRegionOperation(ObserverContext<RegionCoprocessorEnvironment> ctx, Region.Operation operation) throws IOException
Region.startRegionOperation()
.ctx
- operation
- The operation is about to be taken on the regionIOException
void postCloseRegionOperation(ObserverContext<RegionCoprocessorEnvironment> ctx, Region.Operation operation) throws IOException
Region.closeRegionOperation()
.ctx
- operation
- IOException
void postBatchMutateIndispensably(ObserverContext<RegionCoprocessorEnvironment> ctx, MiniBatchOperationInProgress<Mutation> miniBatchOp, boolean success) throws IOException
ctx
- miniBatchOp
- success
- true if batch operation is successful otherwise false.IOException
boolean preCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, ByteArrayComparable comparator, Put put, boolean result) throws IOException
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverrow
- row to checkfamily
- column familyqualifier
- column qualifiercompareOp
- the comparison operationcomparator
- the comparatorput
- data to put if check succeedsresult
- IOException
- if an error occurred on the coprocessorboolean preCheckAndPutAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, ByteArrayComparable comparator, Put put, boolean result) throws IOException
Note: Caution to be taken for not doing any long time operation in this hook. Row will be locked for longer time. Trying to acquire lock on another row, within this, can lead to potential deadlock.
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverrow
- row to checkfamily
- column familyqualifier
- column qualifiercompareOp
- the comparison operationcomparator
- the comparatorput
- data to put if check succeedsresult
- IOException
- if an error occurred on the coprocessorboolean postCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, ByteArrayComparable comparator, Put put, boolean result) throws IOException
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverrow
- row to checkfamily
- column familyqualifier
- column qualifiercompareOp
- the comparison operationcomparator
- the comparatorput
- data to put if check succeedsresult
- from the checkAndPutIOException
- if an error occurred on the coprocessorboolean preCheckAndDelete(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, ByteArrayComparable comparator, Delete delete, boolean result) throws IOException
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverrow
- row to checkfamily
- column familyqualifier
- column qualifiercompareOp
- the comparison operationcomparator
- the comparatordelete
- delete to commit if check succeedsresult
- IOException
- if an error occurred on the coprocessorboolean preCheckAndDeleteAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, ByteArrayComparable comparator, Delete delete, boolean result) throws IOException
Note: Caution to be taken for not doing any long time operation in this hook. Row will be locked for longer time. Trying to acquire lock on another row, within this, can lead to potential deadlock.
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverrow
- row to checkfamily
- column familyqualifier
- column qualifiercompareOp
- the comparison operationcomparator
- the comparatordelete
- delete to commit if check succeedsresult
- IOException
- if an error occurred on the coprocessorboolean postCheckAndDelete(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, ByteArrayComparable comparator, Delete delete, boolean result) throws IOException
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverrow
- row to checkfamily
- column familyqualifier
- column qualifiercompareOp
- the comparison operationcomparator
- the comparatordelete
- delete to commit if check succeedsresult
- from the CheckAndDeleteIOException
- if an error occurred on the coprocessor@Deprecated long preIncrementColumnValue(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, long amount, boolean writeToWAL) throws IOException
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverrow
- row to checkfamily
- column familyqualifier
- column qualifieramount
- long amount to incrementwriteToWAL
- true if the change should be written to the WALIOException
- if an error occurred on the coprocessor@Deprecated long postIncrementColumnValue(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, long amount, boolean writeToWAL, long result) throws IOException
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverrow
- row to checkfamily
- column familyqualifier
- column qualifieramount
- long amount to incrementwriteToWAL
- true if the change should be written to the WALresult
- the result returned by incrementColumnValueIOException
- if an error occurred on the coprocessorResult preAppend(ObserverContext<RegionCoprocessorEnvironment> c, Append append) throws IOException
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverappend
- Append objectIOException
- if an error occurred on the coprocessorResult preAppendAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c, Append append) throws IOException
Note: Caution to be taken for not doing any long time operation in this hook. Row will be locked for longer time. Trying to acquire lock on another row, within this, can lead to potential deadlock.
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverappend
- Append objectIOException
- if an error occurred on the coprocessorResult postAppend(ObserverContext<RegionCoprocessorEnvironment> c, Append append, Result result) throws IOException
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverappend
- Append objectresult
- the result returned by incrementIOException
- if an error occurred on the coprocessorResult preIncrement(ObserverContext<RegionCoprocessorEnvironment> c, Increment increment) throws IOException
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverincrement
- increment objectIOException
- if an error occurred on the coprocessorResult preIncrementAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c, Increment increment) throws IOException
Note: Caution to be taken for not doing any long time operation in this hook. Row will be locked for longer time. Trying to acquire lock on another row, within this, can lead to potential deadlock.
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverincrement
- increment objectIOException
- if an error occurred on the coprocessorResult postIncrement(ObserverContext<RegionCoprocessorEnvironment> c, Increment increment, Result result) throws IOException
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverincrement
- increment objectresult
- the result returned by incrementIOException
- if an error occurred on the coprocessorRegionScanner preScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Scan scan, RegionScanner s) throws IOException
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverscan
- the Scan specifications
- if not null, the base scannerIOException
- if an error occurred on the coprocessorKeyValueScanner preStoreScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Store store, Scan scan, NavigableSet<byte[]> targetCols, KeyValueScanner s) throws IOException
See preFlushScannerOpen(ObserverContext, Store, KeyValueScanner, InternalScanner)
and preCompactScannerOpen(ObserverContext,
Store, List, ScanType, long, InternalScanner)
to override scanners created for flushes or compactions, resp.
Call CoprocessorEnvironment#complete to skip any subsequent chained
coprocessors.
Calling ObserverContext.bypass()
has no
effect in this hook.
c
- the environment provided by the region serverstore
- the store being scannedscan
- the Scan specificationtargetCols
- columns to be used in the scanners
- the base scanner, if not null
, from previous RegionObserver in the chainnull
to use the default implementationIOException
- if an error occurred on the coprocessorRegionScanner postScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Scan scan, RegionScanner s) throws IOException
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region serverscan
- the Scan specifications
- if not null, the base scannerIOException
- if an error occurred on the coprocessorboolean preScannerNext(ObserverContext<RegionCoprocessorEnvironment> c, InternalScanner s, List<Result> result, int limit, boolean hasNext) throws IOException
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region servers
- the scannerresult
- The result to return to the client if default processing
is bypassed. Can be modified. Will not be returned if default processing
is not bypassed.limit
- the maximum number of results to returnhasNext
- the 'has more' indicationIOException
- if an error occurred on the coprocessorboolean postScannerNext(ObserverContext<RegionCoprocessorEnvironment> c, InternalScanner s, List<Result> result, int limit, boolean hasNext) throws IOException
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region servers
- the scannerresult
- the result to return to the client, can be modifiedlimit
- the maximum number of results to returnhasNext
- the 'has more' indicationIOException
- if an error occurred on the coprocessorboolean postScannerFilterRow(ObserverContext<RegionCoprocessorEnvironment> c, InternalScanner s, byte[] currentRow, int offset, short length, boolean hasMore) throws IOException
boolean filterRowKey(byte [] buffer, int offset, int length)
returning trueboolean filterRow()
returning truevoid filterRow(List kvs)
removing all the kvs from the passed Listc
- the environment provided by the region servers
- the scannercurrentRow
- The current rowkey which got filtered outoffset
- offset to rowkeylength
- length of rowkeyhasMore
- the 'has more' indicationIOException
void preScannerClose(ObserverContext<RegionCoprocessorEnvironment> c, InternalScanner s) throws IOException
Call CoprocessorEnvironment#bypass to skip default actions
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region servers
- the scannerIOException
- if an error occurred on the coprocessorvoid postScannerClose(ObserverContext<RegionCoprocessorEnvironment> c, InternalScanner s) throws IOException
Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
c
- the environment provided by the region servers
- the scannerIOException
- if an error occurred on the coprocessorvoid preWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException
WALEdit
replayed for this region.IOException
@Deprecated void preWALRestore(ObserverContext<RegionCoprocessorEnvironment> ctx, HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException
preWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit)
WALEdit
replayed for this region.
This method is left in place to maintain binary compatibility with older
RegionObserver
s. If an implementation directly overrides
preWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit)
then this version
won't be called at all, barring problems with the Security Manager. To work correctly
in the presence of a strict Security Manager, or in the case of an implementation that
relies on a parent class to implement preWALRestore, you should implement this method
as a call to the non-deprecated version.
Users of this method will see all edits that can be treated as HLogKey. If there are
edits that can't be treated as HLogKey they won't be offered to coprocessors that rely
on this method. If a coprocessor gets skipped because of this mechanism, a log message
at ERROR will be generated per coprocessor on the logger for CoprocessorHost
once per
classloader.IOException
void postWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException
WALEdit
replayed for this region.IOException
@Deprecated void postWALRestore(ObserverContext<RegionCoprocessorEnvironment> ctx, HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException
postWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit)
WALEdit
replayed for this region.
This method is left in place to maintain binary compatibility with older
RegionObserver
s. If an implementation directly overrides
postWALRestore(ObserverContext, HRegionInfo, WALKey, WALEdit)
then this version
won't be called at all, barring problems with the Security Manager. To work correctly
in the presence of a strict Security Manager, or in the case of an implementation that
relies on a parent class to implement preWALRestore, you should implement this method
as a call to the non-deprecated version.
Users of this method will see all edits that can be treated as HLogKey. If there are
edits that can't be treated as HLogKey they won't be offered to coprocessors that rely
on this method. If a coprocessor gets skipped because of this mechanism, a log message
at ERROR will be generated per coprocessor on the logger for CoprocessorHost
once per
classloader.IOException
void preBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx, List<Pair<byte[],String>> familyPaths) throws IOException
ctx
- familyPaths
- pairs of { CF, HFile path } submitted for bulk load. Adding
or removing from this list will add or remove HFiles to be bulk loaded.IOException
boolean postBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx, List<Pair<byte[],String>> familyPaths, boolean hasLoaded) throws IOException
ctx
- familyPaths
- pairs of { CF, HFile path } submitted for bulk loadhasLoaded
- whether the bulkLoad was successfulIOException
StoreFile.Reader preStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx, 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
ObserverContext.bypass()
has no
effect in this hook.ctx
- the environment provided by the region serverfs
- 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, if not null
, from previous RegionObserver in the chainIOException
StoreFile.Reader postStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx, 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
ctx
- the environment provided by the region serverfs
- 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
Cell postMutationBeforeWAL(ObserverContext<RegionCoprocessorEnvironment> ctx, RegionObserver.MutationType opType, Mutation mutation, Cell oldCell, Cell newCell) throws IOException
ObserverContext.bypass()
has no
effect in this hook.ctx
- the environment provided by the region serveropType
- the operation typemutation
- the current mutationoldCell
- old cell containing previous valuenewCell
- the new cell containing the computed valueIOException
DeleteTracker postInstantiateDeleteTracker(ObserverContext<RegionCoprocessorEnvironment> ctx, DeleteTracker delTracker) throws IOException
ctx
- the environment provided by the region serverdelTracker
- the deleteTracker that is created by the QueryMatcherIOException
Copyright © 2007-2016 The Apache Software Foundation. All Rights Reserved.