Class AccessController
- All Implemented Interfaces:
Coprocessor
,BulkLoadObserver
,EndpointObserver
,MasterCoprocessor
,MasterObserver
,RegionCoprocessor
,RegionObserver
,RegionServerCoprocessor
,RegionServerObserver
,org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.AccessControlService.Interface
AccessController
performs authorization checks for HBase operations based on:
- the identity of the user performing the operation
- the scope over which the operation is performed, in increasing specificity: global, table, column family, or qualifier
- the type of action being performed (as mapped to
Permission.Action
values)
If the authorization check fails, an AccessDeniedException
will be thrown for the
operation.
To perform authorization checks, AccessController
relies on the RpcServerEngine being
loaded to provide the user identities for remote requests.
The access control lists used for authorization can be manipulated via the exposed
AccessControlProtos.AccessControlService
Interface implementation, and the associated grant
,
revoke
, and user_permission
HBase shell commands.
-
Nested Class Summary
Nested classes/interfaces inherited from interface org.apache.hadoop.hbase.Coprocessor
Coprocessor.State
Nested classes/interfaces inherited from interface org.apache.hadoop.hbase.coprocessor.RegionObserver
RegionObserver.MutationType
-
Field Summary
Modifier and TypeFieldDescriptionprivate AccessChecker
private boolean
flags if we are running on a region of the _acl_ tableprivate boolean
if the ACL table is available, only relevant in the masterprivate static final org.slf4j.Logger
private boolean
if we are active, usually false, only true if "hbase.security.authorization" has been set to true in site configurationprivate boolean
if we are able to support cell ACLsprivate static final String
private boolean
if we should terminate access checks early as soon as table or CF grants allow access; pre-0.98 compatible behaviorprivate boolean
if we have been successfully initializedprivate static final org.slf4j.Logger
private RegionCoprocessorEnvironment
defined only for Endpoint implementation, so it can have way to access region servicesprivate Map<InternalScanner,
String> Mapping of scanner instances to the user who created themprivate boolean
if we should check EXEC permissionsprivate Map<TableName,
List<UserPermission>> private static final String
private static final byte[]
private UserProvider
Provider for mapping principal names to Usersprivate ZKPermissionWatcher
Fields inherited from interface org.apache.hadoop.hbase.Coprocessor
PRIORITY_HIGHEST, PRIORITY_LOWEST, PRIORITY_SYSTEM, PRIORITY_USER, VERSION
-
Constructor Summary
-
Method Summary
Modifier and TypeMethodDescriptionprivate static void
addCellPermissions
(byte[] perms, Map<byte[], List<Cell>> familyMap) private boolean
checkCoveringPermission
(User user, AccessController.OpType request, RegionCoprocessorEnvironment e, byte[] row, Map<byte[], ? extends Collection<?>> familyMap, long opTs, Permission.Action... actions) Determine if cell ACLs covered by the operation grant access.private void
checkForReservedTagPresence
(User user, Mutation m) void
checkLockPermissions
(ObserverContext<?> ctx, String namespace, TableName tableName, RegionInfo[] regionInfos, String reason) void
checkPermissions
(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.CheckPermissionsRequest request, org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.CheckPermissionsResponse> done) Deprecated.since 2.2.0 and will be removed 4.0.0.private void
checkSystemOrSuperUser
(User activeUser) private static void
createACLTable
(Admin admin) Create the ACL tableprivate Cell
createNewCellWithTags
(Mutation mutation, Cell oldCell, Cell newCell) private User
getActiveUser
(ObserverContext<?> ctx) Returns the active user to which authorization checks should be applied.private Region
Observer/Service GettersIterable<org.apache.hbase.thirdparty.com.google.protobuf.Service>
Coprocessor endpoints providing protobuf services should override this method.private TableName
private TableName
getTableName
(Region region) void
getUserPermissions
(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsRequest request, org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsResponse> done) Deprecated.since 2.2.0 and will be removed in 4.0.0.void
grant
(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GrantRequest request, org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GrantResponse> done) Deprecated.since 2.2.0 and will be removed in 4.0.0.private boolean
hasFamilyQualifierPermission
(User user, Permission.Action perm, RegionCoprocessorEnvironment env, Map<byte[], ? extends Collection<byte[]>> familyMap) Returnstrue
if the current user is allowed the given action over at least one of the column qualifiers in the given column families.void
hasPermission
(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.HasPermissionRequest request, org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.HasPermissionResponse> done) Deprecated.since 2.2.0 and will be removed in 4.0.0.private void
private void
internalPreRead
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Query query, AccessController.OpType opType) static boolean
isCellAuthorizationSupported
(org.apache.hadoop.conf.Configuration conf) private Map<byte[],
? extends Collection<byte[]>> makeFamilyMap
(byte[] family, byte[] qualifier) private AuthResult
permissionGranted
(AccessController.OpType opType, User user, RegionCoprocessorEnvironment e, Map<byte[], ? extends Collection<?>> families, Permission.Action... actions) Check the current user for authorization to perform a specific action against the given set of row data.void
Called after a abortProcedure request has been processed.postAppendBeforeWAL
(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, Mutation mutation, List<Pair<Cell, Cell>> cellPairs) Called after a list of new cells has been created during an append operation, but before they are committed to the WAL or memstore.void
postCompletedCreateTableAction
(ObserverContext<MasterCoprocessorEnvironment> c, TableDescriptor desc, RegionInfo[] regions) Called after the createTable operation has been requested.postCreateReplicationEndPoint
(ObserverContext<RegionServerCoprocessorEnvironment> ctx, ReplicationEndpoint endpoint) This will be called after the replication endpoint is instantiated.void
postDelete
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Delete delete, WALEdit edit, Durability durability) Called after the client deletes a value.void
postDeleteNamespace
(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) Called after the deleteNamespace operation has been requested.void
postDeleteTable
(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName) Called after the deleteTable operation has been requested.void
postEndpointInvocation
(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, org.apache.hbase.thirdparty.com.google.protobuf.Service service, String methodName, org.apache.hbase.thirdparty.com.google.protobuf.Message request, org.apache.hbase.thirdparty.com.google.protobuf.Message.Builder responseBuilder) Called after an Endpoint service method is invoked.void
postGetTableDescriptors
(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableName> tableNamesList, List<TableDescriptor> descriptors, String regex) Called after a getTableDescriptors request has been processed.void
postGetTableNames
(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableDescriptor> descriptors, String regex) Called after a getTableNames request has been processed.postIncrementBeforeWAL
(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, Mutation mutation, List<Pair<Cell, Cell>> cellPairs) Called after a list of new cells has been created during an increment operation, but before they are committed to the WAL or memstore.void
postListNamespaceDescriptors
(ObserverContext<MasterCoprocessorEnvironment> ctx, List<NamespaceDescriptor> descriptors) Called after a listNamespaceDescriptors request has been processed.void
postListNamespaces
(ObserverContext<MasterCoprocessorEnvironment> ctx, List<String> namespaces) Called after a listNamespaces request has been processed.void
postModifyTable
(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName, TableDescriptor oldDesc, TableDescriptor currentDesc) Called after the modifyTable operation has been requested.void
postOpen
(ObserverContext<? extends RegionCoprocessorEnvironment> c) Called after the region is reported as open to the master.void
postPut
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Put put, WALEdit edit, Durability durability) Called after the client stores a value.void
This will be called after executing user request to roll a region server WAL.void
postScannerClose
(ObserverContext<? extends RegionCoprocessorEnvironment> c, InternalScanner s) Called after the client closes a scanner.postScannerOpen
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Scan scan, RegionScanner s) Called after the client opens a new scanner.void
Called immediately after an active master instance has completed initialization.void
postTruncateTable
(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) Called after the truncateTable operation has been requested.void
preAbortProcedure
(ObserverContext<MasterCoprocessorEnvironment> ctx, long procId) Called before a abortProcedure request has been processed.void
preAddReplicationPeer
(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId, ReplicationPeerConfig peerConfig) Called before add a replication peervoid
Called before a new region server group is addedpreAppend
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Append append) Called before Append.void
preAssign
(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfo) Called prior to assigning a specific region.void
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, BalanceRequest request) Called before a region server group is removedvoid
preBalanceSwitch
(ObserverContext<MasterCoprocessorEnvironment> c, boolean newValue) Called prior to modifying the flag used to enable/disable region balancing.void
preBatchMutate
(ObserverContext<? extends RegionCoprocessorEnvironment> c, MiniBatchOperationInProgress<Mutation> miniBatchOp) This will be called for every batch mutation operation happening at the server.void
preBulkLoadHFile
(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, List<Pair<byte[], String>> familyPaths) Verifies user has CREATE or ADMIN privileges on the Column Families involved in the bulkLoadHFile request.boolean
preCheckAndDelete
(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Delete delete, boolean result) Called before checkAndDelete.boolean
preCheckAndDeleteAfterRowLock
(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Delete delete, boolean result) Called before checkAndDelete but after acquiring rowock.boolean
preCheckAndPut
(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Put put, boolean result) Called before checkAndPut.boolean
preCheckAndPutAfterRowLock
(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator opp, ByteArrayComparable comparator, Put put, boolean result) Called before checkAndPut but after acquiring rowlock.void
Authorization security check for SecureBulkLoadProtocol.cleanupBulkLoad()void
This will be called before clearing compaction queuesvoid
Called before clear dead region servers.void
Called before clearing the block caches for one or more regionsvoid
preCloneSnapshot
(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor hTableDescriptor) Called before a snapshot is cloned.void
preClose
(ObserverContext<? extends RegionCoprocessorEnvironment> c, boolean abortRequested) Called before the region is reported as closed to the master.preCompact
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store, InternalScanner scanner, ScanType scanType, CompactionLifeCycleTracker tracker, CompactionRequest request) Called prior to writing theStoreFile
s selected for compaction into a newStoreFile
.void
Called before a new namespace is created byHMaster
.void
preCreateTable
(ObserverContext<MasterCoprocessorEnvironment> c, TableDescriptor desc, RegionInfo[] regions) Observer implementationsvoid
preDecommissionRegionServers
(ObserverContext<MasterCoprocessorEnvironment> ctx, List<ServerName> servers, boolean offload) Called before decommission region servers.void
preDelete
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Delete delete, WALEdit edit, Durability durability) Called before the client deletes a value.void
preDeleteNamespace
(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) Called beforeHMaster
deletes a namespacevoid
Called before a snapshot is deleted.void
preDeleteTable
(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName) Called beforeHMaster
deletes a table.void
Called before disable a replication peervoid
preDisableTable
(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName) Called prior to disabling a table.void
Called before enable a replication peervoid
preEnableTable
(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName) Called prior to enabling a table.org.apache.hbase.thirdparty.com.google.protobuf.Message
preEndpointInvocation
(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, org.apache.hbase.thirdparty.com.google.protobuf.Service service, String methodName, org.apache.hbase.thirdparty.com.google.protobuf.Message request) Called before an Endpoint service method is invoked.void
This will be called before executing proceduresboolean
preExists
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Get get, boolean exists) Called before the client tests for existence using a Get.void
preFlush
(ObserverContext<? extends RegionCoprocessorEnvironment> c, FlushLifeCycleTracker tracker) Called before the memstore is flushed to disk.void
preGetConfiguredNamespacesAndTablesInRSGroup
(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName) Called before getting the configured namespaces and tables in the region server group.void
Called before a getLocks request has been processed.void
preGetNamespaceDescriptor
(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) Called before a getNamespaceDescriptor request has been processed.void
preGetOp
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Get get, List<Cell> result) Called before the client performs a Getvoid
Called before a getProcedures request has been processed.void
Called before get the configured ReplicationPeerConfig for the specified peervoid
preGetRSGroupInfo
(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName) Called before getting region server group info of the passed groupName.void
Called before getting region server group info of the passed server.void
preGetRSGroupInfoOfTable
(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) Called before getting region server group info of the passed tableName.void
preGetTableDescriptors
(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableName> tableNamesList, List<TableDescriptor> descriptors, String regex) Called before a getTableDescriptors request has been processed.void
preGetUserPermissions
(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, String namespace, TableName tableName, byte[] family, byte[] qualifier) Called before getting user permissions.private void
preGetUserPermissions
(User caller, String userName, String namespace, TableName tableName, byte[] family, byte[] qualifier) void
preGrant
(ObserverContext<MasterCoprocessorEnvironment> ctx, UserPermission userPermission, boolean mergeExistingPermissions) Called before granting user permissions.private void
preGrantOrRevoke
(User caller, String request, UserPermission userPermission) void
preHasUserPermissions
(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, List<Permission> permissions) private void
preHasUserPermissions
(User caller, String userName, List<Permission> permissions) preIncrement
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Increment increment) Called before Increment.void
Called before getting if is rpc throttle enabled.void
Called before list decommissioned region servers.void
Called before list replication peers.void
Called before listing region server group information.void
Called before listSnapshots request has been processed.void
preListTablesInRSGroup
(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName) Called before listing all tables in the region server group.void
preLockHeartbeat
(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, String description) Called before heartbeat to a lock.void
preMergeRegions
(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge) Called before merge regions request.preModifyColumnFamilyStoreFileTracker
(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName, byte[] family, String dstSFT) Called prior to modifying a family's store file tracker.void
preModifyNamespace
(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor currentNsDesc, NamespaceDescriptor newNsDesc) Called prior to modifying a namespace's properties.preModifyTable
(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName, TableDescriptor currentDesc, TableDescriptor newDesc) Called prior to modifying a table's properties.preModifyTableStoreFileTracker
(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName, String dstSFT) Called prior to modifying a table's store file tracker.void
preMove
(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo region, ServerName srcServer, ServerName destServer) Called prior to moving a given region from one region server to another.void
preMoveServers
(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers, String targetGroup) Called before servers are moved to target region server groupvoid
preMoveServersAndTables
(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers, Set<TableName> tables, String targetGroup) Called before servers are moved to target region server groupvoid
preMoveTables
(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<TableName> tables, String targetGroup) Called before tables are moved to target region server groupvoid
preOpen
(ObserverContext<? extends RegionCoprocessorEnvironment> c) Called before the region is reported as open to the master.void
Authorization check for SecureBulkLoadProtocol.prepareBulkLoad()void
prePut
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Put put, WALEdit edit, Durability durability) Called before the client stores a value.void
preRecommissionRegionServer
(ObserverContext<MasterCoprocessorEnvironment> ctx, ServerName server, List<byte[]> encodedRegionNames) Called before recommission region server.void
preRegionOffline
(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfo) Called prior to marking a given region as offline.void
Called before remove a replication peervoid
Called before a region server group is removedvoid
preRemoveServers
(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers) Called before servers are removed from rsgroupvoid
preRenameRSGroup
(ObserverContext<MasterCoprocessorEnvironment> ctx, String oldName, String newName) Called before rename rsgroup.void
This will be called before executing replication request to shipping log entries.void
preRequestLock
(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace, TableName tableName, RegionInfo[] regionInfos, String description) Called before new LockProcedure is queued.void
preRestoreSnapshot
(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor hTableDescriptor) Called before a snapshot is restored.void
preRevoke
(ObserverContext<MasterCoprocessorEnvironment> ctx, UserPermission userPermission) Called before revoking user permissions.void
This will be called before executing user request to roll a region server WAL.void
preScannerClose
(ObserverContext<? extends RegionCoprocessorEnvironment> c, InternalScanner s) Called before the client closes a scanner.boolean
preScannerNext
(ObserverContext<? extends RegionCoprocessorEnvironment> c, InternalScanner s, List<Result> result, int limit, boolean hasNext) Called before the client asks for the next row on a scanner.void
preScannerOpen
(ObserverContext<? extends RegionCoprocessorEnvironment> c, Scan scan) Called before the client opens a new scanner.void
preSetNamespaceQuota
(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace, GlobalQuotaSettings quotas) Called before the quota for the namespace is stored.void
preSetRegionServerQuota
(ObserverContext<MasterCoprocessorEnvironment> ctx, String regionServer, GlobalQuotaSettings quotas) Called before the quota for the region server is stored.void
preSetSplitOrMergeEnabled
(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean newValue, MasterSwitchType switchType) Called prior to setting split / merge switch Supports Coprocessor 'bypass'.void
preSetTableQuota
(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, GlobalQuotaSettings quotas) Called before the quota for the table is stored.void
preSetUserQuota
(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, String namespace, GlobalQuotaSettings quotas) Called before the quota for the user on the specified namespace is stored.void
preSetUserQuota
(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, GlobalQuotaSettings quotas) Called before the quota for the user is stored.void
preSetUserQuota
(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, TableName tableName, GlobalQuotaSettings quotas) Called before the quota for the user on the specified table is stored.void
Called prior to shutting down the full HBase cluster, including thisHMaster
process.void
preSnapshot
(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor hTableDescriptor) Called before a new snapshot is taken.void
preSplitRegion
(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, byte[] splitRow) Called before the split region procedure is called.void
Called immediately prior to stopping thisHMaster
process.void
Called before stopping region server.void
preSwitchExceedThrottleQuota
(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean enable) Called before switching exceed throttle quota state.void
preSwitchRpcThrottle
(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean enable) Called before switching rpc throttle enabled state.void
preTableFlush
(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) Called before the table memstore is flushed to disk.void
preTransitReplicationPeerSyncReplicationState
(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId, SyncReplicationState clusterState) Called before transit current cluster state for the specified synchronous replication peervoid
preTruncateTable
(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName) Called beforeHMaster
truncates a table.void
preUnassign
(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfo) Called prior to unassigning a given region.void
preUpdateMasterConfiguration
(ObserverContext<MasterCoprocessorEnvironment> ctx, org.apache.hadoop.conf.Configuration preReloadConf) Called before reloading the HMaster'sConfiguration
from diskvoid
preUpdateRegionServerConfiguration
(ObserverContext<RegionServerCoprocessorEnvironment> ctx, org.apache.hadoop.conf.Configuration preReloadConf) Called before reloading the RegionServer'sConfiguration
from diskvoid
preUpdateReplicationPeerConfig
(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId, ReplicationPeerConfig peerConfig) Called before update peerConfig for the specified peervoid
preUpdateRSGroupConfig
(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName, Map<String, String> configuration) Called before update rsgroup config.void
requireAccess
(ObserverContext<?> ctx, String request, TableName tableName, Permission.Action... permissions) void
requireGlobalPermission
(ObserverContext<?> ctx, String request, Permission.Action perm, String namespace) void
requireGlobalPermission
(ObserverContext<?> ctx, String request, Permission.Action perm, TableName tableName, Map<byte[], ? extends Collection<byte[]>> familyMap) void
requireNamespacePermission
(ObserverContext<?> ctx, String request, String namespace, Permission.Action... permissions) void
requireNamespacePermission
(ObserverContext<?> ctx, String request, String namespace, TableName tableName, Map<byte[], ? extends Collection<byte[]>> familyMap, Permission.Action... permissions) void
requirePermission
(ObserverContext<?> ctx, String request, Permission.Action perm) void
requirePermission
(ObserverContext<?> ctx, String request, TableName tableName, byte[] family, byte[] qualifier, Permission.Action... permissions) private void
Verify, when servicing an RPC, that the caller is the scanner owner.void
requireTablePermission
(ObserverContext<?> ctx, String request, TableName tableName, byte[] family, byte[] qualifier, Permission.Action... permissions) void
revoke
(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.RevokeRequest request, org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.RevokeResponse> done) Deprecated.since 2.2.0 and will be removed in 4.0.0.void
Called by theCoprocessorEnvironment
during it's own startup to initialize the coprocessor.void
Called by theCoprocessorEnvironment
during it's own shutdown to stop the coprocessor.private void
updateACL
(RegionCoprocessorEnvironment e, Map<byte[], List<Cell>> familyMap) Writes all table ACLs for the tables in the given Map up into ZooKeeper znodes.Methods inherited from class java.lang.Object
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
Methods inherited from interface org.apache.hadoop.hbase.coprocessor.MasterObserver
postAddReplicationPeer, postAddRSGroup, postAssign, postBalance, postBalanceRSGroup, postBalanceSwitch, postClearDeadServers, postCloneSnapshot, postCompletedDeleteTableAction, postCompletedDisableTableAction, postCompletedEnableTableAction, postCompletedMergeRegionsAction, postCompletedModifyTableAction, postCompletedSnapshotAction, postCompletedSplitRegionAction, postCompletedTruncateTableAction, postCreateNamespace, postCreateTable, postDecommissionRegionServers, postDeleteSnapshot, postDisableReplicationPeer, postDisableTable, postEnableReplicationPeer, postEnableTable, postGetClusterMetrics, postGetConfiguredNamespacesAndTablesInRSGroup, postGetLocks, postGetNamespaceDescriptor, postGetProcedures, postGetReplicationPeerConfig, postGetRSGroupInfo, postGetRSGroupInfoOfServer, postGetRSGroupInfoOfTable, postGetUserPermissions, postGrant, postHasUserPermissions, postIsRpcThrottleEnabled, postListDecommissionedRegionServers, postListReplicationPeers, postListRSGroups, postListSnapshot, postListTablesInRSGroup, postLockHeartbeat, postMasterStoreFlush, postMergeRegions, postMergeRegionsCommitAction, postModifyColumnFamilyStoreFileTracker, postModifyNamespace, postModifyTableStoreFileTracker, postMove, postMoveServers, postMoveServersAndTables, postMoveTables, postRecommissionRegionServer, postRegionOffline, postRemoveReplicationPeer, postRemoveRSGroup, postRemoveServers, postRenameRSGroup, postRequestLock, postRestoreSnapshot, postRevoke, postRollBackMergeRegionsAction, postRollBackSplitRegionAction, postSetNamespaceQuota, postSetRegionServerQuota, postSetSplitOrMergeEnabled, postSetTableQuota, postSetUserQuota, postSetUserQuota, postSetUserQuota, postSnapshot, postSwitchExceedThrottleQuota, postSwitchRpcThrottle, postTableFlush, postTransitReplicationPeerSyncReplicationState, postTruncateRegion, postTruncateRegionAction, postUnassign, postUpdateMasterConfiguration, postUpdateReplicationPeerConfig, postUpdateRSGroupConfig, preCreateTableAction, preCreateTableRegionsInfos, preDeleteTableAction, preDisableTableAction, preEnableTableAction, preGetClusterMetrics, preGetTableNames, preListNamespaceDescriptors, preListNamespaces, preMasterInitialization, preMasterStoreFlush, preMergeRegionsAction, preMergeRegionsCommitAction, preModifyTableAction, preSplitRegionAction, preSplitRegionAfterMETAAction, preSplitRegionBeforeMETAAction, preTruncateRegion, preTruncateRegionAction, preTruncateTableAction
Methods inherited from interface org.apache.hadoop.hbase.coprocessor.RegionObserver
postAppend, postAppend, postBatchMutate, postBatchMutateIndispensably, postBulkLoadHFile, postCheckAndDelete, postCheckAndDelete, postCheckAndMutate, postCheckAndPut, postCheckAndPut, postClose, postCloseRegionOperation, postCommitStoreFile, postCompact, postCompactSelection, postDelete, postExists, postFlush, postFlush, postGetOp, postIncrement, postIncrement, postInstantiateDeleteTracker, postMemStoreCompaction, postMutationBeforeWAL, postPut, postReplayWALs, postScannerFilterRow, postScannerNext, postStartRegionOperation, postStoreFileReaderOpen, preAppend, preAppendAfterRowLock, preCheckAndDelete, preCheckAndDeleteAfterRowLock, preCheckAndMutate, preCheckAndMutateAfterRowLock, preCheckAndPut, preCheckAndPutAfterRowLock, preCommitStoreFile, preCompactScannerOpen, preCompactSelection, preDelete, preFlush, preFlushScannerOpen, preIncrement, preIncrementAfterRowLock, preMemStoreCompaction, preMemStoreCompactionCompact, preMemStoreCompactionCompactScannerOpen, prePrepareTimeStampForDeleteVersion, prePut, preReplayWALs, preStoreFileReaderOpen, preStoreScannerOpen, preWALAppend
Methods inherited from interface org.apache.hadoop.hbase.coprocessor.RegionServerObserver
postClearCompactionQueues, postClearRegionBlockCache, postExecuteProcedures, postReplicateLogEntries, postReplicationSinkBatchMutate, postUpdateRegionServerConfiguration, preReplicationSinkBatchMutate
-
Field Details
-
LOG
-
AUDITLOG
-
CHECK_COVERING_PERM
- See Also:
-
TAG_CHECK_PASSED
- See Also:
-
TRUE
-
accessChecker
-
zkPermissionWatcher
-
aclRegion
flags if we are running on a region of the _acl_ table -
regionEnv
defined only for Endpoint implementation, so it can have way to access region services -
scannerOwners
Mapping of scanner instances to the user who created them -
tableAcls
-
userProvider
Provider for mapping principal names to Users -
authorizationEnabled
if we are active, usually false, only true if "hbase.security.authorization" has been set to true in site configuration -
cellFeaturesEnabled
if we are able to support cell ACLs -
shouldCheckExecPermission
if we should check EXEC permissions -
compatibleEarlyTermination
if we should terminate access checks early as soon as table or CF grants allow access; pre-0.98 compatible behavior -
initialized
if we have been successfully initialized -
aclTabAvailable
if the ACL table is available, only relevant in the master
-
-
Constructor Details
-
AccessController
public AccessController()
-
-
Method Details
-
isCellAuthorizationSupported
-
getRegion
-
getAuthManager
-
initialize
- Throws:
IOException
-
updateACL
Writes all table ACLs for the tables in the given Map up into ZooKeeper znodes. This is called to synchronize ACL changes following_acl_
table updates. -
permissionGranted
private AuthResult permissionGranted(AccessController.OpType opType, User user, RegionCoprocessorEnvironment e, Map<byte[], ? extends Collection<?>> families, Permission.Action... actions) Check the current user for authorization to perform a specific action against the given set of row data.- Parameters:
opType
- the operation typeuser
- the usere
- the coprocessor environmentfamilies
- the map of column families to qualifiers present in the requestactions
- the desired actions- Returns:
- an authorization result
-
requireAccess
public void requireAccess(ObserverContext<?> ctx, String request, TableName tableName, Permission.Action... permissions) throws IOException - Throws:
IOException
-
requirePermission
public void requirePermission(ObserverContext<?> ctx, String request, Permission.Action perm) throws IOException - Throws:
IOException
-
requireGlobalPermission
public void requireGlobalPermission(ObserverContext<?> ctx, String request, Permission.Action perm, TableName tableName, Map<byte[], ? extends Collection<byte[]>> familyMap) throws IOException- Throws:
IOException
-
requireGlobalPermission
public void requireGlobalPermission(ObserverContext<?> ctx, String request, Permission.Action perm, String namespace) throws IOException - Throws:
IOException
-
requireNamespacePermission
public void requireNamespacePermission(ObserverContext<?> ctx, String request, String namespace, Permission.Action... permissions) throws IOException - Throws:
IOException
-
requireNamespacePermission
public void requireNamespacePermission(ObserverContext<?> ctx, String request, String namespace, TableName tableName, Map<byte[], ? extends Collection<byte[]>> familyMap, Permission.Action... permissions) throws IOException- Throws:
IOException
-
requirePermission
public void requirePermission(ObserverContext<?> ctx, String request, TableName tableName, byte[] family, byte[] qualifier, Permission.Action... permissions) throws IOException - Throws:
IOException
-
requireTablePermission
public void requireTablePermission(ObserverContext<?> ctx, String request, TableName tableName, byte[] family, byte[] qualifier, Permission.Action... permissions) throws IOException - Throws:
IOException
-
checkLockPermissions
public void checkLockPermissions(ObserverContext<?> ctx, String namespace, TableName tableName, RegionInfo[] regionInfos, String reason) throws IOException - Throws:
IOException
-
hasFamilyQualifierPermission
private boolean hasFamilyQualifierPermission(User user, Permission.Action perm, RegionCoprocessorEnvironment env, Map<byte[], ? extends Collection<byte[]>> familyMap) throws IOExceptionReturnstrue
if the current user is allowed the given action over at least one of the column qualifiers in the given column families.- Throws:
IOException
-
checkCoveringPermission
private boolean checkCoveringPermission(User user, AccessController.OpType request, RegionCoprocessorEnvironment e, byte[] row, Map<byte[], ? extends Collection<?>> familyMap, long opTs, Permission.Action... actions) throws IOExceptionDetermine if cell ACLs covered by the operation grant access. This is expensive.- Returns:
- false if cell ACLs failed to grant access, true otherwise
- Throws:
IOException
-
addCellPermissions
-
checkForReservedTagPresence
- Throws:
IOException
-
start
Description copied from interface:Coprocessor
Called by theCoprocessorEnvironment
during it's own startup to initialize the coprocessor.- Specified by:
start
in interfaceCoprocessor
- Throws:
IOException
-
stop
Description copied from interface:Coprocessor
Called by theCoprocessorEnvironment
during it's own shutdown to stop the coprocessor.- Specified by:
stop
in interfaceCoprocessor
-
getRegionObserver
Observer/Service Getters- Specified by:
getRegionObserver
in interfaceRegionCoprocessor
-
getMasterObserver
- Specified by:
getMasterObserver
in interfaceMasterCoprocessor
-
getEndpointObserver
- Specified by:
getEndpointObserver
in interfaceRegionCoprocessor
-
getBulkLoadObserver
- Specified by:
getBulkLoadObserver
in interfaceRegionCoprocessor
-
getRegionServerObserver
- Specified by:
getRegionServerObserver
in interfaceRegionServerCoprocessor
-
getServices
Description copied from interface:Coprocessor
Coprocessor endpoints providing protobuf services should override this method.- Specified by:
getServices
in interfaceCoprocessor
- Returns:
- Iterable of
Service
s or empty collection. Implementations should never return null.
-
preCreateTable
public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> c, TableDescriptor desc, RegionInfo[] regions) throws IOException Observer implementations- Specified by:
preCreateTable
in interfaceMasterObserver
- Parameters:
c
- the environment to interact with the framework and masterdesc
- the TableDescriptor for the tableregions
- the initial regions created for the table- Throws:
IOException
-
postCompletedCreateTableAction
public void postCompletedCreateTableAction(ObserverContext<MasterCoprocessorEnvironment> c, TableDescriptor desc, RegionInfo[] regions) throws IOException Description copied from interface:MasterObserver
Called after the createTable operation has been requested. Called as part of create table RPC call. Called as part of create table procedure and it is async to the create RPC call.- Specified by:
postCompletedCreateTableAction
in interfaceMasterObserver
- Parameters:
c
- the environment to interact with the framework and masterdesc
- the TableDescriptor for the tableregions
- the initial regions created for the table- Throws:
IOException
-
preDeleteTable
public void preDeleteTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName) throws IOException Description copied from interface:MasterObserver
Called beforeHMaster
deletes a table. Called as part of delete table RPC call.- Specified by:
preDeleteTable
in interfaceMasterObserver
- Parameters:
c
- the environment to interact with the framework and mastertableName
- the name of the table- Throws:
IOException
-
postDeleteTable
public void postDeleteTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName) throws IOException Description copied from interface:MasterObserver
Called after the deleteTable operation has been requested. Called as part of delete table RPC call.- Specified by:
postDeleteTable
in interfaceMasterObserver
- Parameters:
c
- the environment to interact with the framework and mastertableName
- the name of the table- Throws:
IOException
-
preTruncateTable
public void preTruncateTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName) throws IOException Description copied from interface:MasterObserver
Called beforeHMaster
truncates a table. Called as part of truncate table RPC call.- Specified by:
preTruncateTable
in interfaceMasterObserver
- Parameters:
c
- the environment to interact with the framework and mastertableName
- the name of the table- Throws:
IOException
-
postTruncateTable
public void postTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException Description copied from interface:MasterObserver
Called after the truncateTable operation has been requested. Called as part of truncate table RPC call. The truncate is synchronous, so this method will be called when the truncate operation is terminated.- Specified by:
postTruncateTable
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and mastertableName
- the name of the table- Throws:
IOException
-
preModifyTable
public TableDescriptor preModifyTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName, TableDescriptor currentDesc, TableDescriptor newDesc) throws IOException Description copied from interface:MasterObserver
Called prior to modifying a table's properties. Called as part of modify table RPC call.- Specified by:
preModifyTable
in interfaceMasterObserver
- Parameters:
c
- the environment to interact with the framework and mastertableName
- the name of the tablecurrentDesc
- current TableDescriptor of the tablenewDesc
- after modify operation, table will have this descriptor- Throws:
IOException
-
preModifyTableStoreFileTracker
public String preModifyTableStoreFileTracker(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName, String dstSFT) throws IOException Description copied from interface:MasterObserver
Called prior to modifying a table's store file tracker. Called as part of modify table store file tracker RPC call.- Specified by:
preModifyTableStoreFileTracker
in interfaceMasterObserver
- Parameters:
c
- the environment to interact with the framework and mastertableName
- the name of the tabledstSFT
- the store file tracker- Returns:
- the store file tracker
- Throws:
IOException
-
preModifyColumnFamilyStoreFileTracker
public String preModifyColumnFamilyStoreFileTracker(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName, byte[] family, String dstSFT) throws IOException Description copied from interface:MasterObserver
Called prior to modifying a family's store file tracker. Called as part of modify family store file tracker RPC call.- Specified by:
preModifyColumnFamilyStoreFileTracker
in interfaceMasterObserver
- Parameters:
c
- the environment to interact with the framework and mastertableName
- the name of the tablefamily
- the column familydstSFT
- the store file tracker- Returns:
- the store file tracker
- Throws:
IOException
-
postModifyTable
public void postModifyTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName, TableDescriptor oldDesc, TableDescriptor currentDesc) throws IOException Description copied from interface:MasterObserver
Called after the modifyTable operation has been requested. Called as part of modify table RPC call.- Specified by:
postModifyTable
in interfaceMasterObserver
- Parameters:
c
- the environment to interact with the framework and mastertableName
- the name of the tableoldDesc
- descriptor of table before modify operation happenedcurrentDesc
- current TableDescriptor of the table- Throws:
IOException
-
preEnableTable
public void preEnableTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName) throws IOException Description copied from interface:MasterObserver
Called prior to enabling a table. Called as part of enable table RPC call.- Specified by:
preEnableTable
in interfaceMasterObserver
- Parameters:
c
- the environment to interact with the framework and mastertableName
- the name of the table- Throws:
IOException
-
preDisableTable
public void preDisableTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName) throws IOException Description copied from interface:MasterObserver
Called prior to disabling a table. Called as part of disable table RPC call.- Specified by:
preDisableTable
in interfaceMasterObserver
- Parameters:
c
- the environment to interact with the framework and mastertableName
- the name of the table- Throws:
IOException
-
preAbortProcedure
public void preAbortProcedure(ObserverContext<MasterCoprocessorEnvironment> ctx, long procId) throws IOException Description copied from interface:MasterObserver
Called before a abortProcedure request has been processed.- Specified by:
preAbortProcedure
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and masterprocId
- the Id of the procedure- Throws:
IOException
-
postAbortProcedure
public void postAbortProcedure(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException Description copied from interface:MasterObserver
Called after a abortProcedure request has been processed.- Specified by:
postAbortProcedure
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and master- Throws:
IOException
-
preGetProcedures
Description copied from interface:MasterObserver
Called before a getProcedures request has been processed.- Specified by:
preGetProcedures
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and master- Throws:
IOException
-
preGetLocks
Description copied from interface:MasterObserver
Called before a getLocks request has been processed.- Specified by:
preGetLocks
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and master- Throws:
IOException
- if something went wrong
-
preMove
public void preMove(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo region, ServerName srcServer, ServerName destServer) throws IOException Description copied from interface:MasterObserver
Called prior to moving a given region from one region server to another.- Specified by:
preMove
in interfaceMasterObserver
- Parameters:
c
- the environment to interact with the framework and masterregion
- the RegionInfosrcServer
- the source ServerNamedestServer
- the destination ServerName- Throws:
IOException
-
preAssign
public void preAssign(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfo) throws IOException Description copied from interface:MasterObserver
Called prior to assigning a specific region.- Specified by:
preAssign
in interfaceMasterObserver
- Parameters:
c
- the environment to interact with the framework and masterregionInfo
- the regionInfo of the region- Throws:
IOException
-
preUnassign
public void preUnassign(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfo) throws IOException Description copied from interface:MasterObserver
Called prior to unassigning a given region.- Specified by:
preUnassign
in interfaceMasterObserver
- Parameters:
c
- the environment to interact with the framework and master- Throws:
IOException
-
preRegionOffline
public void preRegionOffline(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfo) throws IOException Description copied from interface:MasterObserver
Called prior to marking a given region as offline.- Specified by:
preRegionOffline
in interfaceMasterObserver
- Parameters:
c
- the environment to interact with the framework and master- Throws:
IOException
-
preSetSplitOrMergeEnabled
public void preSetSplitOrMergeEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean newValue, MasterSwitchType switchType) throws IOException Description copied from interface:MasterObserver
Called prior to setting split / merge switch Supports Coprocessor 'bypass'.- Specified by:
preSetSplitOrMergeEnabled
in interfaceMasterObserver
- Parameters:
ctx
- the coprocessor instance's environmentnewValue
- the new value submitted in the callswitchType
- type of switch- Throws:
IOException
-
preBalance
public void preBalance(ObserverContext<MasterCoprocessorEnvironment> c, BalanceRequest request) throws IOException Description copied from interface:MasterObserver
Called prior to requesting rebalancing of the cluster regions, though after the initial checks for regions in transition and the balance switch flag.- Specified by:
preBalance
in interfaceMasterObserver
- Parameters:
c
- the environment to interact with the framework and masterrequest
- the request used to trigger the balancer- Throws:
IOException
-
preBalanceSwitch
public void preBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> c, boolean newValue) throws IOException Description copied from interface:MasterObserver
Called prior to modifying the flag used to enable/disable region balancing.- Specified by:
preBalanceSwitch
in interfaceMasterObserver
- Parameters:
c
- the coprocessor instance's environment- Throws:
IOException
-
preShutdown
Description copied from interface:MasterObserver
Called prior to shutting down the full HBase cluster, including thisHMaster
process.- Specified by:
preShutdown
in interfaceMasterObserver
- Throws:
IOException
-
preStopMaster
Description copied from interface:MasterObserver
Called immediately prior to stopping thisHMaster
process.- Specified by:
preStopMaster
in interfaceMasterObserver
- Throws:
IOException
-
postStartMaster
Description copied from interface:MasterObserver
Called immediately after an active master instance has completed initialization. Will not be called on standby master instances unless they take over the active role.- Specified by:
postStartMaster
in interfaceMasterObserver
- Throws:
IOException
-
createACLTable
Create the ACL table- Throws:
IOException
-
preSnapshot
public void preSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor hTableDescriptor) throws IOException Description copied from interface:MasterObserver
Called before a new snapshot is taken. Called as part of snapshot RPC call.- Specified by:
preSnapshot
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and mastersnapshot
- the SnapshotDescriptor for the snapshothTableDescriptor
- the TableDescriptor of the table to snapshot- Throws:
IOException
-
preListSnapshot
public void preListSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot) throws IOException Description copied from interface:MasterObserver
Called before listSnapshots request has been processed.- Specified by:
preListSnapshot
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and mastersnapshot
- the SnapshotDescriptor of the snapshot to list- Throws:
IOException
-
preCloneSnapshot
public void preCloneSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor hTableDescriptor) throws IOException Description copied from interface:MasterObserver
Called before a snapshot is cloned. Called as part of restoreSnapshot RPC call.- Specified by:
preCloneSnapshot
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and mastersnapshot
- the SnapshotDescriptor for the snapshothTableDescriptor
- the TableDescriptor of the table to create- Throws:
IOException
-
preRestoreSnapshot
public void preRestoreSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor hTableDescriptor) throws IOException Description copied from interface:MasterObserver
Called before a snapshot is restored. Called as part of restoreSnapshot RPC call.- Specified by:
preRestoreSnapshot
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and mastersnapshot
- the SnapshotDescriptor for the snapshothTableDescriptor
- the TableDescriptor of the table to restore- Throws:
IOException
-
preDeleteSnapshot
public void preDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot) throws IOException Description copied from interface:MasterObserver
Called before a snapshot is deleted. Called as part of deleteSnapshot RPC call.- Specified by:
preDeleteSnapshot
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and mastersnapshot
- the SnapshotDescriptor of the snapshot to delete- Throws:
IOException
-
preCreateNamespace
public void preCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor ns) throws IOException Description copied from interface:MasterObserver
Called before a new namespace is created byHMaster
.- Specified by:
preCreateNamespace
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and masterns
- the NamespaceDescriptor for the table- Throws:
IOException
-
preDeleteNamespace
public void preDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) throws IOException Description copied from interface:MasterObserver
Called beforeHMaster
deletes a namespace- Specified by:
preDeleteNamespace
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and masternamespace
- the name of the namespace- Throws:
IOException
-
postDeleteNamespace
public void postDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) throws IOException Description copied from interface:MasterObserver
Called after the deleteNamespace operation has been requested.- Specified by:
postDeleteNamespace
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and masternamespace
- the name of the namespace- Throws:
IOException
-
preModifyNamespace
public void preModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor currentNsDesc, NamespaceDescriptor newNsDesc) throws IOException Description copied from interface:MasterObserver
Called prior to modifying a namespace's properties.- Specified by:
preModifyNamespace
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and mastercurrentNsDesc
- current NamespaceDescriptor of the namespacenewNsDesc
- after modify operation, namespace will have this descriptor- Throws:
IOException
-
preGetNamespaceDescriptor
public void preGetNamespaceDescriptor(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) throws IOException Description copied from interface:MasterObserver
Called before a getNamespaceDescriptor request has been processed.- Specified by:
preGetNamespaceDescriptor
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and masternamespace
- the name of the namespace- Throws:
IOException
-
postListNamespaces
public void postListNamespaces(ObserverContext<MasterCoprocessorEnvironment> ctx, List<String> namespaces) throws IOException Description copied from interface:MasterObserver
Called after a listNamespaces request has been processed.- Specified by:
postListNamespaces
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and masternamespaces
- the list of namespaces about to be returned- Throws:
IOException
- if something went wrong
-
postListNamespaceDescriptors
public void postListNamespaceDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<NamespaceDescriptor> descriptors) throws IOException Description copied from interface:MasterObserver
Called after a listNamespaceDescriptors request has been processed.- Specified by:
postListNamespaceDescriptors
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and masterdescriptors
- the list of descriptors about to be returned- Throws:
IOException
-
preTableFlush
public void preTableFlush(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException Description copied from interface:MasterObserver
Called before the table memstore is flushed to disk.- Specified by:
preTableFlush
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and mastertableName
- the name of the table- Throws:
IOException
-
preSplitRegion
public void preSplitRegion(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, byte[] splitRow) throws IOException Description copied from interface:MasterObserver
Called before the split region procedure is called.- Specified by:
preSplitRegion
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and mastertableName
- the table where the region belongs tosplitRow
- split point- Throws:
IOException
-
preClearDeadServers
public void preClearDeadServers(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException Description copied from interface:MasterObserver
Called before clear dead region servers.- Specified by:
preClearDeadServers
in interfaceMasterObserver
- Throws:
IOException
-
preDecommissionRegionServers
public void preDecommissionRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx, List<ServerName> servers, boolean offload) throws IOException Description copied from interface:MasterObserver
Called before decommission region servers.- Specified by:
preDecommissionRegionServers
in interfaceMasterObserver
- Throws:
IOException
-
preListDecommissionedRegionServers
public void preListDecommissionedRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException Description copied from interface:MasterObserver
Called before list decommissioned region servers.- Specified by:
preListDecommissionedRegionServers
in interfaceMasterObserver
- Throws:
IOException
-
preRecommissionRegionServer
public void preRecommissionRegionServer(ObserverContext<MasterCoprocessorEnvironment> ctx, ServerName server, List<byte[]> encodedRegionNames) throws IOException Description copied from interface:MasterObserver
Called before recommission region server.- Specified by:
preRecommissionRegionServer
in interfaceMasterObserver
- Throws:
IOException
-
preOpen
Description copied from interface:RegionObserver
Called before the region is reported as open to the master.- Specified by:
preOpen
in interfaceRegionObserver
- Parameters:
c
- the environment provided by the region server- Throws:
IOException
-
postOpen
Description copied from interface:RegionObserver
Called after the region is reported as open to the master.- Specified by:
postOpen
in interfaceRegionObserver
- Parameters:
c
- the environment provided by the region server
-
preFlush
public void preFlush(ObserverContext<? extends RegionCoprocessorEnvironment> c, FlushLifeCycleTracker tracker) throws IOException Description copied from interface:RegionObserver
Called before the memstore is flushed to disk.- Specified by:
preFlush
in interfaceRegionObserver
- Parameters:
c
- the environment provided by the region servertracker
- tracker used to track the life cycle of a flush- Throws:
IOException
-
preCompact
public InternalScanner preCompact(ObserverContext<? extends RegionCoprocessorEnvironment> c, Store store, InternalScanner scanner, ScanType scanType, CompactionLifeCycleTracker tracker, CompactionRequest request) throws IOException Description copied from interface:RegionObserver
Called prior to writing theStoreFile
s selected for compaction into a newStoreFile
.To override or modify the compaction process, implementing classes can wrap the provided
InternalScanner
with a custom implementation that is returned from this method. The custom scanner can then inspectCell
s from the wrapped scanner, applying its own policy to what gets written.If implementations are wrapping the passed in
InternalScanner
, they can also have their implementation implementShipper
and delegate to the original scanner. This will cause compactions to free up memory as they progress, which is especially important for people using off-heap memory pools.Keep in mind that when
Shipper.shipped()
is called, any cell references you maintain in your implementation may get corrupted. As such you should make sure to deep clone any cells that you need to keep reference to across invocations of shipped.- Specified by:
preCompact
in interfaceRegionObserver
- Parameters:
c
- the environment provided by the region serverstore
- the store being compactedscanner
- the scanner over existing data used in the store file rewritingscanType
- type of Scantracker
- tracker used to track the life cycle of a compactionrequest
- the requested compaction- Returns:
- the scanner to use during compaction. Should not be
null
unless the implementation is writing new store files on its own. - Throws:
IOException
-
internalPreRead
private void internalPreRead(ObserverContext<? extends RegionCoprocessorEnvironment> c, Query query, AccessController.OpType opType) throws IOException - Throws:
IOException
-
preGetOp
public void preGetOp(ObserverContext<? extends RegionCoprocessorEnvironment> c, Get get, List<Cell> result) throws IOException Description copied from interface:RegionObserver
Called before the client performs a GetCall CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
- Specified by:
preGetOp
in interfaceRegionObserver
- Parameters:
c
- the environment provided by the region serverget
- the Get requestresult
- The result to return to the client if default processing is bypassed. Can be modified. Will not be used if default processing is not bypassed.- Throws:
IOException
-
preExists
public boolean preExists(ObserverContext<? extends RegionCoprocessorEnvironment> c, Get get, boolean exists) throws IOException Description copied from interface:RegionObserver
Called before the client tests for existence using a Get.Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
- Specified by:
preExists
in interfaceRegionObserver
- Parameters:
c
- the environment provided by the region serverget
- the Get requestexists
- the result returned by the region server- Returns:
- the value to return to the client if bypassing default processing
- Throws:
IOException
-
prePut
public void prePut(ObserverContext<? extends RegionCoprocessorEnvironment> c, Put put, WALEdit edit, Durability durability) throws IOException Description copied from interface:RegionObserver
Called before the client stores a value.Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'put' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
- Specified by:
prePut
in interfaceRegionObserver
- Parameters:
c
- the environment provided by the region serverput
- The Put objectedit
- The WALEdit object that will be written to the waldurability
- Persistence guarantee for this Put- Throws:
IOException
-
postPut
public void postPut(ObserverContext<? extends RegionCoprocessorEnvironment> c, Put put, WALEdit edit, Durability durability) Description copied from interface:RegionObserver
Called after the client stores a value.Note: Do not retain references to any Cells in 'put' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
- Specified by:
postPut
in interfaceRegionObserver
- Parameters:
c
- the environment provided by the region serverput
- The Put objectedit
- The WALEdit object for the waldurability
- Persistence guarantee for this Put
-
preDelete
public void preDelete(ObserverContext<? extends RegionCoprocessorEnvironment> c, Delete delete, WALEdit edit, Durability durability) throws IOException Description copied from interface:RegionObserver
Called before the client deletes a value.Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
- Specified by:
preDelete
in interfaceRegionObserver
- Parameters:
c
- the environment provided by the region serverdelete
- The Delete objectedit
- The WALEdit object for the waldurability
- Persistence guarantee for this Delete- Throws:
IOException
-
preBatchMutate
public void preBatchMutate(ObserverContext<? extends RegionCoprocessorEnvironment> c, MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException Description copied from interface:RegionObserver
This will be called for every batch mutation operation happening at the server. This will be called after acquiring the locks on the mutating rows and after applying the proper timestamp for each Mutation at the server. The batch may contain Put/Delete/Increment/Append. By setting OperationStatus of Mutations (MiniBatchOperationInProgress.setOperationStatus(int, OperationStatus)
),RegionObserver
can make Region to skip these Mutations.Note: Do not retain references to any Cells in Mutations beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
- Specified by:
preBatchMutate
in interfaceRegionObserver
- Parameters:
c
- the environment provided by the region serverminiBatchOp
- batch of Mutations getting applied to region.- Throws:
IOException
-
postDelete
public void postDelete(ObserverContext<? extends RegionCoprocessorEnvironment> c, Delete delete, WALEdit edit, Durability durability) throws IOException Description copied from interface:RegionObserver
Called after the client deletes a value.Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
- Specified by:
postDelete
in interfaceRegionObserver
- Parameters:
c
- the environment provided by the region serverdelete
- The Delete objectedit
- The WALEdit object for the waldurability
- Persistence guarantee for this Delete- Throws:
IOException
-
preCheckAndPut
public boolean preCheckAndPut(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Put put, boolean result) throws IOException Description copied from interface:RegionObserver
Called before checkAndPut.Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'put' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
- Specified by:
preCheckAndPut
in interfaceRegionObserver
- Parameters:
c
- the environment provided by the region serverrow
- row to checkfamily
- column familyqualifier
- column qualifierop
- the comparison operationcomparator
- the comparatorput
- data to put if check succeedsresult
- the default value of the result- Returns:
- the return value to return to client if bypassing default processing
- Throws:
IOException
-
preCheckAndPutAfterRowLock
public boolean preCheckAndPutAfterRowLock(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator opp, ByteArrayComparable comparator, Put put, boolean result) throws IOException Description copied from interface:RegionObserver
Called before checkAndPut but after acquiring rowlock.Note: Caution to be taken for not doing any long time operation in this hook. Row will be locked for longer time. Trying to acquire lock on another row, within this, can lead to potential deadlock.
Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'put' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
- Specified by:
preCheckAndPutAfterRowLock
in interfaceRegionObserver
- Parameters:
c
- the environment provided by the region serverrow
- row to checkfamily
- column familyqualifier
- column qualifieropp
- the comparison operationcomparator
- the comparatorput
- data to put if check succeedsresult
- the default value of the result- Returns:
- the return value to return to client if bypassing default processing
- Throws:
IOException
-
preCheckAndDelete
public boolean preCheckAndDelete(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Delete delete, boolean result) throws IOException Description copied from interface:RegionObserver
Called before checkAndDelete.Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
- Specified by:
preCheckAndDelete
in interfaceRegionObserver
- Parameters:
c
- the environment provided by the region serverrow
- row to checkfamily
- column familyqualifier
- column qualifierop
- the comparison operationcomparator
- the comparatordelete
- delete to commit if check succeedsresult
- the default value of the result- Returns:
- the value to return to client if bypassing default processing
- Throws:
IOException
-
preCheckAndDeleteAfterRowLock
public boolean preCheckAndDeleteAfterRowLock(ObserverContext<? extends RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Delete delete, boolean result) throws IOException Description copied from interface:RegionObserver
Called before checkAndDelete but after acquiring rowock.Note: Caution to be taken for not doing any long time operation in this hook. Row will be locked for longer time. Trying to acquire lock on another row, within this, can lead to potential deadlock.
Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
- Specified by:
preCheckAndDeleteAfterRowLock
in interfaceRegionObserver
- Parameters:
c
- the environment provided by the region serverrow
- row to checkfamily
- column familyqualifier
- column qualifierop
- the comparison operationcomparator
- the comparatordelete
- delete to commit if check succeedsresult
- the default value of the result- Returns:
- the value to return to client if bypassing default processing
- Throws:
IOException
-
preAppend
public Result preAppend(ObserverContext<? extends RegionCoprocessorEnvironment> c, Append append) throws IOException Description copied from interface:RegionObserver
Called before Append.Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'append' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
- Specified by:
preAppend
in interfaceRegionObserver
- Parameters:
c
- the environment provided by the region serverappend
- Append object- Returns:
- result to return to the client if bypassing default processing
- Throws:
IOException
-
preIncrement
public Result preIncrement(ObserverContext<? extends RegionCoprocessorEnvironment> c, Increment increment) throws IOException Description copied from interface:RegionObserver
Called before Increment.Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells in 'increment' beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
- Specified by:
preIncrement
in interfaceRegionObserver
- Parameters:
c
- the environment provided by the region serverincrement
- increment object- Returns:
- result to return to the client if bypassing default processing
- Throws:
IOException
-
postIncrementBeforeWAL
public List<Pair<Cell,Cell>> postIncrementBeforeWAL(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, Mutation mutation, List<Pair<Cell, Cell>> cellPairs) throws IOExceptionDescription copied from interface:RegionObserver
Called after a list of new cells has been created during an increment operation, but before they are committed to the WAL or memstore.- Specified by:
postIncrementBeforeWAL
in interfaceRegionObserver
- Parameters:
ctx
- the environment provided by the region servermutation
- the current mutationcellPairs
- a list of cell pair. The first cell is old cell which may be null. And the second cell is the new cell.- Returns:
- a list of cell pair, possibly changed.
- Throws:
IOException
-
postAppendBeforeWAL
public List<Pair<Cell,Cell>> postAppendBeforeWAL(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, Mutation mutation, List<Pair<Cell, Cell>> cellPairs) throws IOExceptionDescription copied from interface:RegionObserver
Called after a list of new cells has been created during an append operation, but before they are committed to the WAL or memstore.- Specified by:
postAppendBeforeWAL
in interfaceRegionObserver
- Parameters:
ctx
- the environment provided by the region servermutation
- the current mutationcellPairs
- a list of cell pair. The first cell is old cell which may be null. And the second cell is the new cell.- Returns:
- a list of cell pair, possibly changed.
- Throws:
IOException
-
createNewCellWithTags
-
preScannerOpen
public void preScannerOpen(ObserverContext<? extends RegionCoprocessorEnvironment> c, Scan scan) throws IOException Description copied from interface:RegionObserver
Called before the client opens a new scanner.Note: Do not retain references to any Cells returned by scanner, beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
- Specified by:
preScannerOpen
in interfaceRegionObserver
- Parameters:
c
- the environment provided by the region serverscan
- the Scan specification- Throws:
IOException
-
postScannerOpen
public RegionScanner postScannerOpen(ObserverContext<? extends RegionCoprocessorEnvironment> c, Scan scan, RegionScanner s) throws IOException Description copied from interface:RegionObserver
Called after the client opens a new scanner.Note: Do not retain references to any Cells returned by scanner, beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
- Specified by:
postScannerOpen
in interfaceRegionObserver
- Parameters:
c
- the environment provided by the region serverscan
- the Scan specifications
- if not null, the base scanner- Returns:
- the scanner instance to use
- Throws:
IOException
-
preScannerNext
public boolean preScannerNext(ObserverContext<? extends RegionCoprocessorEnvironment> c, InternalScanner s, List<Result> result, int limit, boolean hasNext) throws IOException Description copied from interface:RegionObserver
Called before the client asks for the next row on a scanner.Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
Note: Do not retain references to any Cells returned by scanner, beyond the life of this invocation. If need a Cell reference for later use, copy the cell and use that.
- Specified by:
preScannerNext
in interfaceRegionObserver
- Parameters:
c
- the environment provided by the region servers
- the scannerresult
- The result to return to the client if default processing is bypassed. Can be modified. Will not be returned if default processing is not bypassed.limit
- the maximum number of results to returnhasNext
- the 'has more' indication- Returns:
- 'has more' indication that should be sent to client
- Throws:
IOException
-
preScannerClose
public void preScannerClose(ObserverContext<? extends RegionCoprocessorEnvironment> c, InternalScanner s) throws IOException Description copied from interface:RegionObserver
Called before the client closes a scanner.Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
- Specified by:
preScannerClose
in interfaceRegionObserver
- Parameters:
c
- the environment provided by the region servers
- the scanner- Throws:
IOException
-
postScannerClose
public void postScannerClose(ObserverContext<? extends RegionCoprocessorEnvironment> c, InternalScanner s) throws IOException Description copied from interface:RegionObserver
Called after the client closes a scanner.- Specified by:
postScannerClose
in interfaceRegionObserver
- Parameters:
c
- the environment provided by the region servers
- the scanner- Throws:
IOException
-
requireScannerOwner
Verify, when servicing an RPC, that the caller is the scanner owner. If so, we assume that access control is correctly enforced based on the checks performed in preScannerOpen()- Throws:
AccessDeniedException
-
preBulkLoadHFile
public void preBulkLoadHFile(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, List<Pair<byte[], String>> familyPaths) throws IOExceptionVerifies user has CREATE or ADMIN privileges on the Column Families involved in the bulkLoadHFile request. Specific Column Write privileges are presently ignored.- Specified by:
preBulkLoadHFile
in interfaceRegionObserver
- Parameters:
ctx
- the environment provided by the region serverfamilyPaths
- pairs of { CF, HFile path } submitted for bulk load. Adding or removing from this list will add or remove HFiles to be bulk loaded.- Throws:
IOException
-
prePrepareBulkLoad
public void prePrepareBulkLoad(ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException Authorization check for SecureBulkLoadProtocol.prepareBulkLoad()- Specified by:
prePrepareBulkLoad
in interfaceBulkLoadObserver
- Parameters:
ctx
- the context- Throws:
IOException
-
preCleanupBulkLoad
public void preCleanupBulkLoad(ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException Authorization security check for SecureBulkLoadProtocol.cleanupBulkLoad()- Specified by:
preCleanupBulkLoad
in interfaceBulkLoadObserver
- Parameters:
ctx
- the context- Throws:
IOException
-
preEndpointInvocation
public org.apache.hbase.thirdparty.com.google.protobuf.Message preEndpointInvocation(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, org.apache.hbase.thirdparty.com.google.protobuf.Service service, String methodName, org.apache.hbase.thirdparty.com.google.protobuf.Message request) throws IOException Description copied from interface:EndpointObserver
Called before an Endpoint service method is invoked. The request message can be altered by returning a new instance. Throwing an exception will abort the invocation. CallingObserverContext.bypass()
has no effect in this hook.- Specified by:
preEndpointInvocation
in interfaceEndpointObserver
- Parameters:
ctx
- the environment provided by the region serverservice
- the endpoint servicemethodName
- the invoked service methodrequest
- Request message expected by givenService
's method (by the namemethodName
).- Returns:
- the possibly modified message
- Throws:
IOException
-
postEndpointInvocation
public void postEndpointInvocation(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, org.apache.hbase.thirdparty.com.google.protobuf.Service service, String methodName, org.apache.hbase.thirdparty.com.google.protobuf.Message request, org.apache.hbase.thirdparty.com.google.protobuf.Message.Builder responseBuilder) throws IOException Description copied from interface:EndpointObserver
Called after an Endpoint service method is invoked. The response message can be altered using the builder.- Specified by:
postEndpointInvocation
in interfaceEndpointObserver
- Parameters:
ctx
- the environment provided by the region serverservice
- the endpoint servicemethodName
- the invoked service methodrequest
- Request message expected by givenService
's method (by the namemethodName
).responseBuilder
- Builder for final response to the client, with original response from Service's method merged into it.- Throws:
IOException
-
grant
@Deprecated public void grant(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GrantRequest request, org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GrantResponse> done) Deprecated.since 2.2.0 and will be removed in 4.0.0. UseAdmin.grant(UserPermission, boolean)
instead.- Specified by:
grant
in interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.AccessControlService.Interface
- See Also:
-
revoke
@Deprecated public void revoke(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.RevokeRequest request, org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.RevokeResponse> done) Deprecated.since 2.2.0 and will be removed in 4.0.0. UseAdmin.revoke(UserPermission)
instead.- Specified by:
revoke
in interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.AccessControlService.Interface
- See Also:
-
getUserPermissions
@Deprecated public void getUserPermissions(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsRequest request, org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsResponse> done) Deprecated.since 2.2.0 and will be removed in 4.0.0. UseAdmin.getUserPermissions(GetUserPermissionsRequest)
instead.- Specified by:
getUserPermissions
in interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.AccessControlService.Interface
- See Also:
-
checkPermissions
@Deprecated public void checkPermissions(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.CheckPermissionsRequest request, org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.CheckPermissionsResponse> done) Deprecated.since 2.2.0 and will be removed 4.0.0. UseAdmin.hasUserPermissions(List)
instead.- Specified by:
checkPermissions
in interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.AccessControlService.Interface
- See Also:
-
getRegion
-
getTableName
-
getTableName
-
preClose
public void preClose(ObserverContext<? extends RegionCoprocessorEnvironment> c, boolean abortRequested) throws IOException Description copied from interface:RegionObserver
Called before the region is reported as closed to the master.- Specified by:
preClose
in interfaceRegionObserver
- Parameters:
c
- the environment provided by the region serverabortRequested
- true if the region server is aborting- Throws:
IOException
-
checkSystemOrSuperUser
- Throws:
IOException
-
preStopRegionServer
public void preStopRegionServer(ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException Description copied from interface:RegionServerObserver
Called before stopping region server.- Specified by:
preStopRegionServer
in interfaceRegionServerObserver
- Parameters:
ctx
- the environment to interact with the framework and region server.- Throws:
IOException
-
makeFamilyMap
-
preGetTableDescriptors
public void preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableName> tableNamesList, List<TableDescriptor> descriptors, String regex) throws IOException Description copied from interface:MasterObserver
Called before a getTableDescriptors request has been processed.- Specified by:
preGetTableDescriptors
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and mastertableNamesList
- the list of table names, or null if querying for alldescriptors
- an empty list, can be filled with what to return in coprocessorregex
- regular expression used for filtering the table names- Throws:
IOException
-
postGetTableDescriptors
public void postGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableName> tableNamesList, List<TableDescriptor> descriptors, String regex) throws IOException Description copied from interface:MasterObserver
Called after a getTableDescriptors request has been processed.- Specified by:
postGetTableDescriptors
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and mastertableNamesList
- the list of table names, or null if querying for alldescriptors
- the list of descriptors about to be returnedregex
- regular expression used for filtering the table names- Throws:
IOException
-
postGetTableNames
public void postGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableDescriptor> descriptors, String regex) throws IOException Description copied from interface:MasterObserver
Called after a getTableNames request has been processed.- Specified by:
postGetTableNames
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and masterdescriptors
- the list of descriptors about to be returnedregex
- regular expression used for filtering the table names- Throws:
IOException
-
preMergeRegions
public void preMergeRegions(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge) throws IOException Description copied from interface:MasterObserver
Called before merge regions request.- Specified by:
preMergeRegions
in interfaceMasterObserver
- Parameters:
ctx
- coprocessor environmentregionsToMerge
- regions to be merged- Throws:
IOException
-
preRollWALWriterRequest
public void preRollWALWriterRequest(ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException Description copied from interface:RegionServerObserver
This will be called before executing user request to roll a region server WAL.- Specified by:
preRollWALWriterRequest
in interfaceRegionServerObserver
- Parameters:
ctx
- the environment to interact with the framework and region server.- Throws:
IOException
-
postRollWALWriterRequest
public void postRollWALWriterRequest(ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException Description copied from interface:RegionServerObserver
This will be called after executing user request to roll a region server WAL.- Specified by:
postRollWALWriterRequest
in interfaceRegionServerObserver
- Parameters:
ctx
- the environment to interact with the framework and region server.- Throws:
IOException
-
preSetUserQuota
public void preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, GlobalQuotaSettings quotas) throws IOException Description copied from interface:MasterObserver
Called before the quota for the user is stored.- Specified by:
preSetUserQuota
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and masteruserName
- the name of userquotas
- the current quota for the user- Throws:
IOException
-
preSetUserQuota
public void preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, TableName tableName, GlobalQuotaSettings quotas) throws IOException Description copied from interface:MasterObserver
Called before the quota for the user on the specified table is stored.- Specified by:
preSetUserQuota
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and masteruserName
- the name of usertableName
- the name of the tablequotas
- the current quota for the user on the table- Throws:
IOException
-
preSetUserQuota
public void preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, String namespace, GlobalQuotaSettings quotas) throws IOException Description copied from interface:MasterObserver
Called before the quota for the user on the specified namespace is stored.- Specified by:
preSetUserQuota
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and masteruserName
- the name of usernamespace
- the name of the namespacequotas
- the current quota for the user on the namespace- Throws:
IOException
-
preSetTableQuota
public void preSetTableQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, GlobalQuotaSettings quotas) throws IOException Description copied from interface:MasterObserver
Called before the quota for the table is stored.- Specified by:
preSetTableQuota
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and mastertableName
- the name of the tablequotas
- the current quota for the table- Throws:
IOException
-
preSetNamespaceQuota
public void preSetNamespaceQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace, GlobalQuotaSettings quotas) throws IOException Description copied from interface:MasterObserver
Called before the quota for the namespace is stored.- Specified by:
preSetNamespaceQuota
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and masternamespace
- the name of the namespacequotas
- the current quota for the namespace- Throws:
IOException
-
preSetRegionServerQuota
public void preSetRegionServerQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String regionServer, GlobalQuotaSettings quotas) throws IOException Description copied from interface:MasterObserver
Called before the quota for the region server is stored.- Specified by:
preSetRegionServerQuota
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and masterregionServer
- the name of the region serverquotas
- the current quota for the region server- Throws:
IOException
-
postCreateReplicationEndPoint
public ReplicationEndpoint postCreateReplicationEndPoint(ObserverContext<RegionServerCoprocessorEnvironment> ctx, ReplicationEndpoint endpoint) Description copied from interface:RegionServerObserver
This will be called after the replication endpoint is instantiated.- Specified by:
postCreateReplicationEndPoint
in interfaceRegionServerObserver
- Parameters:
ctx
- the environment to interact with the framework and region server.endpoint
- - the base endpoint for replication- Returns:
- the endpoint to use during replication.
-
preReplicateLogEntries
public void preReplicateLogEntries(ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException Description copied from interface:RegionServerObserver
This will be called before executing replication request to shipping log entries.- Specified by:
preReplicateLogEntries
in interfaceRegionServerObserver
- Parameters:
ctx
- the environment to interact with the framework and region server.- Throws:
IOException
-
preClearCompactionQueues
public void preClearCompactionQueues(ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException Description copied from interface:RegionServerObserver
This will be called before clearing compaction queues- Specified by:
preClearCompactionQueues
in interfaceRegionServerObserver
- Parameters:
ctx
- the environment to interact with the framework and region server.- Throws:
IOException
-
preAddReplicationPeer
public void preAddReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId, ReplicationPeerConfig peerConfig) throws IOException Description copied from interface:MasterObserver
Called before add a replication peer- Specified by:
preAddReplicationPeer
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and masterpeerId
- a short name that identifies the peerpeerConfig
- configuration for the replication peer- Throws:
IOException
-
preRemoveReplicationPeer
public void preRemoveReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException Description copied from interface:MasterObserver
Called before remove a replication peer- Specified by:
preRemoveReplicationPeer
in interfaceMasterObserver
peerId
- a short name that identifies the peer- Throws:
IOException
-
preEnableReplicationPeer
public void preEnableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException Description copied from interface:MasterObserver
Called before enable a replication peer- Specified by:
preEnableReplicationPeer
in interfaceMasterObserver
peerId
- a short name that identifies the peer- Throws:
IOException
-
preDisableReplicationPeer
public void preDisableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException Description copied from interface:MasterObserver
Called before disable a replication peer- Specified by:
preDisableReplicationPeer
in interfaceMasterObserver
peerId
- a short name that identifies the peer- Throws:
IOException
-
preGetReplicationPeerConfig
public void preGetReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException Description copied from interface:MasterObserver
Called before get the configured ReplicationPeerConfig for the specified peer- Specified by:
preGetReplicationPeerConfig
in interfaceMasterObserver
peerId
- a short name that identifies the peer- Throws:
IOException
-
preUpdateReplicationPeerConfig
public void preUpdateReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId, ReplicationPeerConfig peerConfig) throws IOException Description copied from interface:MasterObserver
Called before update peerConfig for the specified peer- Specified by:
preUpdateReplicationPeerConfig
in interfaceMasterObserver
peerId
- a short name that identifies the peer- Throws:
IOException
-
preTransitReplicationPeerSyncReplicationState
public void preTransitReplicationPeerSyncReplicationState(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId, SyncReplicationState clusterState) throws IOException Description copied from interface:MasterObserver
Called before transit current cluster state for the specified synchronous replication peer- Specified by:
preTransitReplicationPeerSyncReplicationState
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and masterpeerId
- a short name that identifies the peerclusterState
- the new state- Throws:
IOException
-
preListReplicationPeers
public void preListReplicationPeers(ObserverContext<MasterCoprocessorEnvironment> ctx, String regex) throws IOException Description copied from interface:MasterObserver
Called before list replication peers.- Specified by:
preListReplicationPeers
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and masterregex
- The regular expression to match peer id- Throws:
IOException
-
preRequestLock
public void preRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace, TableName tableName, RegionInfo[] regionInfos, String description) throws IOException Description copied from interface:MasterObserver
Called before new LockProcedure is queued.- Specified by:
preRequestLock
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and master- Throws:
IOException
-
preLockHeartbeat
public void preLockHeartbeat(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, String description) throws IOException Description copied from interface:MasterObserver
Called before heartbeat to a lock.- Specified by:
preLockHeartbeat
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and master- Throws:
IOException
-
preExecuteProcedures
public void preExecuteProcedures(ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException Description copied from interface:RegionServerObserver
This will be called before executing procedures- Specified by:
preExecuteProcedures
in interfaceRegionServerObserver
- Parameters:
ctx
- the environment to interact with the framework and region server.- Throws:
IOException
-
preSwitchRpcThrottle
public void preSwitchRpcThrottle(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean enable) throws IOException Description copied from interface:MasterObserver
Called before switching rpc throttle enabled state.- Specified by:
preSwitchRpcThrottle
in interfaceMasterObserver
- Parameters:
ctx
- the coprocessor instance's environmentenable
- the rpc throttle value- Throws:
IOException
-
preIsRpcThrottleEnabled
public void preIsRpcThrottleEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException Description copied from interface:MasterObserver
Called before getting if is rpc throttle enabled.- Specified by:
preIsRpcThrottleEnabled
in interfaceMasterObserver
- Parameters:
ctx
- the coprocessor instance's environment- Throws:
IOException
-
preSwitchExceedThrottleQuota
public void preSwitchExceedThrottleQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean enable) throws IOException Description copied from interface:MasterObserver
Called before switching exceed throttle quota state.- Specified by:
preSwitchExceedThrottleQuota
in interfaceMasterObserver
- Parameters:
ctx
- the coprocessor instance's environmentenable
- the exceed throttle quota value- Throws:
IOException
-
getActiveUser
Returns the active user to which authorization checks should be applied. If we are in the context of an RPC call, the remote user is used, otherwise the currently logged in user is used.- Throws:
IOException
-
hasPermission
@Deprecated public void hasPermission(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.HasPermissionRequest request, org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.HasPermissionResponse> done) Deprecated.since 2.2.0 and will be removed in 4.0.0. UseAdmin.hasUserPermissions(String, List)
instead.- Specified by:
hasPermission
in interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.AccessControlService.Interface
- See Also:
-
preGrant
public void preGrant(ObserverContext<MasterCoprocessorEnvironment> ctx, UserPermission userPermission, boolean mergeExistingPermissions) throws IOException Description copied from interface:MasterObserver
Called before granting user permissions.- Specified by:
preGrant
in interfaceMasterObserver
- Parameters:
ctx
- the coprocessor instance's environmentuserPermission
- the user and permissionsmergeExistingPermissions
- True if merge with previous granted permissions- Throws:
IOException
-
preRevoke
public void preRevoke(ObserverContext<MasterCoprocessorEnvironment> ctx, UserPermission userPermission) throws IOException Description copied from interface:MasterObserver
Called before revoking user permissions.- Specified by:
preRevoke
in interfaceMasterObserver
- Parameters:
ctx
- the coprocessor instance's environmentuserPermission
- the user and permissions- Throws:
IOException
-
preGrantOrRevoke
private void preGrantOrRevoke(User caller, String request, UserPermission userPermission) throws IOException - Throws:
IOException
-
preGetUserPermissions
public void preGetUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, String namespace, TableName tableName, byte[] family, byte[] qualifier) throws IOException Description copied from interface:MasterObserver
Called before getting user permissions.- Specified by:
preGetUserPermissions
in interfaceMasterObserver
- Parameters:
ctx
- the coprocessor instance's environmentuserName
- the user name, null if get all user permissionsnamespace
- the namespace, null if don't get namespace permissiontableName
- the table name, null if don't get table permissionfamily
- the table column family, null if don't get table family permissionqualifier
- the table column qualifier, null if don't get table qualifier permission- Throws:
IOException
- if something went wrong
-
preGetUserPermissions
private void preGetUserPermissions(User caller, String userName, String namespace, TableName tableName, byte[] family, byte[] qualifier) throws IOException - Throws:
IOException
-
preHasUserPermissions
public void preHasUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, List<Permission> permissions) throws IOException - Specified by:
preHasUserPermissions
in interfaceMasterObserver
- Throws:
IOException
-
preHasUserPermissions
private void preHasUserPermissions(User caller, String userName, List<Permission> permissions) throws IOException - Throws:
IOException
-
preMoveServersAndTables
public void preMoveServersAndTables(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers, Set<TableName> tables, String targetGroup) throws IOException Description copied from interface:MasterObserver
Called before servers are moved to target region server group- Specified by:
preMoveServersAndTables
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and masterservers
- set of servers to movetargetGroup
- destination group- Throws:
IOException
-
preMoveServers
public void preMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers, String targetGroup) throws IOException Description copied from interface:MasterObserver
Called before servers are moved to target region server group- Specified by:
preMoveServers
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and masterservers
- set of servers to movetargetGroup
- destination group- Throws:
IOException
-
preMoveTables
public void preMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<TableName> tables, String targetGroup) throws IOException Description copied from interface:MasterObserver
Called before tables are moved to target region server group- Specified by:
preMoveTables
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and mastertables
- set of tables to movetargetGroup
- name of group- Throws:
IOException
-
preAddRSGroup
public void preAddRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String name) throws IOException Description copied from interface:MasterObserver
Called before a new region server group is added- Specified by:
preAddRSGroup
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and mastername
- group name- Throws:
IOException
-
preRemoveRSGroup
public void preRemoveRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String name) throws IOException Description copied from interface:MasterObserver
Called before a region server group is removed- Specified by:
preRemoveRSGroup
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and mastername
- group name- Throws:
IOException
-
preBalanceRSGroup
public void preBalanceRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName, BalanceRequest request) throws IOException Description copied from interface:MasterObserver
Called before a region server group is removed- Specified by:
preBalanceRSGroup
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and mastergroupName
- group name- Throws:
IOException
-
preRemoveServers
public void preRemoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers) throws IOException Description copied from interface:MasterObserver
Called before servers are removed from rsgroup- Specified by:
preRemoveServers
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and masterservers
- set of decommissioned servers to remove- Throws:
IOException
-
preGetRSGroupInfo
public void preGetRSGroupInfo(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName) throws IOException Description copied from interface:MasterObserver
Called before getting region server group info of the passed groupName.- Specified by:
preGetRSGroupInfo
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and mastergroupName
- name of the group to get RSGroupInfo for- Throws:
IOException
-
preGetRSGroupInfoOfTable
public void preGetRSGroupInfoOfTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException Description copied from interface:MasterObserver
Called before getting region server group info of the passed tableName.- Specified by:
preGetRSGroupInfoOfTable
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and mastertableName
- name of the table to get RSGroupInfo for- Throws:
IOException
-
preListRSGroups
Description copied from interface:MasterObserver
Called before listing region server group information.- Specified by:
preListRSGroups
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and master- Throws:
IOException
-
preListTablesInRSGroup
public void preListTablesInRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName) throws IOException Description copied from interface:MasterObserver
Called before listing all tables in the region server group.- Specified by:
preListTablesInRSGroup
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and mastergroupName
- name of the region server group- Throws:
IOException
-
preGetConfiguredNamespacesAndTablesInRSGroup
public void preGetConfiguredNamespacesAndTablesInRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName) throws IOException Description copied from interface:MasterObserver
Called before getting the configured namespaces and tables in the region server group.- Specified by:
preGetConfiguredNamespacesAndTablesInRSGroup
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and mastergroupName
- name of the region server group- Throws:
IOException
-
preGetRSGroupInfoOfServer
public void preGetRSGroupInfoOfServer(ObserverContext<MasterCoprocessorEnvironment> ctx, Address server) throws IOException Description copied from interface:MasterObserver
Called before getting region server group info of the passed server.- Specified by:
preGetRSGroupInfoOfServer
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and masterserver
- server to get RSGroupInfo for- Throws:
IOException
-
preRenameRSGroup
public void preRenameRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String oldName, String newName) throws IOException Description copied from interface:MasterObserver
Called before rename rsgroup.- Specified by:
preRenameRSGroup
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and masteroldName
- old rsgroup namenewName
- new rsgroup name- Throws:
IOException
-
preUpdateRSGroupConfig
public void preUpdateRSGroupConfig(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName, Map<String, String> configuration) throws IOExceptionDescription copied from interface:MasterObserver
Called before update rsgroup config.- Specified by:
preUpdateRSGroupConfig
in interfaceMasterObserver
- Parameters:
ctx
- the environment to interact with the framework and mastergroupName
- the group nameconfiguration
- new configuration of the group name to be set- Throws:
IOException
-
preClearRegionBlockCache
public void preClearRegionBlockCache(ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException Description copied from interface:RegionServerObserver
Called before clearing the block caches for one or more regions- Specified by:
preClearRegionBlockCache
in interfaceRegionServerObserver
- Parameters:
ctx
- the coprocessor instance's environment- Throws:
IOException
- if you need to signal an IO error
-
preUpdateRegionServerConfiguration
public void preUpdateRegionServerConfiguration(ObserverContext<RegionServerCoprocessorEnvironment> ctx, org.apache.hadoop.conf.Configuration preReloadConf) throws IOException Description copied from interface:RegionServerObserver
Called before reloading the RegionServer'sConfiguration
from disk- Specified by:
preUpdateRegionServerConfiguration
in interfaceRegionServerObserver
- Parameters:
ctx
- the coprocessor instance's environmentpreReloadConf
- theConfiguration
in use prior to reload- Throws:
IOException
- if you need to signal an IO error
-
preUpdateMasterConfiguration
public void preUpdateMasterConfiguration(ObserverContext<MasterCoprocessorEnvironment> ctx, org.apache.hadoop.conf.Configuration preReloadConf) throws IOException Description copied from interface:MasterObserver
Called before reloading the HMaster'sConfiguration
from disk- Specified by:
preUpdateMasterConfiguration
in interfaceMasterObserver
- Parameters:
ctx
- the coprocessor instance's environmentpreReloadConf
- theConfiguration
in use prior to reload- Throws:
IOException
- if you need to signal an IO error
-