@InterfaceAudience.Private public class RSRpcServices extends Object implements HBaseRPCErrorHandler, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface, org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ClientMetaService.BlockingInterface, PriorityFunction, ConfigurationObserver
Modifier and Type | Class and Description |
---|---|
private static class |
RSRpcServices.RegionScannerCloseCallBack
An Rpc callback for closing a RegionScanner.
|
(package private) static class |
RSRpcServices.RegionScannerHolder
Holder class which holds the RegionScanner, nextCallSeq and RpcCallbacks together.
|
(package private) static class |
RSRpcServices.RegionScannersCloseCallBack
An RpcCallBack that creates a list of scanners that needs to perform callBack operation on
completion of multiGets.
|
private class |
RSRpcServices.RegionScannerShippedCallBack
An Rpc callback for doing shipped() call on a RegionScanner.
|
private class |
RSRpcServices.ScannerListener
Instantiated as a scanner lease.
|
Constructor and Description |
---|
RSRpcServices(HRegionServer rs) |
Modifier and Type | Method and Description |
---|---|
private void |
addResult(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse.Builder builder,
Result result,
HBaseRpcController rpcc,
boolean clientCellBlockSupported) |
private void |
addResults(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse.Builder builder,
List<Result> results,
HBaseRpcController controller,
boolean isDefaultRegion,
boolean clientCellBlockSupported) |
private RSRpcServices.RegionScannerHolder |
addScanner(String scannerName,
RegionScanner s,
Shipper shipper,
HRegion r,
boolean needCursor,
boolean fullRegionScan) |
private void |
addScannerLeaseBack(LeaseManager.Lease lease) |
(package private) Object |
addSize(RpcCallContext context,
Result r,
Object lastBlock)
Method to account for the size of retained cells and retained data blocks.
|
private Result |
append(HRegion region,
OperationQuota quota,
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation,
CellScanner cellScanner,
long nonceGroup,
ActivePolicyEnforcement spaceQuota)
Execute an append mutation.
|
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse |
bulkLoadHFile(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest request)
Atomically bulk load several HFiles into an open region
|
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) |
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) |
private void |
checkBatchSizeAndLogLargeSize(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest request) |
private void |
checkCellSizeLimit(HRegion r,
Mutation m) |
private void |
checkLimitOfRows(int numOfCompleteRows,
int limitOfRows,
boolean moreRows,
ScannerContext scannerContext,
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse.Builder builder) |
boolean |
checkOOME(Throwable e)
Take actions on the event of an OutOfMemoryError.
|
protected void |
checkOpen()
Called to verify that this server is up and running.
|
private void |
checkScanNextCallSeq(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request,
RSRpcServices.RegionScannerHolder rsh) |
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse |
cleanupBulkLoad(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest request) |
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesResponse |
clearCompactionQueues(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesRequest request) |
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheResponse |
clearRegionBlockCache(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheRequest request) |
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearSlowLogResponses |
clearSlowLogsResponses(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest request) |
private void |
closeAllScanners() |
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse |
closeRegion(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest request)
Close a region on the region server.
|
private void |
closeScanner(HRegion region,
RegionScanner scanner,
String scannerName,
RpcCallContext context) |
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchResponse |
compactionSwitch(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchRequest request) |
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse |
compactRegion(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest request)
Compact a region on the region server.
|
protected PriorityFunction |
createPriority() |
protected RpcServerInterface |
createRpcServer(Server server,
RpcSchedulerFactory rpcSchedulerFactory,
InetSocketAddress bindAddress,
String name) |
private void |
delete(HRegion region,
OperationQuota quota,
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation,
CellScanner cellScanner,
ActivePolicyEnforcement spaceQuota) |
private void |
doAtomicBatchOp(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult.Builder builder,
HRegion region,
OperationQuota quota,
List<org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action> mutations,
CellScanner cells,
long nonceGroup,
ActivePolicyEnforcement spaceQuotaEnforcement) |
private void |
doBatchOp(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult.Builder builder,
HRegion region,
OperationQuota quota,
List<org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action> mutations,
CellScanner cells,
long nonceGroup,
ActivePolicyEnforcement spaceQuotaEnforcement,
boolean atomic)
Execute a list of mutations.
|
private void |
doNonAtomicBatchOp(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult.Builder builder,
HRegion region,
OperationQuota quota,
List<org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action> mutations,
CellScanner cells,
ActivePolicyEnforcement spaceQuotaEnforcement) |
private List<CellScannable> |
doNonAtomicRegionMutation(HRegion region,
OperationQuota quota,
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction actions,
CellScanner cellScanner,
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult.Builder builder,
List<CellScannable> cellsToReturn,
long nonceGroup,
RSRpcServices.RegionScannersCloseCallBack closeCallBack,
RpcCallContext context,
ActivePolicyEnforcement spaceQuotaEnforcement)
Run through the regionMutation
rm and per Mutation, do the work, and then when
done, add an instance of a ClientProtos.ResultOrException that corresponds to each Mutation. |
private OperationStatus[] |
doReplayBatchOp(HRegion region,
List<WALSplitUtil.MutationReplay> mutations,
long replaySeqId)
Execute a list of Put/Delete mutations.
|
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse |
execRegionServerService(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest request) |
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse |
execService(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest request) |
private com.google.protobuf.Message |
execServiceOnRegion(HRegion region,
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall serviceCall) |
private void |
executeCloseRegionProcedures(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest request) |
private void |
executeOpenRegionProcedures(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest request,
Map<TableName,TableDescriptor> tdCache) |
private void |
executeProcedures(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RemoteProcedureRequest request) |
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse |
executeProcedures(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresRequest request) |
static boolean |
exitIfOOME(Throwable e) |
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.
|
protected AccessChecker |
getAccessChecker() |
org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetActiveMasterResponse |
getActiveMaster(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetActiveMasterRequest request) |
org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetBootstrapNodesResponse |
getBootstrapNodes(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetBootstrapNodesRequest request) |
org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetClusterIdResponse |
getClusterId(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetClusterIdRequest request) |
org.apache.hadoop.conf.Configuration |
getConfiguration() |
long |
getDeadline(org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader header,
org.apache.hbase.thirdparty.com.google.protobuf.Message param)
Returns the deadline of the specified request.
|
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SlowLogResponses |
getLargeLogResponses(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SlowLogResponseRequest request) |
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LogEntry |
getLogEntries(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LogRequest request) |
org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMastersResponse |
getMasters(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMastersRequest request) |
org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMetaRegionLocationsResponse |
getMetaRegionLocations(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMetaRegionLocationsRequest request) |
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse |
getOnlineRegion(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest request) |
PriorityFunction |
getPriority() |
int |
getPriority(org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader header,
org.apache.hbase.thirdparty.com.google.protobuf.Message param,
User user)
Returns the 'priority type' of the specified request.
|
HRegion |
getRegion(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier regionSpecifier)
Find the HRegion based on a region specifier
|
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse |
getRegionInfo(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest request) |
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadResponse |
getRegionLoad(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest request) |
private List<HRegion> |
getRegions(List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> regionSpecifiers,
CacheEvictionStatsBuilder stats)
Find the List of HRegions based on a list of region specifiers
|
private RSRpcServices.RegionScannerHolder |
getRegionScanner(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request) |
private RSRpcServices.RegionScannerHolder |
getRegionScannerHolder(long scannerId)
Returns The associated RegionScannerHolder for
scannerId or null. |
private long |
getRemainingRpcTimeout(RpcCall call,
HBaseRpcController controller,
long now) |
(package private) static String |
getRemoteClientIpAndPort()
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(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result r,
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(Exception e,
int index) |
private RegionServerRpcQuotaManager |
getRpcQuotaManager() |
RpcScheduler |
getRpcScheduler() |
protected Class<?> |
getRpcSchedulerFactoryClass() |
String |
getScanDetailsWithId(long scannerId) |
String |
getScanDetailsWithRequest(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request) |
(package private) RegionScanner |
getScanner(long scannerId)
Returns The outstanding RegionScanner for
scannerId or null if none found. |
int |
getScannersCount() |
(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> |
getServices()
By default, put up an Admin and a Client Service.
|
private List<org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload> |
getSlowLogPayloads(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SlowLogResponseRequest request,
NamedQueueRecorder namedQueueRecorder) |
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SlowLogResponses |
getSlowLogResponses(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SlowLogResponseRequest request) |
InetSocketAddress |
getSocketAddress() |
private RegionServerSpaceQuotaManager |
getSpaceQuotaManager() |
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 |
getUserName()
Returns Remote client's username.
|
protected ZKPermissionWatcher |
getZkPermissionWatcher() |
private Result |
increment(HRegion region,
OperationQuota quota,
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation,
CellScanner cells,
long nonceGroup,
ActivePolicyEnforcement spaceQuota)
Execute an increment mutation.
|
private boolean |
isClientCellBlockSupport(RpcCallContext context) |
private boolean |
isFullRegionScan(Scan scan,
HRegion region) |
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse |
multi(org.apache.hbase.thirdparty.com.google.protobuf.RpcController rpcc,
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest request)
Execute multiple actions on a table: get, mutate, and/or execCoprocessor
|
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse |
mutate(org.apache.hbase.thirdparty.com.google.protobuf.RpcController rpcc,
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest request)
Mutate data in a table.
|
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) |
void |
onConfigurationChange(org.apache.hadoop.conf.Configuration newConf)
This method would be called by the
ConfigurationManager object when the
Configuration object is reloaded from disk. |
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse |
openRegion(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest request)
Open asynchronously a region or a set of regions on the region server.
|
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse |
prepareBulkLoad(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest request) |
private void |
put(HRegion region,
OperationQuota quota,
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation,
CellScanner cellScanner,
ActivePolicyEnforcement spaceQuota) |
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse |
replay(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request)
Replay the given changes when distributedLogReplay WAL edits from a failed RS.
|
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse |
replicateWALEntry(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request)
Replicate WAL entries on the region server.
|
protected void |
requirePermission(String request,
Permission.Action perm) |
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse |
rollWALWriter(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest request)
Roll the WAL writer of the region server.
|
private void |
rpcPreCheck(String requestName)
Checks for the following pre-checks in order:
RegionServer is running
If authorization is enabled, then RPC caller has ADMIN permissions
|
private void |
runShippedCallback(RSRpcServices.RegionScannerHolder rsh) |
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,
org.apache.commons.lang3.mutable.MutableObject<Object> lastBlock,
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 |
skipCellsForMutation(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action action,
CellScanner cellScanner) |
private void |
skipCellsForMutations(List<org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action> actions,
CellScanner cellScanner) |
(package private) void |
start(ZKWatcher zkWatcher) |
(package private) void |
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(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 void |
throwOnWrongStartCode(long serverStartCode) |
private static String |
toScannerName(long scannerId)
The returned String is used as key doing look up of outstanding Scanners in this Servers'
this.scanners, the Map of outstanding scanners and their current state.
|
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse |
updateConfiguration(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest request) |
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse |
updateFavoredNodes(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest request) |
private void |
updateMutationMetrics(HRegion region,
long starttime,
boolean batchContainsPuts,
boolean batchContainsDelete) |
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse |
warmupRegion(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest request)
Warmup a region on this server.
|
protected static final org.slf4j.Logger LOG
public static final String REGION_SERVER_RPC_SCHEDULER_FACTORY_CLASS
public static final String MASTER_RPC_SCHEDULER_FACTORY_CLASS
private static final String REGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA
static final long DEFAULT_REGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA
REGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA
private static final String REJECT_BATCH_ROWS_OVER_THRESHOLD
HConstants.BATCH_ROWS_THRESHOLD_NAME
private static final boolean DEFAULT_REJECT_BATCH_ROWS_OVER_THRESHOLD
REJECT_BATCH_ROWS_OVER_THRESHOLD
public static final String CLIENT_BOOTSTRAP_NODE_LIMIT
public static final int DEFAULT_CLIENT_BOOTSTRAP_NODE_LIMIT
final LongAdder requestCount
final LongAdder rpcGetRequestCount
final LongAdder rpcScanRequestCount
final LongAdder rpcFullScanRequestCount
final LongAdder rpcMultiRequestCount
final LongAdder rpcMutateRequestCount
final RpcServerInterface rpcServer
final InetSocketAddress isa
protected final HRegionServer regionServer
private final long maxScannerResultSize
private final PriorityFunction priority
private ScannerIdGenerator scannerIdGenerator
private final ConcurrentMap<String,RSRpcServices.RegionScannerHolder> scanners
private final org.apache.hbase.thirdparty.com.google.common.cache.Cache<String,String> closedScanners
private final int scannerLeaseTimeoutPeriod
private final int rpcTimeout
private final long minimumScanTimeLimitDelta
private final int rowSizeWarnThreshold
private final boolean rejectRowsWithSizeOverThreshold
final AtomicBoolean clearCompactionQueues
private AccessChecker accessChecker
private ZKPermissionWatcher zkPermissionWatcher
public static final String REGIONSERVER_ADMIN_SERVICE_CONFIG
public static final String REGIONSERVER_CLIENT_SERVICE_CONFIG
public static final String REGIONSERVER_CLIENT_META_SERVICE_CONFIG
@Deprecated private static final IOException SCANNER_ALREADY_CLOSED
public RSRpcServices(HRegionServer rs) throws IOException
IOException
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 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 void rpcPreCheck(String requestName) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
requestName
- name of rpc request. Used in reporting failures to provide context.org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
- If any of the above listed pre-check fails.private boolean isClientCellBlockSupport(RpcCallContext context)
private void addResult(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse.Builder builder, Result result, HBaseRpcController rpcc, boolean clientCellBlockSupported)
private void addResults(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse.Builder builder, List<Result> results, HBaseRpcController controller, boolean isDefaultRegion, boolean clientCellBlockSupported)
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
IOException
private Result append(HRegion region, OperationQuota quota, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation, CellScanner cellScanner, long nonceGroup, ActivePolicyEnforcement spaceQuota) throws IOException
IOException
private Result increment(HRegion region, OperationQuota quota, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation, CellScanner cells, long nonceGroup, ActivePolicyEnforcement spaceQuota) throws IOException
IOException
private List<CellScannable> doNonAtomicRegionMutation(HRegion region, OperationQuota quota, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction actions, CellScanner cellScanner, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult.Builder builder, List<CellScannable> cellsToReturn, long nonceGroup, RSRpcServices.RegionScannersCloseCallBack closeCallBack, RpcCallContext context, ActivePolicyEnforcement spaceQuotaEnforcement)
rm
and per Mutation, do the work, and then when
done, add an instance of a ClientProtos.ResultOrException
that corresponds to each Mutation.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 RpcCallContextcellScanner
passedprivate void checkCellSizeLimit(HRegion r, Mutation m) throws IOException
IOException
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
IOException
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 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
IOException
private void updateMutationMetrics(HRegion region, long starttime, boolean batchContainsPuts, boolean batchContainsDelete)
private OperationStatus[] doReplayBatchOp(HRegion region, List<WALSplitUtil.MutationReplay> mutations, long replaySeqId) throws IOException
IOException
private void closeAllScanners()
protected RpcServerInterface createRpcServer(Server server, RpcSchedulerFactory rpcSchedulerFactory, InetSocketAddress bindAddress, String name) throws IOException
IOException
protected Class<?> getRpcSchedulerFactoryClass()
public void onConfigurationChange(org.apache.hadoop.conf.Configuration newConf)
ConfigurationObserver
ConfigurationManager
object when the
Configuration
object is reloaded from disk.onConfigurationChange
in interface ConfigurationObserver
protected PriorityFunction createPriority()
protected void requirePermission(String request, Permission.Action perm) throws IOException
IOException
public int getScannersCount()
RegionScanner getScanner(long scannerId)
scannerId
or null if none found.private RSRpcServices.RegionScannerHolder getRegionScannerHolder(long scannerId)
scannerId
or null.public String getScanDetailsWithId(long scannerId)
public String getScanDetailsWithRequest(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request)
long getScannerVirtualTime(long scannerId)
Object addSize(RpcCallContext context, Result r, Object lastBlock)
context
- rpc call contextr
- result to add size.lastBlock
- last block to check whether we need to add the block size in context.static String getRemoteClientIpAndPort()
static String getUserName()
private RSRpcServices.RegionScannerHolder addScanner(String scannerName, RegionScanner s, Shipper shipper, HRegion r, boolean needCursor, boolean fullRegionScan) throws LeaseManager.LeaseStillHeldException
private boolean isFullRegionScan(Scan scan, HRegion region)
public HRegion getRegion(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier regionSpecifier) throws IOException
regionSpecifier
- the region specifierIOException
- if the specifier is not null, but failed to find the regionprivate List<HRegion> getRegions(List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> regionSpecifiers, CacheEvictionStatsBuilder stats)
regionSpecifiers
- the list of region specifiersIOException
- if any of the specifiers is not null, but failed to find the regionpublic PriorityFunction getPriority()
public org.apache.hadoop.conf.Configuration getConfiguration()
private RegionServerRpcQuotaManager getRpcQuotaManager()
private RegionServerSpaceQuotaManager getSpaceQuotaManager()
void stop()
protected void checkOpen() throws IOException
IOException
protected List<RpcServer.BlockingServiceAndInterface> getServices()
hbase.regionserver.admin.executorService
and
hbase.regionserver.client.executorService
if you want to enable/disable services.
Default is that both are enabled.public InetSocketAddress getSocketAddress()
public int getPriority(org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader header, org.apache.hbase.thirdparty.com.google.protobuf.Message param, User user)
PriorityFunction
getPriority
in interface PriorityFunction
public long getDeadline(org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader header, org.apache.hbase.thirdparty.com.google.protobuf.Message param)
PriorityFunction
getDeadline
in interface PriorityFunction
public boolean checkOOME(Throwable e)
HBaseRPCErrorHandler
checkOOME
in interface HBaseRPCErrorHandler
e
- the throwablepublic static boolean exitIfOOME(Throwable e)
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
closeRegion
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
controller
- the RPC controllerrequest
- the request norg.apache.hbase.thirdparty.com.google.protobuf.ServiceException
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
compactRegion
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
controller
- the RPC controllerrequest
- the request norg.apache.hbase.thirdparty.com.google.protobuf.ServiceException
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
compactionSwitch
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
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
flushRegion
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
controller
- the RPC controllerrequest
- the request norg.apache.hbase.thirdparty.com.google.protobuf.ServiceException
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
getOnlineRegion
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
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
getRegionInfo
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
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
getRegionLoad
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
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
clearCompactionQueues
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
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
getServerInfo
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
controller
- the RPC controllerrequest
- the request norg.apache.hbase.thirdparty.com.google.protobuf.ServiceException
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
getStoreFile
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
private void throwOnWrongStartCode(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
private void throwOnWrongStartCode(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
private void throwOnWrongStartCode(long serverStartCode) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
private void throwOnWrongStartCode(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresRequest req) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
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
Different manages states for the region are:
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.
openRegion
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
controller
- the RPC controllerrequest
- the request norg.apache.hbase.thirdparty.com.google.protobuf.ServiceException
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
warmupRegion
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse replay(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
replay
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
controller
- the RPC controllerrequest
- the request norg.apache.hbase.thirdparty.com.google.protobuf.ServiceException
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
replicateWALEntry
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
controller
- the RPC controllerrequest
- the requestorg.apache.hbase.thirdparty.com.google.protobuf.ServiceException
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
rollWALWriter
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
controller
- the RPC controllerrequest
- the request norg.apache.hbase.thirdparty.com.google.protobuf.ServiceException
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
stopServer
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
controller
- the RPC controllerrequest
- the request norg.apache.hbase.thirdparty.com.google.protobuf.ServiceException
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
updateFavoredNodes
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
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
bulkLoadHFile
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
- if failed unrecoverablypublic 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
prepareBulkLoad
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
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
cleanupBulkLoad
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
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
execService
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
private com.google.protobuf.Message execServiceOnRegion(HRegion region, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall serviceCall) throws IOException
IOException
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
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface
controller
- the RPC controllerrequest
- the get request norg.apache.hbase.thirdparty.com.google.protobuf.ServiceException
private Result get(Get get, HRegion region, RSRpcServices.RegionScannersCloseCallBack closeCallBack, RpcCallContext context) throws IOException
IOException
private void checkBatchSizeAndLogLargeSize(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
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)
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
multi
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface
rpcc
- the RPC controllerrequest
- the multi request norg.apache.hbase.thirdparty.com.google.protobuf.ServiceException
private void skipCellsForMutations(List<org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action> actions, CellScanner cellScanner)
private void skipCellsForMutation(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action action, CellScanner cellScanner)
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
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface
rpcc
- the RPC controllerrequest
- the mutate requestorg.apache.hbase.thirdparty.com.google.protobuf.ServiceException
private void put(HRegion region, OperationQuota quota, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation, CellScanner cellScanner, ActivePolicyEnforcement spaceQuota) throws IOException
IOException
private void delete(HRegion region, OperationQuota quota, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation, CellScanner cellScanner, ActivePolicyEnforcement spaceQuota) throws IOException
IOException
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) throws IOException
IOException
private RSRpcServices.RegionScannerHolder getRegionScanner(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request) throws IOException
IOException
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
IOException
private static String toScannerName(long scannerId)
scannerId
- A scanner long id.private void checkScanNextCallSeq(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request, RSRpcServices.RegionScannerHolder rsh) throws OutOfOrderScannerNextException
OutOfOrderScannerNextException
private void addScannerLeaseBack(LeaseManager.Lease lease)
long getTimeLimit(RpcCall rpcCall, HBaseRpcController controller, boolean allowHeartbeatMessages)
private long getRemainingRpcTimeout(RpcCall call, HBaseRpcController controller, long now)
private void checkLimitOfRows(int numOfCompleteRows, int limitOfRows, boolean moreRows, ScannerContext scannerContext, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse.Builder builder)
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, org.apache.commons.lang3.mutable.MutableObject<Object> lastBlock, RpcCall rpcCall) throws IOException
IOException
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
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface
controller
- the RPC controllerrequest
- the scan request norg.apache.hbase.thirdparty.com.google.protobuf.ServiceException
private void runShippedCallback(RSRpcServices.RegionScannerHolder rsh) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
private void closeScanner(HRegion region, RegionScanner scanner, String scannerName, RpcCallContext context) throws IOException
IOException
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
execRegionServerService
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse updateConfiguration(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
updateConfiguration
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
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
getSpaceQuotaSnapshots
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
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
clearRegionBlockCache
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
private void executeOpenRegionProcedures(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest request, Map<TableName,TableDescriptor> tdCache)
private void executeCloseRegionProcedures(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest request)
private void executeProcedures(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RemoteProcedureRequest request)
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
executeProcedures
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SlowLogResponses getSlowLogResponses(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SlowLogResponseRequest request)
getSlowLogResponses
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
private List<org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload> getSlowLogPayloads(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SlowLogResponseRequest request, NamedQueueRecorder namedQueueRecorder)
public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SlowLogResponses getLargeLogResponses(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SlowLogResponseRequest request)
getLargeLogResponses
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearSlowLogResponses clearSlowLogsResponses(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearSlowLogResponseRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
clearSlowLogsResponses
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LogEntry getLogEntries(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LogRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
getLogEntries
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
public RpcScheduler getRpcScheduler()
protected AccessChecker getAccessChecker()
protected ZKPermissionWatcher getZkPermissionWatcher()
public org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetClusterIdResponse getClusterId(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetClusterIdRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
getClusterId
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ClientMetaService.BlockingInterface
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
public org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetActiveMasterResponse getActiveMaster(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetActiveMasterRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
getActiveMaster
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ClientMetaService.BlockingInterface
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
public org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMastersResponse getMasters(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMastersRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
getMasters
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ClientMetaService.BlockingInterface
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
public org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMetaRegionLocationsResponse getMetaRegionLocations(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMetaRegionLocationsRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
getMetaRegionLocations
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ClientMetaService.BlockingInterface
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
public final org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetBootstrapNodesResponse getBootstrapNodes(org.apache.hbase.thirdparty.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetBootstrapNodesRequest request) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
getBootstrapNodes
in interface org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ClientMetaService.BlockingInterface
org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.