Interface RegionObserver
- All Known Implementing Classes:
AccessController
,BackupObserver
,ConstraintProcessor
,ExampleRegionObserverWithMetrics.ExampleRegionObserver
,MetaTableMetrics.ExampleRegionObserverMeta
,ReplicationObserver
,ScanModifyingObserver
,ValueRewritingObserver
,VisibilityController
,WriteHeavyIncrementObserver
,WriteSinkCoprocessor
,ZooKeeperScanPolicyObserver
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.
Exception Handling
For all functions, exception handling is done as follows:
- Exceptions of type
IOException
are reported back to client. - For any other kind of exception:
- If the configuration
CoprocessorHost.ABORT_ON_ERROR_KEY
is set to true, then the server aborts. - Otherwise, coprocessor is removed from the server and
DoNotRetryIOException
is returned to the client.
- If the configuration
For Split Related Hooks
In hbase2/AMv2, master runs splits, so the split related hooks are moved to
MasterObserver
.
Increment Column Value
We do not call this hook anymore.
-
Nested Class Summary
Modifier and TypeInterfaceDescriptionstatic enum
Mutation type for postMutationBeforeWAL hook -
Method Summary
Modifier and TypeMethodDescriptiondefault Result
postAppend
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Append append, Result result) Deprecated.since 3.0.0 and will be removed in 4.0.0.default Result
postAppend
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Append append, Result result, WALEdit edit) Called after AppendpostAppendBeforeWAL
(ObserverContext<? extends 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<? extends RegionCoprocessorEnvironment> c, MiniBatchOperationInProgress<Mutation> miniBatchOp) This will be called after applying a batch of Mutations on a region.default void
postBatchMutateIndispensably
(ObserverContext<? extends 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<? extends RegionCoprocessorEnvironment> ctx, List<Pair<byte[], String>> stagingFamilyPaths, Map<byte[], List<org.apache.hadoop.fs.Path>> finalPaths) Called after bulkLoadHFile.default boolean
postCheckAndDelete
(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Delete delete, boolean result) Deprecated.since 3.0.0 and will be removed in 4.0.0.default boolean
postCheckAndDelete
(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, Filter filter, Delete delete, boolean result) Deprecated.since 3.0.0 and will be removed in 4.0.0.default CheckAndMutateResult
postCheckAndMutate
(ObserverContext<? extends RegionCoprocessorEnvironment> c, CheckAndMutate checkAndMutate, CheckAndMutateResult result) Called after checkAndMutatedefault boolean
postCheckAndPut
(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Put put, boolean result) Deprecated.since 3.0.0 and will be removed in 4.0.0.default boolean
postCheckAndPut
(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, Filter filter, Put put, boolean result) Deprecated.since 3.0.0 and will be removed in 4.0.0.default void
postClose
(ObserverContext<? extends RegionCoprocessorEnvironment> c, boolean abortRequested) Called after the region is reported as closed to the master.default void
postCloseRegionOperation
(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, Region.Operation operation) Called after releasing read lock inRegion.closeRegionOperation()
.default void
postCommitStoreFile
(ObserverContext<? extends 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<? extends 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<? extends RegionCoprocessorEnvironment> c, Store store, List<? extends StoreFile> selected, CompactionLifeCycleTracker tracker, CompactionRequest request) Called after theStoreFile
s to compact have been selected from the available candidates.default void
postDelete
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Delete delete, WALEdit edit) Called after the client deletes a value.default void
postDelete
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Delete delete, WALEdit edit, Durability durability) Deprecated.since 3.0.0 and will be removed in 4.0.0.default boolean
postExists
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Get get, boolean exists) Called after the client tests for existence using a Get.default void
postFlush
(ObserverContext<? extends RegionCoprocessorEnvironment> c, FlushLifeCycleTracker tracker) Called after the memstore is flushed to disk.default void
postFlush
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store, StoreFile resultFile, FlushLifeCycleTracker tracker) Called after a Store's memstore is flushed to disk.default void
postGetOp
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Get get, List<Cell> result) Called after the client performs a Getdefault Result
postIncrement
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Increment increment, Result result) Deprecated.since 3.0.0 and will be removed in 4.0.0.default Result
postIncrement
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Increment increment, Result result, WALEdit edit) Called after incrementpostIncrementBeforeWAL
(ObserverContext<? extends 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<? extends RegionCoprocessorEnvironment> ctx, DeleteTracker delTracker) Deprecated.Since 2.0.0, will be removed in 4.0.0.default void
postMemStoreCompaction
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store) Called after the in memory compaction is finished.default Cell
postMutationBeforeWAL
(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, RegionObserver.MutationType opType, Mutation mutation, Cell oldCell, Cell newCell) Deprecated.since 2.2.0 and will be removedin 4.0.0.default void
postOpen
(ObserverContext<? extends RegionCoprocessorEnvironment> c) Called after the region is reported as open to the master.default void
postPut
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Put put, WALEdit edit) Called after the client stores a value.default void
postPut
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Put put, WALEdit edit, Durability durability) Deprecated.since 3.0.0 and will be removed in 4.0.0.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<? extends RegionCoprocessorEnvironment> ctx, InternalScanner s) Called after the client closes a scanner.default boolean
postScannerFilterRow
(ObserverContext<? extends 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<? extends 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<? extends RegionCoprocessorEnvironment> c, Scan scan, RegionScanner s) Called after the client opens a new scanner.default void
postStartRegionOperation
(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, Region.Operation operation) This will be called for region operations where read lock is acquired inRegion.startRegionOperation()
.default StoreFileReader
postStoreFileReaderOpen
(ObserverContext<? extends 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 Result
preAppend
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Append append) Deprecated.since 3.0.0 and will be removed in 4.0.0.default Result
preAppend
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Append append, WALEdit edit) Called before Append.default Result
preAppendAfterRowLock
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Append append) Deprecated.since 3.0.0 and will be removed in 4.0.0.default void
preBatchMutate
(ObserverContext<? extends RegionCoprocessorEnvironment> c, MiniBatchOperationInProgress<Mutation> miniBatchOp) This will be called for every batch mutation operation happening at the server.default void
preBulkLoadHFile
(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, List<Pair<byte[], String>> familyPaths) Called before bulkLoadHFile.default boolean
preCheckAndDelete
(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Delete delete, boolean result) Deprecated.since 3.0.0 and will be removed in 4.0.0.default boolean
preCheckAndDelete
(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, Filter filter, Delete delete, boolean result) Deprecated.since 3.0.0 and will be removed in 4.0.0.default boolean
preCheckAndDeleteAfterRowLock
(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Delete delete, boolean result) Deprecated.since 3.0.0 and will be removed in 4.0.0.default boolean
preCheckAndDeleteAfterRowLock
(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, Filter filter, Delete delete, boolean result) Deprecated.since 3.0.0 and will be removed in 4.0.0.default CheckAndMutateResult
preCheckAndMutate
(ObserverContext<? extends RegionCoprocessorEnvironment> c, CheckAndMutate checkAndMutate, CheckAndMutateResult result) Called before checkAndMutatedefault CheckAndMutateResult
preCheckAndMutateAfterRowLock
(ObserverContext<? extends RegionCoprocessorEnvironment> c, CheckAndMutate checkAndMutate, CheckAndMutateResult result) Called before checkAndDelete but after acquiring rowlock.default boolean
preCheckAndPut
(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Put put, boolean result) Deprecated.since 3.0.0 and will be removed in 4.0.0.default boolean
preCheckAndPut
(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, Filter filter, Put put, boolean result) Deprecated.since 3.0.0 and will be removed in 4.0.0.default boolean
preCheckAndPutAfterRowLock
(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Put put, boolean result) Deprecated.since 3.0.0 and will be removed in 4.0.0.default boolean
preCheckAndPutAfterRowLock
(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, Filter filter, Put put, boolean result) Deprecated.since 3.0.0 and will be removed in 4.0.0.default void
preClose
(ObserverContext<? extends RegionCoprocessorEnvironment> c, boolean abortRequested) Called before the region is reported as closed to the master.default void
preCommitStoreFile
(ObserverContext<? extends 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<? extends RegionCoprocessorEnvironment> c, Store store, InternalScanner scanner, ScanType scanType, CompactionLifeCycleTracker tracker, CompactionRequest request) Called prior to writing theStoreFile
s selected for compaction into a newStoreFile
.default void
preCompactScannerOpen
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store, ScanType scanType, ScanOptions options, CompactionLifeCycleTracker tracker, CompactionRequest request) Called before we open store scanner for compaction.default void
preCompactSelection
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store, List<? extends StoreFile> candidates, CompactionLifeCycleTracker tracker) Called prior to selecting theStoreFiles
to compact from the list of available candidates.default void
preDelete
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Delete delete, WALEdit edit) Called before the client deletes a value.default void
preDelete
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Delete delete, WALEdit edit, Durability durability) Deprecated.since 3.0.0 and will be removed in 4.0.0.default boolean
preExists
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Get get, boolean exists) Called before the client tests for existence using a Get.default void
preFlush
(ObserverContext<? extends RegionCoprocessorEnvironment> c, FlushLifeCycleTracker tracker) Called before the memstore is flushed to disk.default InternalScanner
preFlush
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store, InternalScanner scanner, FlushLifeCycleTracker tracker) Called before a Store's memstore is flushed to disk.default void
preFlushScannerOpen
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store, ScanOptions options, FlushLifeCycleTracker tracker) Called before we open store scanner for flush.default void
preGetOp
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Get get, List<Cell> result) Called before the client performs a Getdefault Result
preIncrement
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Increment increment) Deprecated.since 3.0.0 and will be removed in 4.0.0.default Result
preIncrement
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Increment increment, WALEdit edit) Called before Increment.default Result
preIncrementAfterRowLock
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Increment increment) Deprecated.since 3.0.0 and will be removed in 4.0.0.default void
preMemStoreCompaction
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store) Called before in memory compaction started.default InternalScanner
preMemStoreCompactionCompact
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store, InternalScanner scanner) Called before we do in memory compaction.default void
preMemStoreCompactionCompactScannerOpen
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store, ScanOptions options) Called before we open store scanner for in memory compaction.default void
preOpen
(ObserverContext<? extends RegionCoprocessorEnvironment> c) Called before the region is reported as open to the master.default void
prePrepareTimeStampForDeleteVersion
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Mutation mutation, Cell cell, byte[] byteNow, Get get) Deprecated.Since hbase-2.0.0.default void
prePut
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Put put, WALEdit edit) Called before the client stores a value.default void
prePut
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Put put, WALEdit edit, Durability durability) Deprecated.since 3.0.0 and will be removed in 4.0.0.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<? extends RegionCoprocessorEnvironment> c, InternalScanner s) Called before the client closes a scanner.default boolean
preScannerNext
(ObserverContext<? extends 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<? extends RegionCoprocessorEnvironment> c, Scan scan) Called before the client opens a new scanner.default StoreFileReader
preStoreFileReaderOpen
(ObserverContext<? extends 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<? extends RegionCoprocessorEnvironment> ctx, Store store, ScanOptions options) Called before a store opens a new scanner.default void
preWALAppend
(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, WALKey key, WALEdit edit) Called just before the WAL Entry is appended to the WAL.
-
Method Details
-
preOpen
Called before the region is reported as open to the master.- Parameters:
c
- the environment provided by the region server- Throws:
IOException
-
postOpen
Called after the region is reported as open to the master.- Parameters:
c
- the environment provided by the region server
-
preFlush
default void preFlush(ObserverContext<? extends RegionCoprocessorEnvironment> c, FlushLifeCycleTracker tracker) throws IOException Called before the memstore is flushed to disk.- Parameters:
c
- the environment provided by the region servertracker
- tracker used to track the life cycle of a flush- Throws:
IOException
-
preFlushScannerOpen
default void preFlushScannerOpen(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store, ScanOptions options, FlushLifeCycleTracker tracker) throws IOException Called before we open store scanner for flush. You can use theoptions
to change max versions and TTL for the scanner being opened.- Parameters:
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 opened- Throws:
IOException
-
preFlush
default InternalScanner preFlush(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store, InternalScanner scanner, FlushLifeCycleTracker tracker) throws IOException Called before a Store's memstore is flushed to disk.- Parameters:
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 flush- Returns:
- the scanner to use during flush. Should not be
null
unless the implementation is writing new store files on its own. - Throws:
IOException
-
postFlush
default void postFlush(ObserverContext<? extends RegionCoprocessorEnvironment> c, FlushLifeCycleTracker tracker) throws IOException Called after the memstore is flushed to disk.- Parameters:
c
- the environment provided by the region servertracker
- tracker used to track the life cycle of a flush- Throws:
IOException
- if an error occurred on the coprocessor
-
postFlush
default void postFlush(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store, StoreFile resultFile, FlushLifeCycleTracker tracker) throws IOException Called after a Store's memstore is flushed to disk.- Parameters:
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 flush- Throws:
IOException
-
preMemStoreCompaction
default void preMemStoreCompaction(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store) throws IOException Called before in memory compaction started.- Parameters:
c
- the environment provided by the region serverstore
- the store where in memory compaction is being requested- Throws:
IOException
-
preMemStoreCompactionCompactScannerOpen
default void preMemStoreCompactionCompactScannerOpen(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store, ScanOptions options) throws IOException Called before we open store scanner for in memory compaction. You can use theoptions
to change max versions and TTL for the scanner being opened. Notice that this method will only be called when you useeager
mode. Forbasic
mode we will not drop any cells thus we do not open a store scanner.- Parameters:
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 opened- Throws:
IOException
-
preMemStoreCompactionCompact
default InternalScanner preMemStoreCompactionCompact(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store, InternalScanner scanner) throws IOException Called before we do in memory compaction. Notice that this method will only be called when you useeager
mode. Forbasic
mode we will not drop any cells thus there is noInternalScanner
.- Parameters:
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 compact- Returns:
- the scanner to use during in memory compaction. Must be non-null.
- Throws:
IOException
-
postMemStoreCompaction
default void postMemStoreCompaction(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store) throws IOException Called after the in memory compaction is finished.- Parameters:
c
- the environment provided by the region serverstore
- the store where in memory compaction is being executed- Throws:
IOException
-
preCompactSelection
default void preCompactSelection(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store, List<? extends StoreFile> candidates, CompactionLifeCycleTracker tracker) throws IOException Called prior to selecting theStoreFiles
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.- Parameters:
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 compaction- Throws:
IOException
-
postCompactSelection
default void postCompactSelection(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store, List<? extends StoreFile> selected, CompactionLifeCycleTracker tracker, CompactionRequest request) Called after theStoreFile
s to compact have been selected from the available candidates.- Parameters:
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 compaction
-
preCompactScannerOpen
default void preCompactScannerOpen(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store, ScanType scanType, ScanOptions options, CompactionLifeCycleTracker tracker, CompactionRequest request) throws IOException Called before we open store scanner for compaction. You can use theoptions
to change max versions and TTL for the scanner being opened.- Parameters:
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 compaction- Throws:
IOException
-
preCompact
default InternalScanner preCompact(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store, InternalScanner scanner, ScanType scanType, CompactionLifeCycleTracker tracker, CompactionRequest request) throws IOException Called prior to writing theStoreFile
s selected for compaction into a newStoreFile
.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 inspectCell
s from the wrapped scanner, applying its own policy to what gets written.If implementations are wrapping the passed in
InternalScanner
, they can also have their implementation implementShipper
and delegate to the original scanner. This will cause compactions to free up memory as they progress, which is especially important for people using off-heap memory pools.Keep in mind that when
Shipper.shipped()
is called, any cell references you maintain in your implementation may get corrupted. As such you should make sure to deep clone any cells that you need to keep reference to across invocations of shipped.- Parameters:
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 compaction- Returns:
- the scanner to use during compaction. Should not be
null
unless the implementation is writing new store files on its own. - Throws:
IOException
-
postCompact
default void postCompact(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store, StoreFile resultFile, CompactionLifeCycleTracker tracker, CompactionRequest request) throws IOException Called after compaction has completed and the new store file has been moved in to place.- Parameters:
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 compaction- Throws:
IOException
-
preClose
default void preClose(ObserverContext<? extends RegionCoprocessorEnvironment> c, boolean abortRequested) throws IOException Called before the region is reported as closed to the master.- Parameters:
c
- the environment provided by the region serverabortRequested
- true if the region server is aborting- Throws:
IOException
-
postClose
default void postClose(ObserverContext<? extends RegionCoprocessorEnvironment> c, boolean abortRequested) Called after the region is reported as closed to the master.- Parameters:
c
- the environment provided by the region serverabortRequested
- true if the region server is aborting
-
preGetOp
default void preGetOp(ObserverContext<? extends RegionCoprocessorEnvironment> c, Get get, List<Cell> result) throws IOException Called before the client performs a GetCall CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
- Parameters:
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.- Throws:
IOException
-
postGetOp
default void postGetOp(ObserverContext<? extends RegionCoprocessorEnvironment> c, Get get, List<Cell> result) throws IOException Called after the client performs a GetNote: 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.
- Parameters:
c
- the environment provided by the region serverget
- the Get requestresult
- the result to return to the client, modify as necessary- Throws:
IOException
-
preExists
default boolean preExists(ObserverContext<? extends RegionCoprocessorEnvironment> c, Get get, boolean exists) throws IOException Called before the client tests for existence using a Get.Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
- Parameters:
c
- the environment provided by the region serverget
- the Get requestexists
- the result returned by the region server- Returns:
- the value to return to the client if bypassing default processing
- Throws:
IOException
-
postExists
default boolean postExists(ObserverContext<? extends RegionCoprocessorEnvironment> c, Get get, boolean exists) throws IOException Called after the client tests for existence using a Get.- Parameters:
c
- the environment provided by the region serverget
- the Get requestexists
- the result returned by the region server- Returns:
- the result to return to the client
- Throws:
IOException
-
prePut
@Deprecated default void prePut(ObserverContext<? extends RegionCoprocessorEnvironment> c, Put put, WALEdit edit, Durability durability) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UseprePut(ObserverContext, Put, WALEdit)
instead.Called before the client stores a value.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.
- Parameters:
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 Put- Throws:
IOException
-
prePut
default void prePut(ObserverContext<? extends RegionCoprocessorEnvironment> c, Put put, WALEdit edit) throws IOException Called before the client stores a value.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.
- Parameters:
c
- the environment provided by the region serverput
- The Put objectedit
- The WALEdit object that will be written to the wal- Throws:
IOException
-
postPut
@Deprecated default void postPut(ObserverContext<? extends RegionCoprocessorEnvironment> c, Put put, WALEdit edit, Durability durability) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsepostPut(ObserverContext, Put, WALEdit)
instead.Called after the client stores a value.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.
- Parameters:
c
- the environment provided by the region serverput
- The Put objectedit
- The WALEdit object for the waldurability
- Persistence guarantee for this Put- Throws:
IOException
-
postPut
default void postPut(ObserverContext<? extends RegionCoprocessorEnvironment> c, Put put, WALEdit edit) throws IOException Called after the client stores a value.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.
- Parameters:
c
- the environment provided by the region serverput
- The Put objectedit
- The WALEdit object for the wal- Throws:
IOException
-
preDelete
@Deprecated default void preDelete(ObserverContext<? extends RegionCoprocessorEnvironment> c, Delete delete, WALEdit edit, Durability durability) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsepreDelete(ObserverContext, Delete, WALEdit)
instead.Called before the client deletes a value.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.
- Parameters:
c
- the environment provided by the region serverdelete
- The Delete objectedit
- The WALEdit object for the waldurability
- Persistence guarantee for this Delete- Throws:
IOException
-
preDelete
default void preDelete(ObserverContext<? extends RegionCoprocessorEnvironment> c, Delete delete, WALEdit edit) throws IOException Called before the client deletes a value.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.
- Parameters:
c
- the environment provided by the region serverdelete
- The Delete objectedit
- The WALEdit object for the wal- Throws:
IOException
-
prePrepareTimeStampForDeleteVersion
@Deprecated default void prePrepareTimeStampForDeleteVersion(ObserverContext<? extends RegionCoprocessorEnvironment> c, Mutation mutation, Cell cell, byte[] byteNow, Get get) throws IOException Deprecated.Since hbase-2.0.0. No replacement. To be removed in hbase-4.0.0 and replaced with something that doesn't expose IntefaceAudience.Private classes. VisibilityController still needs this, need to change the logic there first.Called before the server updates the timestamp for version delete with latest timestamp.Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
- Parameters:
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 qualifier- Throws:
IOException
-
postDelete
@Deprecated default void postDelete(ObserverContext<? extends RegionCoprocessorEnvironment> c, Delete delete, WALEdit edit, Durability durability) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsepostDelete(ObserverContext, Delete, WALEdit)
instead.Called after the client deletes a value.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.
- Parameters:
c
- the environment provided by the region serverdelete
- The Delete objectedit
- The WALEdit object for the waldurability
- Persistence guarantee for this Delete- Throws:
IOException
-
postDelete
default void postDelete(ObserverContext<? extends RegionCoprocessorEnvironment> c, Delete delete, WALEdit edit) throws IOException Called after the client deletes a value.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.
- Parameters:
c
- the environment provided by the region serverdelete
- The Delete objectedit
- The WALEdit object for the wal- Throws:
IOException
-
preBatchMutate
default void preBatchMutate(ObserverContext<? extends RegionCoprocessorEnvironment> c, MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException This will be called for every batch mutation operation happening at the server. This will be called after acquiring the locks on the mutating rows and after applying the proper timestamp for each Mutation at the server. The batch may contain Put/Delete/Increment/Append. By setting OperationStatus of Mutations (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.
- Parameters:
c
- the environment provided by the region serverminiBatchOp
- batch of Mutations getting applied to region.- Throws:
IOException
-
postBatchMutate
default void postBatchMutate(ObserverContext<? extends RegionCoprocessorEnvironment> c, MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException This will be called after applying a batch of Mutations on a region. The Mutations are added to memstore and WAL. The difference of this one withpostPut(ObserverContext, Put, WALEdit)
andpostDelete(ObserverContext, Delete, WALEdit)
andpostIncrement(ObserverContext, Increment, Result, WALEdit)
andpostAppend(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.
- Parameters:
c
- the environment provided by the region serverminiBatchOp
- batch of Mutations applied to region. Coprocessors are discouraged from manipulating its state.- Throws:
IOException
-
postStartRegionOperation
default void postStartRegionOperation(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, Region.Operation operation) throws IOException This will be called for region operations where read lock is acquired inRegion.startRegionOperation()
.- Parameters:
operation
- The operation is about to be taken on the region- Throws:
IOException
-
postCloseRegionOperation
default void postCloseRegionOperation(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, Region.Operation operation) throws IOException Called after releasing read lock inRegion.closeRegionOperation()
.- Throws:
IOException
-
postBatchMutateIndispensably
default void postBatchMutateIndispensably(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, MiniBatchOperationInProgress<Mutation> miniBatchOp, boolean success) throws IOException Called after the completion of batch put/delete/increment/append and will be called even if the batch operation fails.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.
- Parameters:
success
- true if batch operation is successful otherwise false.- Throws:
IOException
-
preCheckAndPut
@Deprecated default boolean preCheckAndPut(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Put put, boolean result) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsepreCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)
instead.Called before checkAndPut.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.
- Parameters:
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 result- Returns:
- the return value to return to client if bypassing default processing
- Throws:
IOException
-
preCheckAndPut
@Deprecated default boolean preCheckAndPut(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, Filter filter, Put put, boolean result) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsepreCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)
instead.Called before checkAndPut.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.
- Parameters:
c
- the environment provided by the region serverrow
- row to checkfilter
- filterput
- data to put if check succeedsresult
- the default value of the result- Returns:
- the return value to return to client if bypassing default processing
- Throws:
IOException
-
preCheckAndPutAfterRowLock
@Deprecated default boolean preCheckAndPutAfterRowLock(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Put put, boolean result) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsepreCheckAndMutateAfterRowLock(ObserverContext, CheckAndMutate,CheckAndMutateResult)
instead.Called before checkAndPut but after acquiring rowlock.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.
- Parameters:
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 result- Returns:
- the return value to return to client if bypassing default processing
- Throws:
IOException
-
preCheckAndPutAfterRowLock
@Deprecated default boolean preCheckAndPutAfterRowLock(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, Filter filter, Put put, boolean result) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsepreCheckAndMutateAfterRowLock(ObserverContext, CheckAndMutate,CheckAndMutateResult)
instead.Called before checkAndPut but after acquiring rowlock.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.
- Parameters:
c
- the environment provided by the region serverrow
- row to checkfilter
- filterput
- data to put if check succeedsresult
- the default value of the result- Returns:
- the return value to return to client if bypassing default processing
- Throws:
IOException
-
postCheckAndPut
@Deprecated default boolean postCheckAndPut(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Put put, boolean result) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsepostCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)
instead.Called after checkAndPutNote: 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.
- Parameters:
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 checkAndPut- Returns:
- the possibly transformed return value to return to client
- Throws:
IOException
-
postCheckAndPut
@Deprecated default boolean postCheckAndPut(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, Filter filter, Put put, boolean result) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsepostCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)
instead.Called after checkAndPutNote: 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.
- Parameters:
c
- the environment provided by the region serverrow
- row to checkfilter
- filterput
- data to put if check succeedsresult
- from the checkAndPut- Returns:
- the possibly transformed return value to return to client
- Throws:
IOException
-
preCheckAndDelete
@Deprecated default boolean preCheckAndDelete(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Delete delete, boolean result) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsepreCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)
instead.Called before checkAndDelete.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.
- Parameters:
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 result- Returns:
- the value to return to client if bypassing default processing
- Throws:
IOException
-
preCheckAndDelete
@Deprecated default boolean preCheckAndDelete(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, Filter filter, Delete delete, boolean result) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsepreCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)
instead.Called before checkAndDelete.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.
- Parameters:
c
- the environment provided by the region serverrow
- row to checkfilter
- column familydelete
- delete to commit if check succeedsresult
- the default value of the result- Returns:
- the value to return to client if bypassing default processing
- Throws:
IOException
-
preCheckAndDeleteAfterRowLock
@Deprecated default boolean preCheckAndDeleteAfterRowLock(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Delete delete, boolean result) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsepreCheckAndMutateAfterRowLock(ObserverContext, CheckAndMutate,CheckAndMutateResult)
instead.Called before checkAndDelete but after acquiring rowock.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.
- Parameters:
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 result- Returns:
- the value to return to client if bypassing default processing
- Throws:
IOException
-
preCheckAndDeleteAfterRowLock
@Deprecated default boolean preCheckAndDeleteAfterRowLock(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, Filter filter, Delete delete, boolean result) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsepreCheckAndMutateAfterRowLock(ObserverContext, CheckAndMutate,CheckAndMutateResult)
instead.Called before checkAndDelete but after acquiring rowock.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.
- Parameters:
c
- the environment provided by the region serverrow
- row to checkfilter
- filterdelete
- delete to commit if check succeedsresult
- the default value of the result- Returns:
- the value to return to client if bypassing default processing
- Throws:
IOException
-
postCheckAndDelete
@Deprecated default boolean postCheckAndDelete(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Delete delete, boolean result) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsepostCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)
instead.Called after checkAndDeleteNote: 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.
- Parameters:
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 CheckAndDelete- Returns:
- the possibly transformed returned value to return to client
- Throws:
IOException
-
postCheckAndDelete
@Deprecated default boolean postCheckAndDelete(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, Filter filter, Delete delete, boolean result) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsepostCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult)
instead.Called after checkAndDeleteNote: 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.
- Parameters:
c
- the environment provided by the region serverrow
- row to checkfilter
- filterdelete
- delete to commit if check succeedsresult
- from the CheckAndDelete- Returns:
- the possibly transformed returned value to return to client
- Throws:
IOException
-
preCheckAndMutate
default CheckAndMutateResult preCheckAndMutate(ObserverContext<? extends RegionCoprocessorEnvironment> c, CheckAndMutate checkAndMutate, CheckAndMutateResult result) throws IOException Called before checkAndMutateCall 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.
- Parameters:
c
- the environment provided by the region servercheckAndMutate
- the CheckAndMutate objectresult
- the default value of the result- Returns:
- the return value to return to client if bypassing default processing
- Throws:
IOException
- if an error occurred on the coprocessor
-
preCheckAndMutateAfterRowLock
default CheckAndMutateResult preCheckAndMutateAfterRowLock(ObserverContext<? extends RegionCoprocessorEnvironment> c, CheckAndMutate checkAndMutate, CheckAndMutateResult result) throws IOException Called before checkAndDelete but after acquiring rowlock.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.
- Parameters:
c
- the environment provided by the region servercheckAndMutate
- the CheckAndMutate objectresult
- the default value of the result- Returns:
- the value to return to client if bypassing default processing
- Throws:
IOException
- if an error occurred on the coprocessor
-
postCheckAndMutate
default CheckAndMutateResult postCheckAndMutate(ObserverContext<? extends RegionCoprocessorEnvironment> c, CheckAndMutate checkAndMutate, CheckAndMutateResult result) throws IOException Called after checkAndMutateNote: 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.
- Parameters:
c
- the environment provided by the region servercheckAndMutate
- the CheckAndMutate objectresult
- from the checkAndMutate- Returns:
- the possibly transformed returned value to return to client
- Throws:
IOException
- if an error occurred on the coprocessor
-
preAppend
@Deprecated default Result preAppend(ObserverContext<? extends RegionCoprocessorEnvironment> c, Append append) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsepreAppend(ObserverContext, Append, WALEdit)
instead.Called before Append.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.
- Parameters:
c
- the environment provided by the region serverappend
- Append object- Returns:
- result to return to the client if bypassing default processing
- Throws:
IOException
-
preAppend
default Result preAppend(ObserverContext<? extends RegionCoprocessorEnvironment> c, Append append, WALEdit edit) throws IOException Called before Append.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.
- Parameters:
c
- the environment provided by the region serverappend
- Append objectedit
- The WALEdit object that will be written to the wal- Returns:
- result to return to the client if bypassing default processing
- Throws:
IOException
-
preAppendAfterRowLock
@Deprecated default Result preAppendAfterRowLock(ObserverContext<? extends RegionCoprocessorEnvironment> c, Append append) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsepreBatchMutate(ObserverContext, MiniBatchOperationInProgress)
instead.Called before Append but after acquiring rowlock.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.
- Parameters:
c
- the environment provided by the region serverappend
- Append object- Returns:
- result to return to the client if bypassing default processing
- Throws:
IOException
-
postAppend
@Deprecated default Result postAppend(ObserverContext<? extends RegionCoprocessorEnvironment> c, Append append, Result result) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsepostAppend(ObserverContext, Append, Result, WALEdit)
instead.Called after AppendNote: 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.
- Parameters:
c
- the environment provided by the region serverappend
- Append objectresult
- the result returned by increment- Returns:
- the result to return to the client
- Throws:
IOException
-
postAppend
default Result postAppend(ObserverContext<? extends RegionCoprocessorEnvironment> c, Append append, Result result, WALEdit edit) throws IOException Called after AppendNote: 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.
- Parameters:
c
- the environment provided by the region serverappend
- Append objectresult
- the result returned by incrementedit
- The WALEdit object for the wal- Returns:
- the result to return to the client
- Throws:
IOException
-
preIncrement
@Deprecated default Result preIncrement(ObserverContext<? extends RegionCoprocessorEnvironment> c, Increment increment) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsepreIncrement(ObserverContext, Increment, WALEdit)
instead.Called before Increment.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.
- Parameters:
c
- the environment provided by the region serverincrement
- increment object- Returns:
- result to return to the client if bypassing default processing
- Throws:
IOException
-
preIncrement
default Result preIncrement(ObserverContext<? extends RegionCoprocessorEnvironment> c, Increment increment, WALEdit edit) throws IOException Called before Increment.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.
- Parameters:
c
- the environment provided by the region serverincrement
- increment objectedit
- The WALEdit object that will be written to the wal- Returns:
- result to return to the client if bypassing default processing
- Throws:
IOException
-
preIncrementAfterRowLock
@Deprecated default Result preIncrementAfterRowLock(ObserverContext<? extends RegionCoprocessorEnvironment> c, Increment increment) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsepreBatchMutate(ObserverContext, MiniBatchOperationInProgress)
instead.Called before Increment but after acquiring rowlock.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.
- Parameters:
c
- the environment provided by the region serverincrement
- increment object- Returns:
- result to return to the client if bypassing default processing
- Throws:
IOException
-
postIncrement
@Deprecated default Result postIncrement(ObserverContext<? extends RegionCoprocessorEnvironment> c, Increment increment, Result result) throws IOException Deprecated.since 3.0.0 and will be removed in 4.0.0. UsepostIncrement(ObserverContext, Increment, Result, WALEdit)
instead.Called after incrementNote: 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.
- Parameters:
c
- the environment provided by the region serverincrement
- increment objectresult
- the result returned by increment- Returns:
- the result to return to the client
- Throws:
IOException
-
postIncrement
default Result postIncrement(ObserverContext<? extends RegionCoprocessorEnvironment> c, Increment increment, Result result, WALEdit edit) throws IOException Called after incrementNote: 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.
- Parameters:
c
- the environment provided by the region serverincrement
- increment objectresult
- the result returned by incrementedit
- The WALEdit object for the wal- Returns:
- the result to return to the client
- Throws:
IOException
-
preScannerOpen
default void preScannerOpen(ObserverContext<? extends RegionCoprocessorEnvironment> c, Scan scan) throws IOException Called before the client opens a new scanner.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.
- Parameters:
c
- the environment provided by the region serverscan
- the Scan specification- Throws:
IOException
-
postScannerOpen
default RegionScanner postScannerOpen(ObserverContext<? extends RegionCoprocessorEnvironment> c, Scan scan, RegionScanner s) throws IOException Called after the client opens a new scanner.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.
- Parameters:
c
- the environment provided by the region serverscan
- the Scan specifications
- if not null, the base scanner- Returns:
- the scanner instance to use
- Throws:
IOException
-
preScannerNext
default boolean preScannerNext(ObserverContext<? extends RegionCoprocessorEnvironment> c, InternalScanner s, List<Result> result, int limit, boolean hasNext) throws IOException Called before the client asks for the next row on a scanner.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.
- Parameters:
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' indication- Returns:
- 'has more' indication that should be sent to client
- Throws:
IOException
-
postScannerNext
default boolean postScannerNext(ObserverContext<? extends RegionCoprocessorEnvironment> c, InternalScanner s, List<Result> result, int limit, boolean hasNext) throws IOException Called after the client asks for the next row on a scanner.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.
- Parameters:
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' indication- Returns:
- 'has more' indication that should be sent to client
- Throws:
IOException
-
postScannerFilterRow
default boolean postScannerFilterRow(ObserverContext<? extends RegionCoprocessorEnvironment> c, InternalScanner s, Cell curRowCell, boolean hasMore) throws IOException This will be called by the scan flow when the current scanned row is being filtered out by the filter. The filter may be filtering out the row via any of the below scenariosboolean filterRowKey(byte [] buffer, int offset, int length)
returning trueboolean filterRow()
returning truedefault void filterRow(List<KeyValue> kvs)
removing all the kvs from the passed List
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.
- Parameters:
c
- the environment provided by the region servers
- the scannercurRowCell
- The cell in the current row which got filtered outhasMore
- the 'has more' indication- Returns:
- whether more rows are available for the scanner or not
- Throws:
IOException
-
preScannerClose
default void preScannerClose(ObserverContext<? extends RegionCoprocessorEnvironment> c, InternalScanner s) throws IOException Called before the client closes a scanner.Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
- Parameters:
c
- the environment provided by the region servers
- the scanner- Throws:
IOException
-
postScannerClose
default void postScannerClose(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, InternalScanner s) throws IOException Called after the client closes a scanner.- Parameters:
ctx
- the environment provided by the region servers
- the scanner- Throws:
IOException
-
preStoreScannerOpen
default void preStoreScannerOpen(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, Store store, ScanOptions options) throws IOException Called before a store opens a new scanner.This hook is called when a "user" scanner is opened. Use
preFlushScannerOpen
andpreCompactScannerOpen
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 inpreScannerOpen
, 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.- Parameters:
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 opened- Throws:
IOException
- See Also:
-
preReplayWALs
default void preReplayWALs(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, RegionInfo info, org.apache.hadoop.fs.Path edits) throws IOException Called before replaying WALs for this region. CallingObserverContext.bypass()
has no effect in this hook.- Parameters:
ctx
- the environment provided by the region serverinfo
- the RegionInfo for this regionedits
- the file of recovered edits- Throws:
IOException
-
postReplayWALs
default void postReplayWALs(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, RegionInfo info, org.apache.hadoop.fs.Path edits) throws IOException Called after replaying WALs for this region.- Parameters:
ctx
- the environment provided by the region serverinfo
- the RegionInfo for this regionedits
- the file of recovered edits- Throws:
IOException
-
preBulkLoadHFile
default void preBulkLoadHFile(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, List<Pair<byte[], String>> familyPaths) throws IOExceptionCalled before bulkLoadHFile. Users can create a StoreFile instance to access the contents of a HFile.- Parameters:
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.- Throws:
IOException
-
preCommitStoreFile
default void preCommitStoreFile(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, byte[] family, List<Pair<org.apache.hadoop.fs.Path, org.apache.hadoop.fs.Path>> pairs) throws IOExceptionCalled before moving bulk loaded hfile to region directory.- Parameters:
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.- Throws:
IOException
-
postCommitStoreFile
default void postCommitStoreFile(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, byte[] family, org.apache.hadoop.fs.Path srcPath, org.apache.hadoop.fs.Path dstPath) throws IOException Called after moving bulk loaded hfile to region directory.- Parameters:
ctx
- the environment provided by the region serverfamily
- column familysrcPath
- Path to file before the movedstPath
- Path to file after the move- Throws:
IOException
-
postBulkLoadHFile
default void postBulkLoadHFile(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, List<Pair<byte[], String>> stagingFamilyPaths, Map<byte[], throws IOExceptionList<org.apache.hadoop.fs.Path>> finalPaths) Called after bulkLoadHFile.- Parameters:
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.- Throws:
IOException
-
preStoreFileReaderOpen
@Deprecated default StoreFileReader preStoreFileReaderOpen(ObserverContext<? extends 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 Deprecated.For Phoenix only, StoreFileReader is not a stable interface.Called before creation of Reader for a store file. CallingObserverContext.bypass()
has no effect in this hook.- Parameters:
ctx
- the environment provided by the region serverfs
- fileystem to read fromp
- path to the filein
-FSDataInputStreamWrapper
size
- Full size of the filer
- original reference file. This will be not null only when reading a split file.reader
- the base reader, if notnull
, from previous RegionObserver in the chain- Returns:
- a Reader instance to use instead of the base reader if overriding default behavior, null otherwise
- Throws:
IOException
-
postStoreFileReaderOpen
@Deprecated default StoreFileReader postStoreFileReaderOpen(ObserverContext<? extends 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 Deprecated.For Phoenix only, StoreFileReader is not a stable interface.Called after the creation of Reader for a store file.- Parameters:
ctx
- the environment provided by the region serverfs
- fileystem to read fromp
- path to the filein
-FSDataInputStreamWrapper
size
- Full size of the filer
- original reference file. This will be not null only when reading a split file.reader
- the base reader instance- Returns:
- The reader to use
- Throws:
IOException
-
postMutationBeforeWAL
@Deprecated default Cell postMutationBeforeWAL(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, RegionObserver.MutationType opType, Mutation mutation, Cell oldCell, Cell newCell) throws IOException Deprecated.since 2.2.0 and will be removedin 4.0.0. UsepostIncrementBeforeWAL(ObserverContext, Mutation, List)
orpostAppendBeforeWAL(ObserverContext, Mutation, List)
instead.Called after a new cell has been created during an increment operation, but before it is committed to the WAL or memstore. CallingObserverContext.bypass()
has no effect in this hook.- Parameters:
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 value- Returns:
- the new cell, possibly changed
- Throws:
IOException
- See Also:
-
postIncrementBeforeWAL
default List<Pair<Cell,Cell>> postIncrementBeforeWAL(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, Mutation mutation, List<Pair<Cell, Cell>> cellPairs) throws IOExceptionCalled after a list of new cells has been created during an increment operation, but before they are committed to the WAL or memstore.- Parameters:
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.- Returns:
- a list of cell pair, possibly changed.
- Throws:
IOException
-
postAppendBeforeWAL
default List<Pair<Cell,Cell>> postAppendBeforeWAL(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, Mutation mutation, List<Pair<Cell, Cell>> cellPairs) throws IOExceptionCalled after a list of new cells has been created during an append operation, but before they are committed to the WAL or memstore.- Parameters:
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.- Returns:
- a list of cell pair, possibly changed.
- Throws:
IOException
-
postInstantiateDeleteTracker
@Deprecated default DeleteTracker postInstantiateDeleteTracker(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, DeleteTracker delTracker) throws IOException Deprecated.Since 2.0.0, will be removed in 4.0.0. Visibility label feature still use this method, so it can not be removed in 3.0.0Called after the ScanQueryMatcher creates ScanDeleteTracker. Implementing this hook would help in creating customised DeleteTracker and returning the newly created DeleteTrackerWarn: This is used by internal coprocessors. Should not be implemented by user coprocessors
- Parameters:
ctx
- the environment provided by the region serverdelTracker
- the deleteTracker that is created by the QueryMatcher- Returns:
- the Delete Tracker
- Throws:
IOException
-
preWALAppend
default void preWALAppend(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, WALKey key, WALEdit edit) throws IOException Called just before the WAL Entry is appended to the WAL. Implementing this hook allows coprocessors to add extended attributes to the WALKey that then get persisted to the WAL, and are available to replication endpoints to use in processing WAL Entries.- Parameters:
ctx
- the environment provided by the region serverkey
- the WALKey associated with a particular append to a WAL- Throws:
IOException
-