Interface RegionObserver

All Known Implementing Classes:
AccessController, BackupObserver, ConstraintProcessor, ExampleRegionObserverWithMetrics.ExampleRegionObserver, MetaTableMetrics.ExampleRegionObserverMeta, ReplicationObserver, ScanModifyingObserver, ValueRewritingObserver, VisibilityController, WriteHeavyIncrementObserver, WriteSinkCoprocessor, ZooKeeperScanPolicyObserver

@LimitedPrivate("Coprocesssor") @Evolving public interface RegionObserver
Coprocessors implement this interface to observe and mediate client actions on the region.

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:

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.
  • 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

      Called before the memstore is flushed to disk.
      Parameters:
      c - the environment provided by the region server
      tracker - tracker used to track the life cycle of a flush
      Throws:
      IOException
    • preFlushScannerOpen

      Called before we open store scanner for flush. You can use the options to change max versions and TTL for the scanner being opened.
      Parameters:
      c - the environment provided by the region server
      store - the store where flush is being requested
      options - used to change max versions and TTL for the scanner being opened
      Throws:
      IOException
    • preFlush

      Called before a Store's memstore is flushed to disk.
      Parameters:
      c - the environment provided by the region server
      store - the store where flush is being requested
      scanner - the scanner over existing data used in the memstore
      tracker - 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

      Called after the memstore is flushed to disk.
      Parameters:
      c - the environment provided by the region server
      tracker - 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 server
      store - the store being flushed
      resultFile - the new store file written out during compaction
      tracker - tracker used to track the life cycle of a flush
      Throws:
      IOException
    • preMemStoreCompaction

      Called before in memory compaction started.
      Parameters:
      c - the environment provided by the region server
      store - the store where in memory compaction is being requested
      Throws:
      IOException
    • preMemStoreCompactionCompactScannerOpen

      Called before we open store scanner for in memory compaction. You can use the 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.
      Parameters:
      c - the environment provided by the region server
      store - the store where in memory compaction is being requested
      options - used to change max versions and TTL for the scanner being opened
      Throws:
      IOException
    • preMemStoreCompactionCompact

      Called before we do in memory compaction. Notice that this method will only be called when you use eager mode. For basic mode we will not drop any cells thus there is no InternalScanner.
      Parameters:
      c - the environment provided by the region server
      store - the store where in memory compaction is being executed
      scanner - 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

      Called after the in memory compaction is finished.
      Parameters:
      c - the environment provided by the region server
      store - 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 the 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.

      Parameters:
      c - the environment provided by the region server
      store - the store where compaction is being requested
      candidates - the store files currently available for compaction
      tracker - 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 the StoreFiles to compact have been selected from the available candidates.
      Parameters:
      c - the environment provided by the region server
      store - the store being compacted
      selected - the store files selected to compact
      tracker - tracker used to track the life cycle of a compaction
      request - the requested compaction
    • preCompactScannerOpen

      Called before we open store scanner for compaction. You can use the options to change max versions and TTL for the scanner being opened.
      Parameters:
      c - the environment provided by the region server
      store - the store being compacted
      scanType - type of Scan
      options - used to change max versions and TTL for the scanner being opened
      tracker - tracker used to track the life cycle of a compaction
      request - the requested compaction
      Throws:
      IOException
    • preCompact

      Called prior to writing the StoreFiles 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 Cells 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 implement Shipper 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 server
      store - the store being compacted
      scanner - the scanner over existing data used in the store file rewriting
      scanType - type of Scan
      tracker - tracker used to track the life cycle of a compaction
      request - 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 server
      store - the store being compacted
      resultFile - the new store file written out during compaction
      tracker - used to track the life cycle of a compaction
      request - 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 server
      abortRequested - 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 server
      abortRequested - 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 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 server
      get - the Get request
      result - 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 Get

      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.

      Parameters:
      c - the environment provided by the region server
      get - the Get request
      result - 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 server
      get - the Get request
      exists - 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 server
      get - the Get request
      exists - 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. Use prePut(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 server
      put - The Put object
      edit - The WALEdit object that will be written to the wal
      durability - 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 server
      put - The Put object
      edit - 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. Use postPut(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 server
      put - The Put object
      edit - The WALEdit object for the wal
      durability - 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 server
      put - The Put object
      edit - 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. Use preDelete(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 server
      delete - The Delete object
      edit - The WALEdit object for the wal
      durability - 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 server
      delete - The Delete object
      edit - 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 server
      mutation - - the parent mutation associated with this delete cell
      cell - - The deleteColumn with latest version cell
      byteNow - - timestamp bytes
      get - - 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. Use postDelete(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 server
      delete - The Delete object
      edit - The WALEdit object for the wal
      durability - 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 server
      delete - The Delete object
      edit - The WALEdit object for the wal
      Throws:
      IOException
    • preBatchMutate

      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 server
      miniBatchOp - batch of Mutations getting applied to region.
      Throws:
      IOException
    • postBatchMutate

      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 with 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.

      Parameters:
      c - the environment provided by the region server
      miniBatchOp - batch of Mutations applied to region. Coprocessors are discouraged from manipulating its state.
      Throws:
      IOException
    • postStartRegionOperation

      This will be called for region operations where read lock is acquired in Region.startRegionOperation().
      Parameters:
      operation - The operation is about to be taken on the region
      Throws:
      IOException
    • postCloseRegionOperation

      Called after releasing read lock in Region.closeRegionOperation().
      Throws:
      IOException
    • postBatchMutateIndispensably

      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. Use preCheckAndMutate(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 server
      row - row to check
      family - column family
      qualifier - column qualifier
      op - the comparison operation
      comparator - the comparator
      put - data to put if check succeeds
      result - 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. Use preCheckAndMutate(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 server
      row - row to check
      filter - filter
      put - data to put if check succeeds
      result - 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.
      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 server
      row - row to check
      family - column family
      qualifier - column qualifier
      op - the comparison operation
      comparator - the comparator
      put - data to put if check succeeds
      result - 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.
      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 server
      row - row to check
      filter - filter
      put - data to put if check succeeds
      result - 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. Use postCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult) instead.
      Called after checkAndPut

      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 server
      row - row to check
      family - column family
      qualifier - column qualifier
      op - the comparison operation
      comparator - the comparator
      put - data to put if check succeeds
      result - 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. Use postCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult) instead.
      Called after checkAndPut

      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 server
      row - row to check
      filter - filter
      put - data to put if check succeeds
      result - 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. Use preCheckAndMutate(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 server
      row - row to check
      family - column family
      qualifier - column qualifier
      op - the comparison operation
      comparator - the comparator
      delete - delete to commit if check succeeds
      result - 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. Use preCheckAndMutate(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 server
      row - row to check
      filter - column family
      delete - delete to commit if check succeeds
      result - 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.
      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 server
      row - row to check
      family - column family
      qualifier - column qualifier
      op - the comparison operation
      comparator - the comparator
      delete - delete to commit if check succeeds
      result - 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.
      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 server
      row - row to check
      filter - filter
      delete - delete to commit if check succeeds
      result - 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. Use postCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult) instead.
      Called after checkAndDelete

      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 server
      row - row to check
      family - column family
      qualifier - column qualifier
      op - the comparison operation
      comparator - the comparator
      delete - delete to commit if check succeeds
      result - 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. Use postCheckAndMutate(ObserverContext, CheckAndMutate, CheckAndMutateResult) instead.
      Called after checkAndDelete

      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 server
      row - row to check
      filter - filter
      delete - delete to commit if check succeeds
      result - from the CheckAndDelete
      Returns:
      the possibly transformed returned value to return to client
      Throws:
      IOException
    • preCheckAndMutate

      Called before checkAndMutate

      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 server
      checkAndMutate - the CheckAndMutate object
      result - 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

      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 server
      checkAndMutate - the CheckAndMutate object
      result - 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

      Called after checkAndMutate

      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 server
      checkAndMutate - the CheckAndMutate object
      result - from the checkAndMutate
      Returns:
      the possibly transformed returned value to return to client
      Throws:
      IOException - if an error occurred on the coprocessor
    • preAppend

      Deprecated.
      since 3.0.0 and will be removed in 4.0.0. Use preAppend(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 server
      append - Append object
      Returns:
      result to return to the client if bypassing default processing
      Throws:
      IOException
    • preAppend

      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 server
      append - Append object
      edit - 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.
      since 3.0.0 and will be removed in 4.0.0. Use preBatchMutate(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 server
      append - Append object
      Returns:
      result to return to the client if bypassing default processing
      Throws:
      IOException
    • postAppend

      Deprecated.
      since 3.0.0 and will be removed in 4.0.0. Use postAppend(ObserverContext, Append, Result, WALEdit) instead.
      Called after Append

      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 server
      append - Append object
      result - 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 Append

      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 server
      append - Append object
      result - the result returned by increment
      edit - The WALEdit object for the wal
      Returns:
      the result to return to the client
      Throws:
      IOException
    • preIncrement

      Deprecated.
      since 3.0.0 and will be removed in 4.0.0. Use preIncrement(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 server
      increment - increment object
      Returns:
      result to return to the client if bypassing default processing
      Throws:
      IOException
    • preIncrement

      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 server
      increment - increment object
      edit - 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.
      since 3.0.0 and will be removed in 4.0.0. Use preBatchMutate(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 server
      increment - increment object
      Returns:
      result to return to the client if bypassing default processing
      Throws:
      IOException
    • postIncrement

      Deprecated.
      since 3.0.0 and will be removed in 4.0.0. Use postIncrement(ObserverContext, Increment, Result, WALEdit) instead.
      Called after increment

      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 server
      increment - increment object
      result - 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 increment

      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 server
      increment - increment object
      result - the result returned by increment
      edit - 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 server
      scan - the Scan specification
      Throws:
      IOException
    • postScannerOpen

      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 server
      scan - the Scan specification
      s - 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 server
      s - the scanner
      result - 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 return
      hasNext - 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 server
      s - the scanner
      result - the result to return to the client, can be modified
      limit - the maximum number of results to return
      hasNext - 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 scenarios
      1. boolean filterRowKey(byte [] buffer, int offset, int length) returning true
      2. boolean filterRow() returning true
      3. default 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 server
      s - the scanner
      curRowCell - The cell in the current row which got filtered out
      hasMore - the 'has more' indication
      Returns:
      whether more rows are available for the scanner or not
      Throws:
      IOException
    • preScannerClose

      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 server
      s - the scanner
      Throws:
      IOException
    • postScannerClose

      Called after the client closes a scanner.
      Parameters:
      ctx - the environment provided by the region server
      s - 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 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.

      Parameters:
      ctx - the environment provided by the region server
      store - the store which we want to get scanner from
      options - 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. Calling ObserverContext.bypass() has no effect in this hook.
      Parameters:
      ctx - the environment provided by the region server
      info - the RegionInfo for this region
      edits - 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 server
      info - the RegionInfo for this region
      edits - the file of recovered edits
      Throws:
      IOException
    • preBulkLoadHFile

      default void preBulkLoadHFile(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, List<Pair<byte[],String>> familyPaths) throws IOException
      Called before bulkLoadHFile. Users can create a StoreFile instance to access the contents of a HFile.
      Parameters:
      ctx - the environment provided by the region server
      familyPaths - pairs of { CF, HFile path } submitted for bulk load. Adding or removing from this list will add or remove HFiles to be bulk loaded.
      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 IOException
      Called before moving bulk loaded hfile to region directory.
      Parameters:
      ctx - the environment provided by the region server
      family - column family
      pairs - 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 server
      family - column family
      srcPath - Path to file before the move
      dstPath - Path to file after the move
      Throws:
      IOException
    • postBulkLoadHFile

      default void postBulkLoadHFile(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, List<Pair<byte[],String>> stagingFamilyPaths, Map<byte[],List<org.apache.hadoop.fs.Path>> finalPaths) throws IOException
      Called after bulkLoadHFile.
      Parameters:
      ctx - the environment provided by the region server
      stagingFamilyPaths - pairs of { CF, HFile path } submitted for bulk load
      finalPaths - 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. Calling ObserverContext.bypass() has no effect in this hook.
      Parameters:
      ctx - the environment provided by the region server
      fs - fileystem to read from
      p - path to the file
      in - FSDataInputStreamWrapper
      size - Full size of the file
      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 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 server
      fs - fileystem to read from
      p - path to the file
      in - FSDataInputStreamWrapper
      size - Full size of the file
      r - 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

      Called after a new cell has been created during an increment operation, but before it is committed to the WAL or memstore. Calling ObserverContext.bypass() has no effect in this hook.
      Parameters:
      ctx - the environment provided by the region server
      opType - the operation type
      mutation - the current mutation
      oldCell - old cell containing previous value
      newCell - the new cell containing the computed value
      Returns:
      the new cell, possibly changed
      Throws:
      IOException
      See Also:
    • postIncrementBeforeWAL

      Called 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 server
      mutation - the current mutation
      cellPairs - 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

      Called 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 server
      mutation - the current mutation
      cellPairs - 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.
      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.0
      Called after the ScanQueryMatcher creates ScanDeleteTracker. Implementing this hook would help in creating customised DeleteTracker and returning the newly created DeleteTracker

      Warn: This is used by internal coprocessors. Should not be implemented by user coprocessors

      Parameters:
      ctx - the environment provided by the region server
      delTracker - 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 server
      key - the WALKey associated with a particular append to a WAL
      Throws:
      IOException