@InterfaceAudience.LimitedPrivate(value="Configuration") public class AccessController extends Object implements MasterCoprocessor, RegionCoprocessor, RegionServerCoprocessor, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService.Interface, MasterObserver, RegionObserver, RegionServerObserver, EndpointObserver, BulkLoadObserver
AccessController
performs authorization checks for HBase operations based on:
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.
Modifier and Type | Class and Description |
---|---|
private static class |
AccessController.OpType |
Coprocessor.State
RegionObserver.MutationType
Modifier and Type | Field and Description |
---|---|
private AccessChecker |
accessChecker |
private boolean |
aclRegion
flags if we are running on a region of the _acl_ table
|
private boolean |
aclTabAvailable
if the ACL table is available, only relevant in the master
|
private static org.slf4j.Logger |
AUDITLOG |
private boolean |
authorizationEnabled
if we are active, usually false, only true if "hbase.security.authorization" has been set to
true in site configuration
|
private boolean |
cellFeaturesEnabled
if we are able to support cell ACLs
|
private static String |
CHECK_COVERING_PERM |
private boolean |
compatibleEarlyTermination
if we should terminate access checks early as soon as table or CF grants allow access; pre-0.98
compatible behavior
|
private boolean |
initialized
if we have been successfully initialized
|
private static org.slf4j.Logger |
LOG |
private RegionCoprocessorEnvironment |
regionEnv
defined only for Endpoint implementation, so it can have way to access region services
|
private Map<InternalScanner,String> |
scannerOwners
Mapping of scanner instances to the user who created them
|
private boolean |
shouldCheckExecPermission
if we should check EXEC permissions
|
private Map<TableName,List<UserPermission>> |
tableAcls |
private static String |
TAG_CHECK_PASSED |
private static byte[] |
TRUE |
private UserProvider |
userProvider
Provider for mapping principal names to Users
|
private ZKPermissionWatcher |
zkPermissionWatcher |
PRIORITY_HIGHEST, PRIORITY_LOWEST, PRIORITY_SYSTEM, PRIORITY_USER, VERSION
Constructor and Description |
---|
AccessController() |
Modifier and Type | Method and Description |
---|---|
private 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(com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse> done)
Deprecated.
since 2.2.0 and will be removed 4.0.0. Use
Admin.hasUserPermissions(List)
instead. |
private void |
checkSystemOrSuperUser(User activeUser) |
private static void |
createACLTable(Admin admin)
Create the ACL table n
|
private Cell |
createNewCellWithTags(Mutation mutation,
Cell oldCell,
Cell newCell) |
private User |
getActiveUser(ObserverContext<?> ctx)
Returns the active user to which authorization checks should be applied.
|
AuthManager |
getAuthManager() |
Optional<BulkLoadObserver> |
getBulkLoadObserver() |
Optional<EndpointObserver> |
getEndpointObserver() |
Optional<MasterObserver> |
getMasterObserver() |
Region |
getRegion() |
private Region |
getRegion(RegionCoprocessorEnvironment e) |
Optional<RegionObserver> |
getRegionObserver()
Observer/Service Getters
|
Optional<RegionServerObserver> |
getRegionServerObserver() |
Iterable<com.google.protobuf.Service> |
getServices()
Coprocessor endpoints providing protobuf services should override this method.
|
private TableName |
getTableName(Region region) |
private TableName |
getTableName(RegionCoprocessorEnvironment e) |
void |
getUserPermissions(com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GetUserPermissionsRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GetUserPermissionsResponse> done)
Deprecated.
since 2.2.0 and will be removed in 4.0.0. Use
Admin.getUserPermissions(GetUserPermissionsRequest) instead. |
void |
grant(com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse> done)
Deprecated.
since 2.2.0 and will be removed in 4.0.0. Use
Admin.grant(UserPermission, boolean) instead. |
private boolean |
hasFamilyQualifierPermission(User user,
Permission.Action perm,
RegionCoprocessorEnvironment env,
Map<byte[],? extends Collection<byte[]>> familyMap)
Returns
true if the current user is allowed the given action over at least one of
the column qualifiers in the given column families. |
void |
hasPermission(com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.HasPermissionRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.HasPermissionResponse> done)
Deprecated.
since 2.2.0 and will be removed in 4.0.0. Use
Admin.hasUserPermissions(String, List) instead. |
private void |
initialize(RegionCoprocessorEnvironment e) |
private void |
internalPreRead(ObserverContext<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 |
postAbortProcedure(ObserverContext<MasterCoprocessorEnvironment> ctx)
Called after a abortProcedure request has been processed.
|
List<Pair<Cell,Cell>> |
postAppendBeforeWAL(ObserverContext<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.
|
ReplicationEndpoint |
postCreateReplicationEndPoint(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
ReplicationEndpoint endpoint)
This will be called after the replication endpoint is instantiated.
|
void |
postDelete(ObserverContext<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<RegionCoprocessorEnvironment> ctx,
com.google.protobuf.Service service,
String methodName,
com.google.protobuf.Message request,
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.
|
List<Pair<Cell,Cell>> |
postIncrementBeforeWAL(ObserverContext<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 htd)
Called after the modifyTable operation has been requested.
|
void |
postOpen(ObserverContext<RegionCoprocessorEnvironment> c)
Called after the region is reported as open to the master.
|
void |
postPut(ObserverContext<RegionCoprocessorEnvironment> c,
Put put,
WALEdit edit,
Durability durability)
Called after the client stores a value.
|
void |
postRollWALWriterRequest(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
This will be called after executing user request to roll a region server WAL.
|
void |
postScannerClose(ObserverContext<RegionCoprocessorEnvironment> c,
InternalScanner s)
Called after the client closes a scanner.
|
RegionScanner |
postScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
Scan scan,
RegionScanner s)
Called after the client opens a new scanner.
|
void |
postStartMaster(ObserverContext<MasterCoprocessorEnvironment> ctx)
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 peer
|
Result |
preAppend(ObserverContext<RegionCoprocessorEnvironment> c,
Append append)
Called before Append.
|
void |
preAssign(ObserverContext<MasterCoprocessorEnvironment> c,
RegionInfo regionInfo)
Called prior to assigning a specific region.
|
void |
preBalance(ObserverContext<MasterCoprocessorEnvironment> c,
BalanceRequest request)
Called prior to requesting rebalancing of the cluster regions, though after the initial checks
for regions in transition and the balance switch flag.
|
void |
preBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> c,
boolean newValue)
Called prior to modifying the flag used to enable/disable region balancing.
|
void |
preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
MiniBatchOperationInProgress<Mutation> miniBatchOp)
This will be called for every batch mutation operation happening at the server.
|
void |
preBulkLoadHFile(ObserverContext<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<RegionCoprocessorEnvironment> c,
byte[] row,
byte[] family,
byte[] qualifier,
CompareOperator op,
ByteArrayComparable comparator,
Delete delete,
boolean result)
Called before checkAndDelete.
|
boolean |
preCheckAndDeleteAfterRowLock(ObserverContext<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<RegionCoprocessorEnvironment> c,
byte[] row,
byte[] family,
byte[] qualifier,
CompareOperator op,
ByteArrayComparable comparator,
Put put,
boolean result)
Called before checkAndPut.
|
boolean |
preCheckAndPutAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] row,
byte[] family,
byte[] qualifier,
CompareOperator opp,
ByteArrayComparable comparator,
Put put,
boolean result)
Called before checkAndPut but after acquiring rowlock.
|
void |
preCleanupBulkLoad(ObserverContext<RegionCoprocessorEnvironment> ctx)
Authorization security check for SecureBulkLoadProtocol.cleanupBulkLoad()
|
void |
preClearCompactionQueues(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
This will be called before clearing compaction queues
|
void |
preClearDeadServers(ObserverContext<MasterCoprocessorEnvironment> ctx)
Called before clear dead region servers.
|
void |
preCloneSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
SnapshotDescription snapshot,
TableDescriptor hTableDescriptor)
Called before a snapshot is cloned.
|
void |
preClose(ObserverContext<RegionCoprocessorEnvironment> c,
boolean abortRequested)
Called before the region is reported as closed to the master.
|
InternalScanner |
preCompact(ObserverContext<RegionCoprocessorEnvironment> c,
Store store,
InternalScanner scanner,
ScanType scanType,
CompactionLifeCycleTracker tracker,
CompactionRequest request)
Called prior to writing the
StoreFile s selected for compaction into a new
StoreFile . |
void |
preCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
NamespaceDescriptor ns)
Called before a new namespace is created by
HMaster . |
void |
preCreateTable(ObserverContext<MasterCoprocessorEnvironment> c,
TableDescriptor desc,
RegionInfo[] regions)
Observer implementations
|
void |
preDecommissionRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
List<ServerName> servers,
boolean offload)
Called before decommission region servers.
|
void |
preDelete(ObserverContext<RegionCoprocessorEnvironment> c,
Delete delete,
WALEdit edit,
Durability durability)
Called before the client deletes a value.
|
void |
preDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
String namespace)
Called before
HMaster deletes a namespace |
void |
preDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
SnapshotDescription snapshot)
Called before a snapshot is deleted.
|
void |
preDeleteTable(ObserverContext<MasterCoprocessorEnvironment> c,
TableName tableName)
Called before
HMaster deletes a table. |
void |
preDisableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx,
String peerId)
Called before disable a replication peer n * @param peerId a short name that identifies the
peer
|
void |
preDisableTable(ObserverContext<MasterCoprocessorEnvironment> c,
TableName tableName)
Called prior to disabling a table.
|
void |
preEnableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx,
String peerId)
Called before enable a replication peer n * @param peerId a short name that identifies the peer
|
void |
preEnableTable(ObserverContext<MasterCoprocessorEnvironment> c,
TableName tableName)
Called prior to enabling a table.
|
com.google.protobuf.Message |
preEndpointInvocation(ObserverContext<RegionCoprocessorEnvironment> ctx,
com.google.protobuf.Service service,
String methodName,
com.google.protobuf.Message request)
Called before an Endpoint service method is invoked.
|
void |
preExecuteProcedures(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
This will be called before executing procedures
|
boolean |
preExists(ObserverContext<RegionCoprocessorEnvironment> c,
Get get,
boolean exists)
Called before the client tests for existence using a Get.
|
void |
preFlush(ObserverContext<RegionCoprocessorEnvironment> c,
FlushLifeCycleTracker tracker)
Called before the memstore is flushed to disk.
|
void |
preGetLocks(ObserverContext<MasterCoprocessorEnvironment> ctx)
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<RegionCoprocessorEnvironment> c,
Get get,
List<Cell> result)
Called before the client performs a Get
|
void |
preGetProcedures(ObserverContext<MasterCoprocessorEnvironment> ctx)
Called before a getProcedures request has been processed.
|
void |
preGetReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx,
String peerId)
Called before get the configured ReplicationPeerConfig for the specified peer n * @param peerId
a short name that identifies the peer
|
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) |
Result |
preIncrement(ObserverContext<RegionCoprocessorEnvironment> c,
Increment increment)
Called before Increment.
|
void |
preIsRpcThrottleEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx)
Called before getting if is rpc throttle enabled.
|
void |
preListDecommissionedRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx)
Called before list decommissioned region servers.
|
void |
preListReplicationPeers(ObserverContext<MasterCoprocessorEnvironment> ctx,
String regex)
Called before list replication peers.
|
void |
preListSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
SnapshotDescription snapshot)
Called before listSnapshots request has been processed.
|
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.
|
String |
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 ns)
Called prior to modifying a namespace's properties.
|
TableDescriptor |
preModifyTable(ObserverContext<MasterCoprocessorEnvironment> c,
TableName tableName,
TableDescriptor currentDesc,
TableDescriptor newDesc)
Called prior to modifying a table's properties.
|
String |
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 |
preOpen(ObserverContext<RegionCoprocessorEnvironment> c)
Called before the region is reported as open to the master.
|
void |
prePrepareBulkLoad(ObserverContext<RegionCoprocessorEnvironment> ctx)
Authorization check for SecureBulkLoadProtocol.prepareBulkLoad()
|
void |
prePut(ObserverContext<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 |
preRemoveReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx,
String peerId)
Called before remove a replication peer n * @param peerId a short name that identifies the peer
|
void |
preReplicateLogEntries(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
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 |
preRollWALWriterRequest(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
This will be called before executing user request to roll a region server WAL.
|
void |
preScannerClose(ObserverContext<RegionCoprocessorEnvironment> c,
InternalScanner s)
Called before the client closes a scanner.
|
boolean |
preScannerNext(ObserverContext<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<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,
GlobalQuotaSettings quotas)
Called before the quota for the user 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,
TableName tableName,
GlobalQuotaSettings quotas)
Called before the quota for the user on the specified table is stored.
|
void |
preShutdown(ObserverContext<MasterCoprocessorEnvironment> c)
Called prior to shutting down the full HBase cluster, including this
HMaster 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 |
preStopMaster(ObserverContext<MasterCoprocessorEnvironment> c)
Called immediately prior to stopping this
HMaster
process. |
void |
preStopRegionServer(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
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 |
preTruncateTable(ObserverContext<MasterCoprocessorEnvironment> c,
TableName tableName)
Called before
HMaster truncates a table. |
void |
preUnassign(ObserverContext<MasterCoprocessorEnvironment> c,
RegionInfo regionInfo)
Called prior to unassigning a given region.
|
void |
preUpdateReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx,
String peerId,
ReplicationPeerConfig peerConfig)
Called before update peerConfig for the specified peer n * @param peerId a short name that
identifies the peer
|
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 |
requireScannerOwner(InternalScanner s)
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(com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse> done)
Deprecated.
since 2.2.0 and will be removed in 4.0.0. Use
Admin.revoke(UserPermission)
instead. |
void |
start(CoprocessorEnvironment env)
Called by the
CoprocessorEnvironment during it's own startup to initialize the
coprocessor. |
void |
stop(CoprocessorEnvironment env)
Called by the
CoprocessorEnvironment 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.
|
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
postAddReplicationPeer, postAddRSGroup, postAssign, postBalance, postBalanceRSGroup, postBalanceSwitch, postClearDeadServers, postCloneSnapshot, postCompletedDeleteTableAction, postCompletedDisableTableAction, postCompletedEnableTableAction, postCompletedMergeRegionsAction, postCompletedModifyTableAction, postCompletedModifyTableAction, postCompletedSnapshotAction, postCompletedSplitRegionAction, postCompletedTruncateTableAction, postCreateNamespace, postCreateTable, postDecommissionRegionServers, postDeleteSnapshot, postDisableReplicationPeer, postDisableTable, postEnableReplicationPeer, postEnableTable, postGetClusterMetrics, postGetLocks, postGetNamespaceDescriptor, postGetProcedures, postGetReplicationPeerConfig, postGetUserPermissions, postGrant, postHasUserPermissions, postIsRpcThrottleEnabled, postListDecommissionedRegionServers, postListReplicationPeers, postListSnapshot, postLockHeartbeat, postMasterStoreFlush, postMergeRegions, postMergeRegionsCommitAction, postModifyColumnFamilyStoreFileTracker, postModifyNamespace, postModifyNamespace, postModifyTable, 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, postUnassign, postUnassign, postUpdateReplicationPeerConfig, postUpdateRSGroupConfig, preAddRSGroup, preBalanceRSGroup, preCreateTableAction, preCreateTableRegionsInfos, preDeleteTableAction, preDisableTableAction, preEnableTableAction, preGetClusterMetrics, preGetTableNames, preListNamespaceDescriptors, preListNamespaces, preMasterInitialization, preMasterStoreFlush, preMergeRegionsAction, preMergeRegionsCommitAction, preModifyNamespace, preModifyTable, preModifyTableAction, preModifyTableAction, preMoveServers, preMoveServersAndTables, preMoveTables, preRemoveRSGroup, preRemoveServers, preRenameRSGroup, preSplitRegionAction, preSplitRegionAfterMETAAction, preSplitRegionBeforeMETAAction, preTruncateTableAction, preUnassign, preUpdateRSGroupConfig
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, postWALRestore, 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, preWALRestore
postClearCompactionQueues, postExecuteProcedures, postReplicateLogEntries
private static final org.slf4j.Logger LOG
private static final org.slf4j.Logger AUDITLOG
private static final String CHECK_COVERING_PERM
private static final String TAG_CHECK_PASSED
private static final byte[] TRUE
private AccessChecker accessChecker
private ZKPermissionWatcher zkPermissionWatcher
private boolean aclRegion
private RegionCoprocessorEnvironment regionEnv
private Map<InternalScanner,String> scannerOwners
private Map<TableName,List<UserPermission>> tableAcls
private UserProvider userProvider
private boolean authorizationEnabled
private boolean cellFeaturesEnabled
private boolean shouldCheckExecPermission
private boolean compatibleEarlyTermination
private volatile boolean initialized
private volatile boolean aclTabAvailable
public AccessController()
public static boolean isCellAuthorizationSupported(org.apache.hadoop.conf.Configuration conf)
public AuthManager getAuthManager()
private void initialize(RegionCoprocessorEnvironment e) throws IOException
IOException
private void updateACL(RegionCoprocessorEnvironment e, Map<byte[],List<Cell>> familyMap)
_acl_
table updates.private AuthResult permissionGranted(AccessController.OpType opType, User user, RegionCoprocessorEnvironment e, Map<byte[],? extends Collection<?>> families, Permission.Action... actions)
opType
- the operation typeuser
- the usere
- the coprocessor environmentfamilies
- the map of column families to qualifiers present in the requestactions
- the desired actionspublic void requireAccess(ObserverContext<?> ctx, String request, TableName tableName, Permission.Action... permissions) throws IOException
IOException
public void requirePermission(ObserverContext<?> ctx, String request, Permission.Action perm) throws IOException
IOException
public void requireGlobalPermission(ObserverContext<?> ctx, String request, Permission.Action perm, TableName tableName, Map<byte[],? extends Collection<byte[]>> familyMap) throws IOException
IOException
public void requireGlobalPermission(ObserverContext<?> ctx, String request, Permission.Action perm, String namespace) throws IOException
IOException
public void requireNamespacePermission(ObserverContext<?> ctx, String request, String namespace, Permission.Action... permissions) throws IOException
IOException
public void requireNamespacePermission(ObserverContext<?> ctx, String request, String namespace, TableName tableName, Map<byte[],? extends Collection<byte[]>> familyMap, Permission.Action... permissions) throws IOException
IOException
public void requirePermission(ObserverContext<?> ctx, String request, TableName tableName, byte[] family, byte[] qualifier, Permission.Action... permissions) throws IOException
IOException
public void requireTablePermission(ObserverContext<?> ctx, String request, TableName tableName, byte[] family, byte[] qualifier, Permission.Action... permissions) throws IOException
IOException
public void checkLockPermissions(ObserverContext<?> ctx, String namespace, TableName tableName, RegionInfo[] regionInfos, String reason) throws IOException
IOException
private boolean hasFamilyQualifierPermission(User user, Permission.Action perm, RegionCoprocessorEnvironment env, Map<byte[],? extends Collection<byte[]>> familyMap) throws IOException
true
if the current user is allowed the given action over at least one of
the column qualifiers in the given column families.IOException
private boolean checkCoveringPermission(User user, AccessController.OpType request, RegionCoprocessorEnvironment e, byte[] row, Map<byte[],? extends Collection<?>> familyMap, long opTs, Permission.Action... actions) throws IOException
IOException
private static void addCellPermissions(byte[] perms, Map<byte[],List<Cell>> familyMap)
private void checkForReservedTagPresence(User user, Mutation m) throws IOException
IOException
public void start(CoprocessorEnvironment env) throws IOException
Coprocessor
CoprocessorEnvironment
during it's own startup to initialize the
coprocessor.start
in interface Coprocessor
IOException
public void stop(CoprocessorEnvironment env)
Coprocessor
CoprocessorEnvironment
during it's own shutdown to stop the coprocessor.stop
in interface Coprocessor
public Optional<RegionObserver> getRegionObserver()
getRegionObserver
in interface RegionCoprocessor
public Optional<MasterObserver> getMasterObserver()
getMasterObserver
in interface MasterCoprocessor
public Optional<EndpointObserver> getEndpointObserver()
getEndpointObserver
in interface RegionCoprocessor
public Optional<BulkLoadObserver> getBulkLoadObserver()
getBulkLoadObserver
in interface RegionCoprocessor
public Optional<RegionServerObserver> getRegionServerObserver()
getRegionServerObserver
in interface RegionServerCoprocessor
public Iterable<com.google.protobuf.Service> getServices()
Coprocessor
getServices
in interface Coprocessor
Service
s or empty collection. Implementations should never return
null.public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> c, TableDescriptor desc, RegionInfo[] regions) throws IOException
preCreateTable
in interface MasterObserver
c
- the environment to interact with the framework and masterdesc
- the TableDescriptor for the tableregions
- the initial regions created for the tableIOException
public void postCompletedCreateTableAction(ObserverContext<MasterCoprocessorEnvironment> c, TableDescriptor desc, RegionInfo[] regions) throws IOException
MasterObserver
postCompletedCreateTableAction
in interface MasterObserver
c
- the environment to interact with the framework and masterdesc
- the TableDescriptor for the tableregions
- the initial regions created for the tableIOException
public void preDeleteTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName) throws IOException
MasterObserver
HMaster
deletes a table. Called as part of
delete table RPC call.preDeleteTable
in interface MasterObserver
c
- the environment to interact with the framework and mastertableName
- the name of the tableIOException
public void postDeleteTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName) throws IOException
MasterObserver
postDeleteTable
in interface MasterObserver
c
- the environment to interact with the framework and mastertableName
- the name of the tableIOException
public void preTruncateTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName) throws IOException
MasterObserver
HMaster
truncates a table. Called as part
of truncate table RPC call.preTruncateTable
in interface MasterObserver
c
- the environment to interact with the framework and mastertableName
- the name of the tableIOException
public void postTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
MasterObserver
postTruncateTable
in interface MasterObserver
ctx
- the environment to interact with the framework and mastertableName
- the name of the tableIOException
public TableDescriptor preModifyTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName, TableDescriptor currentDesc, TableDescriptor newDesc) throws IOException
MasterObserver
preModifyTable
in interface MasterObserver
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 descriptorIOException
public void postModifyTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName, TableDescriptor htd) throws IOException
MasterObserver
postModifyTable
in interface MasterObserver
c
- the environment to interact with the framework and mastertableName
- the name of the tablehtd
- current TableDescriptor of the tableIOException
public String preModifyTableStoreFileTracker(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName, String dstSFT) throws IOException
MasterObserver
preModifyTableStoreFileTracker
in interface MasterObserver
c
- the environment to interact with the framework and mastertableName
- the name of the tabledstSFT
- the store file trackerIOException
public String preModifyColumnFamilyStoreFileTracker(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName, byte[] family, String dstSFT) throws IOException
MasterObserver
preModifyColumnFamilyStoreFileTracker
in interface MasterObserver
c
- the environment to interact with the framework and mastertableName
- the name of the tablefamily
- the column familydstSFT
- the store file trackerIOException
public void preEnableTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName) throws IOException
MasterObserver
preEnableTable
in interface MasterObserver
c
- the environment to interact with the framework and mastertableName
- the name of the tableIOException
public void preDisableTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName) throws IOException
MasterObserver
preDisableTable
in interface MasterObserver
c
- the environment to interact with the framework and mastertableName
- the name of the tableIOException
public void preAbortProcedure(ObserverContext<MasterCoprocessorEnvironment> ctx, long procId) throws IOException
MasterObserver
preAbortProcedure
in interface MasterObserver
ctx
- the environment to interact with the framework and masterprocId
- the Id of the procedureIOException
public void postAbortProcedure(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
MasterObserver
postAbortProcedure
in interface MasterObserver
ctx
- the environment to interact with the framework and masterIOException
public void preGetProcedures(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
MasterObserver
preGetProcedures
in interface MasterObserver
ctx
- the environment to interact with the framework and masterIOException
public void preGetLocks(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
MasterObserver
preGetLocks
in interface MasterObserver
ctx
- the environment to interact with the framework and masterIOException
- if something went wrongpublic void preMove(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo region, ServerName srcServer, ServerName destServer) throws IOException
MasterObserver
preMove
in interface MasterObserver
c
- the environment to interact with the framework and masterregion
- the RegionInfosrcServer
- the source ServerNamedestServer
- the destination ServerNameIOException
public void preAssign(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfo) throws IOException
MasterObserver
preAssign
in interface MasterObserver
c
- the environment to interact with the framework and masterregionInfo
- the regionInfo of the regionIOException
public void preUnassign(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfo) throws IOException
MasterObserver
preUnassign
in interface MasterObserver
c
- the environment to interact with the framework and master nIOException
public void preRegionOffline(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfo) throws IOException
MasterObserver
preRegionOffline
in interface MasterObserver
c
- the environment to interact with the framework and master nIOException
public void preSetSplitOrMergeEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean newValue, MasterSwitchType switchType) throws IOException
MasterObserver
preSetSplitOrMergeEnabled
in interface MasterObserver
ctx
- the coprocessor instance's environmentnewValue
- the new value submitted in the callswitchType
- type of switchIOException
public void preBalance(ObserverContext<MasterCoprocessorEnvironment> c, BalanceRequest request) throws IOException
MasterObserver
preBalance
in interface MasterObserver
c
- the environment to interact with the framework and masterrequest
- the request used to trigger the balancerIOException
public void preBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> c, boolean newValue) throws IOException
MasterObserver
preBalanceSwitch
in interface MasterObserver
c
- the coprocessor instance's environmentIOException
public void preShutdown(ObserverContext<MasterCoprocessorEnvironment> c) throws IOException
MasterObserver
HMaster
process.preShutdown
in interface MasterObserver
IOException
public void preStopMaster(ObserverContext<MasterCoprocessorEnvironment> c) throws IOException
MasterObserver
HMaster
process.preStopMaster
in interface MasterObserver
IOException
public void postStartMaster(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
MasterObserver
postStartMaster
in interface MasterObserver
IOException
private static void createACLTable(Admin admin) throws IOException
IOException
public void preSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor hTableDescriptor) throws IOException
MasterObserver
preSnapshot
in interface MasterObserver
ctx
- the environment to interact with the framework and mastersnapshot
- the SnapshotDescriptor for the snapshothTableDescriptor
- the TableDescriptor of the table to snapshotIOException
public void preListSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot) throws IOException
MasterObserver
preListSnapshot
in interface MasterObserver
ctx
- the environment to interact with the framework and mastersnapshot
- the SnapshotDescriptor of the snapshot to listIOException
public void preCloneSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor hTableDescriptor) throws IOException
MasterObserver
preCloneSnapshot
in interface MasterObserver
ctx
- the environment to interact with the framework and mastersnapshot
- the SnapshotDescriptor for the snapshothTableDescriptor
- the TableDescriptor of the table to createIOException
public void preRestoreSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor hTableDescriptor) throws IOException
MasterObserver
preRestoreSnapshot
in interface MasterObserver
ctx
- the environment to interact with the framework and mastersnapshot
- the SnapshotDescriptor for the snapshothTableDescriptor
- the TableDescriptor of the table to restoreIOException
public void preDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot) throws IOException
MasterObserver
preDeleteSnapshot
in interface MasterObserver
ctx
- the environment to interact with the framework and mastersnapshot
- the SnapshotDescriptor of the snapshot to deleteIOException
public void preCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor ns) throws IOException
MasterObserver
HMaster
.preCreateNamespace
in interface MasterObserver
ctx
- the environment to interact with the framework and masterns
- the NamespaceDescriptor for the tableIOException
public void preDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) throws IOException
MasterObserver
HMaster
deletes a namespacepreDeleteNamespace
in interface MasterObserver
ctx
- the environment to interact with the framework and masternamespace
- the name of the namespaceIOException
public void postDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) throws IOException
MasterObserver
postDeleteNamespace
in interface MasterObserver
ctx
- the environment to interact with the framework and masternamespace
- the name of the namespaceIOException
public void preModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor ns) throws IOException
MasterObserver
preModifyNamespace
in interface MasterObserver
ctx
- the environment to interact with the framework and masterns
- after modify operation, namespace will have this descriptorIOException
public void preGetNamespaceDescriptor(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) throws IOException
MasterObserver
preGetNamespaceDescriptor
in interface MasterObserver
ctx
- the environment to interact with the framework and masternamespace
- the name of the namespaceIOException
public void postListNamespaces(ObserverContext<MasterCoprocessorEnvironment> ctx, List<String> namespaces) throws IOException
MasterObserver
postListNamespaces
in interface MasterObserver
ctx
- the environment to interact with the framework and masternamespaces
- the list of namespaces about to be returnedIOException
- if something went wrongpublic void postListNamespaceDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<NamespaceDescriptor> descriptors) throws IOException
MasterObserver
postListNamespaceDescriptors
in interface MasterObserver
ctx
- the environment to interact with the framework and masterdescriptors
- the list of descriptors about to be returnedIOException
public void preTableFlush(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
MasterObserver
preTableFlush
in interface MasterObserver
ctx
- the environment to interact with the framework and mastertableName
- the name of the tableIOException
public void preSplitRegion(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, byte[] splitRow) throws IOException
MasterObserver
preSplitRegion
in interface MasterObserver
ctx
- the environment to interact with the framework and mastertableName
- the table where the region belongs tosplitRow
- split pointIOException
public void preClearDeadServers(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
MasterObserver
preClearDeadServers
in interface MasterObserver
IOException
public void preDecommissionRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx, List<ServerName> servers, boolean offload) throws IOException
MasterObserver
preDecommissionRegionServers
in interface MasterObserver
IOException
public void preListDecommissionedRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
MasterObserver
preListDecommissionedRegionServers
in interface MasterObserver
IOException
public void preRecommissionRegionServer(ObserverContext<MasterCoprocessorEnvironment> ctx, ServerName server, List<byte[]> encodedRegionNames) throws IOException
MasterObserver
preRecommissionRegionServer
in interface MasterObserver
IOException
public void preOpen(ObserverContext<RegionCoprocessorEnvironment> c) throws IOException
RegionObserver
preOpen
in interface RegionObserver
c
- the environment provided by the region serverIOException
public void postOpen(ObserverContext<RegionCoprocessorEnvironment> c)
RegionObserver
postOpen
in interface RegionObserver
c
- the environment provided by the region serverpublic void preFlush(ObserverContext<RegionCoprocessorEnvironment> c, FlushLifeCycleTracker tracker) throws IOException
RegionObserver
preFlush
in interface RegionObserver
c
- the environment provided by the region servertracker
- tracker used to track the life cycle of a flushIOException
public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> c, Store store, InternalScanner scanner, ScanType scanType, CompactionLifeCycleTracker tracker, CompactionRequest request) throws IOException
RegionObserver
StoreFile
s selected for compaction into a new
StoreFile
.
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 inspect Cell
s from the wrapped scanner,
applying its own policy to what gets written.
preCompact
in interface RegionObserver
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 compactionnull
unless the
implementation is writing new store files on its own.IOException
private void internalPreRead(ObserverContext<RegionCoprocessorEnvironment> c, Query query, AccessController.OpType opType) throws IOException
IOException
public void preGetOp(ObserverContext<RegionCoprocessorEnvironment> c, Get get, List<Cell> result) throws IOException
RegionObserver
Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
preGetOp
in interface RegionObserver
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.IOException
public boolean preExists(ObserverContext<RegionCoprocessorEnvironment> c, Get get, boolean exists) throws IOException
RegionObserver
Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
preExists
in interface RegionObserver
c
- the environment provided by the region serverget
- the Get requestexists
- the result returned by the region serverIOException
public void prePut(ObserverContext<RegionCoprocessorEnvironment> c, Put put, WALEdit edit, Durability durability) throws IOException
RegionObserver
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.
prePut
in interface RegionObserver
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 PutIOException
public void postPut(ObserverContext<RegionCoprocessorEnvironment> c, Put put, WALEdit edit, Durability durability)
RegionObserver
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.
postPut
in interface RegionObserver
c
- the environment provided by the region serverput
- The Put objectedit
- The WALEdit object for the waldurability
- Persistence guarantee for this Putpublic void preDelete(ObserverContext<RegionCoprocessorEnvironment> c, Delete delete, WALEdit edit, Durability durability) throws IOException
RegionObserver
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.
preDelete
in interface RegionObserver
c
- the environment provided by the region serverdelete
- The Delete objectedit
- The WALEdit object for the waldurability
- Persistence guarantee for this DeleteIOException
public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c, MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException
RegionObserver
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.
preBatchMutate
in interface RegionObserver
c
- the environment provided by the region serverminiBatchOp
- batch of Mutations getting applied to region.IOException
public void postDelete(ObserverContext<RegionCoprocessorEnvironment> c, Delete delete, WALEdit edit, Durability durability) throws IOException
RegionObserver
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.
postDelete
in interface RegionObserver
c
- the environment provided by the region serverdelete
- The Delete objectedit
- The WALEdit object for the waldurability
- Persistence guarantee for this DeleteIOException
public boolean preCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Put put, boolean result) throws IOException
RegionObserver
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.
preCheckAndPut
in interface RegionObserver
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 resultIOException
public boolean preCheckAndPutAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator opp, ByteArrayComparable comparator, Put put, boolean result) throws IOException
RegionObserver
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.
preCheckAndPutAfterRowLock
in interface RegionObserver
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 resultIOException
public boolean preCheckAndDelete(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Delete delete, boolean result) throws IOException
RegionObserver
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.
preCheckAndDelete
in interface RegionObserver
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 resultIOException
public boolean preCheckAndDeleteAfterRowLock(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Delete delete, boolean result) throws IOException
RegionObserver
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.
preCheckAndDeleteAfterRowLock
in interface RegionObserver
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 resultIOException
public Result preAppend(ObserverContext<RegionCoprocessorEnvironment> c, Append append) throws IOException
RegionObserver
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.
preAppend
in interface RegionObserver
c
- the environment provided by the region serverappend
- Append objectIOException
public Result preIncrement(ObserverContext<RegionCoprocessorEnvironment> c, Increment increment) throws IOException
RegionObserver
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.
preIncrement
in interface RegionObserver
c
- the environment provided by the region serverincrement
- increment objectIOException
public List<Pair<Cell,Cell>> postIncrementBeforeWAL(ObserverContext<RegionCoprocessorEnvironment> ctx, Mutation mutation, List<Pair<Cell,Cell>> cellPairs) throws IOException
RegionObserver
postIncrementBeforeWAL
in interface RegionObserver
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.IOException
public List<Pair<Cell,Cell>> postAppendBeforeWAL(ObserverContext<RegionCoprocessorEnvironment> ctx, Mutation mutation, List<Pair<Cell,Cell>> cellPairs) throws IOException
RegionObserver
postAppendBeforeWAL
in interface RegionObserver
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.IOException
private Cell createNewCellWithTags(Mutation mutation, Cell oldCell, Cell newCell)
public void preScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Scan scan) throws IOException
RegionObserver
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.
preScannerOpen
in interface RegionObserver
c
- the environment provided by the region serverscan
- the Scan specificationIOException
public RegionScanner postScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Scan scan, RegionScanner s) throws IOException
RegionObserver
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.
postScannerOpen
in interface RegionObserver
c
- the environment provided by the region serverscan
- the Scan specifications
- if not null, the base scannerIOException
public boolean preScannerNext(ObserverContext<RegionCoprocessorEnvironment> c, InternalScanner s, List<Result> result, int limit, boolean hasNext) throws IOException
RegionObserver
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.
preScannerNext
in interface RegionObserver
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' indicationIOException
public void preScannerClose(ObserverContext<RegionCoprocessorEnvironment> c, InternalScanner s) throws IOException
RegionObserver
Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
preScannerClose
in interface RegionObserver
c
- the environment provided by the region servers
- the scannerIOException
public void postScannerClose(ObserverContext<RegionCoprocessorEnvironment> c, InternalScanner s) throws IOException
RegionObserver
postScannerClose
in interface RegionObserver
c
- the environment provided by the region servers
- the scannerIOException
private void requireScannerOwner(InternalScanner s) throws AccessDeniedException
AccessDeniedException
public void preBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx, List<Pair<byte[],String>> familyPaths) throws IOException
preBulkLoadHFile
in interface RegionObserver
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.IOException
public void prePrepareBulkLoad(ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException
prePrepareBulkLoad
in interface BulkLoadObserver
ctx
- the context nIOException
public void preCleanupBulkLoad(ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException
preCleanupBulkLoad
in interface BulkLoadObserver
ctx
- the context nIOException
public com.google.protobuf.Message preEndpointInvocation(ObserverContext<RegionCoprocessorEnvironment> ctx, com.google.protobuf.Service service, String methodName, com.google.protobuf.Message request) throws IOException
EndpointObserver
ObserverContext.bypass()
has no effect in this
hook.preEndpointInvocation
in interface EndpointObserver
ctx
- the environment provided by the region serverservice
- the endpoint servicemethodName
- the invoked service methodrequest
- Request message expected by given Service
's method (by the name
methodName
).IOException
public void postEndpointInvocation(ObserverContext<RegionCoprocessorEnvironment> ctx, com.google.protobuf.Service service, String methodName, com.google.protobuf.Message request, com.google.protobuf.Message.Builder responseBuilder) throws IOException
EndpointObserver
postEndpointInvocation
in interface EndpointObserver
ctx
- the environment provided by the region serverservice
- the endpoint servicemethodName
- the invoked service methodrequest
- Request message expected by given Service
's method (by the name
methodName
).responseBuilder
- Builder for final response to the client, with original response from
Service's method merged into it.IOException
@Deprecated public void grant(com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest request, com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse> done)
Admin.grant(UserPermission, boolean)
instead.grant
in interface org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService.Interface
Admin.grant(UserPermission, boolean)
,
HBASE-21739@Deprecated public void revoke(com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest request, com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse> done)
Admin.revoke(UserPermission)
instead.revoke
in interface org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService.Interface
Admin.revoke(UserPermission)
,
HBASE-21739@Deprecated public void getUserPermissions(com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GetUserPermissionsRequest request, com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GetUserPermissionsResponse> done)
Admin.getUserPermissions(GetUserPermissionsRequest)
instead.getUserPermissions
in interface org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService.Interface
Admin.getUserPermissions(GetUserPermissionsRequest)
,
HBASE-21911@Deprecated public void checkPermissions(com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest request, com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse> done)
Admin.hasUserPermissions(List)
instead.checkPermissions
in interface org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService.Interface
Admin.hasUserPermissions(List)
,
HBASE-22117private Region getRegion(RegionCoprocessorEnvironment e)
private TableName getTableName(RegionCoprocessorEnvironment e)
private TableName getTableName(Region region)
public void preClose(ObserverContext<RegionCoprocessorEnvironment> c, boolean abortRequested) throws IOException
RegionObserver
preClose
in interface RegionObserver
c
- the environment provided by the region serverabortRequested
- true if the region server is abortingIOException
private void checkSystemOrSuperUser(User activeUser) throws IOException
IOException
public void preStopRegionServer(ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException
RegionServerObserver
preStopRegionServer
in interface RegionServerObserver
ctx
- the environment to interact with the framework and region server.IOException
private Map<byte[],? extends Collection<byte[]>> makeFamilyMap(byte[] family, byte[] qualifier)
public void preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableName> tableNamesList, List<TableDescriptor> descriptors, String regex) throws IOException
MasterObserver
preGetTableDescriptors
in interface MasterObserver
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 namesIOException
public void postGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableName> tableNamesList, List<TableDescriptor> descriptors, String regex) throws IOException
MasterObserver
postGetTableDescriptors
in interface MasterObserver
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 namesIOException
public void postGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableDescriptor> descriptors, String regex) throws IOException
MasterObserver
postGetTableNames
in interface MasterObserver
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 namesIOException
public void preMergeRegions(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge) throws IOException
MasterObserver
preMergeRegions
in interface MasterObserver
ctx
- coprocessor environmentregionsToMerge
- regions to be mergedIOException
public void preRollWALWriterRequest(ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException
RegionServerObserver
preRollWALWriterRequest
in interface RegionServerObserver
ctx
- the environment to interact with the framework and region server.IOException
public void postRollWALWriterRequest(ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException
RegionServerObserver
postRollWALWriterRequest
in interface RegionServerObserver
ctx
- the environment to interact with the framework and region server.IOException
public void preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, GlobalQuotaSettings quotas) throws IOException
MasterObserver
preSetUserQuota
in interface MasterObserver
ctx
- the environment to interact with the framework and masteruserName
- the name of userquotas
- the current quota for the userIOException
public void preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, TableName tableName, GlobalQuotaSettings quotas) throws IOException
MasterObserver
preSetUserQuota
in interface MasterObserver
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 tableIOException
public void preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, String namespace, GlobalQuotaSettings quotas) throws IOException
MasterObserver
preSetUserQuota
in interface MasterObserver
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 namespaceIOException
public void preSetTableQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, GlobalQuotaSettings quotas) throws IOException
MasterObserver
preSetTableQuota
in interface MasterObserver
ctx
- the environment to interact with the framework and mastertableName
- the name of the tablequotas
- the current quota for the tableIOException
public void preSetNamespaceQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace, GlobalQuotaSettings quotas) throws IOException
MasterObserver
preSetNamespaceQuota
in interface MasterObserver
ctx
- the environment to interact with the framework and masternamespace
- the name of the namespacequotas
- the current quota for the namespaceIOException
public void preSetRegionServerQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String regionServer, GlobalQuotaSettings quotas) throws IOException
MasterObserver
preSetRegionServerQuota
in interface MasterObserver
ctx
- the environment to interact with the framework and masterregionServer
- the name of the region serverquotas
- the current quota for the region serverIOException
public ReplicationEndpoint postCreateReplicationEndPoint(ObserverContext<RegionServerCoprocessorEnvironment> ctx, ReplicationEndpoint endpoint)
RegionServerObserver
postCreateReplicationEndPoint
in interface RegionServerObserver
ctx
- the environment to interact with the framework and region server.endpoint
- - the base endpoint for replicationpublic void preReplicateLogEntries(ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException
RegionServerObserver
preReplicateLogEntries
in interface RegionServerObserver
ctx
- the environment to interact with the framework and region server.IOException
public void preClearCompactionQueues(ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException
RegionServerObserver
preClearCompactionQueues
in interface RegionServerObserver
ctx
- the environment to interact with the framework and region server.IOException
public void preAddReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId, ReplicationPeerConfig peerConfig) throws IOException
MasterObserver
preAddReplicationPeer
in interface MasterObserver
ctx
- the environment to interact with the framework and masterpeerId
- a short name that identifies the peerpeerConfig
- configuration for the replication peerIOException
public void preRemoveReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException
MasterObserver
preRemoveReplicationPeer
in interface MasterObserver
IOException
public void preEnableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException
MasterObserver
preEnableReplicationPeer
in interface MasterObserver
IOException
public void preDisableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException
MasterObserver
preDisableReplicationPeer
in interface MasterObserver
IOException
public void preGetReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException
MasterObserver
preGetReplicationPeerConfig
in interface MasterObserver
IOException
public void preUpdateReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId, ReplicationPeerConfig peerConfig) throws IOException
MasterObserver
preUpdateReplicationPeerConfig
in interface MasterObserver
IOException
public void preListReplicationPeers(ObserverContext<MasterCoprocessorEnvironment> ctx, String regex) throws IOException
MasterObserver
preListReplicationPeers
in interface MasterObserver
ctx
- the environment to interact with the framework and masterregex
- The regular expression to match peer idIOException
public void preRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace, TableName tableName, RegionInfo[] regionInfos, String description) throws IOException
MasterObserver
preRequestLock
in interface MasterObserver
ctx
- the environment to interact with the framework and masterIOException
public void preLockHeartbeat(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, String description) throws IOException
MasterObserver
preLockHeartbeat
in interface MasterObserver
ctx
- the environment to interact with the framework and masterIOException
public void preExecuteProcedures(ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException
RegionServerObserver
preExecuteProcedures
in interface RegionServerObserver
ctx
- the environment to interact with the framework and region server.IOException
public void preSwitchRpcThrottle(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean enable) throws IOException
MasterObserver
preSwitchRpcThrottle
in interface MasterObserver
ctx
- the coprocessor instance's environmentenable
- the rpc throttle valueIOException
public void preIsRpcThrottleEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
MasterObserver
preIsRpcThrottleEnabled
in interface MasterObserver
ctx
- the coprocessor instance's environmentIOException
public void preSwitchExceedThrottleQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean enable) throws IOException
MasterObserver
preSwitchExceedThrottleQuota
in interface MasterObserver
ctx
- the coprocessor instance's environmentenable
- the exceed throttle quota valueIOException
private User getActiveUser(ObserverContext<?> ctx) throws IOException
IOException
@Deprecated public void hasPermission(com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.HasPermissionRequest request, com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.HasPermissionResponse> done)
Admin.hasUserPermissions(String, List)
instead.hasPermission
in interface org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService.Interface
Admin.hasUserPermissions(String, List)
,
HBASE-22117public void preGrant(ObserverContext<MasterCoprocessorEnvironment> ctx, UserPermission userPermission, boolean mergeExistingPermissions) throws IOException
MasterObserver
preGrant
in interface MasterObserver
ctx
- the coprocessor instance's environmentuserPermission
- the user and permissionsmergeExistingPermissions
- True if merge with previous granted permissionsIOException
public void preRevoke(ObserverContext<MasterCoprocessorEnvironment> ctx, UserPermission userPermission) throws IOException
MasterObserver
preRevoke
in interface MasterObserver
ctx
- the coprocessor instance's environmentuserPermission
- the user and permissionsIOException
private void preGrantOrRevoke(User caller, String request, UserPermission userPermission) throws IOException
IOException
public void preGetUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, String namespace, TableName tableName, byte[] family, byte[] qualifier) throws IOException
MasterObserver
preGetUserPermissions
in interface MasterObserver
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 permissionIOException
- if something went wrongprivate void preGetUserPermissions(User caller, String userName, String namespace, TableName tableName, byte[] family, byte[] qualifier) throws IOException
IOException
public void preHasUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, List<Permission> permissions) throws IOException
preHasUserPermissions
in interface MasterObserver
IOException
private void preHasUserPermissions(User caller, String userName, List<Permission> permissions) throws IOException
IOException
Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.