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.
-
Nested Class Summary
Modifier and TypeClassDescriptionprivate static final class
An Rpc callback for closing a RegionScanner.(package private) static class
(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
Modifier and TypeFieldDescription(package private) final AtomicBoolean
(package private) static final long
Default value ofREGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA
private static final boolean
Default value of configREJECT_BATCH_ROWS_OVER_THRESHOLD
private static final org.slf4j.Logger
private long
private final long
The minimum allowable delta to use for the scan limitprivate 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.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 byHConstants.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
private final int
The RPC timeout period (milliseconds)private static final IOException
Deprecated.private ScannerIdGenerator
private final int
The lease timeout period for client scanners (milliseconds).private final ConcurrentMap<String,
RSRpcServices.RegionScannerHolder> Fields inherited from class org.apache.hadoop.hbase.HBaseRpcServicesBase
CLIENT_BOOTSTRAP_NODE_LIMIT, DEFAULT_CLIENT_BOOTSTRAP_NODE_LIMIT, priority, rpcServer, server
-
Constructor Summary
-
Method Summary
Modifier and TypeMethodDescriptionprivate 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
addSize
(RpcCallContext context, Result r) 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 regionprivate 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) 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) 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) protected void
Called to verify that this server is up and running.checkQuotaAndGetRegionScannerContext
(long scannerId) Returns The associated RegionScannerHolder forscannerId
or null.(package private) RSRpcServices.RegionScannerContext
checkQuotaAndGetRegionScannerContext
(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse.Builder builder) private void
checkScanNextCallSeq
(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request, RSRpcServices.RegionScannerHolder rsh) private void
checkShouldRejectReplicationRequest
(List<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry> entries) 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) 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, boolean isError) 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.protected PriorityFunction
protected RpcServerInterface
createRpcServer
(Server server, RpcSchedulerFactory rpcSchedulerFactory, InetSocketAddress bindAddress, String name) protected boolean
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) 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 regionMutationrm
and per Mutation, do the work, and then when done, add an instance of aClientProtos.ResultOrException
that corresponds to each Mutation.private OperationStatus[]
doReplayBatchOp
(HRegion region, List<WALSplitUtil.MutationReplay> mutations, long replaySeqId) Deprecated.Since 3.0.0, will be removed in 4.0.0.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 org.apache.hbase.thirdparty.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) 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
get
(Get get, HRegion region, RSRpcServices.RegionScannersCloseCallBack closeCallBack, RpcCallContext context) 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.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) private ExtendedCellScanner
getAndReset
(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller) 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) protected int
protected DNS.ServerType
private org.apache.hadoop.fs.FileSystem
getFileSystem
(List<String> filePaths) protected String
getHostname
(org.apache.hadoop.conf.Configuration conf, String defaultHostname) 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) protected String
(package private) PriorityFunction
getRegion
(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier regionSpecifier) Find the HRegion based on a region specifierorg.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) 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 specifiersgetRegionScanner
(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request) 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
getResultOrException
(Exception e, int index) 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) private RegionServerRpcQuotaManager
protected Class<?>
protected Class<?>
getRpcSchedulerFactoryClass
(org.apache.hadoop.conf.Configuration conf) getScanDetailsWithId
(long scannerId) getScanDetailsWithRequest
(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request) (package private) RegionScanner
getScanner
(long scannerId) Returns The outstanding RegionScanner forscannerId
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.protected List<RpcServer.BlockingServiceAndInterface>
By default, put up an Admin and a Client Service.private RegionServerSpaceQuotaManager
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
isClientCellBlockSupport
(RpcCallContext context) private boolean
isFullRegionScan
(Scan scan, HRegion region) private boolean
isReplicationRequest
(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action action) 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 execCoprocessororg.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.private Pair<String,
RSRpcServices.RegionScannerHolder> newRegionScanner
(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request, HRegion region, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse.Builder builder) void
onConfigurationChange
(org.apache.hadoop.conf.Configuration conf) This method would be called by theConfigurationManager
object when theConfiguration
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) private void
rejectIfInStandByState
(HRegion region) 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) Deprecated.Since 3.0.0, will be removed in 4.0.0.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) Replay the given changes on a secondary replicaorg.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.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 permissionsprivate 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 boolean
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
(package private) void
stop()
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.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 org.apache.hadoop.hbase.HBaseRpcServicesBase
checkOOME, clearSlowLogsResponses, getAccessChecker, getActiveMaster, getBootstrapNodes, getClusterId, getConfiguration, getDeadline, getLogEntries, getMasters, getMetaRegionLocations, getPriority, getRpcScheduler, getRpcServer, getServer, getSocketAddress, getZkPermissionWatcher, internalStart, internalStop, requirePermission, updateConfiguration
-
Field Details
-
LOG
-
REGION_SERVER_RPC_SCHEDULER_FACTORY_CLASS
RPC scheduler to use for the region server.- See Also:
-
REGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA
Minimum allowable time limit delta (in milliseconds) that can be enforced during scans. This configuration exists to prevent the scenario where a time limit is specified to be so restrictive that the time limit is reached immediately (before any cells are scanned).- See Also:
-
DEFAULT_REGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA
Default value ofREGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA
- See Also:
-
REJECT_BATCH_ROWS_OVER_THRESHOLD
Whether to reject rows with size > threshold defined byHConstants.BATCH_ROWS_THRESHOLD_NAME
- See Also:
-
DEFAULT_REJECT_BATCH_ROWS_OVER_THRESHOLD
Default value of configREJECT_BATCH_ROWS_OVER_THRESHOLD
- See Also:
-
requestCount
-
rpcGetRequestCount
-
rpcScanRequestCount
-
rpcFullScanRequestCount
-
rpcMultiRequestCount
-
rpcMutateRequestCount
-
maxScannerResultSize
-
scannerIdGenerator
-
scanners
-
closedScanners
-
scannerLeaseTimeoutPeriod
The lease timeout period for client scanners (milliseconds). -
rpcTimeout
The RPC timeout period (milliseconds) -
minimumScanTimeLimitDelta
The minimum allowable delta to use for the scan limit -
rowSizeWarnThreshold
Row size threshold for multi requests above which a warning is logged -
rejectRowsWithSizeOverThreshold
-
clearCompactionQueues
-
REGIONSERVER_ADMIN_SERVICE_CONFIG
Services launched in RSRpcServices. By default they are on but you can use the below booleans to selectively enable/disable these services (Rare is the case where you would ever turn off one or the other).- See Also:
-
REGIONSERVER_CLIENT_SERVICE_CONFIG
- See Also:
-
REGIONSERVER_CLIENT_META_SERVICE_CONFIG
- See Also:
-
REGIONSERVER_BOOTSTRAP_NODES_SERVICE_CONFIG
- See Also:
-
SCANNER_ALREADY_CLOSED
Deprecated.
-
-
Constructor Details
-
RSRpcServices
- Throws:
IOException
-
-
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:- RegionServer is running
- 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
-
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 regionMutationrm
and per Mutation, do the work, and then when done, add an instance of aClientProtos.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 multigetscontext
- the current RpcCallContext- Returns:
- Return the
cellScanner
passed
-
checkCellSizeLimit
- 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, seereplicateToReplica(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
-
defaultReservoirEnabled
- Specified by:
defaultReservoirEnabled
in classHBaseRpcServicesBase<HRegionServer>
-
getDNSServerType
- Specified by:
getDNSServerType
in classHBaseRpcServicesBase<HRegionServer>
-
getHostname
- Specified by:
getHostname
in classHBaseRpcServicesBase<HRegionServer>
-
getPortConfigName
- Specified by:
getPortConfigName
in classHBaseRpcServicesBase<HRegionServer>
-
getDefaultPort
- Specified by:
getDefaultPort
in classHBaseRpcServicesBase<HRegionServer>
-
getRpcSchedulerFactoryClass
- Specified by:
getRpcSchedulerFactoryClass
in classHBaseRpcServicesBase<HRegionServer>
-
createRpcServer
protected RpcServerInterface createRpcServer(Server server, RpcSchedulerFactory rpcSchedulerFactory, InetSocketAddress bindAddress, String name) throws IOException - Throws:
IOException
-
getRpcSchedulerFactoryClass
-
createPriority
- Specified by:
createPriority
in classHBaseRpcServicesBase<HRegionServer>
-
getScannersCount
-
getScanner
Returns The outstanding RegionScanner forscannerId
or null if none found. -
checkQuotaAndGetRegionScannerContext
Returns The associated RegionScannerHolder forscannerId
or null. -
getScanDetailsWithId
-
getScanDetailsWithRequest
public String getScanDetailsWithRequest(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request) -
getScannerVirtualTime
Get the vtime associated with the scanner. Currently the vtime is the number of "next" calls. -
addSize
Method to account for the size of retained cells.- Parameters:
context
- rpc call contextr
- result to add size.
-
getRemoteClientIpAndPort
Returns Remote client's ip and port else null if can't be determined. -
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 -
isFullRegionScan
-
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
-
stop
void stop() -
checkOpen
Called to verify that this server is up and running.- Throws:
IOException
-
getServices
By default, put up an Admin and a Client Service. Set booleanshbase.regionserver.admin.executorService
andhbase.regionserver.client.executorService
if you want to enable/disable services. Default is that both are enabled.- Specified by:
getServices
in classHBaseRpcServicesBase<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 interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
- Parameters:
controller
- the RPC controllerrequest
- 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 interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
- Parameters:
controller
- the RPC controllerrequest
- 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 interfaceorg.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 interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
- Parameters:
controller
- the RPC controllerrequest
- 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 interfaceorg.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 interfaceorg.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 interfaceorg.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 interfaceorg.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 interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
- Parameters:
controller
- the RPC controllerrequest
- 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 interfaceorg.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 interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
- Parameters:
controller
- the RPC controllerrequest
- 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 interfaceorg.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 usereplicateToReplica
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 interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
- Parameters:
controller
- the RPC controllerrequest
- 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 interfaceorg.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 interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
- Parameters:
controller
- the RPC controllerrequest
- 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 interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
- Parameters:
controller
- the RPC controllerrequest
- 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 interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
- Parameters:
controller
- the RPC controllerrequest
- 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 interfaceorg.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 interfaceorg.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 interfaceorg.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 interfaceorg.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 interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface
- Throws:
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
-
getFileSystem
- 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
-
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 interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface
- Parameters:
controller
- the RPC controllerrequest
- 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 interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface
- Parameters:
rpcc
- the RPC controllerrequest
- 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 interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface
- Parameters:
rpcc
- the RPC controllerrequest
- 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, HRegion region, 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
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
-
getRemainingRpcTimeout
-
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 interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface
- Parameters:
controller
- the RPC controllerrequest
- 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 interfaceorg.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 interfaceorg.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 interfaceorg.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 interfaceorg.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 interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.BootstrapNodeProtos.BootstrapNodeService.BlockingInterface
- Throws:
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
-
setReloadableGuardrails
-
onConfigurationChange
Description copied from interface:ConfigurationObserver
This method would be called by theConfigurationManager
object when theConfiguration
object is reloaded from disk.- Specified by:
onConfigurationChange
in interfaceConfigurationObserver
- Overrides:
onConfigurationChange
in classHBaseRpcServicesBase<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 interfaceorg.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
- Throws:
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
-
checkQuotaAndGetRegionScannerContext
RSRpcServices.RegionScannerContext checkQuotaAndGetRegionScannerContext(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse.Builder builder) throws IOException - Throws:
IOException
-