@InterfaceAudience.LimitedPrivate(value="Configuration") public class VisibilityController extends Object implements MasterCoprocessor, RegionCoprocessor, org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService.Interface, MasterObserver, RegionObserver
Modifier and Type | Class and Description |
---|---|
private static class |
VisibilityController.DeleteVersionVisibilityExpressionFilter |
Coprocessor.State
RegionObserver.MutationType
Modifier and Type | Field and Description |
---|---|
private boolean |
accessControllerAvailable |
private static org.slf4j.Logger |
AUDITLOG |
(package 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 |
checkAuths |
private org.apache.hadoop.conf.Configuration |
conf |
private boolean |
initialized |
private boolean |
labelsRegion |
private static org.slf4j.Logger |
LOG |
private static ArrayList<Byte> |
RESERVED_VIS_TAG_TYPES |
private Map<InternalScanner,String> |
scannerOwners
Mapping of scanner instances to the user who created them
|
private VisibilityLabelService |
visibilityLabelService |
PRIORITY_HIGHEST, PRIORITY_LOWEST, PRIORITY_SYSTEM, PRIORITY_USER, VERSION
Constructor and Description |
---|
VisibilityController() |
Modifier and Type | Method and Description |
---|---|
void |
addLabels(com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse> done)
VisibilityEndpoint service related methods
|
private static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair |
buildException(Throwable t)
n * @return NameValuePair of the exception name to stringified version os exception.
|
private void |
checkCallingUserAuth() |
private Pair<Boolean,Tag> |
checkForReservedVisibilityTagPresence(Cell cell,
Pair<Boolean,Tag> pair)
Checks whether cell contains any tag with type as VISIBILITY_TAG_TYPE.
|
void |
clearAuths(com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.SetAuthsRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse> done) |
private Cell |
createNewCellWithTags(Mutation mutation,
Cell newCell) |
void |
getAuths(com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsResponse> done) |
Optional<MasterObserver> |
getMasterObserver() |
Optional<RegionObserver> |
getRegionObserver()
Observer/Service Getters
|
Iterable<com.google.protobuf.Service> |
getServices()
Coprocessor endpoints providing protobuf services should override this method.
|
private void |
initVisibilityLabelService(RegionCoprocessorEnvironment env) |
static boolean |
isCellAuthorizationSupported(org.apache.hadoop.conf.Configuration conf) |
private boolean |
isSystemOrSuperUser() |
void |
listLabels(com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.ListLabelsRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.ListLabelsResponse> done) |
private void |
logResult(boolean isAllowed,
String request,
String reason,
byte[] user,
List<byte[]> labelAuths,
String regex) |
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.
|
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.
|
DeleteTracker |
postInstantiateDeleteTracker(ObserverContext<RegionCoprocessorEnvironment> ctx,
DeleteTracker delTracker)
Called after the ScanQueryMatcher creates ScanDeleteTracker.
|
void |
postOpen(ObserverContext<RegionCoprocessorEnvironment> e)
Region related hooks
|
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 |
postSetSplitOrMergeEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx,
boolean newValue,
MasterSwitchType switchType)
Called after setting split / merge switch
|
void |
postStartMaster(ObserverContext<MasterCoprocessorEnvironment> ctx)
Master related hooks
|
void |
preBatchMutate(ObserverContext<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<RegionCoprocessorEnvironment> e,
Get get,
List<Cell> results)
Called before the client performs a Get
|
TableDescriptor |
preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName,
TableDescriptor currentDescriptor,
TableDescriptor newDescriptor)
Called prior to modifying a table's properties.
|
void |
prePrepareTimeStampForDeleteVersion(ObserverContext<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<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> e,
Scan scan)
Called before the client opens a new scanner.
|
private void |
removeReplicationVisibilityTag(List<Tag> tags) |
private void |
requireScannerOwner(InternalScanner s)
Verify, when servicing an RPC, that the caller is the scanner owner.
|
void |
setAuths(com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.SetAuthsRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse> done) |
private void |
setExceptionResults(int size,
IOException e,
org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse.Builder response) |
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. |
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
getBulkLoadObserver, getEndpointObserver
postAbortProcedure, postAddReplicationPeer, postAddRSGroup, postAssign, postBalance, postBalanceRSGroup, postBalanceSwitch, postClearDeadServers, postCloneSnapshot, postCompletedCreateTableAction, postCompletedDeleteTableAction, postCompletedDisableTableAction, postCompletedEnableTableAction, postCompletedMergeRegionsAction, postCompletedModifyTableAction, postCompletedModifyTableAction, postCompletedSnapshotAction, postCompletedSplitRegionAction, postCompletedTruncateTableAction, postCreateNamespace, postCreateTable, postDecommissionRegionServers, postDeleteNamespace, postDeleteSnapshot, postDeleteTable, postDisableReplicationPeer, postDisableTable, postEnableReplicationPeer, postEnableTable, postGetClusterMetrics, postGetLocks, postGetNamespaceDescriptor, postGetProcedures, postGetReplicationPeerConfig, postGetTableDescriptors, postGetTableNames, postGetUserPermissions, postGrant, postHasUserPermissions, postIsRpcThrottleEnabled, postListDecommissionedRegionServers, postListNamespaceDescriptors, postListNamespaces, postListReplicationPeers, postListSnapshot, postLockHeartbeat, postMasterStoreFlush, postMergeRegions, postMergeRegionsCommitAction, postModifyColumnFamilyStoreFileTracker, postModifyNamespace, postModifyNamespace, postModifyTable, 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, postTruncateTable, postUnassign, postUnassign, 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, preGetLocks, preGetNamespaceDescriptor, preGetProcedures, preGetReplicationPeerConfig, preGetTableDescriptors, preGetTableNames, preGetUserPermissions, preGrant, preHasUserPermissions, preIsRpcThrottleEnabled, preListDecommissionedRegionServers, preListNamespaceDescriptors, preListNamespaces, preListReplicationPeers, preListSnapshot, preLockHeartbeat, preMasterInitialization, preMasterStoreFlush, preMergeRegions, preMergeRegionsAction, preMergeRegionsCommitAction, preModifyColumnFamilyStoreFileTracker, preModifyNamespace, preModifyNamespace, preModifyTable, preModifyTableAction, 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, preTruncateTable, preTruncateTableAction, preUnassign, preUnassign, preUpdateReplicationPeerConfig, preUpdateRSGroupConfig
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, postWALRestore, 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, preWALRestore
private static final org.slf4j.Logger LOG
private static final org.slf4j.Logger AUDITLOG
private boolean labelsRegion
private boolean accessControllerAvailable
private org.apache.hadoop.conf.Configuration conf
private volatile boolean initialized
private boolean checkAuths
private Map<InternalScanner,String> scannerOwners
private VisibilityLabelService visibilityLabelService
boolean authorizationEnabled
private static ArrayList<Byte> RESERVED_VIS_TAG_TYPES
public VisibilityController()
public static boolean isCellAuthorizationSupported(org.apache.hadoop.conf.Configuration conf)
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) throws IOException
Coprocessor
CoprocessorEnvironment
during it's own shutdown to stop the coprocessor.stop
in interface Coprocessor
IOException
public Optional<RegionObserver> getRegionObserver()
getRegionObserver
in interface RegionCoprocessor
public Optional<MasterObserver> getMasterObserver()
getMasterObserver
in interface MasterCoprocessor
public Iterable<com.google.protobuf.Service> getServices()
Coprocessor
getServices
in interface Coprocessor
Service
s or empty collection. Implementations should never return
null.public void postStartMaster(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
postStartMaster
in interface MasterObserver
IOException
public TableDescriptor preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor currentDescriptor, TableDescriptor newDescriptor) throws IOException
MasterObserver
preModifyTable
in interface MasterObserver
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 descriptorIOException
public void preDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
MasterObserver
preDisableTable
in interface MasterObserver
ctx
- the environment to interact with the framework and mastertableName
- the name of the tableIOException
public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e)
postOpen
in interface RegionObserver
e
- the environment provided by the region serverprivate void initVisibilityLabelService(RegionCoprocessorEnvironment env)
public void postSetSplitOrMergeEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean newValue, MasterSwitchType switchType) throws IOException
MasterObserver
postSetSplitOrMergeEnabled
in interface MasterObserver
ctx
- the coprocessor instance's environmentnewValue
- the new value submitted in the callswitchType
- type of switchIOException
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 prePrepareTimeStampForDeleteVersion(ObserverContext<RegionCoprocessorEnvironment> ctx, Mutation delete, Cell cell, byte[] byteNow, Get get) throws IOException
RegionObserver
Call CoprocessorEnvironment#bypass to skip default actions. If 'bypass' is set, we skip out on calling any subsequent chained coprocessors.
prePrepareTimeStampForDeleteVersion
in interface RegionObserver
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 qualifierIOException
private Pair<Boolean,Tag> checkForReservedVisibilityTagPresence(Cell cell, Pair<Boolean,Tag> pair) throws IOException
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 passedIOException
private void removeReplicationVisibilityTag(List<Tag> tags) throws IOException
IOException
public void preScannerOpen(ObserverContext<RegionCoprocessorEnvironment> e, 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
e
- the environment provided by the region serverscan
- the Scan specificationIOException
public DeleteTracker postInstantiateDeleteTracker(ObserverContext<RegionCoprocessorEnvironment> ctx, DeleteTracker delTracker) throws IOException
RegionObserver
Warn: This is used by internal coprocessors. Should not be implemented by user coprocessors
postInstantiateDeleteTracker
in interface RegionObserver
ctx
- the environment provided by the region serverdelTracker
- the deleteTracker that is created by the QueryMatcherIOException
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 preGetOp(ObserverContext<RegionCoprocessorEnvironment> e, Get get, List<Cell> results) 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
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.IOException
private boolean isSystemOrSuperUser() throws IOException
IOException
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 newCell) throws IOException
IOException
public void addLabels(com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsRequest request, com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse> done)
addLabels
in interface org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService.Interface
private void setExceptionResults(int size, IOException e, org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse.Builder response)
public void setAuths(com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.SetAuthsRequest request, com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse> done)
setAuths
in interface org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService.Interface
private void logResult(boolean isAllowed, String request, String reason, byte[] user, List<byte[]> labelAuths, String regex)
public void getAuths(com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsRequest request, com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsResponse> done)
getAuths
in interface org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService.Interface
public void clearAuths(com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.SetAuthsRequest request, com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse> done)
clearAuths
in interface org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService.Interface
public void listLabels(com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.ListLabelsRequest request, com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.ListLabelsResponse> done)
listLabels
in interface org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService.Interface
private void checkCallingUserAuth() throws IOException
IOException
private static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair buildException(Throwable t)
Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.