@InterfaceAudience.LimitedPrivate(value="Coprocesssor") @InterfaceStability.Evolving public interface RegionObserver
Since most implementations will be interested in only a subset of hooks, this class uses 'default' functions to avoid having to add unnecessary overrides. When the functions are non-empty, it's simply to satisfy the compiler by returning value of expected (non-void) type. It is done in a way that these default definitions act as no-op. So our suggestion to implementation would be to not call these 'default' methods from overrides.
IOException
are reported back to client.CoprocessorHost.ABORT_ON_ERROR_KEY
is set to true, then the
server aborts.DoNotRetryIOException
is returned to the client.
MasterObserver
.
Modifier and Type | Interface and Description |
---|---|
static class |
RegionObserver.MutationType
Mutation type for postMutationBeforeWAL hook
|
Modifier and Type | Method and Description |
---|---|
default Result |
postAppend(ObserverContext<RegionCoprocessorEnvironment> c,
Append append,
Result result)
Deprecated.
since 2.5.0 and will be removed in 4.0.0. Use
postAppend(ObserverContext, Append, Result, WALEdit) instead. |
default Result |
postAppend(ObserverContext<RegionCoprocessorEnvironment> c,
Append append,
Result result,
WALEdit edit)
Called after Append
|
default List<Pair<Cell,Cell>> |
postAppendBeforeWAL(ObserverContext<RegionCoprocessorEnvironment> ctx,
Mutation mutation,
List<Pair<Cell,Cell>> cellPairs)
Called after a list of new cells has been created during an append operation, but before they
are committed to the WAL or memstore.
|
default void |
postBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
MiniBatchOperationInProgress<Mutation> miniBatchOp)
This will be called after applying a batch of Mutations on a region.
|
default void |
postBatchMutateIndispensably(ObserverContext<RegionCoprocessorEnvironment> ctx,
MiniBatchOperationInProgress<Mutation> miniBatchOp,
boolean success)
Called after the completion of batch put/delete/increment/append and will be called even if the
batch operation fails.
|
default void |
postBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
List<Pair<byte[],String>> stagingFamilyPaths,
Map<byte[],List<org.apache.hadoop.fs.Path>> finalPaths)
Called after bulkLoadHFile.
|
default boolean |
postCheckAndDelete(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] row,
byte[] family,
byte[] qualifier,
CompareOperator op,
ByteArrayComparable comparator,
Delete delete,
boolean result)
Deprecated.
since 2.4.0 and will be removed in 4.0.0. Use
postCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)
instead. |
default boolean |
postCheckAndDelete(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] row,
Filter filter,
Delete delete,
boolean result)
Deprecated.
since 2.4.0 and will be removed in 4.0.0. Use
postCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)
instead. |
default CheckAndMutateResult |
postCheckAndMutate(ObserverContext<RegionCoprocessorEnvironment> c,
CheckAndMutate checkAndMutate,
CheckAndMutateResult result)
Called after checkAndMutate
|
default boolean |
postCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] row,
byte[] family,
byte[] qualifier,
CompareOperator op,
ByteArrayComparable comparator,
Put put,
boolean result)
Deprecated.
since 2.4.0 and will be removed in 4.0.0. Use
postCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)
instead. |
default boolean |
postCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] row,
Filter filter,
Put put,
boolean result)
Deprecated.
since 2.4.0 and will be removed in 4.0.0. Use
postCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)
instead. |
default void |
postClose(ObserverContext<RegionCoprocessorEnvironment> c,
boolean abortRequested)
Called after the region is reported as closed to the master.
|
default void |
postCloseRegionOperation(ObserverContext<RegionCoprocessorEnvironment> ctx,
Region.Operation operation)
Called after releasing read lock in
Region.closeRegionOperation() . |
default void |
postCommitStoreFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
byte[] family,
org.apache.hadoop.fs.Path srcPath,
org.apache.hadoop.fs.Path dstPath)
Called after moving bulk loaded hfile to region directory.
|
default void |
postCompact(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
StoreFile resultFile,
CompactionLifeCycleTracker tracker,
CompactionRequest request)
Called after compaction has completed and the new store file has been moved in to place.
|
default void |
postCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
List<? extends StoreFile> selected,
CompactionLifeCycleTracker tracker,
CompactionRequest request)
Called after the
StoreFile s to compact have been selected from the available
candidates. |
default void |
postDelete(ObserverContext<RegionCoprocessorEnvironment> c,
Delete delete,
WALEdit edit)
Called after the client deletes a value.
|
default void |
postDelete(ObserverContext<RegionCoprocessorEnvironment> c,
Delete delete,
WALEdit edit,
Durability durability)
Deprecated.
since 2.5.0 and will be removed in 4.0.0. Use
postDelete(ObserverContext, Delete, WALEdit) instead. |
default boolean |
postExists(ObserverContext<RegionCoprocessorEnvironment> c,
Get get,
boolean exists)
Called after the client tests for existence using a Get.
|
default void |
postFlush(ObserverContext<RegionCoprocessorEnvironment> c,
FlushLifeCycleTracker tracker)
Called after the memstore is flushed to disk.
|
default void |
postFlush(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
StoreFile resultFile,
FlushLifeCycleTracker tracker)
Called after a Store's memstore is flushed to disk.
|
default void |
postGetOp(ObserverContext<RegionCoprocessorEnvironment> c,
Get get,
List<Cell> result)
Called after the client performs a Get
|
default Result |
postIncrement(ObserverContext<RegionCoprocessorEnvironment> c,
Increment increment,
Result result)
Deprecated.
since 2.5.0 and will be removed in 4.0.0. Use
postIncrement(ObserverContext, Increment, Result, WALEdit) instead. |
default Result |
postIncrement(ObserverContext<RegionCoprocessorEnvironment> c,
Increment increment,
Result result,
WALEdit edit)
Called after increment
|
default List<Pair<Cell,Cell>> |
postIncrementBeforeWAL(ObserverContext<RegionCoprocessorEnvironment> ctx,
Mutation mutation,
List<Pair<Cell,Cell>> cellPairs)
Called after a list of new cells has been created during an increment operation, but before
they are committed to the WAL or memstore.
|
default DeleteTracker |
postInstantiateDeleteTracker(ObserverContext<RegionCoprocessorEnvironment> ctx,
DeleteTracker delTracker)
Deprecated.
Since 2.0 with out any replacement and will be removed in 3.0
|
default void |
postMemStoreCompaction(ObserverContext<RegionCoprocessorEnvironment> c,
Store store)
Called after the in memory compaction is finished.
|
default Cell |
postMutationBeforeWAL(ObserverContext<RegionCoprocessorEnvironment> ctx,
RegionObserver.MutationType opType,
Mutation mutation,
Cell oldCell,
Cell newCell)
Deprecated.
since 2.2.0 and will be removedin 4.0.0. Use
postIncrementBeforeWAL(ObserverContext, Mutation, List) or
postAppendBeforeWAL(ObserverContext, Mutation, List) instead. |
default void |
postOpen(ObserverContext<RegionCoprocessorEnvironment> c)
Called after the region is reported as open to the master.
|
default void |
postPut(ObserverContext<RegionCoprocessorEnvironment> c,
Put put,
WALEdit edit)
Called after the client stores a value.
|
default void |
postPut(ObserverContext<RegionCoprocessorEnvironment> c,
Put put,
WALEdit edit,
Durability durability)
Deprecated.
since 2.5.0 and will be removed in 4.0.0. Use
postPut(ObserverContext, Put, WALEdit) instead. |
default void |
postReplayWALs(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
RegionInfo info,
org.apache.hadoop.fs.Path edits)
Called after replaying WALs for this region.
|
default void |
postScannerClose(ObserverContext<RegionCoprocessorEnvironment> ctx,
InternalScanner s)
Called after the client closes a scanner.
|
default boolean |
postScannerFilterRow(ObserverContext<RegionCoprocessorEnvironment> c,
InternalScanner s,
Cell curRowCell,
boolean hasMore)
This will be called by the scan flow when the current scanned row is being filtered out by the
filter.
|
default 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.
|
default RegionScanner |
postScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
Scan scan,
RegionScanner s)
Called after the client opens a new scanner.
|
default void |
postStartRegionOperation(ObserverContext<RegionCoprocessorEnvironment> ctx,
Region.Operation operation)
This will be called for region operations where read lock is acquired in
Region.startRegionOperation() . |
default StoreFileReader |
postStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx,
org.apache.hadoop.fs.FileSystem fs,
org.apache.hadoop.fs.Path p,
FSDataInputStreamWrapper in,
long size,
CacheConfig cacheConf,
Reference r,
StoreFileReader reader)
Deprecated.
For Phoenix only, StoreFileReader is not a stable interface.
|
default void |
postWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
RegionInfo info,
WALKey logKey,
WALEdit logEdit)
Called after a
WALEdit replayed for this region. |
default Result |
preAppend(ObserverContext<RegionCoprocessorEnvironment> c,
Append append)
Deprecated.
since 2.5.0 and will be removed in 4.0.0. Use
preAppend(ObserverContext, Append, WALEdit) instead. |
default Result |
preAppend(ObserverContext<RegionCoprocessorEnvironment> c,
Append append,
WALEdit edit)
Called before Append.
|
default Result |
preAppendAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c,
Append append)
Deprecated.
since 2.5.0 and will be removed in 4.0.0. Use
preBatchMutate(ObserverContext, MiniBatchOperationInProgress) instead. |
default void |
preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
MiniBatchOperationInProgress<Mutation> miniBatchOp)
This will be called for every batch mutation operation happening at the server.
|
default void |
preBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
List<Pair<byte[],String>> familyPaths)
Called before bulkLoadHFile.
|
default boolean |
preCheckAndDelete(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] row,
byte[] family,
byte[] qualifier,
CompareOperator op,
ByteArrayComparable comparator,
Delete delete,
boolean result)
Deprecated.
since 2.4.0 and will be removed in 4.0.0. Use
preCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)
instead. |
default boolean |
preCheckAndDelete(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] row,
Filter filter,
Delete delete,
boolean result)
Deprecated.
since 2.4.0 and will be removed in 4.0.0. Use
preCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)
instead. |
default boolean |
preCheckAndDeleteAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] row,
byte[] family,
byte[] qualifier,
CompareOperator op,
ByteArrayComparable comparator,
Delete delete,
boolean result)
Deprecated.
since 2.4.0 and will be removed in 4.0.0. Use
preCheckAndMutateAfterRowLock(ObserverContext, CheckAndMutate,CheckAndMutateResult)
instead. |
default boolean |
preCheckAndDeleteAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] row,
Filter filter,
Delete delete,
boolean result)
Deprecated.
since 2.4.0 and will be removed in 4.0.0. Use
preCheckAndMutateAfterRowLock(ObserverContext, CheckAndMutate,CheckAndMutateResult)
instead. |
default CheckAndMutateResult |
preCheckAndMutate(ObserverContext<RegionCoprocessorEnvironment> c,
CheckAndMutate checkAndMutate,
CheckAndMutateResult result)
Called before checkAndMutate
|
default CheckAndMutateResult |
preCheckAndMutateAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c,
CheckAndMutate checkAndMutate,
CheckAndMutateResult result)
Called before checkAndDelete but after acquiring rowlock.
|
default boolean |
preCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] row,
byte[] family,
byte[] qualifier,
CompareOperator op,
ByteArrayComparable comparator,
Put put,
boolean result)
Deprecated.
since 2.4.0 and will be removed in 4.0.0. Use
preCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)
instead. |
default boolean |
preCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] row,
Filter filter,
Put put,
boolean result)
Deprecated.
since 2.4.0 and will be removed in 4.0.0. Use
preCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)
instead. |
default boolean |
preCheckAndPutAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] row,
byte[] family,
byte[] qualifier,
CompareOperator op,
ByteArrayComparable comparator,
Put put,
boolean result)
Deprecated.
since 2.4.0 and will be removed in 4.0.0. Use
preCheckAndMutateAfterRowLock(ObserverContext, CheckAndMutate,CheckAndMutateResult)
instead. |
default boolean |
preCheckAndPutAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] row,
Filter filter,
Put put,
boolean result)
Deprecated.
since 2.4.0 and will be removed in 4.0.0. Use
preCheckAndMutateAfterRowLock(ObserverContext, CheckAndMutate,CheckAndMutateResult)
instead. |
default void |
preClose(ObserverContext<RegionCoprocessorEnvironment> c,
boolean abortRequested)
Called before the region is reported as closed to the master.
|
default void |
preCommitStoreFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
byte[] family,
List<Pair<org.apache.hadoop.fs.Path,org.apache.hadoop.fs.Path>> pairs)
Called before moving bulk loaded hfile to region directory.
|
default InternalScanner |
preCompact(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
InternalScanner scanner,
ScanType scanType,
CompactionLifeCycleTracker tracker,
CompactionRequest request)
Called prior to writing the
StoreFile s selected for compaction into a new
StoreFile . |
default void |
preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
ScanType scanType,
ScanOptions options,
CompactionLifeCycleTracker tracker,
CompactionRequest request)
Called before we open store scanner for compaction.
|
default void |
preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
List<? extends StoreFile> candidates,
CompactionLifeCycleTracker tracker)
Called prior to selecting the
StoreFiles to compact from the list of
available candidates. |
default void |
preDelete(ObserverContext<RegionCoprocessorEnvironment> c,
Delete delete,
WALEdit edit)
Called before the client deletes a value.
|
default void |
preDelete(ObserverContext<RegionCoprocessorEnvironment> c,
Delete delete,
WALEdit edit,
Durability durability)
Deprecated.
since 2.5.0 and will be removed in 4.0.0. Use
preDelete(ObserverContext, Delete, WALEdit) instead. |
default boolean |
preExists(ObserverContext<RegionCoprocessorEnvironment> c,
Get get,
boolean exists)
Called before the client tests for existence using a Get.
|
default void |
preFlush(ObserverContext<RegionCoprocessorEnvironment> c,
FlushLifeCycleTracker tracker)
Called before the memstore is flushed to disk.
|
default InternalScanner |
preFlush(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
InternalScanner scanner,
FlushLifeCycleTracker tracker)
Called before a Store's memstore is flushed to disk.
|
default void |
preFlushScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
ScanOptions options,
FlushLifeCycleTracker tracker)
Called before we open store scanner for flush.
|
default void |
preGetOp(ObserverContext<RegionCoprocessorEnvironment> c,
Get get,
List<Cell> result)
Called before the client performs a Get
|
default Result |
preIncrement(ObserverContext<RegionCoprocessorEnvironment> c,
Increment increment)
Deprecated.
since 2.5.0 and will be removed in 4.0.0. Use
preIncrement(ObserverContext, Increment, WALEdit) instead. |
default Result |
preIncrement(ObserverContext<RegionCoprocessorEnvironment> c,
Increment increment,
WALEdit edit)
Called before Increment.
|
default Result |
preIncrementAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c,
Increment increment)
Deprecated.
since 2.5.0 and will be removed in 4.0.0. Use
preBatchMutate(ObserverContext, MiniBatchOperationInProgress) instead. |
default void |
preMemStoreCompaction(ObserverContext<RegionCoprocessorEnvironment> c,
Store store)
Called before in memory compaction started.
|
default InternalScanner |
preMemStoreCompactionCompact(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
InternalScanner scanner)
Called before we do in memory compaction.
|
default void |
preMemStoreCompactionCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
ScanOptions options)
Called before we open store scanner for in memory compaction.
|
default void |
preOpen(ObserverContext<RegionCoprocessorEnvironment> c)
Called before the region is reported as open to the master.
|
default void |
prePrepareTimeStampForDeleteVersion(ObserverContext<RegionCoprocessorEnvironment> c,
Mutation mutation,
Cell cell,
byte[] byteNow,
Get get)
Deprecated.
Since hbase-2.0.0. No replacement. To be removed in hbase-3.0.0 and replaced with
something that doesn't expose IntefaceAudience.Private classes.
|
default void |
prePut(ObserverContext<RegionCoprocessorEnvironment> c,
Put put,
WALEdit edit)
Called before the client stores a value.
|
default void |
prePut(ObserverContext<RegionCoprocessorEnvironment> c,
Put put,
WALEdit edit,
Durability durability)
Deprecated.
since 2.5.0 and will be removed in 4.0.0. Use
prePut(ObserverContext, Put, WALEdit) instead. |
default void |
preReplayWALs(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
RegionInfo info,
org.apache.hadoop.fs.Path edits)
Called before replaying WALs for this region.
|
default void |
preScannerClose(ObserverContext<RegionCoprocessorEnvironment> c,
InternalScanner s)
Called before the client closes a scanner.
|
default 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.
|
default void |
preScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
Scan scan)
Called before the client opens a new scanner.
|
default StoreFileReader |
preStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx,
org.apache.hadoop.fs.FileSystem fs,
org.apache.hadoop.fs.Path p,
FSDataInputStreamWrapper in,
long size,
CacheConfig cacheConf,
Reference r,
StoreFileReader reader)
Deprecated.
For Phoenix only, StoreFileReader is not a stable interface.
|
default void |
preStoreScannerOpen(ObserverContext<RegionCoprocessorEnvironment> ctx,
Store store,
ScanOptions options)
Called before a store opens a new scanner.
|
default void |
preWALAppend(ObserverContext<RegionCoprocessorEnvironment> ctx,
WALKey key,
WALEdit edit)
Called just before the WAL Entry is appended to the WAL.
|
default void |
preWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
RegionInfo info,
WALKey logKey,
WALEdit logEdit)
Called before a
WALEdit replayed for this region. |
default void preOpen(ObserverContext<RegionCoprocessorEnvironment> c) throws IOException
c
- the environment provided by the region serverIOException
default void postOpen(ObserverContext<RegionCoprocessorEnvironment> c)
c
- the environment provided by the region serverdefault void preFlush(ObserverContext<RegionCoprocessorEnvironment> c, FlushLifeCycleTracker tracker) throws IOException
c
- the environment provided by the region servertracker
- tracker used to track the life cycle of a flushIOException
default void preFlushScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Store store, ScanOptions options, FlushLifeCycleTracker tracker) throws IOException
options
to change max
versions and TTL for the scanner being opened.c
- the environment provided by the region serverstore
- the store where flush is being requestedoptions
- used to change max versions and TTL for the scanner being openedIOException
default InternalScanner preFlush(ObserverContext<RegionCoprocessorEnvironment> c, Store store, InternalScanner scanner, FlushLifeCycleTracker tracker) throws IOException
c
- the environment provided by the region serverstore
- the store where flush is being requestedscanner
- the scanner over existing data used in the memstoretracker
- tracker used to track the life cycle of a flushnull
unless the implementation
is writing new store files on its own.IOException
default void postFlush(ObserverContext<RegionCoprocessorEnvironment> c, FlushLifeCycleTracker tracker) throws IOException
c
- the environment provided by the region servertracker
- tracker used to track the life cycle of a flushIOException
- if an error occurred on the coprocessordefault void postFlush(ObserverContext<RegionCoprocessorEnvironment> c, Store store, StoreFile resultFile, FlushLifeCycleTracker tracker) throws IOException
c
- the environment provided by the region serverstore
- the store being flushedresultFile
- the new store file written out during compactiontracker
- tracker used to track the life cycle of a flushIOException
default void preMemStoreCompaction(ObserverContext<RegionCoprocessorEnvironment> c, Store store) throws IOException
c
- the environment provided by the region serverstore
- the store where in memory compaction is being requestedIOException
default void preMemStoreCompactionCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Store store, ScanOptions options) throws IOException
options
to change max versions and TTL for the scanner being opened. Notice that this method will only
be called when you use eager
mode. For basic
mode we will not drop any cells
thus we do not open a store scanner.c
- the environment provided by the region serverstore
- the store where in memory compaction is being requestedoptions
- used to change max versions and TTL for the scanner being openedIOException
default InternalScanner preMemStoreCompactionCompact(ObserverContext<RegionCoprocessorEnvironment> c, Store store, InternalScanner scanner) throws IOException
eager
mode. For basic
mode we will not drop any cells thus there is no
InternalScanner
.c
- the environment provided by the region serverstore
- the store where in memory compaction is being executedscanner
- the scanner over existing data used in the memstore segments being compactIOException
default void postMemStoreCompaction(ObserverContext<RegionCoprocessorEnvironment> c, Store store) throws IOException
c
- the environment provided by the region serverstore
- the store where in memory compaction is being executedIOException
default void preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store, List<? extends StoreFile> candidates, CompactionLifeCycleTracker tracker) 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. If you remove all the candidates then the compaction will be canceled.
Supports Coprocessor 'bypass' -- 'bypass' is how this method indicates that it changed the
passed in candidates
. If 'bypass' is set, we skip out on calling any subsequent
chained coprocessors.
c
- the environment provided by the region serverstore
- the store where compaction is being requestedcandidates
- the store files currently available for compactiontracker
- tracker used to track the life cycle of a compactionIOException
default void postCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store, List<? extends StoreFile> selected, CompactionLifeCycleTracker tracker, 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 compacttracker
- tracker used to track the life cycle of a compactionrequest
- the requested compactiondefault void preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Store store, ScanType scanType, ScanOptions options, CompactionLifeCycleTracker tracker, CompactionRequest request) throws IOException
options
to change
max versions and TTL for the scanner being opened.c
- the environment provided by the region serverstore
- the store being compactedscanType
- type of Scanoptions
- used to change max versions and TTL for the scanner being openedtracker
- tracker used to track the life cycle of a compactionrequest
- the requested compactionIOException
default InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> c, Store store, InternalScanner scanner, ScanType scanType, CompactionLifeCycleTracker tracker, CompactionRequest request) throws IOException
StoreFile
s selected for compaction into a new
StoreFile
.
To override or modify the compaction process, implementing classes can wrap the provided
InternalScanner
with a custom implementation that is returned from this method. The
custom scanner can then inspect Cell
s from the wrapped scanner,
applying its own policy to what gets written.
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 Scantracker
- tracker used to track the life cycle of a compactionrequest
- the requested compactionnull
unless the
implementation is writing new store files on its own.IOException
default void postCompact(ObserverContext<RegionCoprocessorEnvironment> c, Store store, StoreFile resultFile, CompactionLifeCycleTracker tracker, CompactionRequest request) throws IOException
c
- the environment provided by the region serverstore
- the store being compactedresultFile
- the new store file written out during compactiontracker
- used to track the life cycle of a compactionrequest
- the requested compactionIOException
default void preClose(ObserverContext<RegionCoprocessorEnvironment> c, boolean abortRequested) throws IOException
c
- the environment provided by the region serverabortRequested
- true if the region server is abortingIOException
default void postClose(ObserverContext<RegionCoprocessorEnvironment> c, boolean abortRequested)
c
- the environment provided by the region serverabortRequested
- true if the region server is abortingdefault void preGetOp(ObserverContext<RegionCoprocessorEnvironment> c, Get get, List<Cell> result) throws IOException
Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling 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
default void postGetOp(ObserverContext<RegionCoprocessorEnvironment> c, Get get, List<Cell> result) throws IOException
Note: Do not retain references to any Cells in 'result' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverget
- the Get requestresult
- the result to return to the client, modify as necessaryIOException
default boolean preExists(ObserverContext<RegionCoprocessorEnvironment> c, Get get, boolean exists) throws IOException
Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
c
- the environment provided by the region serverget
- the Get requestexists
- the result returned by the region serverIOException
default boolean postExists(ObserverContext<RegionCoprocessorEnvironment> c, Get get, boolean exists) throws IOException
c
- the environment provided by the region serverget
- the Get requestexists
- the result returned by the region serverIOException
@Deprecated default void prePut(ObserverContext<RegionCoprocessorEnvironment> c, Put put, WALEdit edit, Durability durability) throws IOException
prePut(ObserverContext, Put, WALEdit)
instead.Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'put' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
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
default void prePut(ObserverContext<RegionCoprocessorEnvironment> c, Put put, WALEdit edit) throws IOException
Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'put' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverput
- The Put objectedit
- The WALEdit object that will be written to the walIOException
@Deprecated default void postPut(ObserverContext<RegionCoprocessorEnvironment> c, Put put, WALEdit edit, Durability durability) throws IOException
postPut(ObserverContext, Put, WALEdit)
instead.Note: Do not retain references to any Cells in 'put' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverput
- The Put objectedit
- The WALEdit object for the waldurability
- Persistence guarantee for this PutIOException
default void postPut(ObserverContext<RegionCoprocessorEnvironment> c, Put put, WALEdit edit) throws IOException
Note: Do not retain references to any Cells in 'put' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverput
- The Put objectedit
- The WALEdit object for the walIOException
@Deprecated default void preDelete(ObserverContext<RegionCoprocessorEnvironment> c, Delete delete, WALEdit edit, Durability durability) throws IOException
preDelete(ObserverContext, Delete, WALEdit)
instead.Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverdelete
- The Delete objectedit
- The WALEdit object for the waldurability
- Persistence guarantee for this DeleteIOException
default void preDelete(ObserverContext<RegionCoprocessorEnvironment> c, Delete delete, WALEdit edit) throws IOException
Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverdelete
- The Delete objectedit
- The WALEdit object for the walIOException
@Deprecated default void prePrepareTimeStampForDeleteVersion(ObserverContext<RegionCoprocessorEnvironment> c, Mutation mutation, Cell cell, byte[] byteNow, Get get) throws IOException
Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling 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
@Deprecated default void postDelete(ObserverContext<RegionCoprocessorEnvironment> c, Delete delete, WALEdit edit, Durability durability) throws IOException
postDelete(ObserverContext, Delete, WALEdit)
instead.Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverdelete
- The Delete objectedit
- The WALEdit object for the waldurability
- Persistence guarantee for this DeleteIOException
default void postDelete(ObserverContext<RegionCoprocessorEnvironment> c, Delete delete, WALEdit edit) throws IOException
Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverdelete
- The Delete objectedit
- The WALEdit object for the walIOException
default void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c, MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException
MiniBatchOperationInProgress.setOperationStatus(int, OperationStatus)
),
RegionObserver
can make Region to skip these Mutations.
Note: Do not retain references to any Cells in Mutations beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverminiBatchOp
- batch of Mutations getting applied to region.IOException
default void postBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c, MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException
postPut(ObserverContext, Put, WALEdit)
and
postDelete(ObserverContext, Delete, WALEdit)
and
postIncrement(ObserverContext, Increment, Result, WALEdit)
and
postAppend(ObserverContext, Append, Result, WALEdit)
is this hook will be executed
before the mvcc transaction completion.
Note: Do not retain references to any Cells in Mutations beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverminiBatchOp
- batch of Mutations applied to region. Coprocessors are discouraged from
manipulating its state.IOException
default void postStartRegionOperation(ObserverContext<RegionCoprocessorEnvironment> ctx, Region.Operation operation) throws IOException
Region.startRegionOperation()
. n * @param operation The operation is about to be taken
on the regionIOException
default void postCloseRegionOperation(ObserverContext<RegionCoprocessorEnvironment> ctx, Region.Operation operation) throws IOException
Region.closeRegionOperation()
. nnIOException
default void postBatchMutateIndispensably(ObserverContext<RegionCoprocessorEnvironment> ctx, MiniBatchOperationInProgress<Mutation> miniBatchOp, boolean success) throws IOException
Note: Do not retain references to any Cells in Mutations beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that. nn * @param success true if batch operation is successful otherwise false.
IOException
@Deprecated default boolean preCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Put put, boolean result) throws IOException
preCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)
instead.Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'put' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverrow
- row to checkfamily
- column familyqualifier
- column qualifierop
- the comparison operationcomparator
- the comparatorput
- data to put if check succeedsresult
- the default value of the resultIOException
@Deprecated default boolean preCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, Filter filter, Put put, boolean result) throws IOException
preCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)
instead.Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'put' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverrow
- row to checkfilter
- filterput
- data to put if check succeedsresult
- the default value of the resultIOException
@Deprecated default boolean preCheckAndPutAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Put put, boolean result) throws IOException
preCheckAndMutateAfterRowLock(ObserverContext, CheckAndMutate,CheckAndMutateResult)
instead.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. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'put' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverrow
- row to checkfamily
- column familyqualifier
- column qualifierop
- the comparison operationcomparator
- the comparatorput
- data to put if check succeedsresult
- the default value of the resultIOException
@Deprecated default boolean preCheckAndPutAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, Filter filter, Put put, boolean result) throws IOException
preCheckAndMutateAfterRowLock(ObserverContext, CheckAndMutate,CheckAndMutateResult)
instead.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. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'put' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverrow
- row to checkfilter
- filterput
- data to put if check succeedsresult
- the default value of the resultIOException
@Deprecated default boolean postCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Put put, boolean result) throws IOException
postCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)
instead.Note: Do not retain references to any Cells in 'put' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverrow
- row to checkfamily
- column familyqualifier
- column qualifierop
- the comparison operationcomparator
- the comparatorput
- data to put if check succeedsresult
- from the checkAndPutIOException
@Deprecated default boolean postCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, Filter filter, Put put, boolean result) throws IOException
postCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)
instead.Note: Do not retain references to any Cells in 'put' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverrow
- row to checkfilter
- filterput
- data to put if check succeedsresult
- from the checkAndPutIOException
@Deprecated default boolean preCheckAndDelete(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Delete delete, boolean result) throws IOException
preCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)
instead.Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverrow
- row to checkfamily
- column familyqualifier
- column qualifierop
- the comparison operationcomparator
- the comparatordelete
- delete to commit if check succeedsresult
- the default value of the resultIOException
@Deprecated default boolean preCheckAndDelete(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, Filter filter, Delete delete, boolean result) throws IOException
preCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)
instead.Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverrow
- row to checkfilter
- column familydelete
- delete to commit if check succeedsresult
- the default value of the resultIOException
@Deprecated default boolean preCheckAndDeleteAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Delete delete, boolean result) throws IOException
preCheckAndMutateAfterRowLock(ObserverContext, CheckAndMutate,CheckAndMutateResult)
instead.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. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverrow
- row to checkfamily
- column familyqualifier
- column qualifierop
- the comparison operationcomparator
- the comparatordelete
- delete to commit if check succeedsresult
- the default value of the resultIOException
@Deprecated default boolean preCheckAndDeleteAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, Filter filter, Delete delete, boolean result) throws IOException
preCheckAndMutateAfterRowLock(ObserverContext, CheckAndMutate,CheckAndMutateResult)
instead.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. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverrow
- row to checkfilter
- filterdelete
- delete to commit if check succeedsresult
- the default value of the resultIOException
@Deprecated default boolean postCheckAndDelete(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Delete delete, boolean result) throws IOException
postCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)
instead.Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverrow
- row to checkfamily
- column familyqualifier
- column qualifierop
- the comparison operationcomparator
- the comparatordelete
- delete to commit if check succeedsresult
- from the CheckAndDeleteIOException
@Deprecated default boolean postCheckAndDelete(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, Filter filter, Delete delete, boolean result) throws IOException
postCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)
instead.Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverrow
- row to checkfilter
- filterdelete
- delete to commit if check succeedsresult
- from the CheckAndDeleteIOException
default CheckAndMutateResult preCheckAndMutate(ObserverContext<RegionCoprocessorEnvironment> c, CheckAndMutate checkAndMutate, CheckAndMutateResult result) throws IOException
Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in actions beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region servercheckAndMutate
- the CheckAndMutate objectresult
- the default value of the resultIOException
- if an error occurred on the coprocessordefault CheckAndMutateResult preCheckAndMutateAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c, CheckAndMutate checkAndMutate, CheckAndMutateResult 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. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in actions beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region servercheckAndMutate
- the CheckAndMutate objectresult
- the default value of the resultIOException
- if an error occurred on the coprocessordefault CheckAndMutateResult postCheckAndMutate(ObserverContext<RegionCoprocessorEnvironment> c, CheckAndMutate checkAndMutate, CheckAndMutateResult result) throws IOException
Note: Do not retain references to any Cells in actions beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region servercheckAndMutate
- the CheckAndMutate objectresult
- from the checkAndMutateIOException
- if an error occurred on the coprocessor@Deprecated default Result preAppend(ObserverContext<RegionCoprocessorEnvironment> c, Append append) throws IOException
preAppend(ObserverContext, Append, WALEdit)
instead.Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'append' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverappend
- Append objectIOException
default Result preAppend(ObserverContext<RegionCoprocessorEnvironment> c, Append append, WALEdit edit) throws IOException
Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'append' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverappend
- Append objectedit
- The WALEdit object that will be written to the walIOException
@Deprecated default Result preAppendAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c, Append append) throws IOException
preBatchMutate(ObserverContext, MiniBatchOperationInProgress)
instead.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. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'append' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverappend
- Append objectIOException
@Deprecated default Result postAppend(ObserverContext<RegionCoprocessorEnvironment> c, Append append, Result result) throws IOException
postAppend(ObserverContext, Append, Result, WALEdit)
instead.Note: Do not retain references to any Cells in 'append' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverappend
- Append objectresult
- the result returned by incrementIOException
default Result postAppend(ObserverContext<RegionCoprocessorEnvironment> c, Append append, Result result, WALEdit edit) throws IOException
Note: Do not retain references to any Cells in 'append' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverappend
- Append objectresult
- the result returned by incrementedit
- The WALEdit object for the walIOException
@Deprecated default Result preIncrement(ObserverContext<RegionCoprocessorEnvironment> c, Increment increment) throws IOException
preIncrement(ObserverContext, Increment, WALEdit)
instead.Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'increment' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverincrement
- increment objectIOException
default Result preIncrement(ObserverContext<RegionCoprocessorEnvironment> c, Increment increment, WALEdit edit) throws IOException
Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'increment' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverincrement
- increment objectedit
- The WALEdit object that will be written to the walIOException
@Deprecated default Result preIncrementAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c, Increment increment) throws IOException
preBatchMutate(ObserverContext, MiniBatchOperationInProgress)
instead.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. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'increment' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverincrement
- increment objectIOException
@Deprecated default Result postIncrement(ObserverContext<RegionCoprocessorEnvironment> c, Increment increment, Result result) throws IOException
postIncrement(ObserverContext, Increment, Result, WALEdit)
instead.Note: Do not retain references to any Cells in 'increment' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverincrement
- increment objectresult
- the result returned by incrementIOException
default Result postIncrement(ObserverContext<RegionCoprocessorEnvironment> c, Increment increment, Result result, WALEdit edit) throws IOException
Note: Do not retain references to any Cells in 'increment' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverincrement
- increment objectresult
- the result returned by incrementedit
- The WALEdit object for the walIOException
default void preScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Scan scan) throws IOException
Note: Do not retain references to any Cells returned by scanner, beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverscan
- the Scan specificationIOException
default RegionScanner postScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Scan scan, RegionScanner s) throws IOException
Note: Do not retain references to any Cells returned by scanner, beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region serverscan
- the Scan specifications
- if not null, the base scannerIOException
default boolean preScannerNext(ObserverContext<RegionCoprocessorEnvironment> c, InternalScanner s, List<Result> result, int limit, boolean hasNext) throws IOException
Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells returned by scanner, beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
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
default boolean postScannerNext(ObserverContext<RegionCoprocessorEnvironment> c, InternalScanner s, List<Result> result, int limit, boolean hasNext) throws IOException
Note: Do not retain references to any Cells returned by scanner, beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
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
default boolean postScannerFilterRow(ObserverContext<RegionCoprocessorEnvironment> c, InternalScanner s, Cell curRowCell, boolean hasMore) throws IOException
boolean filterRowKey(byte [] buffer, int offset, int length)
returning
trueboolean filterRow()
returning truedefault void filterRow(List<KeyValue> kvs)
removing all the kvs from the
passed ListNote: Do not retain references to any Cells returned by scanner, beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
c
- the environment provided by the region servers
- the scannercurRowCell
- The cell in the current row which got filtered outhasMore
- the 'has more' indicationIOException
default void preScannerClose(ObserverContext<RegionCoprocessorEnvironment> c, InternalScanner s) throws IOException
Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
c
- the environment provided by the region servers
- the scannerIOException
default void postScannerClose(ObserverContext<RegionCoprocessorEnvironment> ctx, InternalScanner s) throws IOException
ctx
- the environment provided by the region servers
- the scannerIOException
default void preStoreScannerOpen(ObserverContext<RegionCoprocessorEnvironment> ctx, Store store, ScanOptions options) throws IOException
This hook is called when a "user" scanner is opened. Use preFlushScannerOpen
and
preCompactScannerOpen
to inject flush/compaction.
Notice that, this method is used to change the inherent max versions and TTL for a Store. For
example, you can change the max versions option for a Scan
object to 10 in
preScannerOpen
, but if the max versions config on the Store is 1, then you still can
only read 1 version. You need also to inject here to change the max versions to 10 if you want
to get more versions.
ctx
- the environment provided by the region serverstore
- the store which we want to get scanner fromoptions
- used to change max versions and TTL for the scanner being openedIOException
preFlushScannerOpen(ObserverContext, Store, ScanOptions, FlushLifeCycleTracker)
,
preCompactScannerOpen(ObserverContext, Store, ScanType, ScanOptions,
CompactionLifeCycleTracker, CompactionRequest)
default void preReplayWALs(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, RegionInfo info, org.apache.hadoop.fs.Path edits) throws IOException
ObserverContext.bypass()
has no effect in this
hook.ctx
- the environment provided by the region serverinfo
- the RegionInfo for this regionedits
- the file of recovered editsIOException
default void postReplayWALs(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, RegionInfo info, org.apache.hadoop.fs.Path edits) throws IOException
ctx
- the environment provided by the region serverinfo
- the RegionInfo for this regionedits
- the file of recovered editsIOException
default void preWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException
WALEdit
replayed for this region.ctx
- the environment provided by the region serverIOException
default void postWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException
WALEdit
replayed for this region.ctx
- the environment provided by the region serverIOException
default void preBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx, List<Pair<byte[],String>> familyPaths) throws IOException
ctx
- the environment provided by the region serverfamilyPaths
- 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
default void preCommitStoreFile(ObserverContext<RegionCoprocessorEnvironment> ctx, byte[] family, List<Pair<org.apache.hadoop.fs.Path,org.apache.hadoop.fs.Path>> pairs) throws IOException
ctx
- the environment provided by the region serverfamily
- column familypairs
- List of pairs of { HFile location in staging dir, HFile path in region dir } Each
pair are for the same hfile.IOException
default void postCommitStoreFile(ObserverContext<RegionCoprocessorEnvironment> ctx, byte[] family, org.apache.hadoop.fs.Path srcPath, org.apache.hadoop.fs.Path dstPath) throws IOException
ctx
- the environment provided by the region serverfamily
- column familysrcPath
- Path to file before the movedstPath
- Path to file after the moveIOException
default void postBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx, List<Pair<byte[],String>> stagingFamilyPaths, Map<byte[],List<org.apache.hadoop.fs.Path>> finalPaths) throws IOException
ctx
- the environment provided by the region serverstagingFamilyPaths
- pairs of { CF, HFile path } submitted for bulk loadfinalPaths
- Map of CF to List of file paths for the loaded files if the Map is
not null, the bulkLoad was successful. Otherwise the bulk load
failed. bulkload is done by the time this hook is called.IOException
@Deprecated default StoreFileReader preStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf, Reference r, StoreFileReader 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 file n * @param 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
@Deprecated default StoreFileReader postStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf, Reference r, StoreFileReader 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 file n * @param r original reference file. This will be not null
only when reading a split file.reader
- the base reader instanceIOException
@Deprecated default Cell postMutationBeforeWAL(ObserverContext<RegionCoprocessorEnvironment> ctx, RegionObserver.MutationType opType, Mutation mutation, Cell oldCell, Cell newCell) throws IOException
postIncrementBeforeWAL(ObserverContext, Mutation, List)
or
postAppendBeforeWAL(ObserverContext, Mutation, List)
instead.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
postIncrementBeforeWAL(ObserverContext, Mutation, List)
,
postAppendBeforeWAL(ObserverContext, Mutation, List)
,
HBASE-21643default List<Pair<Cell,Cell>> postIncrementBeforeWAL(ObserverContext<RegionCoprocessorEnvironment> ctx, Mutation mutation, List<Pair<Cell,Cell>> cellPairs) throws IOException
ctx
- the environment provided by the region servermutation
- the current mutationcellPairs
- a list of cell pair. The first cell is old cell which may be null. And the
second cell is the new cell.IOException
default List<Pair<Cell,Cell>> postAppendBeforeWAL(ObserverContext<RegionCoprocessorEnvironment> ctx, Mutation mutation, List<Pair<Cell,Cell>> cellPairs) throws IOException
ctx
- the environment provided by the region servermutation
- the current mutationcellPairs
- a list of cell pair. The first cell is old cell which may be null. And the
second cell is the new cell.IOException
@Deprecated default DeleteTracker postInstantiateDeleteTracker(ObserverContext<RegionCoprocessorEnvironment> ctx, DeleteTracker delTracker) throws IOException
Warn: This is used by internal coprocessors. Should not be implemented by user coprocessors
ctx
- the environment provided by the region serverdelTracker
- the deleteTracker that is created by the QueryMatcherIOException
default void preWALAppend(ObserverContext<RegionCoprocessorEnvironment> ctx, WALKey key, WALEdit edit) throws IOException
ctx
- the environment provided by the region serverkey
- the WALKey associated with a particular append to a WALIOException
Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.