@InterfaceAudience.Private public class MasterCoprocessorHost extends CoprocessorHost<MasterCoprocessorHost.MasterEnvironment>
HMaster
interacts with the loaded coprocessors
through this class.Modifier and Type | Class and Description |
---|---|
private static class |
MasterCoprocessorHost.CoprocessorOperation |
private static class |
MasterCoprocessorHost.CoprocessorOperationWithResult<T> |
(package private) static class |
MasterCoprocessorHost.MasterEnvironment
Coprocessor environment extension providing access to master related
services.
|
CoprocessorHost.Environment
Modifier and Type | Field and Description |
---|---|
private static org.apache.commons.logging.Log |
LOG |
private MasterServices |
masterServices |
ABORT_ON_ERROR_KEY, abortable, conf, coprocessors, COPROCESSORS_ENABLED_CONF_KEY, DEFAULT_ABORT_ON_ERROR, DEFAULT_COPROCESSORS_ENABLED, DEFAULT_USER_COPROCESSORS_ENABLED, loadSequence, MASTER_COPROCESSOR_CONF_KEY, pathPrefix, REGION_COPROCESSOR_CONF_KEY, REGIONSERVER_COPROCESSOR_CONF_KEY, USER_COPROCESSORS_ENABLED_CONF_KEY, USER_REGION_COPROCESSOR_CONF_KEY, WAL_COPROCESSOR_CONF_KEY
Constructor and Description |
---|
MasterCoprocessorHost(MasterServices services,
org.apache.hadoop.conf.Configuration conf) |
Modifier and Type | Method and Description |
---|---|
MasterCoprocessorHost.MasterEnvironment |
createEnvironment(Class<?> implClass,
Coprocessor instance,
int priority,
int seq,
org.apache.hadoop.conf.Configuration conf)
Called when a new Coprocessor class is loaded
|
private boolean |
execOperation(MasterCoprocessorHost.CoprocessorOperation ctx) |
private <T> T |
execOperationWithResult(T defaultValue,
MasterCoprocessorHost.CoprocessorOperationWithResult<T> ctx) |
private boolean |
execShutdown(MasterCoprocessorHost.CoprocessorOperation ctx)
Master coprocessor classes can be configured in any order, based on that priority is set and
chained in a sorted order.
|
void |
postAbortProcedure() |
void |
postAddColumn(TableName tableName,
HColumnDescriptor column) |
void |
postAddColumnHandler(TableName tableName,
HColumnDescriptor column) |
void |
postAssign(HRegionInfo regionInfo) |
void |
postBalance(List<RegionPlan> plans) |
void |
postBalanceSwitch(boolean oldValue,
boolean newValue) |
void |
postCloneSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot,
HTableDescriptor hTableDescriptor) |
void |
postCreateNamespace(NamespaceDescriptor ns) |
void |
postCreateTable(HTableDescriptor htd,
HRegionInfo[] regions) |
void |
postCreateTableHandler(HTableDescriptor htd,
HRegionInfo[] regions) |
void |
postDeleteColumn(TableName tableName,
byte[] c) |
void |
postDeleteColumnHandler(TableName tableName,
byte[] c) |
void |
postDeleteNamespace(String namespaceName) |
void |
postDeleteSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot) |
void |
postDeleteTable(TableName tableName) |
void |
postDeleteTableHandler(TableName tableName) |
void |
postDisableTable(TableName tableName) |
void |
postDisableTableHandler(TableName tableName) |
void |
postEnableTable(TableName tableName) |
void |
postEnableTableHandler(TableName tableName) |
void |
postGetNamespaceDescriptor(NamespaceDescriptor ns) |
void |
postGetTableDescriptors(List<HTableDescriptor> descriptors)
Deprecated.
|
void |
postGetTableDescriptors(List<TableName> tableNamesList,
List<HTableDescriptor> descriptors,
String regex) |
void |
postGetTableNames(List<HTableDescriptor> descriptors,
String regex) |
void |
postListNamespaceDescriptors(List<NamespaceDescriptor> descriptors) |
void |
postListProcedures(List<ProcedureInfo> procInfoList) |
void |
postListSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot) |
void |
postModifyColumn(TableName tableName,
HColumnDescriptor descriptor) |
void |
postModifyColumnHandler(TableName tableName,
HColumnDescriptor descriptor) |
void |
postModifyNamespace(NamespaceDescriptor ns) |
void |
postModifyTable(TableName tableName,
HTableDescriptor htd) |
void |
postModifyTableHandler(TableName tableName,
HTableDescriptor htd) |
void |
postMove(HRegionInfo region,
ServerName srcServer,
ServerName destServer) |
void |
postRegionOffline(HRegionInfo regionInfo) |
void |
postRestoreSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot,
HTableDescriptor hTableDescriptor) |
void |
postSetNamespaceQuota(String namespace,
org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas quotas) |
void |
postSetTableQuota(TableName table,
org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas quotas) |
void |
postSetUserQuota(String user,
org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas quotas) |
void |
postSetUserQuota(String user,
String namespace,
org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas quotas) |
void |
postSetUserQuota(String user,
TableName table,
org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas quotas) |
void |
postSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot,
HTableDescriptor hTableDescriptor) |
void |
postStartMaster() |
void |
postTableFlush(TableName tableName) |
void |
postTruncateTable(TableName tableName) |
void |
postTruncateTableHandler(TableName tableName) |
void |
postUnassign(HRegionInfo regionInfo,
boolean force) |
boolean |
preAbortProcedure(ProcedureExecutor<MasterProcedureEnv> procEnv,
long procId) |
boolean |
preAddColumn(TableName tableName,
HColumnDescriptor column) |
boolean |
preAddColumnHandler(TableName tableName,
HColumnDescriptor column) |
boolean |
preAssign(HRegionInfo regionInfo) |
boolean |
preBalance() |
boolean |
preBalanceSwitch(boolean b) |
void |
preCloneSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot,
HTableDescriptor hTableDescriptor) |
boolean |
preCreateNamespace(NamespaceDescriptor ns) |
void |
preCreateTable(HTableDescriptor htd,
HRegionInfo[] regions) |
void |
preCreateTableHandler(HTableDescriptor htd,
HRegionInfo[] regions) |
boolean |
preDeleteColumn(TableName tableName,
byte[] c) |
boolean |
preDeleteColumnHandler(TableName tableName,
byte[] c) |
boolean |
preDeleteNamespace(String namespaceName) |
void |
preDeleteSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot) |
void |
preDeleteTable(TableName tableName) |
void |
preDeleteTableHandler(TableName tableName) |
void |
preDisableTable(TableName tableName) |
void |
preDisableTableHandler(TableName tableName) |
void |
preEnableTable(TableName tableName) |
void |
preEnableTableHandler(TableName tableName) |
void |
preGetNamespaceDescriptor(String namespaceName) |
boolean |
preGetTableDescriptors(List<TableName> tableNamesList,
List<HTableDescriptor> descriptors)
Deprecated.
|
boolean |
preGetTableDescriptors(List<TableName> tableNamesList,
List<HTableDescriptor> descriptors,
String regex) |
boolean |
preGetTableNames(List<HTableDescriptor> descriptors,
String regex) |
boolean |
preListNamespaceDescriptors(List<NamespaceDescriptor> descriptors) |
boolean |
preListProcedures() |
void |
preListSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot) |
void |
preMasterInitialization() |
boolean |
preModifyColumn(TableName tableName,
HColumnDescriptor descriptor) |
boolean |
preModifyColumnHandler(TableName tableName,
HColumnDescriptor descriptor) |
boolean |
preModifyNamespace(NamespaceDescriptor ns) |
void |
preModifyTable(TableName tableName,
HTableDescriptor htd) |
void |
preModifyTableHandler(TableName tableName,
HTableDescriptor htd) |
boolean |
preMove(HRegionInfo region,
ServerName srcServer,
ServerName destServer) |
void |
preRegionOffline(HRegionInfo regionInfo) |
void |
preRestoreSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot,
HTableDescriptor hTableDescriptor) |
void |
preSetNamespaceQuota(String namespace,
org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas quotas) |
void |
preSetTableQuota(TableName table,
org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas quotas) |
void |
preSetUserQuota(String user,
org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas quotas) |
void |
preSetUserQuota(String user,
String namespace,
org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas quotas) |
void |
preSetUserQuota(String user,
TableName table,
org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas quotas) |
void |
preShutdown() |
void |
preSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot,
HTableDescriptor hTableDescriptor) |
void |
preStopMaster() |
void |
preTableFlush(TableName tableName) |
void |
preTruncateTable(TableName tableName) |
void |
preTruncateTableHandler(TableName tableName) |
boolean |
preUnassign(HRegionInfo regionInfo,
boolean force) |
abortServer, abortServer, findCoprocessor, findCoprocessorEnvironment, findCoprocessors, getCoprocessors, getLoadedCoprocessors, handleCoprocessorThrowable, legacyWarning, load, load, loadInstance, loadSystemCoprocessors, shutdown, useLegacyMethod
private static final org.apache.commons.logging.Log LOG
private MasterServices masterServices
public MasterCoprocessorHost(MasterServices services, org.apache.hadoop.conf.Configuration conf)
public MasterCoprocessorHost.MasterEnvironment createEnvironment(Class<?> implClass, Coprocessor instance, int priority, int seq, org.apache.hadoop.conf.Configuration conf)
CoprocessorHost
createEnvironment
in class CoprocessorHost<MasterCoprocessorHost.MasterEnvironment>
public boolean preCreateNamespace(NamespaceDescriptor ns) throws IOException
IOException
public void postCreateNamespace(NamespaceDescriptor ns) throws IOException
IOException
public boolean preDeleteNamespace(String namespaceName) throws IOException
IOException
public void postDeleteNamespace(String namespaceName) throws IOException
IOException
public boolean preModifyNamespace(NamespaceDescriptor ns) throws IOException
IOException
public void postModifyNamespace(NamespaceDescriptor ns) throws IOException
IOException
public void preGetNamespaceDescriptor(String namespaceName) throws IOException
IOException
public void postGetNamespaceDescriptor(NamespaceDescriptor ns) throws IOException
IOException
public boolean preListNamespaceDescriptors(List<NamespaceDescriptor> descriptors) throws IOException
IOException
public void postListNamespaceDescriptors(List<NamespaceDescriptor> descriptors) throws IOException
IOException
public void preCreateTable(HTableDescriptor htd, HRegionInfo[] regions) throws IOException
IOException
public void postCreateTable(HTableDescriptor htd, HRegionInfo[] regions) throws IOException
IOException
public void preCreateTableHandler(HTableDescriptor htd, HRegionInfo[] regions) throws IOException
IOException
public void postCreateTableHandler(HTableDescriptor htd, HRegionInfo[] regions) throws IOException
IOException
public void preDeleteTable(TableName tableName) throws IOException
IOException
public void postDeleteTable(TableName tableName) throws IOException
IOException
public void preDeleteTableHandler(TableName tableName) throws IOException
IOException
public void postDeleteTableHandler(TableName tableName) throws IOException
IOException
public void preTruncateTable(TableName tableName) throws IOException
IOException
public void postTruncateTable(TableName tableName) throws IOException
IOException
public void preTruncateTableHandler(TableName tableName) throws IOException
IOException
public void postTruncateTableHandler(TableName tableName) throws IOException
IOException
public void preModifyTable(TableName tableName, HTableDescriptor htd) throws IOException
IOException
public void postModifyTable(TableName tableName, HTableDescriptor htd) throws IOException
IOException
public void preModifyTableHandler(TableName tableName, HTableDescriptor htd) throws IOException
IOException
public void postModifyTableHandler(TableName tableName, HTableDescriptor htd) throws IOException
IOException
public boolean preAddColumn(TableName tableName, HColumnDescriptor column) throws IOException
IOException
public void postAddColumn(TableName tableName, HColumnDescriptor column) throws IOException
IOException
public boolean preAddColumnHandler(TableName tableName, HColumnDescriptor column) throws IOException
IOException
public void postAddColumnHandler(TableName tableName, HColumnDescriptor column) throws IOException
IOException
public boolean preModifyColumn(TableName tableName, HColumnDescriptor descriptor) throws IOException
IOException
public void postModifyColumn(TableName tableName, HColumnDescriptor descriptor) throws IOException
IOException
public boolean preModifyColumnHandler(TableName tableName, HColumnDescriptor descriptor) throws IOException
IOException
public void postModifyColumnHandler(TableName tableName, HColumnDescriptor descriptor) throws IOException
IOException
public boolean preDeleteColumn(TableName tableName, byte[] c) throws IOException
IOException
public void postDeleteColumn(TableName tableName, byte[] c) throws IOException
IOException
public boolean preDeleteColumnHandler(TableName tableName, byte[] c) throws IOException
IOException
public void postDeleteColumnHandler(TableName tableName, byte[] c) throws IOException
IOException
public void preEnableTable(TableName tableName) throws IOException
IOException
public void postEnableTable(TableName tableName) throws IOException
IOException
public void preEnableTableHandler(TableName tableName) throws IOException
IOException
public void postEnableTableHandler(TableName tableName) throws IOException
IOException
public void preDisableTable(TableName tableName) throws IOException
IOException
public void postDisableTable(TableName tableName) throws IOException
IOException
public void preDisableTableHandler(TableName tableName) throws IOException
IOException
public void postDisableTableHandler(TableName tableName) throws IOException
IOException
public boolean preAbortProcedure(ProcedureExecutor<MasterProcedureEnv> procEnv, long procId) throws IOException
IOException
public void postAbortProcedure() throws IOException
IOException
public boolean preListProcedures() throws IOException
IOException
public void postListProcedures(List<ProcedureInfo> procInfoList) throws IOException
IOException
public boolean preMove(HRegionInfo region, ServerName srcServer, ServerName destServer) throws IOException
IOException
public void postMove(HRegionInfo region, ServerName srcServer, ServerName destServer) throws IOException
IOException
public boolean preAssign(HRegionInfo regionInfo) throws IOException
IOException
public void postAssign(HRegionInfo regionInfo) throws IOException
IOException
public boolean preUnassign(HRegionInfo regionInfo, boolean force) throws IOException
IOException
public void postUnassign(HRegionInfo regionInfo, boolean force) throws IOException
IOException
public void preRegionOffline(HRegionInfo regionInfo) throws IOException
IOException
public void postRegionOffline(HRegionInfo regionInfo) throws IOException
IOException
public boolean preBalance() throws IOException
IOException
public void postBalance(List<RegionPlan> plans) throws IOException
IOException
public boolean preBalanceSwitch(boolean b) throws IOException
IOException
public void postBalanceSwitch(boolean oldValue, boolean newValue) throws IOException
IOException
public void preShutdown() throws IOException
IOException
public void preStopMaster() throws IOException
IOException
public void preMasterInitialization() throws IOException
IOException
public void postStartMaster() throws IOException
IOException
public void preSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot, HTableDescriptor hTableDescriptor) throws IOException
IOException
public void postSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot, HTableDescriptor hTableDescriptor) throws IOException
IOException
public void preListSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot) throws IOException
IOException
public void postListSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot) throws IOException
IOException
public void preCloneSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot, HTableDescriptor hTableDescriptor) throws IOException
IOException
public void postCloneSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot, HTableDescriptor hTableDescriptor) throws IOException
IOException
public void preRestoreSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot, HTableDescriptor hTableDescriptor) throws IOException
IOException
public void postRestoreSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot, HTableDescriptor hTableDescriptor) throws IOException
IOException
public void preDeleteSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot) throws IOException
IOException
public void postDeleteSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot) throws IOException
IOException
@Deprecated public boolean preGetTableDescriptors(List<TableName> tableNamesList, List<HTableDescriptor> descriptors) throws IOException
IOException
@Deprecated public void postGetTableDescriptors(List<HTableDescriptor> descriptors) throws IOException
IOException
public boolean preGetTableDescriptors(List<TableName> tableNamesList, List<HTableDescriptor> descriptors, String regex) throws IOException
IOException
public void postGetTableDescriptors(List<TableName> tableNamesList, List<HTableDescriptor> descriptors, String regex) throws IOException
IOException
public boolean preGetTableNames(List<HTableDescriptor> descriptors, String regex) throws IOException
IOException
public void postGetTableNames(List<HTableDescriptor> descriptors, String regex) throws IOException
IOException
public void preTableFlush(TableName tableName) throws IOException
IOException
public void postTableFlush(TableName tableName) throws IOException
IOException
public void preSetUserQuota(String user, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas quotas) throws IOException
IOException
public void postSetUserQuota(String user, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas quotas) throws IOException
IOException
public void preSetUserQuota(String user, TableName table, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas quotas) throws IOException
IOException
public void postSetUserQuota(String user, TableName table, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas quotas) throws IOException
IOException
public void preSetUserQuota(String user, String namespace, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas quotas) throws IOException
IOException
public void postSetUserQuota(String user, String namespace, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas quotas) throws IOException
IOException
public void preSetTableQuota(TableName table, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas quotas) throws IOException
IOException
public void postSetTableQuota(TableName table, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas quotas) throws IOException
IOException
public void preSetNamespaceQuota(String namespace, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas quotas) throws IOException
IOException
public void postSetNamespaceQuota(String namespace, org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas quotas) throws IOException
IOException
private <T> T execOperationWithResult(T defaultValue, MasterCoprocessorHost.CoprocessorOperationWithResult<T> ctx) throws IOException
IOException
private boolean execOperation(MasterCoprocessorHost.CoprocessorOperation ctx) throws IOException
IOException
private boolean execShutdown(MasterCoprocessorHost.CoprocessorOperation ctx) throws IOException
ctx
- CoprocessorOperationIOException
Copyright © 2007–2019 The Apache Software Foundation. All rights reserved.