Class RSRpcServices

java.lang.Object
org.apache.hadoop.hbase.HBaseRpcServicesBase<HRegionServer>
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

@Private public class RSRpcServices extends HBaseRpcServicesBase<HRegionServer> implements org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface, org.apache.hadoop.hbase.shaded.protobuf.generated.BootstrapNodeProtos.BootstrapNodeService.BlockingInterface
Implements the regionserver RPC services.
  • 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<ExtendedCellScannable> 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<ExtendedCellScannable> 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

      @Deprecated private OperationStatus[] doReplayBatchOp(HRegion region, List<WALSplitUtil.MutationReplay> mutations, long replaySeqId) throws IOException
      Deprecated.
      Since 3.0.0, will be removed in 4.0.0. We do not use this method for replaying edits for secondary replicas any more, see replicateToReplica(RpcController, ReplicateWALEntryRequest).
      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()
    • defaultReservoirEnabled

      protected boolean defaultReservoirEnabled()
      Specified by:
      defaultReservoirEnabled in class HBaseRpcServicesBase<HRegionServer>
    • getDNSServerType

      Specified by:
      getDNSServerType in class HBaseRpcServicesBase<HRegionServer>
    • getHostname

      protected String getHostname(org.apache.hadoop.conf.Configuration conf, String defaultHostname)
      Specified by:
      getHostname in class HBaseRpcServicesBase<HRegionServer>
    • getPortConfigName

      Specified by:
      getPortConfigName in class HBaseRpcServicesBase<HRegionServer>
    • getDefaultPort

      protected int getDefaultPort()
      Specified by:
      getDefaultPort in class HBaseRpcServicesBase<HRegionServer>
    • getRpcSchedulerFactoryClass

      protected Class<?> getRpcSchedulerFactoryClass(org.apache.hadoop.conf.Configuration conf)
      Specified by:
      getRpcSchedulerFactoryClass in class HBaseRpcServicesBase<HRegionServer>
    • createRpcServer

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

    • createPriority

      Specified by:
      createPriority in class HBaseRpcServicesBase<HRegionServer>
    • 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

    • 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.
      Specified by:
      getServices in class HBaseRpcServicesBase<HRegionServer>
      Returns:
      immutable list of blocking services and the security info classes that this server supports
    • 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
    • getAndReset

      private ExtendedCellScanner getAndReset(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller)
    • replay

      @Deprecated 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
      Deprecated.
      Since 3.0.0, will be removed in 4.0.0. Not used any more, put here only for compatibility with old region replica implementation. Now we will use replicateToReplica method instead.
      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
    • replicateToReplica

      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse replicateToReplica(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 on a secondary replica
      Specified by:
      replicateToReplica in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
      Throws:
      org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
    • checkShouldRejectReplicationRequest

      private void checkShouldRejectReplicationRequest(List<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry> entries) throws IOException
      Throws:
      IOException
    • 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 org.apache.hbase.thirdparty.com.google.protobuf.Message execServiceOnRegion(HRegion region, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall serviceCall) throws IOException
      Throws:
      IOException
    • shouldRejectRequestsFromClient

      private boolean shouldRejectRequestsFromClient(HRegion region)
    • rejectIfInStandByState

      Throws:
      DoNotRetryIOException
    • 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)
    • isReplicationRequest

      private boolean isReplicationRequest(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action action)
    • 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, boolean isError) 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
    • 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
    • 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)
    • onConfigurationChange

      public void onConfigurationChange(org.apache.hadoop.conf.Configuration conf)
      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
      Overrides:
      onConfigurationChange in class HBaseRpcServicesBase<HRegionServer>
    • 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