Interface RegionServerObserver
- All Known Implementing Classes:
AccessController
,VisibilityReplication
Defines coprocessor hooks for interacting with operations on the
HRegionServer
process. 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
-
Method Summary
Modifier and TypeMethodDescriptiondefault void
This will be called after clearing compaction queuesdefault void
postClearRegionBlockCache
(ObserverContext<RegionServerCoprocessorEnvironment> ctx, CacheEvictionStats stats) Called after clearing the block caches for one or more regionsdefault ReplicationEndpoint
postCreateReplicationEndPoint
(ObserverContext<RegionServerCoprocessorEnvironment> ctx, ReplicationEndpoint endpoint) This will be called after the replication endpoint is instantiated.default void
This will be called after executing proceduresdefault void
Deprecated.As of release 2.0.0 with out any replacement.default void
postReplicationSinkBatchMutate
(ObserverContext<RegionServerCoprocessorEnvironment> ctx, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry walEntry, Mutation mutation) This will be called after replication sink mutations are executed on the sink table as part of batch call.default void
This will be called after executing user request to roll a region server WAL.default void
postUpdateRegionServerConfiguration
(ObserverContext<RegionServerCoprocessorEnvironment> ctx, org.apache.hadoop.conf.Configuration postReloadConf) Called after reloading the RegionServer'sConfiguration
from diskdefault void
This will be called before clearing compaction queuesdefault void
default void
This will be called before executing proceduresdefault void
Deprecated.As of release 2.0.0 with out any replacement.default void
preReplicationSinkBatchMutate
(ObserverContext<RegionServerCoprocessorEnvironment> ctx, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry walEntry, Mutation mutation) This will be called before replication sink mutations are executed on the sink table as part of batch call.default void
This will be called before executing user request to roll a region server WAL.default void
Called before stopping region server.default void
preUpdateRegionServerConfiguration
(ObserverContext<RegionServerCoprocessorEnvironment> ctx, org.apache.hadoop.conf.Configuration preReloadConf) Called before reloading the RegionServer'sConfiguration
from disk
-
Method Details
-
preStopRegionServer
default void preStopRegionServer(ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException Called before stopping region server.- Parameters:
ctx
- the environment to interact with the framework and region server.- Throws:
IOException
-
preRollWALWriterRequest
default void preRollWALWriterRequest(ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException This will be called before executing user request to roll a region server WAL.- Parameters:
ctx
- the environment to interact with the framework and region server.- Throws:
IOException
-
postRollWALWriterRequest
default void postRollWALWriterRequest(ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException This will be called after executing user request to roll a region server WAL.- Parameters:
ctx
- the environment to interact with the framework and region server.- Throws:
IOException
-
postCreateReplicationEndPoint
default ReplicationEndpoint postCreateReplicationEndPoint(ObserverContext<RegionServerCoprocessorEnvironment> ctx, ReplicationEndpoint endpoint) This will be called after the replication endpoint is instantiated.- Parameters:
ctx
- the environment to interact with the framework and region server.endpoint
- - the base endpoint for replication- Returns:
- the endpoint to use during replication.
-
preReplicateLogEntries
@Deprecated default void preReplicateLogEntries(ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException Deprecated.As of release 2.0.0 with out any replacement. This is maintained for internal usage by AccessController. Do not use these hooks in custom co-processors.This will be called before executing replication request to shipping log entries.- Parameters:
ctx
- the environment to interact with the framework and region server.- Throws:
IOException
-
postReplicateLogEntries
@Deprecated default void postReplicateLogEntries(ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException Deprecated.As of release 2.0.0 with out any replacement. This is maintained for internal usage by AccessController. Do not use these hooks in custom co-processors.This will be called after executing replication request to shipping log entries.- Parameters:
ctx
- the environment to interact with the framework and region server.- Throws:
IOException
-
preClearCompactionQueues
default void preClearCompactionQueues(ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException This will be called before clearing compaction queues- Parameters:
ctx
- the environment to interact with the framework and region server.- Throws:
IOException
-
postClearCompactionQueues
default void postClearCompactionQueues(ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException This will be called after clearing compaction queues- Parameters:
ctx
- the environment to interact with the framework and region server.- Throws:
IOException
-
preExecuteProcedures
default void preExecuteProcedures(ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException This will be called before executing procedures- Parameters:
ctx
- the environment to interact with the framework and region server.- Throws:
IOException
-
postExecuteProcedures
default void postExecuteProcedures(ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException This will be called after executing procedures- Parameters:
ctx
- the environment to interact with the framework and region server.- Throws:
IOException
-
preReplicationSinkBatchMutate
default void preReplicationSinkBatchMutate(ObserverContext<RegionServerCoprocessorEnvironment> ctx, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry walEntry, Mutation mutation) throws IOException This will be called before replication sink mutations are executed on the sink table as part of batch call.- Parameters:
ctx
- the environment to interact with the framework and region server.walEntry
- wal entry from which mutation is formed.mutation
- mutation to be applied at sink cluster.- Throws:
IOException
- if something goes wrong.
-
postReplicationSinkBatchMutate
default void postReplicationSinkBatchMutate(ObserverContext<RegionServerCoprocessorEnvironment> ctx, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry walEntry, Mutation mutation) throws IOException This will be called after replication sink mutations are executed on the sink table as part of batch call.- Parameters:
ctx
- the environment to interact with the framework and region server.walEntry
- wal entry from which mutation is formed.mutation
- mutation to be applied at sink cluster.- Throws:
IOException
- if something goes wrong.
-
preClearRegionBlockCache
default void preClearRegionBlockCache(ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException - Throws:
IOException
-
postClearRegionBlockCache
default void postClearRegionBlockCache(ObserverContext<RegionServerCoprocessorEnvironment> ctx, CacheEvictionStats stats) throws IOException Called after clearing the block caches for one or more regions- Parameters:
ctx
- the coprocessor instance's environmentstats
- statistics about the cache evictions that happened- Throws:
IOException
- if you need to signal an IO error
-
preUpdateRegionServerConfiguration
default void preUpdateRegionServerConfiguration(ObserverContext<RegionServerCoprocessorEnvironment> ctx, org.apache.hadoop.conf.Configuration preReloadConf) throws IOException Called before reloading the RegionServer'sConfiguration
from disk- Parameters:
ctx
- the coprocessor instance's environmentpreReloadConf
- theConfiguration
in use prior to reload- Throws:
IOException
- if you need to signal an IO error
-
postUpdateRegionServerConfiguration
default void postUpdateRegionServerConfiguration(ObserverContext<RegionServerCoprocessorEnvironment> ctx, org.apache.hadoop.conf.Configuration postReloadConf) throws IOException Called after reloading the RegionServer'sConfiguration
from disk- Parameters:
ctx
- the coprocessor instance's environmentpostReloadConf
- theConfiguration
that was loaded- Throws:
IOException
- if you need to signal an IO error
-