@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, PriorityFunction, ConfigurationObserver
| Modifier and Type | Class and Description |
|---|---|
(package private) static interface |
RSRpcServices.LogDelegate |
private static class |
RSRpcServices.RegionScannerCloseCallBack
An Rpc callback for closing a RegionScanner.
|
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.
|
| Modifier and Type | Field and Description |
|---|---|
private AccessChecker |
accessChecker |
(package private) static int |
BATCH_ROWS_THRESHOLD_DEFAULT
Default value of
BATCH_ROWS_THRESHOLD_NAME |
(package private) static String |
BATCH_ROWS_THRESHOLD_NAME
Number of rows in a batch operation above which a warning will be logged.
|
(package private) AtomicBoolean |
clearCompactionQueues |
private org.apache.hbase.thirdparty.com.google.common.cache.Cache<String,String> |
closedScanners |
private static RSRpcServices.LogDelegate |
DEFAULT_LOG_DELEGATE |
private static long |
DEFAULT_REGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA
Default value of
REGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA |
private static boolean |
DEFAULT_REJECT_BATCH_ROWS_OVER_THRESHOLD |
(package private) InetSocketAddress |
isa |
private RSRpcServices.LogDelegate |
ld |
protected static org.slf4j.Logger |
LOG |
static String |
MASTER_RPC_SCHEDULER_FACTORY_CLASS
RPC scheduler to use for the master.
|
private long |
maxScannerResultSize |
private long |
minimumScanTimeLimitDelta
The minimum allowable delta to use for the scan limit
|
private PriorityFunction |
priority |
private static String |
REGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA
Minimum allowable time limit delta (in milliseconds) that can be enforced during scans.
|
static String |
REGION_SERVER_RPC_SCHEDULER_FACTORY_CLASS
RPC scheduler to use for the region server.
|
protected HRegionServer |
regionServer |
static String |
REGIONSERVER_ADMIN_SERVICE_CONFIG
Services launched in RSRpcServices.
|
static String |
REGIONSERVER_CLIENT_SERVICE_CONFIG |
private static String |
REJECT_BATCH_ROWS_OVER_THRESHOLD |
private boolean |
rejectRowsWithSizeOverThreshold |
(package private) LongAdder |
requestCount |
private int |
rowSizeWarnThreshold
Row size threshold for multi requests above which a warning is logged
|
(package private) LongAdder |
rpcGetRequestCount |
(package private) LongAdder |
rpcMultiRequestCount |
(package private) LongAdder |
rpcMutateRequestCount |
(package private) LongAdder |
rpcScanRequestCount |
(package private) RpcServerInterface |
rpcServer |
private int |
rpcTimeout
The RPC timeout period (milliseconds)
|
private static IOException |
SCANNER_ALREADY_CLOSED
Deprecated.
|
private ScannerIdGenerator |
scannerIdGenerator |
private int |
scannerLeaseTimeoutPeriod
The lease timeout period for client scanners (milliseconds).
|
private ConcurrentMap<String,RSRpcServices.RegionScannerHolder> |
scanners |
private ZKPermissionWatcher |
zkPermissionWatcher |
| Constructor and Description |
|---|
RSRpcServices(HRegionServer rs) |
RSRpcServices(HRegionServer rs,
RSRpcServices.LogDelegate ld) |
| 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) |
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 boolean |
checkAndRowMutate(HRegion region,
List<org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action> actions,
CellScanner cellScanner,
byte[] row,
byte[] family,
byte[] qualifier,
CompareOperator op,
ByteArrayComparable comparator,
Filter filter,
TimeRange timeRange,
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult.Builder builder,
ActivePolicyEnforcement spaceQuotaEnforcement)
Mutate a list of rows atomically.
|
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 |
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,
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,
ActivePolicyEnforcement spaceQuotaEnforcement,
boolean atomic)
Execute a list of Put/Delete 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.
|
private void |
endNonceOperation(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation,
long nonceGroup,
boolean success)
Ends nonce operation for a mutation, if needed.
|
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) |
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.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.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 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) |
RegionScanner |
getScanner(long scannerId) |
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.
|
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) |
private long |
getTimeLimit(HBaseRpcController controller,
boolean allowHeartbeatMessages) |
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) |
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 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) |
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 |
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,
RpcCallContext context) |
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) |
private boolean |
startNonceOperation(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation,
long nonceGroup)
Starts the nonce operation for a mutation, if needed.
|
(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 Get |
toGet(Mutation mutation) |
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)
Wamrmup 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
private static final long DEFAULT_REGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA
REGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTAstatic final String BATCH_ROWS_THRESHOLD_NAME
static final int BATCH_ROWS_THRESHOLD_DEFAULT
BATCH_ROWS_THRESHOLD_NAMEprivate static final String REJECT_BATCH_ROWS_OVER_THRESHOLD
private static final boolean DEFAULT_REJECT_BATCH_ROWS_OVER_THRESHOLD
final LongAdder requestCount
final LongAdder rpcGetRequestCount
final LongAdder rpcScanRequestCount
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
private static RSRpcServices.LogDelegate DEFAULT_LOG_DELEGATE
private final RSRpcServices.LogDelegate ld
@Deprecated private static final IOException SCANNER_ALREADY_CLOSED
public RSRpcServices(HRegionServer rs) throws IOException
IOExceptionRSRpcServices(HRegionServer rs, RSRpcServices.LogDelegate ld) throws IOException
IOExceptionprivate 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 startNonceOperation(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation, long nonceGroup) throws IOException
mutation - Mutation.nonceGroup - Nonce group from the request.IOExceptionprivate void endNonceOperation(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation, long nonceGroup, boolean success)
mutation - Mutation.nonceGroup - Nonce group from the request. Always 0 in initial implementation.success - Whether the operation for this nonce has succeeded.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 boolean checkAndRowMutate(HRegion region, List<org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action> actions, CellScanner cellScanner, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Filter filter, TimeRange timeRange, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult.Builder builder, ActivePolicyEnforcement spaceQuotaEnforcement) throws IOException
cellScanner - if non-null, the mutation data -- the Cell content.IOExceptionprivate Result append(HRegion region, OperationQuota quota, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation, CellScanner cellScanner, long nonceGroup, ActivePolicyEnforcement spaceQuota) throws IOException
IOExceptionprivate Result increment(HRegion region, OperationQuota quota, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation, CellScanner cells, long nonceGroup, ActivePolicyEnforcement spaceQuota) throws IOException
IOExceptionprivate static Get toGet(Mutation mutation) throws IOException
IOExceptionprivate 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
IOExceptionprivate 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, ActivePolicyEnforcement spaceQuotaEnforcement) throws IOException
IOExceptionprivate 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, ActivePolicyEnforcement spaceQuotaEnforcement, boolean atomic) throws IOException
builder - region - mutations - IOExceptionprivate void updateMutationMetrics(HRegion region, long starttime, boolean batchContainsPuts, boolean batchContainsDelete)
private OperationStatus[] doReplayBatchOp(HRegion region, List<WALSplitUtil.MutationReplay> mutations, long replaySeqId) throws IOException
region - mutations - replaySeqId - IOExceptionprivate void closeAllScanners()
protected RpcServerInterface createRpcServer(Server server, RpcSchedulerFactory rpcSchedulerFactory, InetSocketAddress bindAddress, String name) throws IOException
IOExceptionprotected Class<?> getRpcSchedulerFactoryClass()
public void onConfigurationChange(org.apache.hadoop.conf.Configuration newConf)
ConfigurationObserverConfigurationManager
object when the Configuration object is reloaded from disk.onConfigurationChange in interface ConfigurationObserverprotected PriorityFunction createPriority()
protected void requirePermission(String request, Permission.Action perm) throws IOException
IOExceptionpublic int getScannersCount()
public RegionScanner getScanner(long scannerId)
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.private RSRpcServices.RegionScannerHolder addScanner(String scannerName, RegionScanner s, Shipper shipper, HRegion r, boolean needCursor) throws LeaseManager.LeaseStillHeldException
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
IOExceptionprotected 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)
PriorityFunctiongetPriority in interface PriorityFunctionpublic long getDeadline(org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader header, org.apache.hbase.thirdparty.com.google.protobuf.Message param)
PriorityFunctiongetDeadline in interface PriorityFunctionpublic boolean checkOOME(Throwable e)
HBaseRPCErrorHandlercheckOOME in interface HBaseRPCErrorHandlere - 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.BlockingInterfacecontroller - the RPC controllerrequest - the requestorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionpublic 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.BlockingInterfacecontroller - the RPC controllerrequest - the requestorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionpublic 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.BlockingInterfaceorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionpublic 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.BlockingInterfacecontroller - the RPC controllerrequest - the requestorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionpublic 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.BlockingInterfaceorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionpublic 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.BlockingInterfaceorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionpublic 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.BlockingInterfaceorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionpublic 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.BlockingInterfaceorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionpublic 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.BlockingInterfacecontroller - the RPC controllerrequest - the requestorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionpublic 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.BlockingInterfaceorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionprivate 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.ServiceExceptionprivate 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.ServiceExceptionprivate void throwOnWrongStartCode(long serverStartCode) throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException
org.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionprivate 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.ServiceExceptionpublic 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.BlockingInterfacecontroller - the RPC controllerrequest - the requestorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionpublic 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.BlockingInterfacecontroller - the RPC controllerrequest - the requestorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionpublic 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.BlockingInterfacecontroller - the RPC controllerrequest - the requestorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionpublic 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.BlockingInterfacecontroller - the RPC controllerrequest - the requestorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionpublic 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.BlockingInterfacecontroller - the RPC controllerrequest - the requestorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionpublic 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.BlockingInterfacecontroller - the RPC controllerrequest - the requestorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionpublic 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.BlockingInterfaceorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionpublic 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.BlockingInterfaceorg.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.BlockingInterfaceorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionpublic 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.BlockingInterfaceorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionpublic 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.BlockingInterfaceorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionprivate com.google.protobuf.Message execServiceOnRegion(HRegion region, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall serviceCall) throws IOException
IOExceptionpublic 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.BlockingInterfacecontroller - the RPC controllerrequest - the get requestorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionprivate Result get(Get get, HRegion region, RSRpcServices.RegionScannersCloseCallBack closeCallBack, RpcCallContext context) throws IOException
IOExceptionprivate 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.ServiceExceptionpublic 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.BlockingInterfacerpcc - the RPC controllerrequest - the multi requestorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionprivate 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.BlockingInterfacerpcc - the RPC controllerrequest - the mutate requestorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionprivate RSRpcServices.RegionScannerHolder getRegionScanner(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request) throws IOException
IOExceptionprivate 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
IOExceptionprivate void checkScanNextCallSeq(org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request, RSRpcServices.RegionScannerHolder rsh) throws OutOfOrderScannerNextException
OutOfOrderScannerNextExceptionprivate void addScannerLeaseBack(LeaseManager.Lease lease)
private long getTimeLimit(HBaseRpcController controller, boolean allowHeartbeatMessages)
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, RpcCallContext context) throws IOException
IOExceptionpublic 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.BlockingInterfacecontroller - the RPC controllerrequest - the scan requestorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionprivate void closeScanner(HRegion region, RegionScanner scanner, String scannerName, RpcCallContext context) throws IOException
IOExceptionpublic 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.BlockingInterfaceorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionpublic 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.BlockingInterfaceorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionpublic 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.BlockingInterfaceorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionpublic 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)
clearRegionBlockCache in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterfaceprivate 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.BlockingInterfaceorg.apache.hbase.thirdparty.com.google.protobuf.ServiceExceptionpublic 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.BlockingInterfacepublic 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.BlockingInterfacepublic 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)
clearSlowLogsResponses in interface org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterfacepublic RpcScheduler getRpcScheduler()
protected AccessChecker getAccessChecker()
protected ZKPermissionWatcher getZkPermissionWatcher()
Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.