@InterfaceAudience.LimitedPrivate(value="Coprocesssor") @InterfaceStability.Evolving public interface MasterObserver
HMaster process.
IOException are reported back to client.CoprocessorHost.ABORT_ON_ERROR_KEY is set to true, then
the server aborts.DoNotRetryIOException is returned to the client.| Modifier and Type | Method and Description |
|---|---|
default void |
postAbortProcedure(ObserverContext<MasterCoprocessorEnvironment> ctx)
Called after a abortProcedure request has been processed.
|
default void |
postAddReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx,
String peerId,
ReplicationPeerConfig peerConfig)
Called after add a replication peer
|
default void |
postAddRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
String name)
Called after a new region server group is added
|
default void |
postAssign(ObserverContext<MasterCoprocessorEnvironment> ctx,
RegionInfo regionInfo)
Called after the region assignment has been requested.
|
default void |
postBalance(ObserverContext<MasterCoprocessorEnvironment> ctx,
List<RegionPlan> plans)
Called after the balancing plan has been submitted.
|
default void |
postBalanceRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
String groupName,
boolean balancerRan)
Called after a region server group is removed
|
default void |
postBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> ctx,
boolean oldValue,
boolean newValue)
Called after the flag to enable/disable balancing has changed.
|
default void |
postClearDeadServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
List<ServerName> servers,
List<ServerName> notClearedServers)
Called after clear dead region servers.
|
default void |
postCloneSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
SnapshotDescription snapshot,
TableDescriptor tableDescriptor)
Called after a snapshot clone operation has been requested.
|
default void |
postCompletedCreateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableDescriptor desc,
RegionInfo[] regions)
Called after the createTable operation has been requested.
|
default void |
postCompletedDeleteTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called after
HMaster deletes a
table. |
default void |
postCompletedDisableTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called after the disableTable operation has been requested.
|
default void |
postCompletedEnableTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called after the enableTable operation has been requested.
|
default void |
postCompletedMergeRegionsAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
RegionInfo[] regionsToMerge,
RegionInfo mergedRegion)
called after the regions merge.
|
default void |
postCompletedModifyTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
TableDescriptor currentDescriptor)
Deprecated.
Since 2.1. Will be removed in 3.0.
|
default void |
postCompletedModifyTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
TableDescriptor oldDescriptor,
TableDescriptor currentDescriptor)
Called after to modifying a table's properties.
|
default void |
postCompletedSplitRegionAction(ObserverContext<MasterCoprocessorEnvironment> c,
RegionInfo regionInfoA,
RegionInfo regionInfoB)
Called after the region is split.
|
default void |
postCompletedTruncateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called after
HMaster truncates a
table. |
default void |
postCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
NamespaceDescriptor ns)
Called after the createNamespace operation has been requested.
|
default void |
postCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableDescriptor desc,
RegionInfo[] regions)
Called after the createTable operation has been requested.
|
default void |
postDecommissionRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
List<ServerName> servers,
boolean offload)
Called after decommission region servers.
|
default void |
postDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
String namespace)
Called after the deleteNamespace operation has been requested.
|
default void |
postDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
SnapshotDescription snapshot)
Called after the delete snapshot operation has been requested.
|
default void |
postDeleteTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called after the deleteTable operation has been requested.
|
default void |
postDisableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx,
String peerId)
Called after disable a replication peer
|
default void |
postDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called after the disableTable operation has been requested.
|
default void |
postEnableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx,
String peerId)
Called after enable a replication peer
|
default void |
postEnableTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called after the enableTable operation has been requested.
|
default void |
postGetClusterMetrics(ObserverContext<MasterCoprocessorEnvironment> ctx,
ClusterMetrics status)
Called after get cluster status.
|
default void |
postGetLocks(ObserverContext<MasterCoprocessorEnvironment> ctx)
Called after a getLocks request has been processed.
|
default void |
postGetNamespaceDescriptor(ObserverContext<MasterCoprocessorEnvironment> ctx,
NamespaceDescriptor ns)
Called after a getNamespaceDescriptor request has been processed.
|
default void |
postGetProcedures(ObserverContext<MasterCoprocessorEnvironment> ctx)
Called after a getProcedures request has been processed.
|
default void |
postGetReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx,
String peerId)
Called after get the configured ReplicationPeerConfig for the specified peer
|
default void |
postGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
List<TableName> tableNamesList,
List<TableDescriptor> descriptors,
String regex)
Called after a getTableDescriptors request has been processed.
|
default void |
postGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx,
List<TableDescriptor> descriptors,
String regex)
Called after a getTableNames request has been processed.
|
default void |
postListDecommissionedRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx)
Called after list decommissioned region servers.
|
default void |
postListNamespaceDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
List<NamespaceDescriptor> descriptors)
Called after a listNamespaceDescriptors request has been processed.
|
default void |
postListReplicationPeers(ObserverContext<MasterCoprocessorEnvironment> ctx,
String regex)
Called after list replication peers.
|
default void |
postListSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
SnapshotDescription snapshot)
Called after listSnapshots request has been processed.
|
default void |
postLockHeartbeat(ObserverContext<MasterCoprocessorEnvironment> ctx)
Called after heartbeat to a lock.
|
default void |
postMergeRegions(ObserverContext<MasterCoprocessorEnvironment> c,
RegionInfo[] regionsToMerge)
called after merge regions request.
|
default void |
postMergeRegionsCommitAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
RegionInfo[] regionsToMerge,
RegionInfo mergedRegion)
This will be called after META step as part of regions merge transaction.
|
default void |
postModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
NamespaceDescriptor currentNsDescriptor)
Deprecated.
Since 2.1. Will be removed in 3.0.
|
default void |
postModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
NamespaceDescriptor oldNsDescriptor,
NamespaceDescriptor currentNsDescriptor)
Called after the modifyNamespace operation has been requested.
|
default void |
postModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
TableDescriptor currentDescriptor)
Deprecated.
Since 2.1. Will be removed in 3.0.
|
default void |
postModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
TableDescriptor oldDescriptor,
TableDescriptor currentDescriptor)
Called after the modifyTable operation has been requested.
|
default void |
postMove(ObserverContext<MasterCoprocessorEnvironment> ctx,
RegionInfo region,
ServerName srcServer,
ServerName destServer)
Called after the region move has been requested.
|
default void |
postMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
Set<Address> servers,
String targetGroup)
Called after servers are moved to target region server group
|
default void |
postMoveServersAndTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
Set<Address> servers,
Set<TableName> tables,
String targetGroup)
Called after servers are moved to target region server group
|
default void |
postMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
Set<TableName> tables,
String targetGroup)
Called after servers are moved to target region server group
|
default void |
postRecommissionRegionServer(ObserverContext<MasterCoprocessorEnvironment> ctx,
ServerName server,
List<byte[]> encodedRegionNames)
Called after recommission region server.
|
default void |
postRegionOffline(ObserverContext<MasterCoprocessorEnvironment> ctx,
RegionInfo regionInfo)
Called after the region has been marked offline.
|
default void |
postRemoveReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx,
String peerId)
Called after remove a replication peer
|
default void |
postRemoveRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
String name)
Called after a region server group is removed
|
default void |
postRemoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
Set<Address> servers)
Called after servers are removed from rsgroup
|
default void |
postRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx,
String namespace,
TableName tableName,
RegionInfo[] regionInfos,
String description)
Called after new LockProcedure is queued.
|
default void |
postRestoreSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
SnapshotDescription snapshot,
TableDescriptor tableDescriptor)
Called after a snapshot restore operation has been requested.
|
default void |
postRollBackMergeRegionsAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
RegionInfo[] regionsToMerge)
This will be called after the roll back of the regions merge.
|
default void |
postRollBackSplitRegionAction(ObserverContext<MasterCoprocessorEnvironment> ctx)
This will be called after the roll back of the split region is completed
|
default void |
postSetNamespaceQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
String namespace,
GlobalQuotaSettings quotas)
Called after the quota for the namespace is stored.
|
default void |
postSetSplitOrMergeEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx,
boolean newValue,
MasterSwitchType switchType)
Called after setting split / merge switch
|
default void |
postSetTableQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
GlobalQuotaSettings quotas)
Called after the quota for the table is stored.
|
default void |
postSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
String userName,
GlobalQuotaSettings quotas)
Called after the quota for the user is stored.
|
default void |
postSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
String userName,
String namespace,
GlobalQuotaSettings quotas)
Called after the quota for the user on the specified namespace is stored.
|
default void |
postSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
String userName,
TableName tableName,
GlobalQuotaSettings quotas)
Called after the quota for the user on the specified table is stored.
|
default void |
postSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
SnapshotDescription snapshot,
TableDescriptor tableDescriptor)
Called after the snapshot operation has been requested.
|
default void |
postStartMaster(ObserverContext<MasterCoprocessorEnvironment> ctx)
Called immediately after an active master instance has completed
initialization.
|
default void |
postTableFlush(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called after the table memstore is flushed to disk.
|
default void |
postTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called after the truncateTable operation has been requested.
|
default void |
postUnassign(ObserverContext<MasterCoprocessorEnvironment> ctx,
RegionInfo regionInfo,
boolean force)
Called after the region unassignment has been requested.
|
default void |
postUpdateReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx,
String peerId,
ReplicationPeerConfig peerConfig)
Called after update peerConfig for the specified peer
|
default void |
preAbortProcedure(ObserverContext<MasterCoprocessorEnvironment> ctx,
long procId)
Called before a abortProcedure request has been processed.
|
default void |
preAddReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx,
String peerId,
ReplicationPeerConfig peerConfig)
Called before add a replication peer
|
default void |
preAddRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
String name)
Called before a new region server group is added
|
default void |
preAssign(ObserverContext<MasterCoprocessorEnvironment> ctx,
RegionInfo regionInfo)
Called prior to assigning a specific region.
|
default void |
preBalance(ObserverContext<MasterCoprocessorEnvironment> ctx)
Called prior to requesting rebalancing of the cluster regions, though after
the initial checks for regions in transition and the balance switch flag.
|
default void |
preBalanceRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
String groupName)
Called before a region server group is removed
|
default void |
preBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> ctx,
boolean newValue)
Called prior to modifying the flag used to enable/disable region balancing.
|
default void |
preClearDeadServers(ObserverContext<MasterCoprocessorEnvironment> ctx)
Called before clear dead region servers.
|
default void |
preCloneSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
SnapshotDescription snapshot,
TableDescriptor tableDescriptor)
Called before a snapshot is cloned.
|
default void |
preCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
NamespaceDescriptor ns)
Called before a new namespace is created by
HMaster. |
default void |
preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableDescriptor desc,
RegionInfo[] regions)
Called before a new table is created by
HMaster. |
default void |
preCreateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableDescriptor desc,
RegionInfo[] regions)
Called before a new table is created by
HMaster. |
default void |
preDecommissionRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
List<ServerName> servers,
boolean offload)
Called before decommission region servers.
|
default void |
preDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
String namespace)
Called before
HMaster deletes a
namespace |
default void |
preDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
SnapshotDescription snapshot)
Called before a snapshot is deleted.
|
default void |
preDeleteTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called before
HMaster deletes a
table. |
default void |
preDeleteTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called before
HMaster deletes a
table. |
default void |
preDisableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx,
String peerId)
Called before disable a replication peer
|
default void |
preDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called prior to disabling a table.
|
default void |
preDisableTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called prior to disabling a table.
|
default void |
preEnableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx,
String peerId)
Called before enable a replication peer
|
default void |
preEnableTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called prior to enabling a table.
|
default void |
preEnableTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called prior to enabling a table.
|
default void |
preGetClusterMetrics(ObserverContext<MasterCoprocessorEnvironment> ctx)
Called before get cluster status.
|
default void |
preGetLocks(ObserverContext<MasterCoprocessorEnvironment> ctx)
Called before a getLocks request has been processed.
|
default void |
preGetNamespaceDescriptor(ObserverContext<MasterCoprocessorEnvironment> ctx,
String namespace)
Called before a getNamespaceDescriptor request has been processed.
|
default void |
preGetProcedures(ObserverContext<MasterCoprocessorEnvironment> ctx)
Called before a getProcedures request has been processed.
|
default void |
preGetReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx,
String peerId)
Called before get the configured ReplicationPeerConfig for the specified peer
|
default void |
preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
List<TableName> tableNamesList,
List<TableDescriptor> descriptors,
String regex)
Called before a getTableDescriptors request has been processed.
|
default void |
preGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx,
List<TableDescriptor> descriptors,
String regex)
Called before a getTableNames request has been processed.
|
default void |
preListDecommissionedRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx)
Called before list decommissioned region servers.
|
default void |
preListNamespaceDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
List<NamespaceDescriptor> descriptors)
Called before a listNamespaceDescriptors request has been processed.
|
default void |
preListReplicationPeers(ObserverContext<MasterCoprocessorEnvironment> ctx,
String regex)
Called before list replication peers.
|
default void |
preListSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
SnapshotDescription snapshot)
Called before listSnapshots request has been processed.
|
default void |
preLockHeartbeat(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tn,
String description)
Called before heartbeat to a lock.
|
default void |
preMasterInitialization(ObserverContext<MasterCoprocessorEnvironment> ctx)
Call before the master initialization is set to true.
|
default void |
preMergeRegions(ObserverContext<MasterCoprocessorEnvironment> ctx,
RegionInfo[] regionsToMerge)
Called before merge regions request.
|
default void |
preMergeRegionsAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
RegionInfo[] regionsToMerge)
Called before the regions merge.
|
default void |
preMergeRegionsCommitAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
RegionInfo[] regionsToMerge,
List<Mutation> metaEntries)
This will be called before update META step as part of regions merge transaction.
|
default void |
preModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
NamespaceDescriptor newNsDescriptor)
Deprecated.
Since 2.1. Will be removed in 3.0.
|
default void |
preModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
NamespaceDescriptor currentNsDescriptor,
NamespaceDescriptor newNsDescriptor)
Called prior to modifying a namespace's properties.
|
default void |
preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
TableDescriptor newDescriptor)
Deprecated.
Since 2.1. Will be removed in 3.0.
|
default void |
preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
TableDescriptor currentDescriptor,
TableDescriptor newDescriptor)
Called prior to modifying a table's properties.
|
default void |
preModifyTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
TableDescriptor newDescriptor)
Deprecated.
Since 2.1. Will be removed in 3.0.
|
default void |
preModifyTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
TableDescriptor currentDescriptor,
TableDescriptor newDescriptor)
Called prior to modifying a table's properties.
|
default void |
preMove(ObserverContext<MasterCoprocessorEnvironment> ctx,
RegionInfo region,
ServerName srcServer,
ServerName destServer)
Called prior to moving a given region from one region server to another.
|
default void |
preMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
Set<Address> servers,
String targetGroup)
Called before servers are moved to target region server group
|
default void |
preMoveServersAndTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
Set<Address> servers,
Set<TableName> tables,
String targetGroup)
Called before servers are moved to target region server group
|
default void |
preMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
Set<TableName> tables,
String targetGroup)
Called before tables are moved to target region server group
|
default void |
preRecommissionRegionServer(ObserverContext<MasterCoprocessorEnvironment> ctx,
ServerName server,
List<byte[]> encodedRegionNames)
Called before recommission region server.
|
default void |
preRegionOffline(ObserverContext<MasterCoprocessorEnvironment> ctx,
RegionInfo regionInfo)
Called prior to marking a given region as offline.
|
default void |
preRemoveReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx,
String peerId)
Called before remove a replication peer
|
default void |
preRemoveRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
String name)
Called before a region server group is removed
|
default void |
preRemoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
Set<Address> servers)
Called before servers are removed from rsgroup
|
default void |
preRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx,
String namespace,
TableName tableName,
RegionInfo[] regionInfos,
String description)
Called before new LockProcedure is queued.
|
default void |
preRestoreSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
SnapshotDescription snapshot,
TableDescriptor tableDescriptor)
Called before a snapshot is restored.
|
default void |
preSetNamespaceQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
String namespace,
GlobalQuotaSettings quotas)
Called before the quota for the namespace is stored.
|
default void |
preSetSplitOrMergeEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx,
boolean newValue,
MasterSwitchType switchType)
Called prior to setting split / merge switch
Supports Coprocessor 'bypass'.
|
default void |
preSetTableQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
GlobalQuotaSettings quotas)
Called before the quota for the table is stored.
|
default void |
preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
String userName,
GlobalQuotaSettings quotas)
Called before the quota for the user is stored.
|
default void |
preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
String userName,
String namespace,
GlobalQuotaSettings quotas)
Called before the quota for the user on the specified namespace is stored.
|
default void |
preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
String userName,
TableName tableName,
GlobalQuotaSettings quotas)
Called before the quota for the user on the specified table is stored.
|
default void |
preShutdown(ObserverContext<MasterCoprocessorEnvironment> ctx)
Called prior to shutting down the full HBase cluster, including this
HMaster process. |
default void |
preSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
SnapshotDescription snapshot,
TableDescriptor tableDescriptor)
Called before a new snapshot is taken.
|
default void |
preSplitRegion(ObserverContext<MasterCoprocessorEnvironment> c,
TableName tableName,
byte[] splitRow)
Called before the split region procedure is called.
|
default void |
preSplitRegionAction(ObserverContext<MasterCoprocessorEnvironment> c,
TableName tableName,
byte[] splitRow)
Called before the region is split.
|
default void |
preSplitRegionAfterMETAAction(ObserverContext<MasterCoprocessorEnvironment> ctx)
This will be called after update META step as part of split transaction
|
default void |
preSplitRegionBeforeMETAAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
byte[] splitKey,
List<Mutation> metaEntries)
This will be called before update META step as part of split transaction.
|
default void |
preStopMaster(ObserverContext<MasterCoprocessorEnvironment> ctx)
Called immediately prior to stopping this
HMaster process. |
default void |
preTableFlush(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called before the table memstore is flushed to disk.
|
default void |
preTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called before
HMaster truncates a
table. |
default void |
preTruncateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called before
HMaster truncates a
table. |
default void |
preUnassign(ObserverContext<MasterCoprocessorEnvironment> ctx,
RegionInfo regionInfo,
boolean force)
Called prior to unassigning a given region.
|
default void |
preUpdateReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx,
String peerId,
ReplicationPeerConfig peerConfig)
Called before update peerConfig for the specified peer
|
default void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableDescriptor desc, RegionInfo[] regions) throws IOException
HMaster. Called as part of create
table RPC call.ctx - the environment to interact with the framework and masterdesc - the TableDescriptor for the tableregions - the initial regions created for the tableIOExceptiondefault void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableDescriptor desc, RegionInfo[] regions) throws IOException
ctx - the environment to interact with the framework and masterdesc - the TableDescriptor for the tableregions - the initial regions created for the tableIOExceptiondefault void preCreateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableDescriptor desc, RegionInfo[] regions) throws IOException
HMaster. Called as part of create
table procedure and it is async to the create RPC call.ctx - the environment to interact with the framework and masterdesc - the TableDescriptor for the tableregions - the initial regions created for the tableIOExceptiondefault void postCompletedCreateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableDescriptor desc, RegionInfo[] regions) throws IOException
ctx - the environment to interact with the framework and masterdesc - the TableDescriptor for the tableregions - the initial regions created for the tableIOExceptiondefault void preDeleteTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
HMaster deletes a
table. Called as part of delete table RPC call.ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void postDeleteTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void preDeleteTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
HMaster deletes a
table. Called as part of delete table procedure and
it is async to the delete RPC call.ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void postCompletedDeleteTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
HMaster deletes a
table. Called as part of delete table procedure and it is async to the
delete RPC call.ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void preTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
HMaster truncates a
table. Called as part of truncate table RPC call.ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void postTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void preTruncateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
HMaster truncates a
table. Called as part of truncate table procedure and it is async
to the truncate RPC call.ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void postCompletedTruncateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
HMaster truncates a
table. Called as part of truncate table procedure and it is async to the
truncate RPC call.ctx - the environment to interact with the framework and mastertableName - the name of the tableIOException@Deprecated default void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor newDescriptor) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablenewDescriptor - after modify operation, table will have this descriptorIOExceptiondefault void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor currentDescriptor, TableDescriptor newDescriptor) throws IOException
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 descriptorIOException@Deprecated default void postModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor currentDescriptor) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablecurrentDescriptor - current TableDescriptor of the tableIOExceptiondefault void postModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor oldDescriptor, TableDescriptor currentDescriptor) throws IOException
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 tableIOException@Deprecated default void preModifyTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor newDescriptor) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablenewDescriptor - after modify operation, table will have this descriptorIOExceptiondefault void preModifyTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor currentDescriptor, TableDescriptor newDescriptor) throws IOException
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 descriptorIOException@Deprecated default void postCompletedModifyTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor currentDescriptor) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablecurrentDescriptor - current TableDescriptor of the tableIOExceptiondefault void postCompletedModifyTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor oldDescriptor, TableDescriptor currentDescriptor) throws IOException
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 tableIOExceptiondefault void preEnableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void postEnableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void preEnableTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void postCompletedEnableTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void preDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void postDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void preDisableTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void postCompletedDisableTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void preAbortProcedure(ObserverContext<MasterCoprocessorEnvironment> ctx, long procId) throws IOException
ctx - the environment to interact with the framework and masterprocId - the Id of the procedureIOExceptiondefault void postAbortProcedure(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void preGetProcedures(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void postGetProcedures(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void preGetLocks(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
ctx - the environment to interact with the framework and masterIOException - if something went wrongdefault void postGetLocks(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
ctx - the environment to interact with the framework and masterIOException - if something went wrongdefault void preMove(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo region, ServerName srcServer, ServerName destServer) throws IOException
ctx - the environment to interact with the framework and masterregion - the RegionInfosrcServer - the source ServerNamedestServer - the destination ServerNameIOExceptiondefault void postMove(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo region, ServerName srcServer, ServerName destServer) throws IOException
ctx - the environment to interact with the framework and masterregion - the RegionInfosrcServer - the source ServerNamedestServer - the destination ServerNameIOExceptiondefault void preAssign(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo) throws IOException
ctx - the environment to interact with the framework and masterregionInfo - the regionInfo of the regionIOExceptiondefault void postAssign(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo) throws IOException
ctx - the environment to interact with the framework and masterregionInfo - the regionInfo of the regionIOExceptiondefault void preUnassign(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo, boolean force) throws IOException
ctx - the environment to interact with the framework and masterregionInfo - force - whether to force unassignment or notIOExceptiondefault void postUnassign(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo, boolean force) throws IOException
ctx - the environment to interact with the framework and masterregionInfo - force - whether to force unassignment or notIOExceptiondefault void preRegionOffline(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo) throws IOException
ctx - the environment to interact with the framework and masterregionInfo - IOExceptiondefault void postRegionOffline(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo) throws IOException
ctx - the environment to interact with the framework and masterregionInfo - IOExceptiondefault void preBalance(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void postBalance(ObserverContext<MasterCoprocessorEnvironment> ctx, List<RegionPlan> plans) throws IOException
ctx - the environment to interact with the framework and masterplans - 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 assignmentIOExceptiondefault void preSetSplitOrMergeEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean newValue, MasterSwitchType switchType) throws IOException
ctx - the coprocessor instance's environmentnewValue - the new value submitted in the callswitchType - type of switchIOExceptiondefault void postSetSplitOrMergeEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean newValue, MasterSwitchType switchType) throws IOException
ctx - the coprocessor instance's environmentnewValue - the new value submitted in the callswitchType - type of switchIOExceptiondefault void preSplitRegion(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName, byte[] splitRow) throws IOException
c - the environment to interact with the framework and mastertableName - the table where the region belongs tosplitRow - split pointIOExceptiondefault void preSplitRegionAction(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName, byte[] splitRow) throws IOException
c - the environment to interact with the framework and mastertableName - the table where the region belongs tosplitRow - split pointIOExceptiondefault void postCompletedSplitRegionAction(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfoA, RegionInfo regionInfoB) throws IOException
c - the environment to interact with the framework and masterregionInfoA - the left daughter regionregionInfoB - the right daughter regionIOExceptiondefault void preSplitRegionBeforeMETAAction(ObserverContext<MasterCoprocessorEnvironment> ctx, byte[] splitKey, List<Mutation> metaEntries) throws IOException
ctx - the environment to interact with the framework and mastersplitKey - metaEntries - IOExceptiondefault void preSplitRegionAfterMETAAction(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void postRollBackSplitRegionAction(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void preMergeRegionsAction(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void postCompletedMergeRegionsAction(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge, RegionInfo mergedRegion) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void preMergeRegionsCommitAction(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge, @MetaMutationAnnotation List<Mutation> metaEntries) throws IOException
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.IOExceptiondefault void postMergeRegionsCommitAction(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge, RegionInfo mergedRegion) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void postRollBackMergeRegionsAction(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void preBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean newValue) throws IOException
ctx - the coprocessor instance's environmentIOExceptiondefault void postBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean oldValue, boolean newValue) throws IOException
ctx - the coprocessor instance's environmentoldValue - the previously set balanceSwitch valuenewValue - the newly set balanceSwitch valueIOExceptiondefault void preShutdown(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
HMaster process.IOExceptiondefault void preStopMaster(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
HMaster process.IOExceptiondefault void postStartMaster(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
IOExceptiondefault void preMasterInitialization(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
HMaster process.IOExceptiondefault void preSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor for the snapshottableDescriptor - the TableDescriptor of the table to snapshotIOExceptiondefault void postSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor for the snapshottableDescriptor - the TableDescriptor of the table to snapshotIOExceptiondefault void preListSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor of the snapshot to listIOExceptiondefault void postListSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor of the snapshot to listIOExceptiondefault void preCloneSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor for the snapshottableDescriptor - the TableDescriptor of the table to createIOExceptiondefault void postCloneSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor for the snapshottableDescriptor - the v of the table to createIOExceptiondefault void preRestoreSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor for the snapshottableDescriptor - the TableDescriptor of the table to restoreIOExceptiondefault void postRestoreSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor for the snapshottableDescriptor - the TableDescriptor of the table to restoreIOExceptiondefault void preDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor of the snapshot to deleteIOExceptiondefault void postDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor of the snapshot to deleteIOExceptiondefault void preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableName> tableNamesList, List<TableDescriptor> descriptors, String regex) throws IOException
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 namesIOExceptiondefault void postGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableName> tableNamesList, List<TableDescriptor> descriptors, String regex) throws IOException
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 namesIOExceptiondefault void preGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableDescriptor> descriptors, String regex) throws IOException
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 namesIOExceptiondefault void postGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableDescriptor> descriptors, String regex) throws IOException
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 namesIOExceptiondefault void preCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor ns) throws IOException
HMaster.ctx - the environment to interact with the framework and masterns - the NamespaceDescriptor for the tableIOExceptiondefault void postCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor ns) throws IOException
ctx - the environment to interact with the framework and masterns - the NamespaceDescriptor for the tableIOExceptiondefault void preDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) throws IOException
HMaster deletes a
namespacectx - the environment to interact with the framework and masternamespace - the name of the namespaceIOExceptiondefault void postDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) throws IOException
ctx - the environment to interact with the framework and masternamespace - the name of the namespaceIOException@Deprecated default void preModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor newNsDescriptor) throws IOException
ctx - the environment to interact with the framework and masternewNsDescriptor - after modify operation, namespace will have this descriptorIOExceptiondefault void preModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor currentNsDescriptor, NamespaceDescriptor newNsDescriptor) throws IOException
ctx - the environment to interact with the framework and mastercurrentNsDescriptor - current NamespaceDescriptor of the namespacenewNsDescriptor - after modify operation, namespace will have this descriptorIOException@Deprecated default void postModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor currentNsDescriptor) throws IOException
ctx - the environment to interact with the framework and mastercurrentNsDescriptor - current NamespaceDescriptor of the namespaceIOExceptiondefault void postModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor oldNsDescriptor, NamespaceDescriptor currentNsDescriptor) throws IOException
ctx - the environment to interact with the framework and masteroldNsDescriptor - descriptor of namespace before modify operation happenedcurrentNsDescriptor - current NamespaceDescriptor of the namespaceIOExceptiondefault void preGetNamespaceDescriptor(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) throws IOException
ctx - the environment to interact with the framework and masternamespace - the name of the namespaceIOExceptiondefault void postGetNamespaceDescriptor(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor ns) throws IOException
ctx - the environment to interact with the framework and masterns - the NamespaceDescriptorIOExceptiondefault void preListNamespaceDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<NamespaceDescriptor> descriptors) throws IOException
ctx - the environment to interact with the framework and masterdescriptors - an empty list, can be filled with what to return by coprocessorIOExceptiondefault void postListNamespaceDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<NamespaceDescriptor> descriptors) throws IOException
ctx - the environment to interact with the framework and masterdescriptors - the list of descriptors about to be returnedIOExceptiondefault void preTableFlush(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void postTableFlush(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, GlobalQuotaSettings quotas) throws IOException
ctx - the environment to interact with the framework and masteruserName - the name of userquotas - the current quota for the userIOExceptiondefault void postSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, GlobalQuotaSettings quotas) throws IOException
ctx - the environment to interact with the framework and masteruserName - the name of userquotas - the resulting quota for the userIOExceptiondefault void preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, TableName tableName, GlobalQuotaSettings quotas) throws IOException
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 tableIOExceptiondefault void postSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, TableName tableName, GlobalQuotaSettings quotas) throws IOException
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 tableIOExceptiondefault void preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, String namespace, GlobalQuotaSettings quotas) throws IOException
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 namespaceIOExceptiondefault void postSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, String namespace, GlobalQuotaSettings quotas) throws IOException
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 namespaceIOExceptiondefault void preSetTableQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, GlobalQuotaSettings quotas) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablequotas - the current quota for the tableIOExceptiondefault void postSetTableQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, GlobalQuotaSettings quotas) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablequotas - the resulting quota for the tableIOExceptiondefault void preSetNamespaceQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace, GlobalQuotaSettings quotas) throws IOException
ctx - the environment to interact with the framework and masternamespace - the name of the namespacequotas - the current quota for the namespaceIOExceptiondefault void postSetNamespaceQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace, GlobalQuotaSettings quotas) throws IOException
ctx - the environment to interact with the framework and masternamespace - the name of the namespacequotas - the resulting quota for the namespaceIOExceptiondefault void preMergeRegions(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge) throws IOException
ctx - coprocessor environmentregionsToMerge - regions to be mergedIOExceptiondefault void postMergeRegions(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo[] regionsToMerge) throws IOException
c - coprocessor environmentregionsToMerge - regions to be mergedIOExceptiondefault void preMoveServersAndTables(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers, Set<TableName> tables, String targetGroup) throws IOException
ctx - the environment to interact with the framework and masterservers - set of servers to movetargetGroup - destination groupIOExceptiondefault void postMoveServersAndTables(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers, Set<TableName> tables, String targetGroup) throws IOException
ctx - the environment to interact with the framework and masterservers - set of servers to movetargetGroup - name of groupIOExceptiondefault void preMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers, String targetGroup) throws IOException
ctx - the environment to interact with the framework and masterservers - set of servers to movetargetGroup - destination groupIOExceptiondefault void postMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers, String targetGroup) throws IOException
ctx - the environment to interact with the framework and masterservers - set of servers to movetargetGroup - name of groupIOExceptiondefault void preMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<TableName> tables, String targetGroup) throws IOException
ctx - the environment to interact with the framework and mastertables - set of tables to movetargetGroup - name of groupIOExceptiondefault void postMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<TableName> tables, String targetGroup) throws IOException
ctx - the environment to interact with the framework and mastertables - set of tables to movetargetGroup - name of groupIOExceptiondefault void preAddRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String name) throws IOException
ctx - the environment to interact with the framework and mastername - group nameIOExceptiondefault void postAddRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String name) throws IOException
ctx - the environment to interact with the framework and mastername - group nameIOExceptiondefault void preRemoveRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String name) throws IOException
ctx - the environment to interact with the framework and mastername - group nameIOExceptiondefault void postRemoveRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String name) throws IOException
ctx - the environment to interact with the framework and mastername - group nameIOExceptiondefault void preBalanceRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName) throws IOException
ctx - the environment to interact with the framework and mastergroupName - group nameIOExceptiondefault void postBalanceRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName, boolean balancerRan) throws IOException
ctx - the environment to interact with the framework and mastergroupName - group nameIOExceptiondefault void preRemoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers) throws IOException
ctx - the environment to interact with the framework and masterservers - set of decommissioned servers to removeIOExceptiondefault void postRemoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers) throws IOException
ctx - the environment to interact with the framework and masterservers - set of servers to removeIOExceptiondefault void preAddReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId, ReplicationPeerConfig peerConfig) throws IOException
ctx - the environment to interact with the framework and masterpeerId - a short name that identifies the peerpeerConfig - configuration for the replication peerIOExceptiondefault void postAddReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId, ReplicationPeerConfig peerConfig) throws IOException
ctx - the environment to interact with the framework and masterpeerId - a short name that identifies the peerpeerConfig - configuration for the replication peerIOExceptiondefault void preRemoveReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException
ctx - peerId - a short name that identifies the peerIOExceptiondefault void postRemoveReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException
ctx - peerId - a short name that identifies the peerIOExceptiondefault void preEnableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException
ctx - peerId - a short name that identifies the peerIOExceptiondefault void postEnableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException
ctx - peerId - a short name that identifies the peerIOExceptiondefault void preDisableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException
ctx - peerId - a short name that identifies the peerIOExceptiondefault void postDisableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException
ctx - peerId - a short name that identifies the peerIOExceptiondefault void preGetReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException
ctx - peerId - a short name that identifies the peerIOExceptiondefault void postGetReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException
ctx - peerId - a short name that identifies the peerIOExceptiondefault void preUpdateReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId, ReplicationPeerConfig peerConfig) throws IOException
ctx - peerId - a short name that identifies the peerIOExceptiondefault void postUpdateReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId, ReplicationPeerConfig peerConfig) throws IOException
ctx - the environment to interact with the framework and masterpeerId - a short name that identifies the peerIOExceptiondefault void preListReplicationPeers(ObserverContext<MasterCoprocessorEnvironment> ctx, String regex) throws IOException
ctx - the environment to interact with the framework and masterregex - The regular expression to match peer idIOExceptiondefault void postListReplicationPeers(ObserverContext<MasterCoprocessorEnvironment> ctx, String regex) throws IOException
ctx - the environment to interact with the framework and masterregex - The regular expression to match peer idIOExceptiondefault void preRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace, TableName tableName, RegionInfo[] regionInfos, String description) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void postRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace, TableName tableName, RegionInfo[] regionInfos, String description) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void preLockHeartbeat(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tn, String description) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void postLockHeartbeat(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void preGetClusterMetrics(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
IOExceptiondefault void postGetClusterMetrics(ObserverContext<MasterCoprocessorEnvironment> ctx, ClusterMetrics status) throws IOException
IOExceptiondefault void preClearDeadServers(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
IOExceptiondefault void postClearDeadServers(ObserverContext<MasterCoprocessorEnvironment> ctx, List<ServerName> servers, List<ServerName> notClearedServers) throws IOException
IOExceptiondefault void preDecommissionRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx, List<ServerName> servers, boolean offload) throws IOException
IOExceptiondefault void postDecommissionRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx, List<ServerName> servers, boolean offload) throws IOException
IOExceptiondefault void preListDecommissionedRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
IOExceptiondefault void postListDecommissionedRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
IOExceptiondefault void preRecommissionRegionServer(ObserverContext<MasterCoprocessorEnvironment> ctx, ServerName server, List<byte[]> encodedRegionNames) throws IOException
IOExceptiondefault void postRecommissionRegionServer(ObserverContext<MasterCoprocessorEnvironment> ctx, ServerName server, List<byte[]> encodedRegionNames) throws IOException
IOExceptionCopyright © 2007–2020 The Apache Software Foundation. All rights reserved.