2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
18 package org
.apache
.hadoop
.hbase
.master
;
20 import java
.io
.IOException
;
21 import java
.net
.InetSocketAddress
;
22 import java
.util
.ArrayList
;
23 import java
.util
.Collection
;
24 import java
.util
.Collections
;
25 import java
.util
.HashMap
;
26 import java
.util
.List
;
28 import java
.util
.Map
.Entry
;
29 import java
.util
.Optional
;
30 import java
.util
.Random
;
31 import java
.util
.TreeMap
;
32 import java
.util
.concurrent
.ConcurrentSkipListMap
;
33 import org
.apache
.hadoop
.conf
.Configuration
;
34 import org
.apache
.hadoop
.fs
.FileSystem
;
35 import org
.apache
.hadoop
.hbase
.Abortable
;
36 import org
.apache
.hadoop
.hbase
.CellScannable
;
37 import org
.apache
.hadoop
.hbase
.CellUtil
;
38 import org
.apache
.hadoop
.hbase
.ChoreService
;
39 import org
.apache
.hadoop
.hbase
.CoordinatedStateManager
;
40 import org
.apache
.hadoop
.hbase
.ServerName
;
41 import org
.apache
.hadoop
.hbase
.TableDescriptors
;
42 import org
.apache
.hadoop
.hbase
.TableName
;
43 import org
.apache
.hadoop
.hbase
.ZooKeeperConnectionException
;
44 import org
.apache
.hadoop
.hbase
.client
.AsyncClusterConnection
;
45 import org
.apache
.hadoop
.hbase
.client
.Connection
;
46 import org
.apache
.hadoop
.hbase
.client
.RegionInfo
;
47 import org
.apache
.hadoop
.hbase
.client
.RegionInfoBuilder
;
48 import org
.apache
.hadoop
.hbase
.client
.Result
;
49 import org
.apache
.hadoop
.hbase
.client
.Scan
;
50 import org
.apache
.hadoop
.hbase
.client
.locking
.EntityLock
;
51 import org
.apache
.hadoop
.hbase
.executor
.ExecutorService
;
52 import org
.apache
.hadoop
.hbase
.io
.hfile
.BlockCache
;
53 import org
.apache
.hadoop
.hbase
.ipc
.HBaseRpcController
;
54 import org
.apache
.hadoop
.hbase
.ipc
.RpcServerInterface
;
55 import org
.apache
.hadoop
.hbase
.mob
.MobFileCache
;
56 import org
.apache
.hadoop
.hbase
.quotas
.RegionServerRpcQuotaManager
;
57 import org
.apache
.hadoop
.hbase
.quotas
.RegionServerSpaceQuotaManager
;
58 import org
.apache
.hadoop
.hbase
.quotas
.RegionSizeStore
;
59 import org
.apache
.hadoop
.hbase
.regionserver
.FlushRequester
;
60 import org
.apache
.hadoop
.hbase
.regionserver
.HRegion
;
61 import org
.apache
.hadoop
.hbase
.regionserver
.HeapMemoryManager
;
62 import org
.apache
.hadoop
.hbase
.regionserver
.LeaseManager
;
63 import org
.apache
.hadoop
.hbase
.regionserver
.MetricsRegionServer
;
64 import org
.apache
.hadoop
.hbase
.regionserver
.Region
;
65 import org
.apache
.hadoop
.hbase
.regionserver
.RegionServerAccounting
;
66 import org
.apache
.hadoop
.hbase
.regionserver
.RegionServerServices
;
67 import org
.apache
.hadoop
.hbase
.regionserver
.ReplicationSourceService
;
68 import org
.apache
.hadoop
.hbase
.regionserver
.SecureBulkLoadManager
;
69 import org
.apache
.hadoop
.hbase
.regionserver
.ServerNonceManager
;
70 import org
.apache
.hadoop
.hbase
.regionserver
.compactions
.CompactionRequester
;
71 import org
.apache
.hadoop
.hbase
.regionserver
.throttle
.ThroughputController
;
72 import org
.apache
.hadoop
.hbase
.security
.access
.AccessChecker
;
73 import org
.apache
.hadoop
.hbase
.security
.access
.ZKPermissionWatcher
;
74 import org
.apache
.hadoop
.hbase
.util
.Bytes
;
75 import org
.apache
.hadoop
.hbase
.wal
.WAL
;
76 import org
.apache
.hadoop
.hbase
.zookeeper
.ZKWatcher
;
78 import org
.apache
.hbase
.thirdparty
.com
.google
.protobuf
.RpcController
;
79 import org
.apache
.hbase
.thirdparty
.com
.google
.protobuf
.ServiceException
;
81 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.ProtobufUtil
;
82 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
;
83 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.ClearCompactionQueuesRequest
;
84 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.ClearCompactionQueuesResponse
;
85 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.ClearRegionBlockCacheRequest
;
86 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.ClearRegionBlockCacheResponse
;
87 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.ClearSlowLogResponseRequest
;
88 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.ClearSlowLogResponses
;
89 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.CloseRegionRequest
;
90 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.CloseRegionResponse
;
91 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.CompactRegionRequest
;
92 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.CompactRegionResponse
;
93 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.CompactionSwitchRequest
;
94 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.CompactionSwitchResponse
;
95 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.ExecuteProceduresRequest
;
96 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.ExecuteProceduresResponse
;
97 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.FlushRegionRequest
;
98 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.FlushRegionResponse
;
99 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.GetOnlineRegionRequest
;
100 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.GetOnlineRegionResponse
;
101 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.GetRegionInfoRequest
;
102 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.GetRegionInfoResponse
;
103 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.GetRegionLoadRequest
;
104 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.GetRegionLoadResponse
;
105 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.GetServerInfoRequest
;
106 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.GetServerInfoResponse
;
107 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.GetStoreFileRequest
;
108 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.GetStoreFileResponse
;
109 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.OpenRegionRequest
;
110 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.OpenRegionResponse
;
111 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.ReplicateWALEntryRequest
;
112 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.ReplicateWALEntryResponse
;
113 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.RollWALWriterRequest
;
114 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.RollWALWriterResponse
;
115 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.SlowLogResponseRequest
;
116 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.SlowLogResponses
;
117 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.StopServerRequest
;
118 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.StopServerResponse
;
119 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.UpdateConfigurationRequest
;
120 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.UpdateConfigurationResponse
;
121 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.UpdateFavoredNodesRequest
;
122 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.UpdateFavoredNodesResponse
;
123 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.WarmupRegionRequest
;
124 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.AdminProtos
.WarmupRegionResponse
;
125 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ClientProtos
;
126 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ClientProtos
.BulkLoadHFileRequest
;
127 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ClientProtos
.BulkLoadHFileResponse
;
128 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ClientProtos
.CleanupBulkLoadRequest
;
129 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ClientProtos
.CleanupBulkLoadResponse
;
130 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ClientProtos
.CoprocessorServiceRequest
;
131 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ClientProtos
.CoprocessorServiceResponse
;
132 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ClientProtos
.GetRequest
;
133 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ClientProtos
.GetResponse
;
134 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ClientProtos
.MultiRequest
;
135 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ClientProtos
.MutateRequest
;
136 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ClientProtos
.MutateResponse
;
137 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ClientProtos
.PrepareBulkLoadRequest
;
138 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ClientProtos
.PrepareBulkLoadResponse
;
139 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ClientProtos
.ScanRequest
;
140 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ClientProtos
.ScanResponse
;
141 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.QuotaProtos
.GetSpaceQuotaSnapshotsRequest
;
142 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.QuotaProtos
.GetSpaceQuotaSnapshotsResponse
;
145 * A mock RegionServer implementation.
146 * Use this when you can't bend Mockito to your liking (e.g. return null result
147 * when 'scanning' until master timesout and then return a coherent meta row
148 * result thereafter. Have some facility for faking gets and scans. See
149 * setGetResult(byte[], byte[], Result) for how to fill the backing data
150 * store that the get pulls from.
152 class MockRegionServer
implements AdminProtos
.AdminService
.BlockingInterface
,
153 ClientProtos
.ClientService
.BlockingInterface
, RegionServerServices
{
154 private final ServerName sn
;
155 private final ZKWatcher zkw
;
156 private final Configuration conf
;
157 private final Random random
= new Random();
160 * Map of regions to map of rows and {@link Result}. Used as data source when
161 * {@link #get(RpcController, ClientProtos.GetRequest)} is called. Because we have a byte
162 * key, need to use TreeMap and provide a Comparator. Use
163 * {@link #setGetResult(byte[], byte[], Result)} filling this map.
165 private final Map
<byte [], Map
<byte [], Result
>> gets
= new TreeMap
<>(Bytes
.BYTES_COMPARATOR
);
168 * Map of regions to results to return when scanning.
170 private final Map
<byte [], Result
[]> nexts
= new TreeMap
<>(Bytes
.BYTES_COMPARATOR
);
173 * Data structure that holds regionname and index used scanning.
175 class RegionNameAndIndex
{
176 private final byte[] regionName
;
177 private int index
= 0;
179 RegionNameAndIndex(final byte[] regionName
) {
180 this.regionName
= regionName
;
183 byte[] getRegionName() {
184 return this.regionName
;
187 int getThenIncrement() {
188 int currentIndex
= this.index
;
195 * Outstanding scanners and their offset into <code>nexts</code>
197 private final Map
<Long
, RegionNameAndIndex
> scannersAndOffsets
= new HashMap
<>();
200 * @param sn Name of this mock regionserver
201 * @throws IOException
202 * @throws org.apache.hadoop.hbase.ZooKeeperConnectionException
204 MockRegionServer(final Configuration conf
, final ServerName sn
)
205 throws ZooKeeperConnectionException
, IOException
{
208 this.zkw
= new ZKWatcher(conf
, sn
.toString(), this, true);
212 * Use this method filling the backing data source used by
213 * {@link #get(RpcController, ClientProtos.GetRequest)}
214 * @param regionName the region name to assign
215 * @param row the row key
216 * @param r the single row result
218 void setGetResult(final byte [] regionName
, final byte [] row
, final Result r
) {
219 Map
<byte [], Result
> value
= this.gets
.get(regionName
);
221 // If no value already, create one. Needs to be treemap because we are
222 // using byte array as key. Not thread safe.
223 value
= new TreeMap
<>(Bytes
.BYTES_COMPARATOR
);
224 this.gets
.put(regionName
, value
);
230 * Use this method to set what a scanner will reply as we next through
234 void setNextResults(final byte [] regionName
, final Result
[] rs
) {
235 this.nexts
.put(regionName
, rs
);
239 public boolean isStopped() {
244 public void abort(String why
, Throwable e
) {
245 throw new RuntimeException(this.sn
+ ": " + why
, e
);
249 public boolean isAborted() {
253 public long openScanner(byte[] regionName
, Scan scan
) throws IOException
{
254 long scannerId
= this.random
.nextLong();
255 this.scannersAndOffsets
.put(scannerId
, new RegionNameAndIndex(regionName
));
259 public Result
next(long scannerId
) throws IOException
{
260 RegionNameAndIndex rnai
= this.scannersAndOffsets
.get(scannerId
);
261 int index
= rnai
.getThenIncrement();
262 Result
[] results
= this.nexts
.get(rnai
.getRegionName());
263 if (results
== null) return null;
264 return index
< results
.length? results
[index
]: null;
267 public Result
[] next(long scannerId
, int numberOfRows
) throws IOException
{
268 // Just return one result whatever they ask for.
269 Result r
= next(scannerId
);
270 return r
== null?
null: new Result
[] {r
};
273 public void close(final long scannerId
) throws IOException
{
274 this.scannersAndOffsets
.remove(scannerId
);
278 public void stop(String why
) {
283 public void addRegion(HRegion r
) {
287 public boolean removeRegion(HRegion r
, ServerName destination
) {
292 public HRegion
getRegion(String encodedRegionName
) {
297 public Configuration
getConfiguration() {
302 public ZKWatcher
getZooKeeper() {
307 public CoordinatedStateManager
getCoordinatedStateManager() {
312 public Connection
getConnection() {
317 public ServerName
getServerName() {
322 public boolean isStopping() {
327 public FlushRequester
getFlushRequester() {
331 public CompactionRequester
getCompactionRequestor() {
335 public RegionServerAccounting
getRegionServerAccounting() {
340 public RegionServerRpcQuotaManager
getRegionServerRpcQuotaManager() {
345 public void postOpenDeployTasks(PostOpenDeployContext context
) throws IOException
{
349 public RpcServerInterface
getRpcServer() {
354 public ConcurrentSkipListMap
<byte[], Boolean
> getRegionsInTransitionInRS() {
359 public FileSystem
getFileSystem() {
364 public GetResponse
get(RpcController controller
, GetRequest request
)
365 throws ServiceException
{
366 byte[] regionName
= request
.getRegion().getValue().toByteArray();
367 Map
<byte [], Result
> m
= this.gets
.get(regionName
);
368 GetResponse
.Builder builder
= GetResponse
.newBuilder();
370 byte[] row
= request
.getGet().getRow().toByteArray();
371 builder
.setResult(ProtobufUtil
.toResult(m
.get(row
)));
373 return builder
.build();
377 public MutateResponse
mutate(RpcController controller
, MutateRequest request
)
378 throws ServiceException
{
383 public ScanResponse
scan(RpcController controller
, ScanRequest request
)
384 throws ServiceException
{
385 ScanResponse
.Builder builder
= ScanResponse
.newBuilder();
387 if (request
.hasScan()) {
388 byte[] regionName
= request
.getRegion().getValue().toByteArray();
389 builder
.setScannerId(openScanner(regionName
, null));
390 builder
.setMoreResults(true);
393 long scannerId
= request
.getScannerId();
394 Result result
= next(scannerId
);
395 if (result
!= null) {
396 builder
.addCellsPerResult(result
.size());
397 List
<CellScannable
> results
= new ArrayList
<>(1);
399 ((HBaseRpcController
) controller
).setCellScanner(CellUtil
400 .createCellScanner(results
));
401 builder
.setMoreResults(true);
404 builder
.setMoreResults(false);
408 } catch (IOException ie
) {
409 throw new ServiceException(ie
);
411 return builder
.build();
415 public BulkLoadHFileResponse
bulkLoadHFile(RpcController controller
,
416 BulkLoadHFileRequest request
) throws ServiceException
{
421 public ClientProtos
.CoprocessorServiceResponse
execService(RpcController controller
,
422 ClientProtos
.CoprocessorServiceRequest request
) throws ServiceException
{
427 public ClientProtos
.MultiResponse
multi(
428 RpcController controller
, MultiRequest request
) throws ServiceException
{
433 public GetRegionInfoResponse
getRegionInfo(RpcController controller
,
434 GetRegionInfoRequest request
) throws ServiceException
{
435 GetRegionInfoResponse
.Builder builder
= GetRegionInfoResponse
.newBuilder();
436 builder
.setRegionInfo(ProtobufUtil
.toRegionInfo(RegionInfoBuilder
.FIRST_META_REGIONINFO
));
437 return builder
.build();
441 public GetRegionLoadResponse
getRegionLoad(RpcController controller
,
442 GetRegionLoadRequest request
) throws ServiceException
{
443 GetRegionLoadResponse
.Builder builder
= GetRegionLoadResponse
.newBuilder();
444 return builder
.build();
448 public ClearCompactionQueuesResponse
clearCompactionQueues(RpcController controller
,
449 ClearCompactionQueuesRequest request
) throws ServiceException
{
454 public GetStoreFileResponse
getStoreFile(RpcController controller
,
455 GetStoreFileRequest request
) throws ServiceException
{
460 public GetOnlineRegionResponse
getOnlineRegion(RpcController controller
,
461 GetOnlineRegionRequest request
) throws ServiceException
{
466 public List
<Region
> getRegions() {
471 public OpenRegionResponse
openRegion(RpcController controller
,
472 OpenRegionRequest request
) throws ServiceException
{
477 public WarmupRegionResponse
warmupRegion(RpcController controller
,
478 WarmupRegionRequest request
) throws ServiceException
{
482 public CloseRegionResponse
closeRegion(RpcController controller
,
483 CloseRegionRequest request
) throws ServiceException
{
488 public FlushRegionResponse
flushRegion(RpcController controller
,
489 FlushRegionRequest request
) throws ServiceException
{
494 public CompactionSwitchResponse
compactionSwitch(RpcController controller
,
495 CompactionSwitchRequest request
) throws ServiceException
{
500 public CompactRegionResponse
compactRegion(RpcController controller
,
501 CompactRegionRequest request
) throws ServiceException
{
506 public ReplicateWALEntryResponse
replicateWALEntry(RpcController controller
,
507 ReplicateWALEntryRequest request
) throws ServiceException
{
512 public RollWALWriterResponse
rollWALWriter(RpcController controller
,
513 RollWALWriterRequest request
) throws ServiceException
{
518 public GetServerInfoResponse
getServerInfo(RpcController controller
,
519 GetServerInfoRequest request
) throws ServiceException
{
524 public StopServerResponse
stopServer(RpcController controller
,
525 StopServerRequest request
) throws ServiceException
{
530 public List
<Region
> getRegions(TableName tableName
) throws IOException
{
535 public LeaseManager
getLeaseManager() {
540 public List
<WAL
> getWALs() throws IOException
{
541 return Collections
.emptyList();
545 public WAL
getWAL(RegionInfo regionInfo
) throws IOException
{
550 public ExecutorService
getExecutorService() {
555 public ChoreService
getChoreService() {
560 public void updateRegionFavoredNodesMapping(String encodedRegionName
,
561 List
<org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.HBaseProtos
.ServerName
> favoredNodes
) {
565 public InetSocketAddress
[] getFavoredNodesForRegion(String encodedRegionName
) {
570 public ReplicateWALEntryResponse
571 replay(RpcController controller
, ReplicateWALEntryRequest request
)
572 throws ServiceException
{
577 public UpdateFavoredNodesResponse
updateFavoredNodes(RpcController controller
,
578 UpdateFavoredNodesRequest request
) throws ServiceException
{
583 public ServerNonceManager
getNonceManager() {
588 public boolean reportRegionStateTransition(RegionStateTransitionContext context
) {
593 public boolean registerService(com
.google
.protobuf
.Service service
) {
598 public CoprocessorServiceResponse
execRegionServerService(RpcController controller
,
599 CoprocessorServiceRequest request
) throws ServiceException
{
604 public UpdateConfigurationResponse
updateConfiguration(
605 RpcController controller
, UpdateConfigurationRequest request
)
606 throws ServiceException
{
611 public ClearRegionBlockCacheResponse
clearRegionBlockCache(RpcController controller
,
612 ClearRegionBlockCacheRequest request
)
613 throws ServiceException
{
618 public HeapMemoryManager
getHeapMemoryManager() {
623 public double getCompactionPressure() {
628 public ThroughputController
getFlushThroughputController() {
633 public double getFlushPressure() {
638 public MetricsRegionServer
getMetrics() {
643 public EntityLock
regionLock(List
<RegionInfo
> regionInfos
, String description
, Abortable abort
)
649 public PrepareBulkLoadResponse
prepareBulkLoad(RpcController controller
,
650 PrepareBulkLoadRequest request
) throws ServiceException
{
655 public CleanupBulkLoadResponse
cleanupBulkLoad(RpcController controller
,
656 CleanupBulkLoadRequest request
) throws ServiceException
{
661 public SecureBulkLoadManager
getSecureBulkLoadManager() {
666 public void unassign(byte[] regionName
) throws IOException
{
670 public RegionServerSpaceQuotaManager
getRegionServerSpaceQuotaManager() {
675 public ExecuteProceduresResponse
executeProcedures(RpcController controller
,
676 ExecuteProceduresRequest request
) throws ServiceException
{
681 public SlowLogResponses
getSlowLogResponses(RpcController controller
,
682 SlowLogResponseRequest request
) throws ServiceException
{
687 public ClearSlowLogResponses
clearSlowLogsResponses(RpcController controller
,
688 ClearSlowLogResponseRequest request
) throws ServiceException
{
693 public GetSpaceQuotaSnapshotsResponse
getSpaceQuotaSnapshots(
694 RpcController controller
, GetSpaceQuotaSnapshotsRequest request
)
695 throws ServiceException
{
700 public Connection
createConnection(Configuration conf
) throws IOException
{
705 public boolean reportRegionSizesForQuotas(RegionSizeStore sizeStore
) {
710 public boolean reportFileArchivalForQuotas(
711 TableName tableName
, Collection
<Entry
<String
, Long
>> archivedFiles
) {
715 public boolean isClusterUp() {
720 public ReplicationSourceService
getReplicationSourceService() {
725 public TableDescriptors
getTableDescriptors() {
730 public Optional
<BlockCache
> getBlockCache() {
731 return Optional
.empty();
735 public Optional
<MobFileCache
> getMobFileCache() {
736 return Optional
.empty();
740 public AccessChecker
getAccessChecker() {
745 public ZKPermissionWatcher
getZKPermissionWatcher() {
750 public AsyncClusterConnection
getAsyncClusterConnection() {