Class RSRpcServices

java.lang.Object
org.apache.hadoop.hbase.regionserver.RSRpcServices
All Implemented Interfaces:
ConfigurationObserver, HBaseRPCErrorHandler, PriorityFunction, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface, org.apache.hadoop.hbase.shaded.protobuf.generated.BootstrapNodeProtos.BootstrapNodeService.BlockingInterface, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface, org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ClientMetaService.BlockingInterface
Direct Known Subclasses:
MasterRpcServices

@Private public class RSRpcServices extends Object implements HBaseRPCErrorHandler, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface, org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ClientMetaService.BlockingInterface, org.apache.hadoop.hbase.shaded.protobuf.generated.BootstrapNodeProtos.BootstrapNodeService.BlockingInterface, PriorityFunction, ConfigurationObserver
Implements the regionserver RPC services.
  • Nested Class Summary

    Nested Classes
    Modifier and Type
    Class
    Description
    private static final class 
    An Rpc callback for closing a RegionScanner.
    (package private) static final class 
    Holder class which holds the RegionScanner, nextCallSeq and RpcCallbacks together.
    (package private) static class 
    An RpcCallBack that creates a list of scanners that needs to perform callBack operation on completion of multiGets.
    private class 
    An Rpc callback for doing shipped() call on a RegionScanner.
    private class 
    Instantiated as a scanner lease.
  • Field Summary

    Fields
    Modifier and Type
    Field
    Description
     
    (package private) final AtomicBoolean
     
    static final String
     
    private final org.apache.hbase.thirdparty.com.google.common.cache.Cache<String,String>
     
    static final int
     
    (package private) static final long
    private static final boolean
    Default value of config REJECT_BATCH_ROWS_OVER_THRESHOLD
    (package private) final InetSocketAddress
     
    protected static final org.slf4j.Logger
     
    static final String
    RPC scheduler to use for the master.
    private long
     
    private final long
    The minimum allowable delta to use for the scan limit
    private final PriorityFunction
     
    private static final String
    Minimum allowable time limit delta (in milliseconds) that can be enforced during scans.
    static final String
    RPC scheduler to use for the region server.
    protected final HRegionServer
     
    static final String
    Services launched in RSRpcServices.
    static final String
     
    static final String
     
    static final String
     
    private static final String
    Whether to reject rows with size > threshold defined by HConstants.BATCH_ROWS_THRESHOLD_NAME
    private boolean
     
    (package private) final LongAdder
     
    private int
    Row size threshold for multi requests above which a warning is logged
    (package private) final LongAdder
     
    (package private) final LongAdder
     
    (package private) final LongAdder
     
    (package private) final LongAdder
     
    (package private) final LongAdder
     
    (package private) final RpcServerInterface
     
    private final int
    The RPC timeout period (milliseconds)
    private static final IOException
    Deprecated.
     
    private final int
    The lease timeout period for client scanners (milliseconds).
     
     
  • Constructor Summary

    Constructors
    Constructor
    Description
     
  • Method Summary

    Modifier and Type
    Method
    Description
    private void
    addResult(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse.Builder builder, Result result, HBaseRpcController rpcc, boolean clientCellBlockSupported)
     
    private void
    addResults(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse.Builder builder, List<Result> results, HBaseRpcController controller, boolean isDefaultRegion, boolean clientCellBlockSupported)
     
    addScanner(String scannerName, RegionScanner s, Shipper shipper, HRegion r, boolean needCursor, boolean fullRegionScan)
     
    private void
     
    (package private) void
    Method to account for the size of retained cells.
    private Result
    append(HRegion region, OperationQuota quota, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation, CellScanner cellScanner, long nonceGroup, ActivePolicyEnforcement spaceQuota, RpcCallContext context)
    Execute an append mutation.
    org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse
    bulkLoadHFile(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest request)
    Atomically bulk load several HFiles into an open region
    checkAndMutate(HRegion region, List<org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action> actions, CellScanner cellScanner, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition condition, long nonceGroup, ActivePolicyEnforcement spaceQuotaEnforcement)
     
    checkAndMutate(HRegion region, OperationQuota quota, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation, CellScanner cellScanner, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition condition, long nonceGroup, ActivePolicyEnforcement spaceQuota, RpcCallContext context)
     
    private void
    checkBatchSizeAndLogLargeSize(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest request)
     
    private void
     
    private void
    checkLimitOfRows(int numOfCompleteRows, int limitOfRows, boolean moreRows, ScannerContext scannerContext, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse.Builder builder)
     
    boolean
    Take actions on the event of an OutOfMemoryError.
    protected void
    Called to verify that this server is up and running.
    private void
    checkScanNextCallSeq(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request, RSRpcServices.RegionScannerHolder rsh)
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse
    cleanupBulkLoad(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest request)
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesResponse
    clearCompactionQueues(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesRequest request)
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheResponse
    clearRegionBlockCache(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheRequest request)
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearSlowLogResponses
    clearSlowLogsResponses(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest request)
     
    private void
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse
    closeRegion(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest request)
    Close a region on the region server.
    private void
    closeScanner(HRegion region, RegionScanner scanner, String scannerName, RpcCallContext context)
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchResponse
    compactionSwitch(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchRequest request)
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse
    compactRegion(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest request)
    Compact a region on the region server.
     
    createRpcServer(Server server, RpcSchedulerFactory rpcSchedulerFactory, InetSocketAddress bindAddress, String name)
     
    private void
    delete(HRegion region, OperationQuota quota, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation, CellScanner cellScanner, ActivePolicyEnforcement spaceQuota)
     
    private void
    doAtomicBatchOp(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult.Builder builder, HRegion region, OperationQuota quota, List<org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action> mutations, CellScanner cells, long nonceGroup, ActivePolicyEnforcement spaceQuotaEnforcement)
     
    private void
    doBatchOp(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult.Builder builder, HRegion region, OperationQuota quota, List<org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action> mutations, CellScanner cells, long nonceGroup, ActivePolicyEnforcement spaceQuotaEnforcement, boolean atomic)
    Execute a list of mutations.
    private void
    doNonAtomicBatchOp(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult.Builder builder, HRegion region, OperationQuota quota, List<org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action> mutations, CellScanner cells, ActivePolicyEnforcement spaceQuotaEnforcement)
     
    doNonAtomicRegionMutation(HRegion region, OperationQuota quota, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction actions, CellScanner cellScanner, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult.Builder builder, List<CellScannable> cellsToReturn, long nonceGroup, RSRpcServices.RegionScannersCloseCallBack closeCallBack, RpcCallContext context, ActivePolicyEnforcement spaceQuotaEnforcement)
    Run through the regionMutation rm and per Mutation, do the work, and then when done, add an instance of a ClientProtos.ResultOrException that corresponds to each Mutation.
    private OperationStatus[]
    doReplayBatchOp(HRegion region, List<WALSplitUtil.MutationReplay> mutations, long replaySeqId)
    Execute a list of Put/Delete mutations.
    org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse
    execRegionServerService(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest request)
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse
    execService(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest request)
     
    private com.google.protobuf.Message
    execServiceOnRegion(HRegion region, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall serviceCall)
     
    private void
    executeCloseRegionProcedures(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest request)
     
    private void
    executeOpenRegionProcedures(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest request, Map<TableName,TableDescriptor> tdCache)
     
    private void
    executeProcedures(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RemoteProcedureRequest request)
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse
    executeProcedures(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresRequest request)
     
    static boolean
     
    private void
    failRegionAction(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse.Builder responseBuilder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult.Builder regionActionResultBuilder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction regionAction, CellScanner cellScanner, Throwable error)
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse
    flushRegion(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest request)
    Flush a region on the region server.
    private Result
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse
    get(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest request)
    Get data from a table.
    protected AccessChecker
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetActiveMasterResponse
    getActiveMaster(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetActiveMasterRequest request)
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.BootstrapNodeProtos.GetAllBootstrapNodesResponse
    getAllBootstrapNodes(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.BootstrapNodeProtos.GetAllBootstrapNodesRequest request)
     
    final org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetBootstrapNodesResponse
    getBootstrapNodes(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetBootstrapNodesRequest request)
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetCachedFilesListResponse
    getCachedFilesList(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetCachedFilesListRequest request)
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetClusterIdResponse
    getClusterId(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetClusterIdRequest request)
     
    org.apache.hadoop.conf.Configuration
     
    long
    getDeadline(org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader header, org.apache.hbase.thirdparty.com.google.protobuf.Message param)
    Returns the deadline of the specified request.
    private org.apache.hadoop.fs.FileSystem
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SlowLogResponses
    getLargeLogResponses(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SlowLogResponseRequest request)
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LogEntry
    getLogEntries(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LogRequest request)
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMastersResponse
    getMasters(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMastersRequest request)
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMetaRegionLocationsResponse
    getMetaRegionLocations(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMetaRegionLocationsRequest request)
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse
    getOnlineRegion(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest request)
     
     
    int
    getPriority(org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader header, org.apache.hbase.thirdparty.com.google.protobuf.Message param, User user)
    Returns the 'priority type' of the specified request.
    getRegion(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier regionSpecifier)
    Find the HRegion based on a region specifier
    org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse
    getRegionInfo(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest request)
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadResponse
    getRegionLoad(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest request)
     
    private List<HRegion>
    getRegions(List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> regionSpecifiers, CacheEvictionStatsBuilder stats)
    Find the List of HRegions based on a list of region specifiers
    getRegionScanner(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request)
     
    getRegionScannerHolder(long scannerId)
    Returns The associated RegionScannerHolder for scannerId or null.
    private long
    getRemainingRpcTimeout(RpcCall call, HBaseRpcController controller, long now)
     
    (package private) static String
    Returns Remote client's ip and port else null if can't be determined.
    private static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException
     
    private static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException
    getResultOrException(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException.Builder builder, int index)
     
    private static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException
    getResultOrException(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result r, int index)
     
     
     
    protected Class<?>
     
    getScanDetailsWithId(long scannerId)
     
    getScanDetailsWithRequest(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request)
     
    (package private) RegionScanner
    getScanner(long scannerId)
    Returns The outstanding RegionScanner for scannerId or null if none found.
    int
     
    (package private) long
    getScannerVirtualTime(long scannerId)
    Get the vtime associated with the scanner.
    org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse
    getServerInfo(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest request)
    Get some information of the region server.
    By default, put up an Admin and a Client Service.
    private List<org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload>
    getSlowLogPayloads(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SlowLogResponseRequest request, NamedQueueRecorder namedQueueRecorder)
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SlowLogResponses
    getSlowLogResponses(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SlowLogResponseRequest request)
     
     
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse
    getSpaceQuotaSnapshots(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest request)
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse
    getStoreFile(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest request)
     
    (package private) long
    getTimeLimit(RpcCall rpcCall, HBaseRpcController controller, boolean allowHeartbeatMessages)
     
    (package private) static String
    Returns Remote client's username.
     
    private Result
    increment(HRegion region, OperationQuota quota, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation, CellScanner cells, long nonceGroup, ActivePolicyEnforcement spaceQuota, RpcCallContext context)
    Execute an increment mutation.
    private boolean
     
    private boolean
    isFullRegionScan(Scan scan, HRegion region)
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse
    multi(org.apache.hbase.thirdparty.com.google.protobuf.RpcController rpcc, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest request)
    Execute multiple actions on a table: get, mutate, and/or execCoprocessor
    org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse
    mutate(org.apache.hbase.thirdparty.com.google.protobuf.RpcController rpcc, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest request)
    Mutate data in a table.
    newRegionScanner(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse.Builder builder)
     
    void
    onConfigurationChange(org.apache.hadoop.conf.Configuration newConf)
    This method would be called by the ConfigurationManager object when the Configuration object is reloaded from disk.
    org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse
    openRegion(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest request)
    Open asynchronously a region or a set of regions on the region server.
    org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse
    prepareBulkLoad(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest request)
     
    private void
    put(HRegion region, OperationQuota quota, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation, CellScanner cellScanner, ActivePolicyEnforcement spaceQuota)
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse
    replay(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request)
    Replay the given changes when distributedLogReplay WAL edits from a failed RS.
    org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse
    replicateWALEntry(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request)
    Replicate WAL entries on the region server.
    protected void
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse
    rollWALWriter(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest request)
    Roll the WAL writer of the region server.
    private void
    rpcPreCheck(String requestName)
    Checks for the following pre-checks in order: RegionServer is running If authorization is enabled, then RPC caller has ADMIN permissions
    private void
     
    private void
    scan(HBaseRpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request, RSRpcServices.RegionScannerHolder rsh, long maxQuotaResultSize, int maxResults, int limitOfRows, List<Result> results, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse.Builder builder, RpcCall rpcCall)
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse
    scan(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request)
    Scan data in a table.
    private void
    setReloadableGuardrails(org.apache.hadoop.conf.Configuration conf)
     
    private void
    skipCellsForMutation(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action action, CellScanner cellScanner)
     
    private void
    skipCellsForMutations(List<org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action> actions, CellScanner cellScanner)
     
    (package private) void
    start(ZKWatcher zkWatcher)
     
    (package private) void
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse
    stopServer(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest request)
    Stop the region server.
    private void
    throwOnWrongStartCode(long serverStartCode)
     
    private void
    throwOnWrongStartCode(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest request)
     
    private void
    throwOnWrongStartCode(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresRequest req)
     
    private void
    throwOnWrongStartCode(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest request)
     
    private static String
    toScannerName(long scannerId)
    The returned String is used as key doing look up of outstanding Scanners in this Servers' this.scanners, the Map of outstanding scanners and their current state.
    org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse
    updateConfiguration(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest request)
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse
    updateFavoredNodes(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest request)
     
    private void
    updateMutationMetrics(HRegion region, long starttime, boolean batchContainsPuts, boolean batchContainsDelete)
     
    org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse
    warmupRegion(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest request)
    Warmup a region on this server.

    Methods inherited from class java.lang.Object

    clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
  • Field Details

  • Constructor Details

  • Method Details

    • getResultOrException

      private static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException getResultOrException(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result r, int index)
    • getResultOrException

      private static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException getResultOrException(Exception e, int index)
    • getResultOrException

      private static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException getResultOrException(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException.Builder builder, int index)
    • rpcPreCheck

      private void rpcPreCheck(String requestName) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Checks for the following pre-checks in order:
      1. RegionServer is running
      2. If authorization is enabled, then RPC caller has ADMIN permissions
      Parameters:
      requestName - name of rpc request. Used in reporting failures to provide context.
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException - If any of the above listed pre-check fails.
    • isClientCellBlockSupport

      private boolean isClientCellBlockSupport(RpcCallContext context)
    • addResult

      private void addResult(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse.Builder builder, Result result, HBaseRpcController rpcc, boolean clientCellBlockSupported)
    • addResults

      private void addResults(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse.Builder builder, List<Result> results, HBaseRpcController controller, boolean isDefaultRegion, boolean clientCellBlockSupported)
    • checkAndMutate

      private CheckAndMutateResult checkAndMutate(HRegion region, List<org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action> actions, CellScanner cellScanner, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition condition, long nonceGroup, ActivePolicyEnforcement spaceQuotaEnforcement) throws IOException
      Throws:
      IOException
    • append

      private Result append(HRegion region, OperationQuota quota, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation, CellScanner cellScanner, long nonceGroup, ActivePolicyEnforcement spaceQuota, RpcCallContext context) throws IOException
      Execute an append mutation.
      Returns:
      result to return to client if default operation should be bypassed as indicated by RegionObserver, null otherwise
      Throws:
      IOException
    • increment

      private Result increment(HRegion region, OperationQuota quota, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation, CellScanner cells, long nonceGroup, ActivePolicyEnforcement spaceQuota, RpcCallContext context) throws IOException
      Execute an increment mutation.
      Throws:
      IOException
    • doNonAtomicRegionMutation

      private List<CellScannable> doNonAtomicRegionMutation(HRegion region, OperationQuota quota, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction actions, CellScanner cellScanner, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult.Builder builder, List<CellScannable> cellsToReturn, long nonceGroup, RSRpcServices.RegionScannersCloseCallBack closeCallBack, RpcCallContext context, ActivePolicyEnforcement spaceQuotaEnforcement)
      Run through the regionMutation rm and per Mutation, do the work, and then when done, add an instance of a ClientProtos.ResultOrException that corresponds to each Mutation.
      Parameters:
      cellsToReturn - Could be null. May be allocated in this method. This is what this method returns as a 'result'.
      closeCallBack - the callback to be used with multigets
      context - the current RpcCallContext
      Returns:
      Return the cellScanner passed
    • checkCellSizeLimit

      private void checkCellSizeLimit(HRegion r, Mutation m) throws IOException
      Throws:
      IOException
    • doAtomicBatchOp

      private void doAtomicBatchOp(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult.Builder builder, HRegion region, OperationQuota quota, List<org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action> mutations, CellScanner cells, long nonceGroup, ActivePolicyEnforcement spaceQuotaEnforcement) throws IOException
      Throws:
      IOException
    • doNonAtomicBatchOp

      private void doNonAtomicBatchOp(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult.Builder builder, HRegion region, OperationQuota quota, List<org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action> mutations, CellScanner cells, ActivePolicyEnforcement spaceQuotaEnforcement)
    • doBatchOp

      private void doBatchOp(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult.Builder builder, HRegion region, OperationQuota quota, List<org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action> mutations, CellScanner cells, long nonceGroup, ActivePolicyEnforcement spaceQuotaEnforcement, boolean atomic) throws IOException
      Execute a list of mutations.
      Throws:
      IOException
    • updateMutationMetrics

      private void updateMutationMetrics(HRegion region, long starttime, boolean batchContainsPuts, boolean batchContainsDelete)
    • doReplayBatchOp

      private OperationStatus[] doReplayBatchOp(HRegion region, List<WALSplitUtil.MutationReplay> mutations, long replaySeqId) throws IOException
      Execute a list of Put/Delete mutations. The function returns OperationStatus instead of constructing MultiResponse to save a possible loop if caller doesn't need MultiResponse.
      Returns:
      an array of OperationStatus which internally contains the OperationStatusCode and the exceptionMessage if any
      Throws:
      IOException
    • closeAllScanners

      private void closeAllScanners()
    • createRpcServer

      protected RpcServerInterface createRpcServer(Server server, RpcSchedulerFactory rpcSchedulerFactory, InetSocketAddress bindAddress, String name) throws IOException
      Throws:
      IOException
    • getRpcSchedulerFactoryClass

    • onConfigurationChange

      public void onConfigurationChange(org.apache.hadoop.conf.Configuration newConf)
      Description copied from interface: ConfigurationObserver
      This method would be called by the ConfigurationManager object when the Configuration object is reloaded from disk.
      Specified by:
      onConfigurationChange in interface ConfigurationObserver
    • createPriority

    • requirePermission

      protected void requirePermission(String request, Permission.Action perm) throws IOException
      Throws:
      IOException
    • getScannersCount

      public int getScannersCount()
    • getScanner

      RegionScanner getScanner(long scannerId)
      Returns The outstanding RegionScanner for scannerId or null if none found.
    • getRegionScannerHolder

      Returns The associated RegionScannerHolder for scannerId or null.
    • getScanDetailsWithId

      public String getScanDetailsWithId(long scannerId)
    • getScanDetailsWithRequest

      public String getScanDetailsWithRequest(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request)
    • getScannerVirtualTime

      long getScannerVirtualTime(long scannerId)
      Get the vtime associated with the scanner. Currently the vtime is the number of "next" calls.
    • addSize

      void addSize(RpcCallContext context, Result r)
      Method to account for the size of retained cells.
      Parameters:
      context - rpc call context
      r - result to add size.
    • getRemoteClientIpAndPort

      Returns Remote client's ip and port else null if can't be determined.
    • getUserName

      static String getUserName()
      Returns Remote client's username.
    • addScanner

      private RSRpcServices.RegionScannerHolder addScanner(String scannerName, RegionScanner s, Shipper shipper, HRegion r, boolean needCursor, boolean fullRegionScan) throws LeaseManager.LeaseStillHeldException
      Throws:
      LeaseManager.LeaseStillHeldException
    • isFullRegionScan

      private boolean isFullRegionScan(Scan scan, HRegion region)
    • getRegion

      public HRegion getRegion(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier regionSpecifier) throws IOException
      Find the HRegion based on a region specifier
      Parameters:
      regionSpecifier - the region specifier
      Returns:
      the corresponding region
      Throws:
      IOException - if the specifier is not null, but failed to find the region
    • getRegions

      private List<HRegion> getRegions(List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> regionSpecifiers, CacheEvictionStatsBuilder stats)
      Find the List of HRegions based on a list of region specifiers
      Parameters:
      regionSpecifiers - the list of region specifiers
      Returns:
      the corresponding list of regions
      Throws:
      IOException - if any of the specifiers is not null, but failed to find the region
    • getPriority

    • getConfiguration

      public org.apache.hadoop.conf.Configuration getConfiguration()
    • getRpcQuotaManager

    • getSpaceQuotaManager

    • start

      void start(ZKWatcher zkWatcher)
    • stop

      void stop()
    • checkOpen

      protected void checkOpen() throws IOException
      Called to verify that this server is up and running.
      Throws:
      IOException
    • getServices

      By default, put up an Admin and a Client Service. Set booleans hbase.regionserver.admin.executorService and hbase.regionserver.client.executorService if you want to enable/disable services. Default is that both are enabled.
      Returns:
      immutable list of blocking services and the security info classes that this server supports
    • getSocketAddress

    • getPriority

      public int getPriority(org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader header, org.apache.hbase.thirdparty.com.google.protobuf.Message param, User user)
      Description copied from interface: PriorityFunction
      Returns the 'priority type' of the specified request. The returned value is mainly used to select the dispatch queue.
      Specified by:
      getPriority in interface PriorityFunction
      Returns:
      Priority of this request.
    • getDeadline

      public long getDeadline(org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader header, org.apache.hbase.thirdparty.com.google.protobuf.Message param)
      Description copied from interface: PriorityFunction
      Returns the deadline of the specified request. The returned value is used to sort the dispatch queue.
      Specified by:
      getDeadline in interface PriorityFunction
      Returns:
      Deadline of this request. 0 now, otherwise msec of 'delay'
    • checkOOME

      public boolean checkOOME(Throwable e)
      Description copied from interface: HBaseRPCErrorHandler
      Take actions on the event of an OutOfMemoryError.
      Specified by:
      checkOOME in interface HBaseRPCErrorHandler
      Parameters:
      e - the throwable
      Returns:
      if the server should be shut down
    • exitIfOOME

      public static boolean exitIfOOME(Throwable e)
    • closeRegion

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse closeRegion(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Close a region on the region server.
      Specified by:
      closeRegion in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Parameters:
      controller - the RPC controller
      request - the request
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • compactRegion

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse compactRegion(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Compact a region on the region server.
      Specified by:
      compactRegion in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Parameters:
      controller - the RPC controller
      request - the request
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • compactionSwitch

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchResponse compactionSwitch(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Specified by:
      compactionSwitch in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • flushRegion

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse flushRegion(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Flush a region on the region server.
      Specified by:
      flushRegion in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Parameters:
      controller - the RPC controller
      request - the request
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • getOnlineRegion

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse getOnlineRegion(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Specified by:
      getOnlineRegion in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • getRegionInfo

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse getRegionInfo(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Specified by:
      getRegionInfo in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • getRegionLoad

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadResponse getRegionLoad(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Specified by:
      getRegionLoad in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • clearCompactionQueues

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesResponse clearCompactionQueues(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Specified by:
      clearCompactionQueues in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • getServerInfo

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse getServerInfo(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Get some information of the region server.
      Specified by:
      getServerInfo in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Parameters:
      controller - the RPC controller
      request - the request
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • getStoreFile

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse getStoreFile(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Specified by:
      getStoreFile in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • throwOnWrongStartCode

      private void throwOnWrongStartCode(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • throwOnWrongStartCode

      private void throwOnWrongStartCode(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • throwOnWrongStartCode

      private void throwOnWrongStartCode(long serverStartCode) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • throwOnWrongStartCode

      private void throwOnWrongStartCode(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresRequest req) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • openRegion

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse openRegion(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Open asynchronously a region or a set of regions on the region server. The opening is coordinated by ZooKeeper, and this method requires the znode to be created before being called. As a consequence, this method should be called only from the master.

      Different manages states for the region are:

      • region not opened: the region opening will start asynchronously.
      • a close is already in progress: this is considered as an error.
      • an open is already in progress: this new open request will be ignored. This is important because the Master can do multiple requests if it crashes.
      • the region is already opened: this new open request will be ignored.

      Bulk assign: If there are more than 1 region to open, it will be considered as a bulk assign. For a single region opening, errors are sent through a ServiceException. For bulk assign, errors are put in the response as FAILED_OPENING.

      Specified by:
      openRegion in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Parameters:
      controller - the RPC controller
      request - the request
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • warmupRegion

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse warmupRegion(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Warmup a region on this server. This method should only be called by Master. It synchronously opens the region and closes the region bringing the most important pages in cache.
      Specified by:
      warmupRegion in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • replay

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse replay(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Replay the given changes when distributedLogReplay WAL edits from a failed RS. The guarantee is that the given mutations will be durable on the receiving RS if this method returns without any exception.
      Specified by:
      replay in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Parameters:
      controller - the RPC controller
      request - the request
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • replicateWALEntry

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse replicateWALEntry(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Replicate WAL entries on the region server.
      Specified by:
      replicateWALEntry in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Parameters:
      controller - the RPC controller
      request - the request
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • rollWALWriter

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse rollWALWriter(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Roll the WAL writer of the region server.
      Specified by:
      rollWALWriter in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Parameters:
      controller - the RPC controller
      request - the request
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • stopServer

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse stopServer(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Stop the region server.
      Specified by:
      stopServer in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Parameters:
      controller - the RPC controller
      request - the request
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • updateFavoredNodes

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse updateFavoredNodes(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Specified by:
      updateFavoredNodes in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • bulkLoadHFile

      public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse bulkLoadHFile(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Atomically bulk load several HFiles into an open region
      Specified by:
      bulkLoadHFile in interface org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface
      Returns:
      true if successful, false is failed but recoverably (no action)
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException - if failed unrecoverably
    • prepareBulkLoad

      public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse prepareBulkLoad(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Specified by:
      prepareBulkLoad in interface org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • cleanupBulkLoad

      public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse cleanupBulkLoad(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Specified by:
      cleanupBulkLoad in interface org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • execService

      public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse execService(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Specified by:
      execService in interface org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • getFileSystem

      private org.apache.hadoop.fs.FileSystem getFileSystem(List<String> filePaths) throws IOException
      Throws:
      IOException
    • execServiceOnRegion

      private com.google.protobuf.Message execServiceOnRegion(HRegion region, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall serviceCall) throws IOException
      Throws:
      IOException
    • get

      public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse get(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Get data from a table.
      Specified by:
      get in interface org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface
      Parameters:
      controller - the RPC controller
      request - the get request
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • get

      private Result get(Get get, HRegion region, RSRpcServices.RegionScannersCloseCallBack closeCallBack, RpcCallContext context) throws IOException
      Throws:
      IOException
    • checkBatchSizeAndLogLargeSize

      private void checkBatchSizeAndLogLargeSize(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • failRegionAction

      private void failRegionAction(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse.Builder responseBuilder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult.Builder regionActionResultBuilder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction regionAction, CellScanner cellScanner, Throwable error)
    • multi

      public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse multi(org.apache.hbase.thirdparty.com.google.protobuf.RpcController rpcc, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Execute multiple actions on a table: get, mutate, and/or execCoprocessor
      Specified by:
      multi in interface org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface
      Parameters:
      rpcc - the RPC controller
      request - the multi request
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • skipCellsForMutations

      private void skipCellsForMutations(List<org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action> actions, CellScanner cellScanner)
    • skipCellsForMutation

      private void skipCellsForMutation(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action action, CellScanner cellScanner)
    • mutate

      public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse mutate(org.apache.hbase.thirdparty.com.google.protobuf.RpcController rpcc, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Mutate data in a table.
      Specified by:
      mutate in interface org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface
      Parameters:
      rpcc - the RPC controller
      request - the mutate request
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • put

      private void put(HRegion region, OperationQuota quota, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation, CellScanner cellScanner, ActivePolicyEnforcement spaceQuota) throws IOException
      Throws:
      IOException
    • delete

      private void delete(HRegion region, OperationQuota quota, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation, CellScanner cellScanner, ActivePolicyEnforcement spaceQuota) throws IOException
      Throws:
      IOException
    • checkAndMutate

      private CheckAndMutateResult checkAndMutate(HRegion region, OperationQuota quota, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation, CellScanner cellScanner, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition condition, long nonceGroup, ActivePolicyEnforcement spaceQuota, RpcCallContext context) throws IOException
      Throws:
      IOException
    • getRegionScanner

      private RSRpcServices.RegionScannerHolder getRegionScanner(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request) throws IOException
      Throws:
      IOException
    • newRegionScanner

      private Pair<String,RSRpcServices.RegionScannerHolder> newRegionScanner(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse.Builder builder) throws IOException
      Returns:
      Pair with scannerName key to use with this new Scanner and its RegionScannerHolder value.
      Throws:
      IOException
    • toScannerName

      private static String toScannerName(long scannerId)
      The returned String is used as key doing look up of outstanding Scanners in this Servers' this.scanners, the Map of outstanding scanners and their current state.
      Parameters:
      scannerId - A scanner long id.
      Returns:
      The long id as a String.
    • checkScanNextCallSeq

      private void checkScanNextCallSeq(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request, RSRpcServices.RegionScannerHolder rsh) throws OutOfOrderScannerNextException
      Throws:
      OutOfOrderScannerNextException
    • addScannerLeaseBack

    • getTimeLimit

      long getTimeLimit(RpcCall rpcCall, HBaseRpcController controller, boolean allowHeartbeatMessages)
    • getRemainingRpcTimeout

      private long getRemainingRpcTimeout(RpcCall call, HBaseRpcController controller, long now)
    • checkLimitOfRows

      private void checkLimitOfRows(int numOfCompleteRows, int limitOfRows, boolean moreRows, ScannerContext scannerContext, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse.Builder builder)
    • scan

      private void scan(HBaseRpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request, RSRpcServices.RegionScannerHolder rsh, long maxQuotaResultSize, int maxResults, int limitOfRows, List<Result> results, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse.Builder builder, RpcCall rpcCall) throws IOException
      Throws:
      IOException
    • scan

      public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse scan(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Scan data in a table.
      Specified by:
      scan in interface org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface
      Parameters:
      controller - the RPC controller
      request - the scan request
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • runShippedCallback

      private void runShippedCallback(RSRpcServices.RegionScannerHolder rsh) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • closeScanner

      private void closeScanner(HRegion region, RegionScanner scanner, String scannerName, RpcCallContext context) throws IOException
      Throws:
      IOException
    • execRegionServerService

      public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse execRegionServerService(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Specified by:
      execRegionServerService in interface org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • updateConfiguration

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse updateConfiguration(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Specified by:
      updateConfiguration in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • getSpaceQuotaSnapshots

      public org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse getSpaceQuotaSnapshots(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Specified by:
      getSpaceQuotaSnapshots in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • clearRegionBlockCache

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheResponse clearRegionBlockCache(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Specified by:
      clearRegionBlockCache in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • executeOpenRegionProcedures

      private void executeOpenRegionProcedures(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest request, Map<TableName,TableDescriptor> tdCache)
    • executeCloseRegionProcedures

      private void executeCloseRegionProcedures(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest request)
    • executeProcedures

      private void executeProcedures(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RemoteProcedureRequest request)
    • executeProcedures

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse executeProcedures(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Specified by:
      executeProcedures in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • getSlowLogResponses

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SlowLogResponses getSlowLogResponses(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SlowLogResponseRequest request)
      Specified by:
      getSlowLogResponses in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
    • getSlowLogPayloads

      private List<org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload> getSlowLogPayloads(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SlowLogResponseRequest request, NamedQueueRecorder namedQueueRecorder)
    • getLargeLogResponses

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SlowLogResponses getLargeLogResponses(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SlowLogResponseRequest request)
      Specified by:
      getLargeLogResponses in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
    • clearSlowLogsResponses

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearSlowLogResponses clearSlowLogsResponses(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Specified by:
      clearSlowLogsResponses in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • getLogEntries

      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LogEntry getLogEntries(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LogRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Specified by:
      getLogEntries in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • getCachedFilesList

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetCachedFilesListResponse getCachedFilesList(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetCachedFilesListRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Specified by:
      getCachedFilesList in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • getRpcScheduler

    • getAccessChecker

    • getZkPermissionWatcher

    • getClusterId

      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetClusterIdResponse getClusterId(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetClusterIdRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Specified by:
      getClusterId in interface org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ClientMetaService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • getActiveMaster

      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetActiveMasterResponse getActiveMaster(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetActiveMasterRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Specified by:
      getActiveMaster in interface org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ClientMetaService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • getMasters

      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMastersResponse getMasters(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMastersRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Specified by:
      getMasters in interface org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ClientMetaService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • getMetaRegionLocations

      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMetaRegionLocationsResponse getMetaRegionLocations(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMetaRegionLocationsRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Specified by:
      getMetaRegionLocations in interface org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ClientMetaService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • getBootstrapNodes

      public final org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetBootstrapNodesResponse getBootstrapNodes(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetBootstrapNodesRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Specified by:
      getBootstrapNodes in interface org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ClientMetaService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • getAllBootstrapNodes

      public org.apache.hadoop.hbase.shaded.protobuf.generated.BootstrapNodeProtos.GetAllBootstrapNodesResponse getAllBootstrapNodes(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.BootstrapNodeProtos.GetAllBootstrapNodesRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
      Specified by:
      getAllBootstrapNodes in interface org.apache.hadoop.hbase.shaded.protobuf.generated.BootstrapNodeProtos.BootstrapNodeService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • setReloadableGuardrails

      private void setReloadableGuardrails(org.apache.hadoop.conf.Configuration conf)