Class VisibilityController
- All Implemented Interfaces:
Coprocessor
,MasterCoprocessor
,MasterObserver
,RegionCoprocessor
,RegionObserver
,org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService.Interface
-
Nested Class Summary
Modifier and TypeClassDescriptionprivate static class
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 boolean
private static final org.slf4j.Logger
(package private) boolean
if we are active, usually false, only true if "hbase.security.authorization" has been set to true in site configurationprivate boolean
private org.apache.hadoop.conf.Configuration
private boolean
private boolean
private static final org.slf4j.Logger
private Map<InternalScanner,
String> Mapping of scanner instances to the user who created themprivate VisibilityLabelService
Fields inherited from interface org.apache.hadoop.hbase.Coprocessor
PRIORITY_HIGHEST, PRIORITY_LOWEST, PRIORITY_SYSTEM, PRIORITY_USER, VERSION
-
Constructor Summary
-
Method Summary
Modifier and TypeMethodDescriptionvoid
addLabels
(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsRequest request, org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse> done) VisibilityEndpoint service related methodsprivate static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair
Returns NameValuePair of the exception name to stringified version os exception.private void
checkForReservedVisibilityTagPresence
(ExtendedCell cell, Pair<Boolean, Tag> pair) Checks whether cell contains any tag with type as VISIBILITY_TAG_TYPE.void
clearAuths
(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.SetAuthsRequest request, org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse> done) private Cell
createNewCellWithTags
(Mutation mutation, ExtendedCell newCell) void
getAuths
(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.GetAuthsRequest request, org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.GetAuthsResponse> done) Observer/Service GettersIterable<org.apache.hbase.thirdparty.com.google.protobuf.Service>
Coprocessor endpoints providing protobuf services should override this method.private void
static boolean
isCellAuthorizationSupported
(org.apache.hadoop.conf.Configuration conf) private boolean
void
listLabels
(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.ListLabelsRequest request, org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.ListLabelsResponse> done) private void
logResult
(boolean isAllowed, String request, String reason, byte[] user, List<byte[]> labelAuths, String regex) 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.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.postInstantiateDeleteTracker
(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, DeleteTracker delTracker) Called after the ScanQueryMatcher creates ScanDeleteTracker.void
postOpen
(ObserverContext<? extends RegionCoprocessorEnvironment> e) Region related hooksvoid
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
postSetSplitOrMergeEnabled
(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean newValue, MasterSwitchType switchType) Called after setting split / merge switchvoid
Master related hooksvoid
preBatchMutate
(ObserverContext<? extends RegionCoprocessorEnvironment> c, MiniBatchOperationInProgress<Mutation> miniBatchOp) This will be called for every batch mutation operation happening at the server.void
preDisableTable
(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) Called prior to disabling a table.void
preGetOp
(ObserverContext<? extends RegionCoprocessorEnvironment> e, Get get, List<Cell> results) Called before the client performs a GetpreModifyTable
(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor currentDescriptor, TableDescriptor newDescriptor) Called prior to modifying a table's properties.void
prePrepareTimeStampForDeleteVersion
(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, Mutation delete, Cell cell, byte[] byteNow, Get get) Called before the server updates the timestamp for version delete with latest timestamp.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> e, Scan scan) Called before the client opens a new scanner.private void
private void
Verify, when servicing an RPC, that the caller is the scanner owner.void
setAuths
(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.SetAuthsRequest request, org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse> done) private void
setExceptionResults
(int size, IOException e, org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse.Builder response) 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.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
postAbortProcedure, postAddReplicationPeer, postAddRSGroup, postAssign, postBalance, postBalanceRSGroup, postBalanceSwitch, postClearDeadServers, postCloneSnapshot, postCompletedCreateTableAction, postCompletedDeleteTableAction, postCompletedDisableTableAction, postCompletedEnableTableAction, postCompletedMergeRegionsAction, postCompletedModifyTableAction, postCompletedSnapshotAction, postCompletedSplitRegionAction, postCompletedTruncateTableAction, postCreateNamespace, postCreateTable, postDecommissionRegionServers, postDeleteNamespace, postDeleteSnapshot, postDeleteTable, postDisableReplicationPeer, postDisableTable, postEnableReplicationPeer, postEnableTable, postGetClusterMetrics, postGetConfiguredNamespacesAndTablesInRSGroup, postGetLocks, postGetNamespaceDescriptor, postGetProcedures, postGetReplicationPeerConfig, postGetRSGroupInfo, postGetRSGroupInfoOfServer, postGetRSGroupInfoOfTable, postGetTableDescriptors, postGetTableNames, postGetUserPermissions, postGrant, postHasUserPermissions, postIsRpcThrottleEnabled, postListDecommissionedRegionServers, postListNamespaceDescriptors, postListNamespaces, postListReplicationPeers, postListRSGroups, postListSnapshot, postListTablesInRSGroup, postLockHeartbeat, postMasterStoreFlush, postMergeRegions, postMergeRegionsCommitAction, postModifyColumnFamilyStoreFileTracker, postModifyNamespace, postModifyTable, postModifyTableStoreFileTracker, postMove, postMoveServers, postMoveServersAndTables, postMoveTables, postRecommissionRegionServer, postRegionOffline, postRemoveReplicationPeer, postRemoveRSGroup, postRemoveServers, postRenameRSGroup, postRequestLock, postRestoreSnapshot, postRevoke, postRollBackMergeRegionsAction, postRollBackSplitRegionAction, postSetNamespaceQuota, postSetRegionServerQuota, postSetTableQuota, postSetUserQuota, postSetUserQuota, postSetUserQuota, postSnapshot, postSwitchExceedThrottleQuota, postSwitchRpcThrottle, postTableFlush, postTransitReplicationPeerSyncReplicationState, postTruncateRegion, postTruncateRegionAction, postTruncateTable, postUnassign, postUpdateMasterConfiguration, postUpdateReplicationPeerConfig, postUpdateRSGroupConfig, preAbortProcedure, preAddReplicationPeer, preAddRSGroup, preAssign, preBalance, preBalanceRSGroup, preBalanceSwitch, preClearDeadServers, preCloneSnapshot, preCreateNamespace, preCreateTable, preCreateTableAction, preCreateTableRegionsInfos, preDecommissionRegionServers, preDeleteNamespace, preDeleteSnapshot, preDeleteTable, preDeleteTableAction, preDisableReplicationPeer, preDisableTableAction, preEnableReplicationPeer, preEnableTable, preEnableTableAction, preGetClusterMetrics, preGetConfiguredNamespacesAndTablesInRSGroup, preGetLocks, preGetNamespaceDescriptor, preGetProcedures, preGetReplicationPeerConfig, preGetRSGroupInfo, preGetRSGroupInfoOfServer, preGetRSGroupInfoOfTable, preGetTableDescriptors, preGetTableNames, preGetUserPermissions, preGrant, preHasUserPermissions, preIsRpcThrottleEnabled, preListDecommissionedRegionServers, preListNamespaceDescriptors, preListNamespaces, preListReplicationPeers, preListRSGroups, preListSnapshot, preListTablesInRSGroup, preLockHeartbeat, preMasterInitialization, preMasterStoreFlush, preMergeRegions, preMergeRegionsAction, preMergeRegionsCommitAction, preModifyColumnFamilyStoreFileTracker, preModifyNamespace, preModifyTableAction, preModifyTableStoreFileTracker, preMove, preMoveServers, preMoveServersAndTables, preMoveTables, preRecommissionRegionServer, preRegionOffline, preRemoveReplicationPeer, preRemoveRSGroup, preRemoveServers, preRenameRSGroup, preRequestLock, preRestoreSnapshot, preRevoke, preSetNamespaceQuota, preSetRegionServerQuota, preSetSplitOrMergeEnabled, preSetTableQuota, preSetUserQuota, preSetUserQuota, preSetUserQuota, preShutdown, preSnapshot, preSplitRegion, preSplitRegionAction, preSplitRegionAfterMETAAction, preSplitRegionBeforeMETAAction, preStopMaster, preSwitchExceedThrottleQuota, preSwitchRpcThrottle, preTableFlush, preTransitReplicationPeerSyncReplicationState, preTruncateRegion, preTruncateRegionAction, preTruncateTable, preTruncateTableAction, preUnassign, preUpdateMasterConfiguration, preUpdateReplicationPeerConfig, preUpdateRSGroupConfig
Methods inherited from interface org.apache.hadoop.hbase.coprocessor.RegionCoprocessor
getBulkLoadObserver, getEndpointObserver
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, postDelete, postExists, postFlush, postFlush, postGetOp, postIncrement, postIncrement, postMemStoreCompaction, postMutationBeforeWAL, postPut, postPut, postReplayWALs, postScannerFilterRow, postScannerNext, postStartRegionOperation, postStoreFileReaderOpen, preAppend, preAppend, preAppendAfterRowLock, preBulkLoadHFile, preCheckAndDelete, preCheckAndDelete, preCheckAndDeleteAfterRowLock, preCheckAndDeleteAfterRowLock, preCheckAndMutate, preCheckAndMutateAfterRowLock, preCheckAndPut, preCheckAndPut, preCheckAndPutAfterRowLock, preCheckAndPutAfterRowLock, preClose, preCommitStoreFile, preCompact, preCompactScannerOpen, preCompactSelection, preDelete, preDelete, preExists, preFlush, preFlush, preFlushScannerOpen, preIncrement, preIncrement, preIncrementAfterRowLock, preMemStoreCompaction, preMemStoreCompactionCompact, preMemStoreCompactionCompactScannerOpen, preOpen, prePut, prePut, preReplayWALs, preStoreFileReaderOpen, preStoreScannerOpen, preWALAppend
-
Field Details
-
LOG
-
AUDITLOG
-
labelsRegion
-
accessControllerAvailable
-
conf
-
initialized
-
checkAuths
-
scannerOwners
Mapping of scanner instances to the user who created them -
visibilityLabelService
-
authorizationEnabled
boolean authorizationEnabledif we are active, usually false, only true if "hbase.security.authorization" has been set to true in site configuration -
RESERVED_VIS_TAG_TYPES
-
-
Constructor Details
-
VisibilityController
public VisibilityController()
-
-
Method Details
-
isCellAuthorizationSupported
-
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
- Throws:
IOException
-
getRegionObserver
Observer/Service Getters- Specified by:
getRegionObserver
in interfaceRegionCoprocessor
-
getMasterObserver
- Specified by:
getMasterObserver
in interfaceMasterCoprocessor
-
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.
-
postStartMaster
Master related hooks- Specified by:
postStartMaster
in interfaceMasterObserver
- Throws:
IOException
-
preModifyTable
public TableDescriptor preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor currentDescriptor, TableDescriptor newDescriptor) 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:
ctx
- the environment to interact with the framework and mastertableName
- the name of the tablecurrentDescriptor
- current TableDescriptor of the tablenewDescriptor
- after modify operation, table will have this descriptor- Throws:
IOException
-
preDisableTable
public void preDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, 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:
ctx
- the environment to interact with the framework and mastertableName
- the name of the table- Throws:
IOException
-
postOpen
Region related hooks- Specified by:
postOpen
in interfaceRegionObserver
- Parameters:
e
- the environment provided by the region server
-
initVisibilityLabelService
-
postSetSplitOrMergeEnabled
public void postSetSplitOrMergeEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean newValue, MasterSwitchType switchType) throws IOException Description copied from interface:MasterObserver
Called after setting split / merge switch- Specified by:
postSetSplitOrMergeEnabled
in interfaceMasterObserver
- Parameters:
ctx
- the coprocessor instance's environmentnewValue
- the new value submitted in the callswitchType
- type of switch- 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
-
prePrepareTimeStampForDeleteVersion
public void prePrepareTimeStampForDeleteVersion(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, Mutation delete, Cell cell, byte[] byteNow, Get get) throws IOException Description copied from interface:RegionObserver
Called before the server updates the timestamp for version delete with latest timestamp.Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
- Specified by:
prePrepareTimeStampForDeleteVersion
in interfaceRegionObserver
- Parameters:
ctx
- the environment provided by the region serverdelete
- - the parent mutation associated with this delete cellcell
- - The deleteColumn with latest version cellbyteNow
- - timestamp bytesget
- - the get formed using the current cell's row. Note that the get does not specify the family and qualifier- Throws:
IOException
-
checkForReservedVisibilityTagPresence
private Pair<Boolean,Tag> checkForReservedVisibilityTagPresence(ExtendedCell cell, Pair<Boolean, Tag> pair) throws IOExceptionChecks whether cell contains any tag with type as VISIBILITY_TAG_TYPE. This tag type is reserved and should not be explicitly set by user.- Parameters:
cell
- The cell under considerationpair
- An optional pair of type<Boolean, Tag>
which would be reused if already set and new one will be created if NULL is passed- Returns:
- If the boolean is false then it indicates that the cell has a RESERVERD_VIS_TAG and with boolean as true, not null tag indicates that a string modified tag was found.
- Throws:
IOException
-
removeReplicationVisibilityTag
- Throws:
IOException
-
preScannerOpen
public void preScannerOpen(ObserverContext<? extends RegionCoprocessorEnvironment> e, 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:
e
- the environment provided by the region serverscan
- the Scan specification- Throws:
IOException
-
postInstantiateDeleteTracker
public DeleteTracker postInstantiateDeleteTracker(ObserverContext<? extends RegionCoprocessorEnvironment> ctx, DeleteTracker delTracker) throws IOException Description copied from interface:RegionObserver
Called after the ScanQueryMatcher creates ScanDeleteTracker. Implementing this hook would help in creating customised DeleteTracker and returning the newly created DeleteTrackerWarn: This is used by internal coprocessors. Should not be implemented by user coprocessors
- Specified by:
postInstantiateDeleteTracker
in interfaceRegionObserver
- Parameters:
ctx
- the environment provided by the region serverdelTracker
- the deleteTracker that is created by the QueryMatcher- Returns:
- the Delete Tracker
- 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
-
preGetOp
public void preGetOp(ObserverContext<? extends RegionCoprocessorEnvironment> e, Get get, List<Cell> results) 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:
e
- the environment provided by the region serverget
- the Get requestresults
- 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
-
isSystemOrSuperUser
- 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
- Throws:
IOException
-
addLabels
public void addLabels(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsRequest request, org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse> done) VisibilityEndpoint service related methods- Specified by:
addLabels
in interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService.Interface
-
setExceptionResults
private void setExceptionResults(int size, IOException e, org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse.Builder response) -
setAuths
public void setAuths(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.SetAuthsRequest request, org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse> done) - Specified by:
setAuths
in interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService.Interface
-
logResult
-
getAuths
public void getAuths(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.GetAuthsRequest request, org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.GetAuthsResponse> done) - Specified by:
getAuths
in interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService.Interface
-
clearAuths
public void clearAuths(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.SetAuthsRequest request, org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse> done) - Specified by:
clearAuths
in interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService.Interface
-
listLabels
public void listLabels(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.ListLabelsRequest request, org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.ListLabelsResponse> done) - Specified by:
listLabels
in interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService.Interface
-
checkCallingUserAuth
- Throws:
IOException
-
buildException
private static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair buildException(Throwable t) Returns NameValuePair of the exception name to stringified version os exception.
-