Interface MasterObserver
- All Known Implementing Classes:
AccessController,CoprocessorWhitelistMasterObserver,ExampleMasterObserverWithMetrics,MasterQuotasObserver,RSGroupAdminEndpoint,SnapshotScannerHDFSAclController,VisibilityController
Defines coprocessor hooks for interacting with operations on the
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.
HMaster 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
IOExceptionare reported back to client. - For any other kind of exception:
- If the configuration
CoprocessorHost.ABORT_ON_ERROR_KEYis set to true, then the server aborts. - Otherwise, coprocessor is removed from the server and
DoNotRetryIOExceptionis returned to the client.
- If the configuration
-
Method Summary
Modifier and TypeMethodDescriptiondefault voidCalled after a abortProcedure request has been processed.default voidpostAddReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId, ReplicationPeerConfig peerConfig) Called after add a replication peerdefault voidCalled after a new region server group is addeddefault voidpostAssign(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo) Called after the region assignment has been requested.default voidpostBalance(ObserverContext<MasterCoprocessorEnvironment> ctx, BalanceRequest request, List<RegionPlan> plans) Called after the balancing plan has been submitted.default voidpostBalanceRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName, BalanceRequest request, BalanceResponse response) Called after a region server group is removeddefault voidpostBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean oldValue, boolean newValue) Called after the flag to enable/disable balancing has changed.default voidpostClearDeadServers(ObserverContext<MasterCoprocessorEnvironment> ctx, List<ServerName> servers, List<ServerName> notClearedServers) Called after clear dead region servers.default voidpostCloneSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) Called after a snapshot clone operation has been requested.default voidpostCompletedCreateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableDescriptor desc, RegionInfo[] regions) Called after the createTable operation has been requested.default voidpostCompletedDeleteTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) Called afterHMasterdeletes a table.default voidpostCompletedDisableTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) Called after the disableTable operation has been requested.default voidpostCompletedEnableTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) Called after the enableTable operation has been requested.default voidpostCompletedMergeRegionsAction(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge, RegionInfo mergedRegion) called after the regions merge.default voidpostCompletedModifyTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor currentDescriptor) Deprecated.Since 2.1.default voidpostCompletedModifyTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor oldDescriptor, TableDescriptor currentDescriptor) Called after to modifying a table's properties.default voidpostCompletedSnapshotAction(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) Called after the snapshot operation has been completed.default voidpostCompletedSplitRegionAction(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfoA, RegionInfo regionInfoB) Called after the region is split.default voidpostCompletedTruncateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) Called afterHMastertruncates a table.default voidCalled after the createNamespace operation has been requested.default voidpostCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableDescriptor desc, RegionInfo[] regions) Called after the createTable operation has been requested.default voidpostDecommissionRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx, List<ServerName> servers, boolean offload) Called after decommission region servers.default voidpostDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) Called after the deleteNamespace operation has been requested.default voidCalled after the delete snapshot operation has been requested.default voidpostDeleteTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) Called after the deleteTable operation has been requested.default voidCalled after disable a replication peerdefault voidpostDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) Called after the disableTable operation has been requested.default voidCalled after enable a replication peerdefault voidpostEnableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) Called after the enableTable operation has been requested.default voidCalled after get cluster status.default voidCalled after a getLocks request has been processed.default voidpostGetNamespaceDescriptor(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor ns) Called after a getNamespaceDescriptor request has been processed.default voidCalled after a getProcedures request has been processed.default voidCalled after get the configured ReplicationPeerConfig for the specified peerdefault voidpostGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableName> tableNamesList, List<TableDescriptor> descriptors, String regex) Called after a getTableDescriptors request has been processed.default voidpostGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableDescriptor> descriptors, String regex) Called after a getTableNames request has been processed.default voidpostGetUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, String namespace, TableName tableName, byte[] family, byte[] qualifier) Called after getting user permissions.default voidpostGrant(ObserverContext<MasterCoprocessorEnvironment> ctx, UserPermission userPermission, boolean mergeExistingPermissions) Called after granting user permissions.default voidpostHasUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, List<Permission> permissions) Called after checking if user has permissions.default voidpostIsRpcThrottleEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean rpcThrottleEnabled) Called after getting if is rpc throttle enabled.default voidCalled after list decommissioned region servers.default voidpostListNamespaceDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<NamespaceDescriptor> descriptors) Called after a listNamespaceDescriptors request has been processed.default voidpostListNamespaces(ObserverContext<MasterCoprocessorEnvironment> ctx, List<String> namespaces) Called after a listNamespaces request has been processed.default voidCalled after list replication peers.default voidCalled after listSnapshots request has been processed.default voidCalled after heartbeat to a lock.default voidCalled after the master local region memstore is flushed to disk.default voidpostMergeRegions(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo[] regionsToMerge) called after merge regions request.default voidpostMergeRegionsCommitAction(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge, RegionInfo mergedRegion) This will be called after META step as part of regions merge transaction.default voidpostModifyColumnFamilyStoreFileTracker(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, byte[] family, String dstSFT) Called after modifying a family store file tracker.default voidpostModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor currentNsDescriptor) Deprecated.Since 2.1.default voidpostModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor oldNsDescriptor, NamespaceDescriptor currentNsDescriptor) Called after the modifyNamespace operation has been requested.default voidpostModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor currentDescriptor) Deprecated.Since 2.1.default voidpostModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor oldDescriptor, TableDescriptor currentDescriptor) Called after the modifyTable operation has been requested.default voidpostModifyTableStoreFileTracker(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, String dstSFT) Called after modifying a table's store file tracker.default voidpostMove(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo region, ServerName srcServer, ServerName destServer) Called after the region move has been requested.default voidpostMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers, String targetGroup) Called after servers are moved to target region server groupdefault voidpostMoveServersAndTables(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers, Set<TableName> tables, String targetGroup) Called after servers are moved to target region server groupdefault voidpostMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<TableName> tables, String targetGroup) Called after servers are moved to target region server groupdefault voidpostRecommissionRegionServer(ObserverContext<MasterCoprocessorEnvironment> ctx, ServerName server, List<byte[]> encodedRegionNames) Called after recommission region server.default voidpostRegionOffline(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo) Called after the region has been marked offline.default voidCalled after remove a replication peerdefault voidCalled after a region server group is removeddefault voidpostRemoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers) Called after servers are removed from rsgroupdefault voidpostRenameRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String oldName, String newName) Called after rename rsgroup.default voidpostRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace, TableName tableName, RegionInfo[] regionInfos, String description) Called after new LockProcedure is queued.default voidpostRestoreSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) Called after a snapshot restore operation has been requested.default voidpostRevoke(ObserverContext<MasterCoprocessorEnvironment> ctx, UserPermission userPermission) Called after revoking user permissions.default voidpostRollBackMergeRegionsAction(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge) This will be called after the roll back of the regions merge.default voidThis will be called after the roll back of the split region is completeddefault voidpostSetNamespaceQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace, GlobalQuotaSettings quotas) Called after the quota for the namespace is stored.default voidpostSetRegionServerQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String regionServer, GlobalQuotaSettings quotas) Called after the quota for the region server is stored.default voidpostSetSplitOrMergeEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean newValue, MasterSwitchType switchType) Called after setting split / merge switchdefault voidpostSetTableQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, GlobalQuotaSettings quotas) Called after the quota for the table is stored.default voidpostSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, String namespace, GlobalQuotaSettings quotas) Called after the quota for the user on the specified namespace is stored.default voidpostSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, GlobalQuotaSettings quotas) Called after the quota for the user is stored.default voidpostSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, TableName tableName, GlobalQuotaSettings quotas) Called after the quota for the user on the specified table is stored.default voidpostSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) Called after the snapshot operation has been requested.default voidCalled immediately after an active master instance has completed initialization.default voidpostSwitchExceedThrottleQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean oldValue, boolean newValue) Called after switching exceed throttle quota state.default voidpostSwitchRpcThrottle(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean oldValue, boolean newValue) Called after switching rpc throttle enabled state.default voidpostTableFlush(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) Called after the table memstore is flushed to disk.default voidpostTruncateRegion(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfo) Called after the truncate region procedure is called.default voidCalled post the region is truncated.default voidpostTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) Called after the truncateTable operation has been requested.default voidpostUnassign(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo) Called after the region unassignment has been requested.default voidpostUnassign(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo, boolean force) Deprecated.in 2.4.0.default voidpostUpdateMasterConfiguration(ObserverContext<MasterCoprocessorEnvironment> ctx, org.apache.hadoop.conf.Configuration postReloadConf) Called after reloading the HMaster'sConfigurationfrom diskdefault voidpostUpdateReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId, ReplicationPeerConfig peerConfig) Called after update peerConfig for the specified peerdefault voidpostUpdateRSGroupConfig(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName, Map<String, String> configuration) Called after update rsgroup config.default voidpreAbortProcedure(ObserverContext<MasterCoprocessorEnvironment> ctx, long procId) Called before a abortProcedure request has been processed.default voidpreAddReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId, ReplicationPeerConfig peerConfig) Called before add a replication peerdefault voidCalled before a new region server group is addeddefault voidpreAssign(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo) Called prior to assigning a specific region.default voidpreBalance(ObserverContext<MasterCoprocessorEnvironment> ctx, BalanceRequest request) Called prior to requesting rebalancing of the cluster regions, though after the initial checks for regions in transition and the balance switch flag.default voidpreBalanceRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName, BalanceRequest request) Called before a region server group is removeddefault voidpreBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean newValue) Called prior to modifying the flag used to enable/disable region balancing.default voidCalled before clear dead region servers.default voidpreCloneSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) Called before a snapshot is cloned.default voidCalled before a new namespace is created byHMaster.default voidpreCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableDescriptor desc, RegionInfo[] regions) Called before a new table is created byHMaster.default voidpreCreateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableDescriptor desc, RegionInfo[] regions) Called before a new table is created byHMaster.default TableDescriptorCalled before we create the region infos for this table.default voidpreDecommissionRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx, List<ServerName> servers, boolean offload) Called before decommission region servers.default voidpreDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) Called beforeHMasterdeletes a namespacedefault voidCalled before a snapshot is deleted.default voidpreDeleteTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) Called beforeHMasterdeletes a table.default voidpreDeleteTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) Called beforeHMasterdeletes a table.default voidCalled before disable a replication peerdefault voidpreDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) Called prior to disabling a table.default voidpreDisableTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) Called prior to disabling a table.default voidCalled before enable a replication peerdefault voidpreEnableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) Called prior to enabling a table.default voidpreEnableTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) Called prior to enabling a table.default voidCalled before get cluster status.default voidCalled before a getLocks request has been processed.default voidpreGetNamespaceDescriptor(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) Called before a getNamespaceDescriptor request has been processed.default voidCalled before a getProcedures request has been processed.default voidCalled before get the configured ReplicationPeerConfig for the specified peerdefault voidpreGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableName> tableNamesList, List<TableDescriptor> descriptors, String regex) Called before a getTableDescriptors request has been processed.default voidpreGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableDescriptor> descriptors, String regex) Called before a getTableNames request has been processed.default voidpreGetUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, String namespace, TableName tableName, byte[] family, byte[] qualifier) Called before getting user permissions.default voidpreGrant(ObserverContext<MasterCoprocessorEnvironment> ctx, UserPermission userPermission, boolean mergeExistingPermissions) Called before granting user permissions.default voidpreHasUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, List<Permission> permissions) default voidCalled before getting if is rpc throttle enabled.default voidCalled before list decommissioned region servers.default voidpreListNamespaceDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<NamespaceDescriptor> descriptors) Called before a listNamespaceDescriptors request has been processed.default voidpreListNamespaces(ObserverContext<MasterCoprocessorEnvironment> ctx, List<String> namespaces) Called before a listNamespaces request has been processed.default voidCalled before list replication peers.default voidCalled before listSnapshots request has been processed.default voidpreLockHeartbeat(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tn, String description) Called before heartbeat to a lock.default voidCall before the master initialization is set to true.default voidCalled before the master local region memstore is flushed to disk.default voidpreMergeRegions(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge) Called before merge regions request.default voidpreMergeRegionsAction(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge) Called before the regions merge.default voidpreMergeRegionsCommitAction(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge, List<Mutation> metaEntries) This will be called before update META step as part of regions merge transaction.default StringpreModifyColumnFamilyStoreFileTracker(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, byte[] family, String dstSFT) Called prior to modifying a family's store file tracker.default voidpreModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor newNsDescriptor) Deprecated.Since 2.1.default voidpreModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor currentNsDescriptor, NamespaceDescriptor newNsDescriptor) Called prior to modifying a namespace's properties.default voidpreModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor newDescriptor) Deprecated.Since 2.1.default TableDescriptorpreModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor currentDescriptor, TableDescriptor newDescriptor) Called prior to modifying a table's properties.default voidpreModifyTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor newDescriptor) Deprecated.Since 2.1.default voidpreModifyTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor currentDescriptor, TableDescriptor newDescriptor) Called prior to modifying a table's properties.default StringpreModifyTableStoreFileTracker(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, String dstSFT) Called prior to modifying a table's store file tracker.default voidpreMove(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo region, ServerName srcServer, ServerName destServer) Called prior to moving a given region from one region server to another.default voidpreMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers, String targetGroup) Called before servers are moved to target region server groupdefault voidpreMoveServersAndTables(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers, Set<TableName> tables, String targetGroup) Called before servers are moved to target region server groupdefault voidpreMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<TableName> tables, String targetGroup) Called before tables are moved to target region server groupdefault voidpreRecommissionRegionServer(ObserverContext<MasterCoprocessorEnvironment> ctx, ServerName server, List<byte[]> encodedRegionNames) Called before recommission region server.default voidpreRegionOffline(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo) Called prior to marking a given region as offline.default voidCalled before remove a replication peerdefault voidCalled before a region server group is removeddefault voidpreRemoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers) Called before servers are removed from rsgroupdefault voidpreRenameRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String oldName, String newName) Called before rename rsgroup.default voidpreRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace, TableName tableName, RegionInfo[] regionInfos, String description) Called before new LockProcedure is queued.default voidpreRestoreSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) Called before a snapshot is restored.default voidpreRevoke(ObserverContext<MasterCoprocessorEnvironment> ctx, UserPermission userPermission) Called before revoking user permissions.default voidpreSetNamespaceQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace, GlobalQuotaSettings quotas) Called before the quota for the namespace is stored.default voidpreSetRegionServerQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String regionServer, GlobalQuotaSettings quotas) Called before the quota for the region server is stored.default voidpreSetSplitOrMergeEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean newValue, MasterSwitchType switchType) Called prior to setting split / merge switch Supports Coprocessor 'bypass'.default voidpreSetTableQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, GlobalQuotaSettings quotas) Called before the quota for the table is stored.default voidpreSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, String namespace, GlobalQuotaSettings quotas) Called before the quota for the user on the specified namespace is stored.default voidpreSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, GlobalQuotaSettings quotas) Called before the quota for the user is stored.default voidpreSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, TableName tableName, GlobalQuotaSettings quotas) Called before the quota for the user on the specified table is stored.default voidCalled prior to shutting down the full HBase cluster, including thisHMasterprocess.default voidpreSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) Called before a new snapshot is taken.default voidpreSplitRegion(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName, byte[] splitRow) Called before the split region procedure is called.default voidpreSplitRegionAction(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName, byte[] splitRow) Called before the region is split.default voidThis will be called after update META step as part of split transactiondefault voidpreSplitRegionBeforeMETAAction(ObserverContext<MasterCoprocessorEnvironment> ctx, byte[] splitKey, List<Mutation> metaEntries) This will be called before update META step as part of split transaction.default voidCalled immediately prior to stopping thisHMasterprocess.default voidpreSwitchExceedThrottleQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean enable) Called before switching exceed throttle quota state.default voidpreSwitchRpcThrottle(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean enable) Called before switching rpc throttle enabled state.default voidpreTableFlush(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) Called before the table memstore is flushed to disk.default voidpreTruncateRegion(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfo) Called before the truncate region procedure is called.default voidpreTruncateRegionAction(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfo) Called before the region is truncated.default voidpreTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) Called beforeHMastertruncates a table.default voidpreTruncateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) Called beforeHMastertruncates a table.default voidpreUnassign(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo) Called prior to unassigning a given region.default voidpreUnassign(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo, boolean force) Deprecated.in 2.4.0.default voidpreUpdateMasterConfiguration(ObserverContext<MasterCoprocessorEnvironment> ctx, org.apache.hadoop.conf.Configuration preReloadConf) default voidpreUpdateReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId, ReplicationPeerConfig peerConfig) Called before update peerConfig for the specified peerdefault voidpreUpdateRSGroupConfig(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName, Map<String, String> configuration) Called before update rsgroup config.
-
Method Details
-
preCreateTableRegionsInfos
default TableDescriptor preCreateTableRegionsInfos(ObserverContext<MasterCoprocessorEnvironment> ctx, TableDescriptor desc) throws IOException Called before we create the region infos for this table. Called as part of create table RPC call.- Parameters:
ctx- the environment to interact with the framework and masterdesc- the TableDescriptor for the table- Returns:
- the TableDescriptor used to create the table. Default is the one passed in. Return
nullmeans cancel the creation. - Throws:
IOException
-
preCreateTable
default void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableDescriptor desc, RegionInfo[] regions) throws IOException Called before a new table is created byHMaster. Called as part of create table RPC call.- Parameters:
ctx- the environment to interact with the framework and masterdesc- the TableDescriptor for the tableregions- the initial regions created for the table- Throws:
IOException
-
postCreateTable
default void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableDescriptor desc, RegionInfo[] regions) throws IOException Called after the createTable operation has been requested. Called as part of create table RPC call.- Parameters:
ctx- the environment to interact with the framework and masterdesc- the TableDescriptor for the tableregions- the initial regions created for the table- Throws:
IOException
-
preCreateTableAction
default void preCreateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableDescriptor desc, RegionInfo[] regions) throws IOException Called before a new table is created byHMaster. Called as part of create table procedure and it is async to the create RPC call.- Parameters:
ctx- the environment to interact with the framework and masterdesc- the TableDescriptor for the tableregions- the initial regions created for the table- Throws:
IOException
-
postCompletedCreateTableAction
default void postCompletedCreateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableDescriptor desc, RegionInfo[] regions) throws IOException Called after the createTable operation has been requested. Called as part of create table RPC call. Called as part of create table procedure and it is async to the create RPC call.- Parameters:
ctx- the environment to interact with the framework and masterdesc- the TableDescriptor for the tableregions- the initial regions created for the table- Throws:
IOException
-
preDeleteTable
default void preDeleteTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException Called beforeHMasterdeletes a table. Called as part of delete table RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the table- Throws:
IOException
-
postDeleteTable
default void postDeleteTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException Called after the deleteTable operation has been requested. Called as part of delete table RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the table- Throws:
IOException
-
preDeleteTableAction
default void preDeleteTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException Called beforeHMasterdeletes a table. Called as part of delete table procedure and it is async to the delete RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the table- Throws:
IOException
-
postCompletedDeleteTableAction
default void postCompletedDeleteTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException Called afterHMasterdeletes a table. Called as part of delete table procedure and it is async to the delete RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the table- Throws:
IOException
-
preTruncateTable
default void preTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException Called beforeHMastertruncates a table. Called as part of truncate table RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the table- Throws:
IOException
-
postTruncateTable
default void postTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException Called after the truncateTable operation has been requested. Called as part of truncate table RPC call. The truncate is synchronous, so this method will be called when the truncate operation is terminated.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the table- Throws:
IOException
-
preTruncateTableAction
default void preTruncateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException Called beforeHMastertruncates a table. Called as part of truncate table procedure and it is async to the truncate RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the table- Throws:
IOException
-
postCompletedTruncateTableAction
default void postCompletedTruncateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException Called afterHMastertruncates a table. Called as part of truncate table procedure and it is async to the truncate RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the table- Throws:
IOException
-
preModifyTable
@Deprecated default void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor newDescriptor) throws IOException Deprecated.Since 2.1. Will be removed in 3.0.Called prior to modifying a table's properties. Called as part of modify table RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the tablenewDescriptor- after modify operation, table will have this descriptor- Throws:
IOException
-
preModifyTable
default TableDescriptor preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor currentDescriptor, TableDescriptor newDescriptor) throws IOException Called prior to modifying a table's properties. Called as part of modify table RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the tablecurrentDescriptor- current TableDescriptor of the tablenewDescriptor- after modify operation, table will have this descriptor- Throws:
IOException
-
postModifyTable
@Deprecated default void postModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor currentDescriptor) throws IOException Deprecated.Since 2.1. Will be removed in 3.0.Called after the modifyTable operation has been requested. Called as part of modify table RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the tablecurrentDescriptor- current TableDescriptor of the table- Throws:
IOException
-
postModifyTable
default void postModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor oldDescriptor, TableDescriptor currentDescriptor) throws IOException Called after the modifyTable operation has been requested. Called as part of modify table RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the tableoldDescriptor- descriptor of table before modify operation happenedcurrentDescriptor- current TableDescriptor of the table- Throws:
IOException
-
preModifyTableAction
@Deprecated default void preModifyTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor newDescriptor) throws IOException Deprecated.Since 2.1. Will be removed in 3.0.Called prior to modifying a table's properties. Called as part of modify table procedure and it is async to the modify table RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the tablenewDescriptor- after modify operation, table will have this descriptor- Throws:
IOException
-
preModifyTableStoreFileTracker
default String preModifyTableStoreFileTracker(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, String dstSFT) throws IOException Called prior to modifying a table's store file tracker. Called as part of modify table store file tracker RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the tabledstSFT- the store file tracker- Returns:
- the store file tracker
- Throws:
IOException
-
postModifyTableStoreFileTracker
default void postModifyTableStoreFileTracker(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, String dstSFT) throws IOException Called after modifying a table's store file tracker. Called as part of modify table store file tracker RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the tabledstSFT- the store file tracker- Throws:
IOException
-
preModifyColumnFamilyStoreFileTracker
default String preModifyColumnFamilyStoreFileTracker(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, byte[] family, String dstSFT) throws IOException Called prior to modifying a family's store file tracker. Called as part of modify family store file tracker RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the tablefamily- the column familydstSFT- the store file tracker- Returns:
- the store file tracker
- Throws:
IOException
-
postModifyColumnFamilyStoreFileTracker
default void postModifyColumnFamilyStoreFileTracker(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, byte[] family, String dstSFT) throws IOException Called after modifying a family store file tracker. Called as part of modify family store file tracker RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the tablefamily- the column familydstSFT- the store file tracker- Throws:
IOException
-
preModifyTableAction
default void preModifyTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor currentDescriptor, TableDescriptor newDescriptor) throws IOException Called prior to modifying a table's properties. Called as part of modify table procedure and it is async to the modify table RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the tablecurrentDescriptor- current TableDescriptor of the tablenewDescriptor- after modify operation, table will have this descriptor- Throws:
IOException
-
postCompletedModifyTableAction
@Deprecated default void postCompletedModifyTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor currentDescriptor) throws IOException Deprecated.Since 2.1. Will be removed in 3.0.Called after to modifying a table's properties. Called as part of modify table procedure and it is async to the modify table RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the tablecurrentDescriptor- current TableDescriptor of the table- Throws:
IOException
-
postCompletedModifyTableAction
default void postCompletedModifyTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor oldDescriptor, TableDescriptor currentDescriptor) throws IOException Called after to modifying a table's properties. Called as part of modify table procedure and it is async to the modify table RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the tableoldDescriptor- descriptor of table before modify operation happenedcurrentDescriptor- current TableDescriptor of the table- Throws:
IOException
-
preEnableTable
default void preEnableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException Called prior to enabling a table. Called as part of enable table RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the table- Throws:
IOException
-
postEnableTable
default void postEnableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException Called after the enableTable operation has been requested. Called as part of enable table RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the table- Throws:
IOException
-
preEnableTableAction
default void preEnableTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException Called prior to enabling a table. Called as part of enable table procedure and it is async to the enable table RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the table- Throws:
IOException
-
postCompletedEnableTableAction
default void postCompletedEnableTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException Called after the enableTable operation has been requested. Called as part of enable table procedure and it is async to the enable table RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the table- Throws:
IOException
-
preDisableTable
default void preDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException Called prior to disabling a table. Called as part of disable table RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the table- Throws:
IOException
-
postDisableTable
default void postDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException Called after the disableTable operation has been requested. Called as part of disable table RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the table- Throws:
IOException
-
preDisableTableAction
default void preDisableTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException Called prior to disabling a table. Called as part of disable table procedure and it is asyn to the disable table RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the table- Throws:
IOException
-
postCompletedDisableTableAction
default void postCompletedDisableTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException Called after the disableTable operation has been requested. Called as part of disable table procedure and it is asyn to the disable table RPC call.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the table- Throws:
IOException
-
preAbortProcedure
default void preAbortProcedure(ObserverContext<MasterCoprocessorEnvironment> ctx, long procId) throws IOException Called before a abortProcedure request has been processed.- Parameters:
ctx- the environment to interact with the framework and masterprocId- the Id of the procedure- Throws:
IOException
-
postAbortProcedure
default void postAbortProcedure(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException Called after a abortProcedure request has been processed.- Parameters:
ctx- the environment to interact with the framework and master- Throws:
IOException
-
preGetProcedures
Called before a getProcedures request has been processed.- Parameters:
ctx- the environment to interact with the framework and master- Throws:
IOException
-
postGetProcedures
default void postGetProcedures(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException Called after a getProcedures request has been processed.- Parameters:
ctx- the environment to interact with the framework and master- Throws:
IOException
-
preGetLocks
Called before a getLocks request has been processed.- Parameters:
ctx- the environment to interact with the framework and master- Throws:
IOException- if something went wrong
-
postGetLocks
Called after a getLocks request has been processed.- Parameters:
ctx- the environment to interact with the framework and master- Throws:
IOException- if something went wrong
-
preMove
default void preMove(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo region, ServerName srcServer, ServerName destServer) throws IOException Called prior to moving a given region from one region server to another.- Parameters:
ctx- the environment to interact with the framework and masterregion- the RegionInfosrcServer- the source ServerNamedestServer- the destination ServerName- Throws:
IOException
-
postMove
default void postMove(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo region, ServerName srcServer, ServerName destServer) throws IOException Called after the region move has been requested.- Parameters:
ctx- the environment to interact with the framework and masterregion- the RegionInfosrcServer- the source ServerNamedestServer- the destination ServerName- Throws:
IOException
-
preAssign
default void preAssign(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo) throws IOException Called prior to assigning a specific region.- Parameters:
ctx- the environment to interact with the framework and masterregionInfo- the regionInfo of the region- Throws:
IOException
-
postAssign
default void postAssign(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo) throws IOException Called after the region assignment has been requested.- Parameters:
ctx- the environment to interact with the framework and masterregionInfo- the regionInfo of the region- Throws:
IOException
-
preUnassign
default void preUnassign(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo, boolean force) throws IOException Deprecated.in 2.4.0. replaced by preUnassign(ctx, regionInfo). removed in hbase 3. until then safe to either leave implementation here or move it to the new method. default impl of that method calls this one.Called prior to unassigning a given region.- Parameters:
ctx- the environment to interact with the framework and masterforce- whether to force unassignment or not- Throws:
IOException
-
preUnassign
default void preUnassign(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo) throws IOException Called prior to unassigning a given region.- Parameters:
ctx- the environment to interact with the framework and master- Throws:
IOException
-
postUnassign
default void postUnassign(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo, boolean force) throws IOException Deprecated.in 2.4.0. replaced by postUnassign(ctx, regionInfo). removed in hbase 3. until then safe to either leave implementation here or move it to the new method. default impl of that method calls this one.Called after the region unassignment has been requested.- Parameters:
ctx- the environment to interact with the framework and masterforce- whether to force unassignment or not- Throws:
IOException
-
postUnassign
default void postUnassign(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo) throws IOException Called after the region unassignment has been requested.- Parameters:
ctx- the environment to interact with the framework and master- Throws:
IOException
-
preRegionOffline
default void preRegionOffline(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo) throws IOException Called prior to marking a given region as offline.- Parameters:
ctx- the environment to interact with the framework and master- Throws:
IOException
-
postRegionOffline
default void postRegionOffline(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo) throws IOException Called after the region has been marked offline.- Parameters:
ctx- the environment to interact with the framework and master- Throws:
IOException
-
preBalance
default void preBalance(ObserverContext<MasterCoprocessorEnvironment> ctx, BalanceRequest request) throws IOException Called prior to requesting rebalancing of the cluster regions, though after the initial checks for regions in transition and the balance switch flag.- Parameters:
ctx- the environment to interact with the framework and masterrequest- the request used to trigger the balancer- Throws:
IOException
-
postBalance
default void postBalance(ObserverContext<MasterCoprocessorEnvironment> ctx, BalanceRequest request, List<RegionPlan> plans) throws IOException Called after the balancing plan has been submitted.- Parameters:
ctx- the environment to interact with the framework and masterrequest- the request used to trigger the balanceplans- the RegionPlans which master has executed. RegionPlan serves as hint as for the final destination for the underlying region but may not represent the final state of assignment- Throws:
IOException
-
preSetSplitOrMergeEnabled
default void preSetSplitOrMergeEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean newValue, MasterSwitchType switchType) throws IOException Called prior to setting split / merge switch Supports Coprocessor 'bypass'.- Parameters:
ctx- the coprocessor instance's environmentnewValue- the new value submitted in the callswitchType- type of switch- Throws:
IOException
-
postSetSplitOrMergeEnabled
default void postSetSplitOrMergeEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean newValue, MasterSwitchType switchType) throws IOException Called after setting split / merge switch- Parameters:
ctx- the coprocessor instance's environmentnewValue- the new value submitted in the callswitchType- type of switch- Throws:
IOException
-
preSplitRegion
default void preSplitRegion(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName, byte[] splitRow) throws IOException Called before the split region procedure is called.- Parameters:
c- the environment to interact with the framework and mastertableName- the table where the region belongs tosplitRow- split point- Throws:
IOException
-
preSplitRegionAction
default void preSplitRegionAction(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName, byte[] splitRow) throws IOException Called before the region is split.- Parameters:
c- the environment to interact with the framework and mastertableName- the table where the region belongs tosplitRow- split point- Throws:
IOException
-
preTruncateRegionAction
default void preTruncateRegionAction(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfo) Called before the region is truncated.- Parameters:
c- The environment to interact with the framework and masterregionInfo- The Region being truncated
-
preTruncateRegion
default void preTruncateRegion(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfo) Called before the truncate region procedure is called.- Parameters:
c- The environment to interact with the framework and masterregionInfo- The Region being truncated
-
postTruncateRegion
default void postTruncateRegion(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfo) Called after the truncate region procedure is called.- Parameters:
c- The environment to interact with the framework and masterregionInfo- The Region being truncated
-
postTruncateRegionAction
default void postTruncateRegionAction(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfo) Called post the region is truncated.- Parameters:
c- The environment to interact with the framework and masterregionInfo- The Region To be truncated
-
postCompletedSplitRegionAction
default void postCompletedSplitRegionAction(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfoA, RegionInfo regionInfoB) throws IOException Called after the region is split.- Parameters:
c- the environment to interact with the framework and masterregionInfoA- the left daughter regionregionInfoB- the right daughter region- Throws:
IOException
-
preSplitRegionBeforeMETAAction
default void preSplitRegionBeforeMETAAction(ObserverContext<MasterCoprocessorEnvironment> ctx, byte[] splitKey, List<Mutation> metaEntries) throws IOException This will be called before update META step as part of split transaction.- Parameters:
ctx- the environment to interact with the framework and master- Throws:
IOException
-
preSplitRegionAfterMETAAction
default void preSplitRegionAfterMETAAction(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException This will be called after update META step as part of split transaction- Parameters:
ctx- the environment to interact with the framework and master- Throws:
IOException
-
postRollBackSplitRegionAction
default void postRollBackSplitRegionAction(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException This will be called after the roll back of the split region is completed- Parameters:
ctx- the environment to interact with the framework and master- Throws:
IOException
-
preMergeRegionsAction
default void preMergeRegionsAction(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge) throws IOException Called before the regions merge.- Parameters:
ctx- the environment to interact with the framework and master- Throws:
IOException
-
postCompletedMergeRegionsAction
default void postCompletedMergeRegionsAction(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge, RegionInfo mergedRegion) throws IOException called after the regions merge.- Parameters:
ctx- the environment to interact with the framework and master- Throws:
IOException
-
preMergeRegionsCommitAction
default void preMergeRegionsCommitAction(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge, @MetaMutationAnnotation List<Mutation> metaEntries) throws IOException This will be called before update META step as part of regions merge transaction.- Parameters:
ctx- the environment to interact with the framework and mastermetaEntries- mutations to execute on hbase:meta atomically with regions merge updates. Any puts or deletes to execute on hbase:meta can be added to the mutations.- Throws:
IOException
-
postMergeRegionsCommitAction
default void postMergeRegionsCommitAction(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge, RegionInfo mergedRegion) throws IOException This will be called after META step as part of regions merge transaction.- Parameters:
ctx- the environment to interact with the framework and master- Throws:
IOException
-
postRollBackMergeRegionsAction
default void postRollBackMergeRegionsAction(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge) throws IOException This will be called after the roll back of the regions merge.- Parameters:
ctx- the environment to interact with the framework and master- Throws:
IOException
-
preBalanceSwitch
default void preBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean newValue) throws IOException Called prior to modifying the flag used to enable/disable region balancing.- Parameters:
ctx- the coprocessor instance's environment- Throws:
IOException
-
postBalanceSwitch
default void postBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean oldValue, boolean newValue) throws IOException Called after the flag to enable/disable balancing has changed.- Parameters:
ctx- the coprocessor instance's environmentoldValue- the previously set balanceSwitch valuenewValue- the newly set balanceSwitch value- Throws:
IOException
-
preShutdown
Called prior to shutting down the full HBase cluster, including thisHMasterprocess.- Throws:
IOException
-
preStopMaster
Called immediately prior to stopping thisHMasterprocess.- Throws:
IOException
-
postStartMaster
Called immediately after an active master instance has completed initialization. Will not be called on standby master instances unless they take over the active role.- Throws:
IOException
-
preMasterInitialization
default void preMasterInitialization(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException Call before the master initialization is set to true.HMasterprocess.- Throws:
IOException
-
preSnapshot
default void preSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) throws IOException Called before a new snapshot is taken. Called as part of snapshot RPC call.- Parameters:
ctx- the environment to interact with the framework and mastersnapshot- the SnapshotDescriptor for the snapshottableDescriptor- the TableDescriptor of the table to snapshot- Throws:
IOException
-
postSnapshot
default void postSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) throws IOException Called after the snapshot operation has been requested. Called as part of snapshot RPC call.- Parameters:
ctx- the environment to interact with the framework and mastersnapshot- the SnapshotDescriptor for the snapshottableDescriptor- the TableDescriptor of the table to snapshot- Throws:
IOException
-
postCompletedSnapshotAction
default void postCompletedSnapshotAction(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) throws IOException Called after the snapshot operation has been completed.- Parameters:
ctx- the environment to interact with the framework and mastersnapshot- the SnapshotDescriptor for the snapshottableDescriptor- the TableDescriptor of the table to snapshot- Throws:
IOException
-
preListSnapshot
default void preListSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot) throws IOException Called before listSnapshots request has been processed.- Parameters:
ctx- the environment to interact with the framework and mastersnapshot- the SnapshotDescriptor of the snapshot to list- Throws:
IOException
-
postListSnapshot
default void postListSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot) throws IOException Called after listSnapshots request has been processed.- Parameters:
ctx- the environment to interact with the framework and mastersnapshot- the SnapshotDescriptor of the snapshot to list- Throws:
IOException
-
preCloneSnapshot
default void preCloneSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) throws IOException Called before a snapshot is cloned. Called as part of restoreSnapshot RPC call.- Parameters:
ctx- the environment to interact with the framework and mastersnapshot- the SnapshotDescriptor for the snapshottableDescriptor- the TableDescriptor of the table to create- Throws:
IOException
-
postCloneSnapshot
default void postCloneSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) throws IOException Called after a snapshot clone operation has been requested. Called as part of restoreSnapshot RPC call.- Parameters:
ctx- the environment to interact with the framework and mastersnapshot- the SnapshotDescriptor for the snapshottableDescriptor- the v of the table to create- Throws:
IOException
-
preRestoreSnapshot
default void preRestoreSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) throws IOException Called before a snapshot is restored. Called as part of restoreSnapshot RPC call.- Parameters:
ctx- the environment to interact with the framework and mastersnapshot- the SnapshotDescriptor for the snapshottableDescriptor- the TableDescriptor of the table to restore- Throws:
IOException
-
postRestoreSnapshot
default void postRestoreSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) throws IOException Called after a snapshot restore operation has been requested. Called as part of restoreSnapshot RPC call.- Parameters:
ctx- the environment to interact with the framework and mastersnapshot- the SnapshotDescriptor for the snapshottableDescriptor- the TableDescriptor of the table to restore- Throws:
IOException
-
preDeleteSnapshot
default void preDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot) throws IOException Called before a snapshot is deleted. Called as part of deleteSnapshot RPC call.- Parameters:
ctx- the environment to interact with the framework and mastersnapshot- the SnapshotDescriptor of the snapshot to delete- Throws:
IOException
-
postDeleteSnapshot
default void postDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot) throws IOException Called after the delete snapshot operation has been requested. Called as part of deleteSnapshot RPC call.- Parameters:
ctx- the environment to interact with the framework and mastersnapshot- the SnapshotDescriptor of the snapshot to delete- Throws:
IOException
-
preGetTableDescriptors
default void preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableName> tableNamesList, List<TableDescriptor> descriptors, String regex) throws IOException Called before a getTableDescriptors request has been processed.- Parameters:
ctx- the environment to interact with the framework and mastertableNamesList- the list of table names, or null if querying for alldescriptors- an empty list, can be filled with what to return in coprocessorregex- regular expression used for filtering the table names- Throws:
IOException
-
postGetTableDescriptors
default void postGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableName> tableNamesList, List<TableDescriptor> descriptors, String regex) throws IOException Called after a getTableDescriptors request has been processed.- Parameters:
ctx- the environment to interact with the framework and mastertableNamesList- the list of table names, or null if querying for alldescriptors- the list of descriptors about to be returnedregex- regular expression used for filtering the table names- Throws:
IOException
-
preGetTableNames
default void preGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableDescriptor> descriptors, String regex) throws IOException Called before a getTableNames request has been processed.- Parameters:
ctx- the environment to interact with the framework and masterdescriptors- an empty list, can be filled with what to return by coprocessorregex- regular expression used for filtering the table names- Throws:
IOException
-
postGetTableNames
default void postGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableDescriptor> descriptors, String regex) throws IOException Called after a getTableNames request has been processed.- Parameters:
ctx- the environment to interact with the framework and masterdescriptors- the list of descriptors about to be returnedregex- regular expression used for filtering the table names- Throws:
IOException
-
preCreateNamespace
default void preCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor ns) throws IOException Called before a new namespace is created byHMaster.- Parameters:
ctx- the environment to interact with the framework and masterns- the NamespaceDescriptor for the table- Throws:
IOException
-
postCreateNamespace
default void postCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor ns) throws IOException Called after the createNamespace operation has been requested.- Parameters:
ctx- the environment to interact with the framework and masterns- the NamespaceDescriptor for the table- Throws:
IOException
-
preDeleteNamespace
default void preDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) throws IOException Called beforeHMasterdeletes a namespace- Parameters:
ctx- the environment to interact with the framework and masternamespace- the name of the namespace- Throws:
IOException
-
postDeleteNamespace
default void postDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) throws IOException Called after the deleteNamespace operation has been requested.- Parameters:
ctx- the environment to interact with the framework and masternamespace- the name of the namespace- Throws:
IOException
-
preModifyNamespace
@Deprecated default void preModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor newNsDescriptor) throws IOException Deprecated.Since 2.1. Will be removed in 3.0.Called prior to modifying a namespace's properties.- Parameters:
ctx- the environment to interact with the framework and masternewNsDescriptor- after modify operation, namespace will have this descriptor- Throws:
IOException
-
preModifyNamespace
default void preModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor currentNsDescriptor, NamespaceDescriptor newNsDescriptor) throws IOException Called prior to modifying a namespace's properties.- Parameters:
ctx- the environment to interact with the framework and mastercurrentNsDescriptor- current NamespaceDescriptor of the namespacenewNsDescriptor- after modify operation, namespace will have this descriptor- Throws:
IOException
-
postModifyNamespace
@Deprecated default void postModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor currentNsDescriptor) throws IOException Deprecated.Since 2.1. Will be removed in 3.0.Called after the modifyNamespace operation has been requested.- Parameters:
ctx- the environment to interact with the framework and mastercurrentNsDescriptor- current NamespaceDescriptor of the namespace- Throws:
IOException
-
postModifyNamespace
default void postModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor oldNsDescriptor, NamespaceDescriptor currentNsDescriptor) throws IOException Called after the modifyNamespace operation has been requested.- Parameters:
ctx- the environment to interact with the framework and masteroldNsDescriptor- descriptor of namespace before modify operation happenedcurrentNsDescriptor- current NamespaceDescriptor of the namespace- Throws:
IOException
-
preGetNamespaceDescriptor
default void preGetNamespaceDescriptor(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) throws IOException Called before a getNamespaceDescriptor request has been processed.- Parameters:
ctx- the environment to interact with the framework and masternamespace- the name of the namespace- Throws:
IOException
-
postGetNamespaceDescriptor
default void postGetNamespaceDescriptor(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor ns) throws IOException Called after a getNamespaceDescriptor request has been processed.- Parameters:
ctx- the environment to interact with the framework and masterns- the NamespaceDescriptor- Throws:
IOException
-
preListNamespaces
default void preListNamespaces(ObserverContext<MasterCoprocessorEnvironment> ctx, List<String> namespaces) throws IOException Called before a listNamespaces request has been processed.- Parameters:
ctx- the environment to interact with the framework and masternamespaces- an empty list, can be filled with what to return if bypassing- Throws:
IOException- if something went wrong
-
postListNamespaces
default void postListNamespaces(ObserverContext<MasterCoprocessorEnvironment> ctx, List<String> namespaces) throws IOException Called after a listNamespaces request has been processed.- Parameters:
ctx- the environment to interact with the framework and masternamespaces- the list of namespaces about to be returned- Throws:
IOException- if something went wrong
-
preListNamespaceDescriptors
default void preListNamespaceDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<NamespaceDescriptor> descriptors) throws IOException Called before a listNamespaceDescriptors request has been processed.- Parameters:
ctx- the environment to interact with the framework and masterdescriptors- an empty list, can be filled with what to return by coprocessor- Throws:
IOException
-
postListNamespaceDescriptors
default void postListNamespaceDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<NamespaceDescriptor> descriptors) throws IOException Called after a listNamespaceDescriptors request has been processed.- Parameters:
ctx- the environment to interact with the framework and masterdescriptors- the list of descriptors about to be returned- Throws:
IOException
-
preTableFlush
default void preTableFlush(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException Called before the table memstore is flushed to disk.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the table- Throws:
IOException
-
postTableFlush
default void postTableFlush(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException Called after the table memstore is flushed to disk.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the table- Throws:
IOException
-
preMasterStoreFlush
default void preMasterStoreFlush(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException Called before the master local region memstore is flushed to disk.- Parameters:
ctx- the environment to interact with the framework and master- Throws:
IOException
-
postMasterStoreFlush
default void postMasterStoreFlush(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException Called after the master local region memstore is flushed to disk.- Parameters:
ctx- the environment to interact with the framework and master- Throws:
IOException
-
preSetUserQuota
default void preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, GlobalQuotaSettings quotas) throws IOException Called before the quota for the user is stored.- Parameters:
ctx- the environment to interact with the framework and masteruserName- the name of userquotas- the current quota for the user- Throws:
IOException
-
postSetUserQuota
default void postSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, GlobalQuotaSettings quotas) throws IOException Called after the quota for the user is stored.- Parameters:
ctx- the environment to interact with the framework and masteruserName- the name of userquotas- the resulting quota for the user- Throws:
IOException
-
preSetUserQuota
default void preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, TableName tableName, GlobalQuotaSettings quotas) throws IOException Called before the quota for the user on the specified table is stored.- Parameters:
ctx- the environment to interact with the framework and masteruserName- the name of usertableName- the name of the tablequotas- the current quota for the user on the table- Throws:
IOException
-
postSetUserQuota
default void postSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, TableName tableName, GlobalQuotaSettings quotas) throws IOException Called after the quota for the user on the specified table is stored.- Parameters:
ctx- the environment to interact with the framework and masteruserName- the name of usertableName- the name of the tablequotas- the resulting quota for the user on the table- Throws:
IOException
-
preSetUserQuota
default void preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, String namespace, GlobalQuotaSettings quotas) throws IOException Called before the quota for the user on the specified namespace is stored.- Parameters:
ctx- the environment to interact with the framework and masteruserName- the name of usernamespace- the name of the namespacequotas- the current quota for the user on the namespace- Throws:
IOException
-
postSetUserQuota
default void postSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, String namespace, GlobalQuotaSettings quotas) throws IOException Called after the quota for the user on the specified namespace is stored.- Parameters:
ctx- the environment to interact with the framework and masteruserName- the name of usernamespace- the name of the namespacequotas- the resulting quota for the user on the namespace- Throws:
IOException
-
preSetTableQuota
default void preSetTableQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, GlobalQuotaSettings quotas) throws IOException Called before the quota for the table is stored.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the tablequotas- the current quota for the table- Throws:
IOException
-
postSetTableQuota
default void postSetTableQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, GlobalQuotaSettings quotas) throws IOException Called after the quota for the table is stored.- Parameters:
ctx- the environment to interact with the framework and mastertableName- the name of the tablequotas- the resulting quota for the table- Throws:
IOException
-
preSetNamespaceQuota
default void preSetNamespaceQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace, GlobalQuotaSettings quotas) throws IOException Called before the quota for the namespace is stored.- Parameters:
ctx- the environment to interact with the framework and masternamespace- the name of the namespacequotas- the current quota for the namespace- Throws:
IOException
-
postSetNamespaceQuota
default void postSetNamespaceQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace, GlobalQuotaSettings quotas) throws IOException Called after the quota for the namespace is stored.- Parameters:
ctx- the environment to interact with the framework and masternamespace- the name of the namespacequotas- the resulting quota for the namespace- Throws:
IOException
-
preSetRegionServerQuota
default void preSetRegionServerQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String regionServer, GlobalQuotaSettings quotas) throws IOException Called before the quota for the region server is stored.- Parameters:
ctx- the environment to interact with the framework and masterregionServer- the name of the region serverquotas- the current quota for the region server- Throws:
IOException
-
postSetRegionServerQuota
default void postSetRegionServerQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String regionServer, GlobalQuotaSettings quotas) throws IOException Called after the quota for the region server is stored.- Parameters:
ctx- the environment to interact with the framework and masterregionServer- the name of the region serverquotas- the resulting quota for the region server- Throws:
IOException
-
preMergeRegions
default void preMergeRegions(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge) throws IOException Called before merge regions request.- Parameters:
ctx- coprocessor environmentregionsToMerge- regions to be merged- Throws:
IOException
-
postMergeRegions
default void postMergeRegions(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo[] regionsToMerge) throws IOException called after merge regions request.- Parameters:
c- coprocessor environmentregionsToMerge- regions to be merged- Throws:
IOException
-
preMoveServersAndTables
default void preMoveServersAndTables(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers, Set<TableName> tables, String targetGroup) throws IOException Called before servers are moved to target region server group- Parameters:
ctx- the environment to interact with the framework and masterservers- set of servers to movetargetGroup- destination group- Throws:
IOException
-
postMoveServersAndTables
default void postMoveServersAndTables(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers, Set<TableName> tables, String targetGroup) throws IOException Called after servers are moved to target region server group- Parameters:
ctx- the environment to interact with the framework and masterservers- set of servers to movetargetGroup- name of group- Throws:
IOException
-
preMoveServers
default void preMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers, String targetGroup) throws IOException Called before servers are moved to target region server group- Parameters:
ctx- the environment to interact with the framework and masterservers- set of servers to movetargetGroup- destination group- Throws:
IOException
-
postMoveServers
default void postMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers, String targetGroup) throws IOException Called after servers are moved to target region server group- Parameters:
ctx- the environment to interact with the framework and masterservers- set of servers to movetargetGroup- name of group- Throws:
IOException
-
preMoveTables
default void preMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<TableName> tables, String targetGroup) throws IOException Called before tables are moved to target region server group- Parameters:
ctx- the environment to interact with the framework and mastertables- set of tables to movetargetGroup- name of group- Throws:
IOException
-
postMoveTables
default void postMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<TableName> tables, String targetGroup) throws IOException Called after servers are moved to target region server group- Parameters:
ctx- the environment to interact with the framework and mastertables- set of tables to movetargetGroup- name of group- Throws:
IOException
-
preAddRSGroup
default void preAddRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String name) throws IOException Called before a new region server group is added- Parameters:
ctx- the environment to interact with the framework and mastername- group name- Throws:
IOException
-
postAddRSGroup
default void postAddRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String name) throws IOException Called after a new region server group is added- Parameters:
ctx- the environment to interact with the framework and mastername- group name- Throws:
IOException
-
preRemoveRSGroup
default void preRemoveRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String name) throws IOException Called before a region server group is removed- Parameters:
ctx- the environment to interact with the framework and mastername- group name- Throws:
IOException
-
postRemoveRSGroup
default void postRemoveRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String name) throws IOException Called after a region server group is removed- Parameters:
ctx- the environment to interact with the framework and mastername- group name- Throws:
IOException
-
preBalanceRSGroup
default void preBalanceRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName, BalanceRequest request) throws IOException Called before a region server group is removed- Parameters:
ctx- the environment to interact with the framework and mastergroupName- group name- Throws:
IOException
-
postBalanceRSGroup
default void postBalanceRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName, BalanceRequest request, BalanceResponse response) throws IOException Called after a region server group is removed- Parameters:
ctx- the environment to interact with the framework and mastergroupName- group namerequest- the request sent to the balancerresponse- the response returned by the balancer- Throws:
IOException
-
preRemoveServers
default void preRemoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers) throws IOException Called before servers are removed from rsgroup- Parameters:
ctx- the environment to interact with the framework and masterservers- set of decommissioned servers to remove- Throws:
IOException
-
postRemoveServers
default void postRemoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers) throws IOException Called after servers are removed from rsgroup- Parameters:
ctx- the environment to interact with the framework and masterservers- set of servers to remove- Throws:
IOException
-
preAddReplicationPeer
default void preAddReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId, ReplicationPeerConfig peerConfig) throws IOException Called before add a replication peer- Parameters:
ctx- the environment to interact with the framework and masterpeerId- a short name that identifies the peerpeerConfig- configuration for the replication peer- Throws:
IOException
-
postAddReplicationPeer
default void postAddReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId, ReplicationPeerConfig peerConfig) throws IOException Called after add a replication peer- Parameters:
ctx- the environment to interact with the framework and masterpeerId- a short name that identifies the peerpeerConfig- configuration for the replication peer- Throws:
IOException
-
preRemoveReplicationPeer
default void preRemoveReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException Called before remove a replication peer- Parameters:
peerId- a short name that identifies the peer- Throws:
IOException
-
postRemoveReplicationPeer
default void postRemoveReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException Called after remove a replication peer- Parameters:
peerId- a short name that identifies the peer- Throws:
IOException
-
preEnableReplicationPeer
default void preEnableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException Called before enable a replication peer- Parameters:
peerId- a short name that identifies the peer- Throws:
IOException
-
postEnableReplicationPeer
default void postEnableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException Called after enable a replication peer- Parameters:
peerId- a short name that identifies the peer- Throws:
IOException
-
preDisableReplicationPeer
default void preDisableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException Called before disable a replication peer- Parameters:
peerId- a short name that identifies the peer- Throws:
IOException
-
postDisableReplicationPeer
default void postDisableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException Called after disable a replication peer- Parameters:
peerId- a short name that identifies the peer- Throws:
IOException
-
preGetReplicationPeerConfig
default void preGetReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException Called before get the configured ReplicationPeerConfig for the specified peer- Parameters:
peerId- a short name that identifies the peer- Throws:
IOException
-
postGetReplicationPeerConfig
default void postGetReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException Called after get the configured ReplicationPeerConfig for the specified peer- Parameters:
peerId- a short name that identifies the peer- Throws:
IOException
-
preUpdateReplicationPeerConfig
default void preUpdateReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId, ReplicationPeerConfig peerConfig) throws IOException Called before update peerConfig for the specified peer- Parameters:
peerId- a short name that identifies the peer- Throws:
IOException
-
postUpdateReplicationPeerConfig
default void postUpdateReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId, ReplicationPeerConfig peerConfig) throws IOException Called after update peerConfig for the specified peer- Parameters:
ctx- the environment to interact with the framework and masterpeerId- a short name that identifies the peer- Throws:
IOException
-
preListReplicationPeers
default void preListReplicationPeers(ObserverContext<MasterCoprocessorEnvironment> ctx, String regex) throws IOException Called before list replication peers.- Parameters:
ctx- the environment to interact with the framework and masterregex- The regular expression to match peer id- Throws:
IOException
-
postListReplicationPeers
default void postListReplicationPeers(ObserverContext<MasterCoprocessorEnvironment> ctx, String regex) throws IOException Called after list replication peers.- Parameters:
ctx- the environment to interact with the framework and masterregex- The regular expression to match peer id- Throws:
IOException
-
preRequestLock
default void preRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace, TableName tableName, RegionInfo[] regionInfos, String description) throws IOException Called before new LockProcedure is queued.- Parameters:
ctx- the environment to interact with the framework and master- Throws:
IOException
-
postRequestLock
default void postRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace, TableName tableName, RegionInfo[] regionInfos, String description) throws IOException Called after new LockProcedure is queued.- Parameters:
ctx- the environment to interact with the framework and master- Throws:
IOException
-
preLockHeartbeat
default void preLockHeartbeat(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tn, String description) throws IOException Called before heartbeat to a lock.- Parameters:
ctx- the environment to interact with the framework and master- Throws:
IOException
-
postLockHeartbeat
default void postLockHeartbeat(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException Called after heartbeat to a lock.- Parameters:
ctx- the environment to interact with the framework and master- Throws:
IOException
-
preGetClusterMetrics
default void preGetClusterMetrics(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException Called before get cluster status.- Throws:
IOException
-
postGetClusterMetrics
default void postGetClusterMetrics(ObserverContext<MasterCoprocessorEnvironment> ctx, ClusterMetrics status) throws IOException Called after get cluster status.- Throws:
IOException
-
preClearDeadServers
default void preClearDeadServers(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException Called before clear dead region servers.- Throws:
IOException
-
postClearDeadServers
default void postClearDeadServers(ObserverContext<MasterCoprocessorEnvironment> ctx, List<ServerName> servers, List<ServerName> notClearedServers) throws IOException Called after clear dead region servers.- Throws:
IOException
-
preDecommissionRegionServers
default void preDecommissionRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx, List<ServerName> servers, boolean offload) throws IOException Called before decommission region servers.- Throws:
IOException
-
postDecommissionRegionServers
default void postDecommissionRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx, List<ServerName> servers, boolean offload) throws IOException Called after decommission region servers.- Throws:
IOException
-
preListDecommissionedRegionServers
default void preListDecommissionedRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException Called before list decommissioned region servers.- Throws:
IOException
-
postListDecommissionedRegionServers
default void postListDecommissionedRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException Called after list decommissioned region servers.- Throws:
IOException
-
preRecommissionRegionServer
default void preRecommissionRegionServer(ObserverContext<MasterCoprocessorEnvironment> ctx, ServerName server, List<byte[]> encodedRegionNames) throws IOException Called before recommission region server.- Throws:
IOException
-
postRecommissionRegionServer
default void postRecommissionRegionServer(ObserverContext<MasterCoprocessorEnvironment> ctx, ServerName server, List<byte[]> encodedRegionNames) throws IOException Called after recommission region server.- Throws:
IOException
-
preSwitchRpcThrottle
default void preSwitchRpcThrottle(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean enable) throws IOException Called before switching rpc throttle enabled state.- Parameters:
ctx- the coprocessor instance's environmentenable- the rpc throttle value- Throws:
IOException
-
postSwitchRpcThrottle
default void postSwitchRpcThrottle(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean oldValue, boolean newValue) throws IOException Called after switching rpc throttle enabled state.- Parameters:
ctx- the coprocessor instance's environmentoldValue- the previously rpc throttle valuenewValue- the newly rpc throttle value- Throws:
IOException
-
preIsRpcThrottleEnabled
default void preIsRpcThrottleEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException Called before getting if is rpc throttle enabled.- Parameters:
ctx- the coprocessor instance's environment- Throws:
IOException
-
postIsRpcThrottleEnabled
default void postIsRpcThrottleEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean rpcThrottleEnabled) throws IOException Called after getting if is rpc throttle enabled.- Parameters:
ctx- the coprocessor instance's environmentrpcThrottleEnabled- the rpc throttle enabled value- Throws:
IOException
-
preSwitchExceedThrottleQuota
default void preSwitchExceedThrottleQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean enable) throws IOException Called before switching exceed throttle quota state.- Parameters:
ctx- the coprocessor instance's environmentenable- the exceed throttle quota value- Throws:
IOException
-
postSwitchExceedThrottleQuota
default void postSwitchExceedThrottleQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean oldValue, boolean newValue) throws IOException Called after switching exceed throttle quota state.- Parameters:
ctx- the coprocessor instance's environmentoldValue- the previously exceed throttle quota valuenewValue- the newly exceed throttle quota value- Throws:
IOException
-
preGrant
default void preGrant(ObserverContext<MasterCoprocessorEnvironment> ctx, UserPermission userPermission, boolean mergeExistingPermissions) throws IOException Called before granting user permissions.- Parameters:
ctx- the coprocessor instance's environmentuserPermission- the user and permissionsmergeExistingPermissions- True if merge with previous granted permissions- Throws:
IOException
-
postGrant
default void postGrant(ObserverContext<MasterCoprocessorEnvironment> ctx, UserPermission userPermission, boolean mergeExistingPermissions) throws IOException Called after granting user permissions.- Parameters:
ctx- the coprocessor instance's environmentuserPermission- the user and permissionsmergeExistingPermissions- True if merge with previous granted permissions- Throws:
IOException
-
preRevoke
default void preRevoke(ObserverContext<MasterCoprocessorEnvironment> ctx, UserPermission userPermission) throws IOException Called before revoking user permissions.- Parameters:
ctx- the coprocessor instance's environmentuserPermission- the user and permissions- Throws:
IOException
-
postRevoke
default void postRevoke(ObserverContext<MasterCoprocessorEnvironment> ctx, UserPermission userPermission) throws IOException Called after revoking user permissions.- Parameters:
ctx- the coprocessor instance's environmentuserPermission- the user and permissions- Throws:
IOException
-
preGetUserPermissions
default void preGetUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, String namespace, TableName tableName, byte[] family, byte[] qualifier) throws IOException Called before getting user permissions.- Parameters:
ctx- the coprocessor instance's environmentuserName- the user name, null if get all user permissionsnamespace- the namespace, null if don't get namespace permissiontableName- the table name, null if don't get table permissionfamily- the table column family, null if don't get table family permissionqualifier- the table column qualifier, null if don't get table qualifier permission- Throws:
IOException- if something went wrong
-
postGetUserPermissions
default void postGetUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, String namespace, TableName tableName, byte[] family, byte[] qualifier) throws IOException Called after getting user permissions.- Parameters:
ctx- the coprocessor instance's environmentuserName- the user name, null if get all user permissionsnamespace- the namespace, null if don't get namespace permissiontableName- the table name, null if don't get table permissionfamily- the table column family, null if don't get table family permissionqualifier- the table column qualifier, null if don't get table qualifier permission- Throws:
IOException- if something went wrong
-
preHasUserPermissions
default void preHasUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, List<Permission> permissions) throws IOException - Throws:
IOException
-
postHasUserPermissions
default void postHasUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, List<Permission> permissions) throws IOException Called after checking if user has permissions.- Parameters:
ctx- the coprocessor instance's environmentuserName- the user namepermissions- the permission list- Throws:
IOException
-
preRenameRSGroup
default void preRenameRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String oldName, String newName) throws IOException Called before rename rsgroup.- Parameters:
ctx- the environment to interact with the framework and masteroldName- old rsgroup namenewName- new rsgroup name- Throws:
IOException- on failure
-
postRenameRSGroup
default void postRenameRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String oldName, String newName) throws IOException Called after rename rsgroup.- Parameters:
ctx- the environment to interact with the framework and masteroldName- old rsgroup namenewName- new rsgroup name- Throws:
IOException- on failure
-
preUpdateRSGroupConfig
default void preUpdateRSGroupConfig(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName, Map<String, String> configuration) throws IOExceptionCalled before update rsgroup config.- Parameters:
ctx- the environment to interact with the framework and mastergroupName- the group nameconfiguration- new configuration of the group name to be set- Throws:
IOException
-
postUpdateRSGroupConfig
default void postUpdateRSGroupConfig(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName, Map<String, String> configuration) throws IOExceptionCalled after update rsgroup config.- Parameters:
ctx- the environment to interact with the framework and mastergroupName- the group nameconfiguration- new configuration of the group name to be set- Throws:
IOException
-
preUpdateMasterConfiguration
default void preUpdateMasterConfiguration(ObserverContext<MasterCoprocessorEnvironment> ctx, org.apache.hadoop.conf.Configuration preReloadConf) throws IOException - Throws:
IOException
-
postUpdateMasterConfiguration
default void postUpdateMasterConfiguration(ObserverContext<MasterCoprocessorEnvironment> ctx, org.apache.hadoop.conf.Configuration postReloadConf) throws IOException Called after reloading the HMaster'sConfigurationfrom disk- Parameters:
ctx- the coprocessor instance's environmentpostReloadConf- theConfigurationthat was loaded- Throws:
IOException- if you need to signal an IO error
-