@InterfaceAudience.LimitedPrivate(value="Coprocesssor") @InterfaceStability.Evolving public interface MasterObserver extends Coprocessor
HMaster process.Coprocessor.StatePRIORITY_HIGHEST, PRIORITY_LOWEST, PRIORITY_SYSTEM, PRIORITY_USER, VERSION| Modifier and Type | Method and Description |
|---|---|
void |
postAbortProcedure(ObserverContext<MasterCoprocessorEnvironment> ctx)
Called after a abortProcedure request has been processed.
|
void |
postAddColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
HColumnDescriptor column)
Called after the new column family has been created.
|
void |
postAddColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
HColumnDescriptor column)
Called after the new column family has been created.
|
void |
postAddRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
String name)
Called after a new region server group is added
|
void |
postAssign(ObserverContext<MasterCoprocessorEnvironment> ctx,
HRegionInfo regionInfo)
Called after the region assignment has been requested.
|
void |
postBalance(ObserverContext<MasterCoprocessorEnvironment> ctx,
List<RegionPlan> plans)
Called after the balancing plan has been submitted.
|
void |
postBalanceRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
String groupName,
boolean balancerRan)
Called after a region server group is removed
|
void |
postBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> ctx,
boolean oldValue,
boolean newValue)
Called after the flag to enable/disable balancing has changed.
|
void |
postCloneSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
HBaseProtos.SnapshotDescription snapshot,
HTableDescriptor hTableDescriptor)
Called after a snapshot clone operation has been requested.
|
void |
postCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
NamespaceDescriptor ns)
Called after the createNamespace operation has been requested.
|
void |
postCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
HTableDescriptor desc,
HRegionInfo[] regions)
Called after the createTable operation has been requested.
|
void |
postCreateTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
HTableDescriptor desc,
HRegionInfo[] regions)
Called after the createTable operation has been requested.
|
void |
postDeleteColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
byte[] c)
Called after the column family has been deleted.
|
void |
postDeleteColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
byte[] c)
Called after the column family has been deleted.
|
void |
postDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
String namespace)
Called after the deleteNamespace operation has been requested.
|
void |
postDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
HBaseProtos.SnapshotDescription snapshot)
Called after the delete snapshot operation has been requested.
|
void |
postDeleteTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called after the deleteTable operation has been requested.
|
void |
postDeleteTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called after
HMaster deletes a
table. |
void |
postDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called after the disableTable operation has been requested.
|
void |
postDisableTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called after the disableTable operation has been requested.
|
void |
postEnableTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called after the enableTable operation has been requested.
|
void |
postEnableTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called after the enableTable operation has been requested.
|
void |
postGetNamespaceDescriptor(ObserverContext<MasterCoprocessorEnvironment> ctx,
NamespaceDescriptor ns)
Called after a getNamespaceDescriptor request has been processed.
|
void |
postGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
List<HTableDescriptor> descriptors)
Deprecated.
Use postGetTableDescriptors with regex instead.
|
void |
postGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
List<TableName> tableNamesList,
List<HTableDescriptor> descriptors,
String regex)
Called after a getTableDescriptors request has been processed.
|
void |
postGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx,
List<HTableDescriptor> descriptors,
String regex)
Called after a getTableNames request has been processed.
|
void |
postListNamespaceDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
List<NamespaceDescriptor> descriptors)
Called after a listNamespaceDescriptors request has been processed.
|
void |
postListProcedures(ObserverContext<MasterCoprocessorEnvironment> ctx,
List<ProcedureInfo> procInfoList)
Called after a listProcedures request has been processed.
|
void |
postListSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
HBaseProtos.SnapshotDescription snapshot)
Called after listSnapshots request has been processed.
|
void |
postModifyColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
HColumnDescriptor descriptor)
Called after the column family has been updated.
|
void |
postModifyColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
HColumnDescriptor descriptor)
Called after the column family has been updated.
|
void |
postModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
NamespaceDescriptor ns)
Called after the modifyNamespace operation has been requested.
|
void |
postModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
HTableDescriptor htd)
Called after the modifyTable operation has been requested.
|
void |
postModifyTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
HTableDescriptor htd)
Called after to modifying a table's properties.
|
void |
postMove(ObserverContext<MasterCoprocessorEnvironment> ctx,
HRegionInfo region,
ServerName srcServer,
ServerName destServer)
Called after the region move has been requested.
|
void |
postMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
Set<HostAndPort> servers,
String targetGroup)
Called after servers are moved to target region server group
|
void |
postMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
Set<TableName> tables,
String targetGroup)
Called after servers are moved to target region server group
|
void |
postRegionOffline(ObserverContext<MasterCoprocessorEnvironment> ctx,
HRegionInfo regionInfo)
Called after the region has been marked offline.
|
void |
postRemoveRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
String name)
Called after a region server group is removed
|
void |
postRestoreSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
HBaseProtos.SnapshotDescription snapshot,
HTableDescriptor hTableDescriptor)
Called after a snapshot restore operation has been requested.
|
void |
postSetNamespaceQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
String namespace,
QuotaProtos.Quotas quotas)
Called after the quota for the namespace is stored.
|
void |
postSetTableQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
QuotaProtos.Quotas quotas)
Called after the quota for the table is stored.
|
void |
postSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
String userName,
QuotaProtos.Quotas quotas)
Called after the quota for the user is stored.
|
void |
postSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
String userName,
String namespace,
QuotaProtos.Quotas quotas)
Called after the quota for the user on the specified namespace is stored.
|
void |
postSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
String userName,
TableName tableName,
QuotaProtos.Quotas quotas)
Called after the quota for the user on the specified table is stored.
|
void |
postSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
HBaseProtos.SnapshotDescription snapshot,
HTableDescriptor hTableDescriptor)
Called after the snapshot operation has been requested.
|
void |
postStartMaster(ObserverContext<MasterCoprocessorEnvironment> ctx)
Called immediately after an active master instance has completed
initialization.
|
void |
postTableFlush(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called after the table memstore is flushed to disk.
|
void |
postTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called after the truncateTable operation has been requested.
|
void |
postTruncateTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called after
HMaster truncates a
table. |
void |
postUnassign(ObserverContext<MasterCoprocessorEnvironment> ctx,
HRegionInfo regionInfo,
boolean force)
Called after the region unassignment has been requested.
|
void |
preAbortProcedure(ObserverContext<MasterCoprocessorEnvironment> ctx,
ProcedureExecutor<MasterProcedureEnv> procEnv,
long procId)
Called before a abortProcedure request has been processed.
|
void |
preAddColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
HColumnDescriptor column)
Called prior to adding a new column family to the table.
|
void |
preAddColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
HColumnDescriptor column)
Called prior to adding a new column family to the table.
|
void |
preAddRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
String name)
Called before a new region server group is added
|
void |
preAssign(ObserverContext<MasterCoprocessorEnvironment> ctx,
HRegionInfo regionInfo)
Called prior to assigning a specific region.
|
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.
|
void |
preBalanceRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
String groupName)
Called before a region server group is removed
|
boolean |
preBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> ctx,
boolean newValue)
Called prior to modifying the flag used to enable/disable region balancing.
|
void |
preCloneSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
HBaseProtos.SnapshotDescription snapshot,
HTableDescriptor hTableDescriptor)
Called before a snapshot is cloned.
|
void |
preCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
NamespaceDescriptor ns)
Called before a new namespace is created by
HMaster. |
void |
preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
HTableDescriptor desc,
HRegionInfo[] regions)
Called before a new table is created by
HMaster. |
void |
preCreateTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
HTableDescriptor desc,
HRegionInfo[] regions)
Called before a new table is created by
HMaster. |
void |
preDeleteColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
byte[] c)
Called prior to deleting the entire column family.
|
void |
preDeleteColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
byte[] c)
Called prior to deleting the entire column family.
|
void |
preDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
String namespace)
Called before
HMaster deletes a
namespace
It can't bypass the default action, e.g., ctx.bypass() won't have effect. |
void |
preDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
HBaseProtos.SnapshotDescription snapshot)
Called before a snapshot is deleted.
|
void |
preDeleteTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called before
HMaster deletes a
table. |
void |
preDeleteTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called before
HMaster deletes a
table. |
void |
preDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called prior to disabling a table.
|
void |
preDisableTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called prior to disabling a table.
|
void |
preEnableTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called prior to enabling a table.
|
void |
preEnableTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called prior to enabling a table.
|
void |
preGetNamespaceDescriptor(ObserverContext<MasterCoprocessorEnvironment> ctx,
String namespace)
Called before a getNamespaceDescriptor request has been processed.
|
void |
preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
List<TableName> tableNamesList,
List<HTableDescriptor> descriptors)
Deprecated.
Use preGetTableDescriptors with regex instead.
|
void |
preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
List<TableName> tableNamesList,
List<HTableDescriptor> descriptors,
String regex)
Called before a getTableDescriptors request has been processed.
|
void |
preGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx,
List<HTableDescriptor> descriptors,
String regex)
Called before a getTableNames request has been processed.
|
void |
preListNamespaceDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
List<NamespaceDescriptor> descriptors)
Called before a listNamespaceDescriptors request has been processed.
|
void |
preListProcedures(ObserverContext<MasterCoprocessorEnvironment> ctx)
Called before a listProcedures request has been processed.
|
void |
preListSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
HBaseProtos.SnapshotDescription snapshot)
Called before listSnapshots request has been processed.
|
void |
preMasterInitialization(ObserverContext<MasterCoprocessorEnvironment> ctx)
Call before the master initialization is set to true.
|
void |
preModifyColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
HColumnDescriptor descriptor)
Called prior to modifying a column family's attributes.
|
void |
preModifyColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
HColumnDescriptor descriptor)
Called prior to modifying a column family's attributes.
|
void |
preModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
NamespaceDescriptor ns)
Called prior to modifying a namespace's properties.
|
void |
preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
HTableDescriptor htd)
Called prior to modifying a table's properties.
|
void |
preModifyTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
HTableDescriptor htd)
Called prior to modifying a table's properties.
|
void |
preMove(ObserverContext<MasterCoprocessorEnvironment> ctx,
HRegionInfo region,
ServerName srcServer,
ServerName destServer)
Called prior to moving a given region from one region server to another.
|
void |
preMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
Set<HostAndPort> servers,
String targetGroup)
Called before servers are moved to target region server group
|
void |
preMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
Set<TableName> tables,
String targetGroup)
Called before tables are moved to target region server group
|
void |
preRegionOffline(ObserverContext<MasterCoprocessorEnvironment> ctx,
HRegionInfo regionInfo)
Called prior to marking a given region as offline.
|
void |
preRemoveRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
String name)
Called before a region server group is removed
|
void |
preRestoreSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
HBaseProtos.SnapshotDescription snapshot,
HTableDescriptor hTableDescriptor)
Called before a snapshot is restored.
|
void |
preSetNamespaceQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
String namespace,
QuotaProtos.Quotas quotas)
Called before the quota for the namespace is stored.
|
void |
preSetTableQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
QuotaProtos.Quotas quotas)
Called before the quota for the table is stored.
|
void |
preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
String userName,
QuotaProtos.Quotas quotas)
Called before the quota for the user is stored.
|
void |
preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
String userName,
String namespace,
QuotaProtos.Quotas quotas)
Called before the quota for the user on the specified namespace is stored.
|
void |
preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
String userName,
TableName tableName,
QuotaProtos.Quotas quotas)
Called before the quota for the user on the specified table is stored.
|
void |
preShutdown(ObserverContext<MasterCoprocessorEnvironment> ctx)
Called prior to shutting down the full HBase cluster, including this
HMaster process. |
void |
preSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
HBaseProtos.SnapshotDescription snapshot,
HTableDescriptor hTableDescriptor)
Called before a new snapshot is taken.
|
void |
preStopMaster(ObserverContext<MasterCoprocessorEnvironment> ctx)
Called immediately prior to stopping this
HMaster process. |
void |
preTableFlush(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called before the table memstore is flushed to disk.
|
void |
preTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called before
HMaster truncates a
table. |
void |
preTruncateTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName)
Called before
HMaster truncates a
table. |
void |
preUnassign(ObserverContext<MasterCoprocessorEnvironment> ctx,
HRegionInfo regionInfo,
boolean force)
Called prior to unassigning a given region.
|
start, stopvoid preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx, HTableDescriptor desc, HRegionInfo[] regions) throws IOException
HMaster. Called as part of create
table RPC call.
It can't bypass the default action, e.g., ctx.bypass() won't have effect.ctx - the environment to interact with the framework and masterdesc - the HTableDescriptor for the tableregions - the initial regions created for the tableIOExceptionvoid postCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx, HTableDescriptor desc, HRegionInfo[] regions) throws IOException
ctx - the environment to interact with the framework and masterdesc - the HTableDescriptor for the tableregions - the initial regions created for the tableIOExceptionvoid preCreateTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx, HTableDescriptor desc, HRegionInfo[] regions) throws IOException
HMaster. Called as part of create
table handler and it is async to the create RPC call.
It can't bypass the default action, e.g., ctx.bypass() won't have effect.ctx - the environment to interact with the framework and masterdesc - the HTableDescriptor for the tableregions - the initial regions created for the tableIOExceptionvoid postCreateTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx, HTableDescriptor desc, HRegionInfo[] regions) throws IOException
ctx - the environment to interact with the framework and masterdesc - the HTableDescriptor for the tableregions - the initial regions created for the tableIOExceptionvoid preDeleteTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
HMaster deletes a
table. Called as part of delete table RPC call.
It can't bypass the default action, e.g., ctx.bypass() won't have effect.ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptionvoid postDeleteTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptionvoid preDeleteTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
HMaster deletes a
table. Called as part of delete table handler and
it is async to the delete RPC call.
It can't bypass the default action, e.g., ctx.bypass() won't have effect.ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptionvoid postDeleteTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
HMaster deletes a
table. Called as part of delete table handler and it is async to the
delete RPC call.
It can't bypass the default action, e.g., ctx.bypass() won't have effect.ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptionvoid preTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
HMaster truncates a
table. Called as part of truncate table RPC call.
It can't bypass the default action, e.g., ctx.bypass() won't have effect.ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptionvoid postTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptionvoid preTruncateTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
HMaster truncates a
table. Called as part of truncate table handler and it is sync
to the truncate RPC call.
It can't bypass the default action, e.g., ctx.bypass() won't have effect.ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptionvoid postTruncateTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
HMaster truncates a
table. Called as part of truncate table handler and it is sync to the
truncate RPC call.
It can't bypass the default action, e.g., ctx.bypass() won't have effect.ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptionvoid preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, HTableDescriptor htd) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablehtd - the HTableDescriptorIOExceptionvoid postModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, HTableDescriptor htd) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablehtd - the HTableDescriptorIOExceptionvoid preModifyTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, HTableDescriptor htd) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablehtd - the HTableDescriptorIOExceptionvoid postModifyTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, HTableDescriptor htd) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablehtd - the HTableDescriptorIOExceptionvoid preAddColumn(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, HColumnDescriptor column) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablecolumn - the HColumnDescriptorIOExceptionvoid postAddColumn(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, HColumnDescriptor column) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablecolumn - the HColumnDescriptorIOExceptionvoid preAddColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, HColumnDescriptor column) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablecolumn - the HColumnDescriptorIOExceptionvoid postAddColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, HColumnDescriptor column) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablecolumn - the HColumnDescriptorIOExceptionvoid preModifyColumn(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, HColumnDescriptor descriptor) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tabledescriptor - the HColumnDescriptorIOExceptionvoid postModifyColumn(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, HColumnDescriptor descriptor) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tabledescriptor - the HColumnDescriptorIOExceptionvoid preModifyColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, HColumnDescriptor descriptor) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tabledescriptor - the HColumnDescriptorIOExceptionvoid postModifyColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, HColumnDescriptor descriptor) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tabledescriptor - the HColumnDescriptorIOExceptionvoid preDeleteColumn(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, byte[] c) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablec - the columnIOExceptionvoid postDeleteColumn(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, byte[] c) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablec - the columnIOExceptionvoid preDeleteColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, byte[] c) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablec - the columnIOExceptionvoid postDeleteColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, byte[] c) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablec - the columnIOExceptionvoid preEnableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptionvoid postEnableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptionvoid preEnableTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptionvoid postEnableTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptionvoid preDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptionvoid postDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptionvoid preDisableTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptionvoid postDisableTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptionvoid preMove(ObserverContext<MasterCoprocessorEnvironment> ctx, HRegionInfo region, ServerName srcServer, ServerName destServer) throws IOException
ctx - the environment to interact with the framework and masterregion - the HRegionInfosrcServer - the source ServerNamedestServer - the destination ServerNameIOExceptionvoid postMove(ObserverContext<MasterCoprocessorEnvironment> ctx, HRegionInfo region, ServerName srcServer, ServerName destServer) throws IOException
ctx - the environment to interact with the framework and masterregion - the HRegionInfosrcServer - the source ServerNamedestServer - the destination ServerNameIOExceptionvoid preAbortProcedure(ObserverContext<MasterCoprocessorEnvironment> ctx, ProcedureExecutor<MasterProcedureEnv> procEnv, long procId) throws IOException
ctx - the environment to interact with the framework and masterIOExceptionvoid postAbortProcedure(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
ctx - the environment to interact with the framework and masterIOExceptionvoid preListProcedures(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
ctx - the environment to interact with the framework and masterIOExceptionvoid postListProcedures(ObserverContext<MasterCoprocessorEnvironment> ctx, List<ProcedureInfo> procInfoList) throws IOException
ctx - the environment to interact with the framework and masterprocInfoList - the list of procedures about to be returnedIOExceptionvoid preAssign(ObserverContext<MasterCoprocessorEnvironment> ctx, HRegionInfo regionInfo) throws IOException
ctx - the environment to interact with the framework and masterregionInfo - the regionInfo of the regionIOExceptionvoid postAssign(ObserverContext<MasterCoprocessorEnvironment> ctx, HRegionInfo regionInfo) throws IOException
ctx - the environment to interact with the framework and masterregionInfo - the regionInfo of the regionIOExceptionvoid preUnassign(ObserverContext<MasterCoprocessorEnvironment> ctx, HRegionInfo regionInfo, boolean force) throws IOException
ctx - the environment to interact with the framework and masterregionInfo - force - whether to force unassignment or notIOExceptionvoid postUnassign(ObserverContext<MasterCoprocessorEnvironment> ctx, HRegionInfo regionInfo, boolean force) throws IOException
ctx - the environment to interact with the framework and masterregionInfo - force - whether to force unassignment or notIOExceptionvoid preRegionOffline(ObserverContext<MasterCoprocessorEnvironment> ctx, HRegionInfo regionInfo) throws IOException
ctx.bypass() will not have any
impact on this hook.ctx - the environment to interact with the framework and masterregionInfo - IOExceptionvoid postRegionOffline(ObserverContext<MasterCoprocessorEnvironment> ctx, HRegionInfo regionInfo) throws IOException
ctx - the environment to interact with the framework and masterregionInfo - IOExceptionvoid preBalance(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
ctx - the environment to interact with the framework and masterIOExceptionvoid 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 assignmentIOExceptionboolean preBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean newValue) throws IOException
ctx - the coprocessor instance's environmentnewValue - the new flag value submitted in the callIOExceptionvoid 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 valueIOExceptionvoid preShutdown(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
HMaster process.IOExceptionvoid preStopMaster(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
HMaster process.IOExceptionvoid postStartMaster(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
IOExceptionvoid preMasterInitialization(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
HMaster process.IOExceptionvoid preSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, HBaseProtos.SnapshotDescription snapshot, HTableDescriptor hTableDescriptor) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor for the snapshothTableDescriptor - the hTableDescriptor of the table to snapshotIOExceptionvoid postSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, HBaseProtos.SnapshotDescription snapshot, HTableDescriptor hTableDescriptor) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor for the snapshothTableDescriptor - the hTableDescriptor of the table to snapshotIOExceptionvoid preListSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, HBaseProtos.SnapshotDescription snapshot) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor of the snapshot to listIOExceptionvoid postListSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, HBaseProtos.SnapshotDescription snapshot) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor of the snapshot to listIOExceptionvoid preCloneSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, HBaseProtos.SnapshotDescription snapshot, HTableDescriptor hTableDescriptor) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor for the snapshothTableDescriptor - the hTableDescriptor of the table to createIOExceptionvoid postCloneSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, HBaseProtos.SnapshotDescription snapshot, HTableDescriptor hTableDescriptor) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor for the snapshothTableDescriptor - the hTableDescriptor of the table to createIOExceptionvoid preRestoreSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, HBaseProtos.SnapshotDescription snapshot, HTableDescriptor hTableDescriptor) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor for the snapshothTableDescriptor - the hTableDescriptor of the table to restoreIOExceptionvoid postRestoreSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, HBaseProtos.SnapshotDescription snapshot, HTableDescriptor hTableDescriptor) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor for the snapshothTableDescriptor - the hTableDescriptor of the table to restoreIOExceptionvoid preDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, HBaseProtos.SnapshotDescription snapshot) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor of the snapshot to deleteIOExceptionvoid postDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, HBaseProtos.SnapshotDescription snapshot) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor of the snapshot to deleteIOException@Deprecated void preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableName> tableNamesList, List<HTableDescriptor> descriptors) 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 if bypassingIOException@Deprecated void postGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<HTableDescriptor> descriptors) throws IOException
ctx - the environment to interact with the framework and masterdescriptors - the list of descriptors about to be returnedIOExceptionvoid preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableName> tableNamesList, List<HTableDescriptor> 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 if bypassingregex - regular expression used for filtering the table namesIOExceptionvoid postGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableName> tableNamesList, List<HTableDescriptor> 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 namesIOExceptionvoid preGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx, List<HTableDescriptor> 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 if bypassingregex - regular expression used for filtering the table namesIOExceptionvoid postGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx, List<HTableDescriptor> 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 namesIOExceptionvoid preCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor ns) throws IOException
HMaster.
It can't bypass the default action, e.g., ctx.bypass() won't have effect.ctx - the environment to interact with the framework and masterns - the NamespaceDescriptor for the tableIOExceptionvoid postCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor ns) throws IOException
ctx - the environment to interact with the framework and masterns - the NamespaceDescriptor for the tableIOExceptionvoid preDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) throws IOException
HMaster deletes a
namespace
It can't bypass the default action, e.g., ctx.bypass() won't have effect.ctx - the environment to interact with the framework and masternamespace - the name of the namespaceIOExceptionvoid postDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) throws IOException
ctx - the environment to interact with the framework and masternamespace - the name of the namespaceIOExceptionvoid preModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor ns) throws IOException
ctx - the environment to interact with the framework and masterns - the NamespaceDescriptorIOExceptionvoid postModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor ns) throws IOException
ctx - the environment to interact with the framework and masterns - the NamespaceDescriptorIOExceptionvoid preGetNamespaceDescriptor(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) throws IOException
ctx - the environment to interact with the framework and masternamespace - the name of the namespaceIOExceptionvoid postGetNamespaceDescriptor(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor ns) throws IOException
ctx - the environment to interact with the framework and masterns - the NamespaceDescriptorIOExceptionvoid 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 if bypassingIOExceptionvoid 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 returnedIOExceptionvoid preTableFlush(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptionvoid postTableFlush(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptionvoid preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, QuotaProtos.Quotas quotas) throws IOException
ctx - the environment to interact with the framework and masteruserName - the name of userquotas - the quota settingsIOExceptionvoid postSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, QuotaProtos.Quotas quotas) throws IOException
ctx - the environment to interact with the framework and masteruserName - the name of userquotas - the quota settingsIOExceptionvoid preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, TableName tableName, QuotaProtos.Quotas quotas) throws IOException
ctx - the environment to interact with the framework and masteruserName - the name of usertableName - the name of the tablequotas - the quota settingsIOExceptionvoid postSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, TableName tableName, QuotaProtos.Quotas quotas) throws IOException
ctx - the environment to interact with the framework and masteruserName - the name of usertableName - the name of the tablequotas - the quota settingsIOExceptionvoid preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, String namespace, QuotaProtos.Quotas quotas) throws IOException
ctx - the environment to interact with the framework and masteruserName - the name of usernamespace - the name of the namespacequotas - the quota settingsIOExceptionvoid postSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, String namespace, QuotaProtos.Quotas quotas) throws IOException
ctx - the environment to interact with the framework and masteruserName - the name of usernamespace - the name of the namespacequotas - the quota settingsIOExceptionvoid preSetTableQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, QuotaProtos.Quotas quotas) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablequotas - the quota settingsIOExceptionvoid postSetTableQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, QuotaProtos.Quotas quotas) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablequotas - the quota settingsIOExceptionvoid preSetNamespaceQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace, QuotaProtos.Quotas quotas) throws IOException
ctx - the environment to interact with the framework and masternamespace - the name of the namespacequotas - the quota settingsIOExceptionvoid postSetNamespaceQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace, QuotaProtos.Quotas quotas) throws IOException
ctx - the environment to interact with the framework and masternamespace - the name of the namespacequotas - the quota settingsIOExceptionvoid preMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<HostAndPort> servers, String targetGroup) throws IOException
ctx - the environment to interact with the framework and masterservers - set of servers to movetargetGroup - destination groupIOException - on failurevoid postMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<HostAndPort> servers, String targetGroup) throws IOException
ctx - the environment to interact with the framework and masterservers - set of servers to movetargetGroup - name of groupIOException - on failurevoid 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 groupIOException - on failurevoid 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 groupIOException - on failurevoid preAddRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String name) throws IOException
ctx - the environment to interact with the framework and mastername - group nameIOException - on failurevoid postAddRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String name) throws IOException
ctx - the environment to interact with the framework and mastername - group nameIOException - on failurevoid preRemoveRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String name) throws IOException
ctx - the environment to interact with the framework and mastername - group nameIOException - on failurevoid postRemoveRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String name) throws IOException
ctx - the environment to interact with the framework and mastername - group nameIOException - on failurevoid preBalanceRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName) throws IOException
ctx - the environment to interact with the framework and mastergroupName - group nameIOException - on failurevoid postBalanceRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName, boolean balancerRan) throws IOException
ctx - the environment to interact with the framework and mastergroupName - group nameIOException - on failure