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
.client
;
20 import static org
.apache
.hadoop
.hbase
.client
.ConnectionUtils
.setCoprocessorError
;
21 import static org
.apache
.hadoop
.hbase
.util
.FutureUtils
.get
;
23 import java
.io
.IOException
;
24 import java
.util
.Arrays
;
25 import java
.util
.EnumSet
;
26 import java
.util
.List
;
29 import java
.util
.concurrent
.Future
;
30 import java
.util
.regex
.Pattern
;
31 import org
.apache
.hadoop
.conf
.Configuration
;
32 import org
.apache
.hadoop
.hbase
.CacheEvictionStats
;
33 import org
.apache
.hadoop
.hbase
.ClusterMetrics
;
34 import org
.apache
.hadoop
.hbase
.ClusterMetrics
.Option
;
35 import org
.apache
.hadoop
.hbase
.HConstants
;
36 import org
.apache
.hadoop
.hbase
.NamespaceDescriptor
;
37 import org
.apache
.hadoop
.hbase
.NamespaceNotFoundException
;
38 import org
.apache
.hadoop
.hbase
.RegionMetrics
;
39 import org
.apache
.hadoop
.hbase
.ServerName
;
40 import org
.apache
.hadoop
.hbase
.TableExistsException
;
41 import org
.apache
.hadoop
.hbase
.TableName
;
42 import org
.apache
.hadoop
.hbase
.TableNotFoundException
;
43 import org
.apache
.hadoop
.hbase
.client
.replication
.TableCFs
;
44 import org
.apache
.hadoop
.hbase
.client
.security
.SecurityCapability
;
45 import org
.apache
.hadoop
.hbase
.ipc
.CoprocessorRpcChannel
;
46 import org
.apache
.hadoop
.hbase
.net
.Address
;
47 import org
.apache
.hadoop
.hbase
.quotas
.QuotaFilter
;
48 import org
.apache
.hadoop
.hbase
.quotas
.QuotaSettings
;
49 import org
.apache
.hadoop
.hbase
.quotas
.SpaceQuotaSnapshotView
;
50 import org
.apache
.hadoop
.hbase
.regionserver
.wal
.FailedLogCloseException
;
51 import org
.apache
.hadoop
.hbase
.replication
.ReplicationPeerConfig
;
52 import org
.apache
.hadoop
.hbase
.replication
.ReplicationPeerDescription
;
53 import org
.apache
.hadoop
.hbase
.replication
.SyncReplicationState
;
54 import org
.apache
.hadoop
.hbase
.rsgroup
.RSGroupInfo
;
55 import org
.apache
.hadoop
.hbase
.security
.access
.GetUserPermissionsRequest
;
56 import org
.apache
.hadoop
.hbase
.security
.access
.Permission
;
57 import org
.apache
.hadoop
.hbase
.security
.access
.UserPermission
;
58 import org
.apache
.hadoop
.hbase
.snapshot
.HBaseSnapshotException
;
59 import org
.apache
.hadoop
.hbase
.snapshot
.RestoreSnapshotException
;
60 import org
.apache
.hadoop
.hbase
.snapshot
.SnapshotCreationException
;
61 import org
.apache
.hadoop
.hbase
.snapshot
.UnknownSnapshotException
;
62 import org
.apache
.hadoop
.hbase
.util
.Bytes
;
63 import org
.apache
.hadoop
.hbase
.util
.Pair
;
64 import org
.apache
.yetus
.audience
.InterfaceAudience
;
65 import org
.slf4j
.Logger
;
66 import org
.slf4j
.LoggerFactory
;
68 import org
.apache
.hbase
.thirdparty
.com
.google
.protobuf
.Descriptors
.MethodDescriptor
;
69 import org
.apache
.hbase
.thirdparty
.com
.google
.protobuf
.Message
;
70 import org
.apache
.hbase
.thirdparty
.com
.google
.protobuf
.RpcCallback
;
71 import org
.apache
.hbase
.thirdparty
.com
.google
.protobuf
.RpcChannel
;
72 import org
.apache
.hbase
.thirdparty
.com
.google
.protobuf
.RpcController
;
73 import org
.apache
.hbase
.thirdparty
.com
.google
.protobuf
.ServiceException
;
76 * The {@link Admin} implementation which is based on an {@link AsyncAdmin}.
78 @InterfaceAudience.Private
79 class AdminOverAsyncAdmin
implements Admin
{
81 private static final Logger LOG
= LoggerFactory
.getLogger(AdminOverAsyncAdmin
.class);
83 private volatile boolean aborted
= false;
85 private final Connection conn
;
87 private final RawAsyncHBaseAdmin admin
;
89 private final int operationTimeout
;
91 private final int syncWaitTimeout
;
93 public AdminOverAsyncAdmin(Connection conn
, RawAsyncHBaseAdmin admin
) {
96 this.operationTimeout
= conn
.getConfiguration().getInt(
97 HConstants
.HBASE_CLIENT_OPERATION_TIMEOUT
, HConstants
.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT
);
98 this.syncWaitTimeout
=
99 conn
.getConfiguration().getInt("hbase.client.sync.wait.timeout.msec", 10 * 60000); // 10min
103 public int getOperationTimeout() {
104 return operationTimeout
;
108 public int getSyncWaitTimeout() {
109 return syncWaitTimeout
;
113 public void abort(String why
, Throwable e
) {
114 LOG
.warn("Aborting becasue of {}", why
, e
);
119 public boolean isAborted() {
124 public Connection
getConnection() {
129 public boolean tableExists(TableName tableName
) throws IOException
{
130 return get(admin
.tableExists(tableName
));
134 public List
<TableDescriptor
> listTableDescriptors() throws IOException
{
135 return get(admin
.listTableDescriptors());
139 public List
<TableDescriptor
> listTableDescriptors(boolean includeSysTables
)
141 return get(admin
.listTableDescriptors(includeSysTables
));
145 public List
<TableDescriptor
> listTableDescriptors(Pattern pattern
, boolean includeSysTables
)
147 return get(admin
.listTableDescriptors(pattern
, includeSysTables
));
151 public TableName
[] listTableNames() throws IOException
{
152 return get(admin
.listTableNames()).toArray(new TableName
[0]);
156 public TableName
[] listTableNames(Pattern pattern
, boolean includeSysTables
) throws IOException
{
157 return get(admin
.listTableNames(pattern
, includeSysTables
)).toArray(new TableName
[0]);
161 public TableDescriptor
getDescriptor(TableName tableName
)
162 throws TableNotFoundException
, IOException
{
163 return get(admin
.getDescriptor(tableName
));
167 public void createTable(TableDescriptor desc
, byte[] startKey
, byte[] endKey
, int numRegions
)
169 get(admin
.createTable(desc
, startKey
, endKey
, numRegions
));
173 public Future
<Void
> createTableAsync(TableDescriptor desc
) throws IOException
{
174 return admin
.createTable(desc
);
178 public Future
<Void
> createTableAsync(TableDescriptor desc
, byte[][] splitKeys
)
180 return admin
.createTable(desc
, splitKeys
);
184 public Future
<Void
> deleteTableAsync(TableName tableName
) throws IOException
{
185 return admin
.deleteTable(tableName
);
189 public Future
<Void
> truncateTableAsync(TableName tableName
, boolean preserveSplits
)
191 return admin
.truncateTable(tableName
, preserveSplits
);
195 public Future
<Void
> enableTableAsync(TableName tableName
) throws IOException
{
196 return admin
.enableTable(tableName
);
200 public Future
<Void
> disableTableAsync(TableName tableName
) throws IOException
{
201 return admin
.disableTable(tableName
);
205 public boolean isTableEnabled(TableName tableName
) throws IOException
{
206 return get(admin
.isTableEnabled(tableName
));
210 public boolean isTableDisabled(TableName tableName
) throws IOException
{
211 return get(admin
.isTableDisabled(tableName
));
215 public boolean isTableAvailable(TableName tableName
) throws IOException
{
216 return get(admin
.isTableAvailable(tableName
));
220 public Future
<Void
> addColumnFamilyAsync(TableName tableName
, ColumnFamilyDescriptor columnFamily
)
222 return admin
.addColumnFamily(tableName
, columnFamily
);
226 public Future
<Void
> deleteColumnFamilyAsync(TableName tableName
, byte[] columnFamily
)
228 return admin
.deleteColumnFamily(tableName
, columnFamily
);
232 public Future
<Void
> modifyColumnFamilyAsync(TableName tableName
,
233 ColumnFamilyDescriptor columnFamily
) throws IOException
{
234 return admin
.modifyColumnFamily(tableName
, columnFamily
);
238 public List
<RegionInfo
> getRegions(ServerName serverName
) throws IOException
{
239 return get(admin
.getRegions(serverName
));
243 public void flush(TableName tableName
) throws IOException
{
244 get(admin
.flush(tableName
));
248 public void flushRegion(byte[] regionName
) throws IOException
{
249 get(admin
.flushRegion(regionName
));
253 public void flushRegionServer(ServerName serverName
) throws IOException
{
254 get(admin
.flushRegionServer(serverName
));
258 public void compact(TableName tableName
) throws IOException
{
259 get(admin
.compact(tableName
));
263 public void compactRegion(byte[] regionName
) throws IOException
{
264 get(admin
.compactRegion(regionName
));
268 public void compact(TableName tableName
, byte[] columnFamily
) throws IOException
{
269 get(admin
.compact(tableName
, columnFamily
));
273 public void compactRegion(byte[] regionName
, byte[] columnFamily
) throws IOException
{
274 get(admin
.compactRegion(regionName
, columnFamily
));
278 public void compact(TableName tableName
, CompactType compactType
)
279 throws IOException
, InterruptedException
{
280 get(admin
.compact(tableName
, compactType
));
284 public void compact(TableName tableName
, byte[] columnFamily
, CompactType compactType
)
285 throws IOException
, InterruptedException
{
286 get(admin
.compact(tableName
, columnFamily
, compactType
));
290 public void majorCompact(TableName tableName
) throws IOException
{
291 get(admin
.majorCompact(tableName
));
295 public void majorCompactRegion(byte[] regionName
) throws IOException
{
296 get(admin
.majorCompactRegion(regionName
));
300 public void majorCompact(TableName tableName
, byte[] columnFamily
) throws IOException
{
301 get(admin
.majorCompact(tableName
, columnFamily
));
305 public void majorCompactRegion(byte[] regionName
, byte[] columnFamily
) throws IOException
{
306 get(admin
.majorCompactRegion(regionName
, columnFamily
));
310 public void majorCompact(TableName tableName
, CompactType compactType
)
311 throws IOException
, InterruptedException
{
312 get(admin
.majorCompact(tableName
, compactType
));
316 public void majorCompact(TableName tableName
, byte[] columnFamily
, CompactType compactType
)
317 throws IOException
, InterruptedException
{
318 get(admin
.majorCompact(tableName
, columnFamily
, compactType
));
322 public Map
<ServerName
, Boolean
> compactionSwitch(boolean switchState
,
323 List
<String
> serverNamesList
) throws IOException
{
324 return get(admin
.compactionSwitch(switchState
, serverNamesList
));
328 public void compactRegionServer(ServerName serverName
) throws IOException
{
329 get(admin
.compactRegionServer(serverName
));
333 public void majorCompactRegionServer(ServerName serverName
) throws IOException
{
334 get(admin
.majorCompactRegionServer(serverName
));
338 public void move(byte[] encodedRegionName
) throws IOException
{
339 get(admin
.move(encodedRegionName
));
343 public void move(byte[] encodedRegionName
, ServerName destServerName
) throws IOException
{
344 get(admin
.move(encodedRegionName
, destServerName
));
348 public void assign(byte[] regionName
) throws IOException
{
349 get(admin
.assign(regionName
));
353 public void unassign(byte[] regionName
, boolean force
) throws IOException
{
354 get(admin
.unassign(regionName
, force
));
358 public void offline(byte[] regionName
) throws IOException
{
359 get(admin
.offline(regionName
));
363 public boolean balancerSwitch(boolean onOrOff
, boolean synchronous
) throws IOException
{
364 return get(admin
.balancerSwitch(onOrOff
, synchronous
));
368 public boolean balance() throws IOException
{
369 return get(admin
.balance());
373 public boolean balance(boolean force
) throws IOException
{
374 return get(admin
.balance(force
));
378 public boolean isBalancerEnabled() throws IOException
{
379 return get(admin
.isBalancerEnabled());
383 public CacheEvictionStats
clearBlockCache(TableName tableName
) throws IOException
{
384 return get(admin
.clearBlockCache(tableName
));
388 public boolean normalize() throws IOException
{
389 return get(admin
.normalize());
393 public boolean isNormalizerEnabled() throws IOException
{
394 return get(admin
.isNormalizerEnabled());
398 public boolean normalizerSwitch(boolean on
) throws IOException
{
399 return get(admin
.normalizerSwitch(on
));
403 public boolean catalogJanitorSwitch(boolean onOrOff
) throws IOException
{
404 return get(admin
.catalogJanitorSwitch(onOrOff
));
408 public int runCatalogJanitor() throws IOException
{
409 return get(admin
.runCatalogJanitor());
413 public boolean isCatalogJanitorEnabled() throws IOException
{
414 return get(admin
.isCatalogJanitorEnabled());
418 public boolean cleanerChoreSwitch(boolean onOrOff
) throws IOException
{
419 return get(admin
.cleanerChoreSwitch(onOrOff
));
423 public boolean runCleanerChore() throws IOException
{
424 return get(admin
.runCleanerChore());
428 public boolean isCleanerChoreEnabled() throws IOException
{
429 return get(admin
.isCleanerChoreEnabled());
433 public Future
<Void
> mergeRegionsAsync(byte[][] nameOfRegionsToMerge
, boolean forcible
)
435 return admin
.mergeRegions(Arrays
.asList(nameOfRegionsToMerge
), forcible
);
439 public void split(TableName tableName
) throws IOException
{
440 get(admin
.split(tableName
));
444 public void split(TableName tableName
, byte[] splitPoint
) throws IOException
{
445 get(admin
.split(tableName
, splitPoint
));
449 public Future
<Void
> splitRegionAsync(byte[] regionName
) throws IOException
{
450 return admin
.splitRegion(regionName
);
454 public Future
<Void
> splitRegionAsync(byte[] regionName
, byte[] splitPoint
) throws IOException
{
455 return admin
.splitRegion(regionName
, splitPoint
);
459 public Future
<Void
> modifyTableAsync(TableDescriptor td
) throws IOException
{
460 return admin
.modifyTable(td
);
464 public void shutdown() throws IOException
{
465 get(admin
.shutdown());
469 public void stopMaster() throws IOException
{
470 get(admin
.stopMaster());
474 public boolean isMasterInMaintenanceMode() throws IOException
{
475 return get(admin
.isMasterInMaintenanceMode());
479 public void stopRegionServer(String hostnamePort
) throws IOException
{
480 get(admin
.stopRegionServer(ServerName
.valueOf(hostnamePort
, 0)));
484 public ClusterMetrics
getClusterMetrics(EnumSet
<Option
> options
) throws IOException
{
485 return get(admin
.getClusterMetrics(options
));
489 public List
<RegionMetrics
> getRegionMetrics(ServerName serverName
) throws IOException
{
490 return get(admin
.getRegionMetrics(serverName
));
494 public List
<RegionMetrics
> getRegionMetrics(ServerName serverName
, TableName tableName
)
496 return get(admin
.getRegionMetrics(serverName
, tableName
));
500 public Configuration
getConfiguration() {
501 return conn
.getConfiguration();
505 public Future
<Void
> createNamespaceAsync(NamespaceDescriptor descriptor
) throws IOException
{
506 return admin
.createNamespace(descriptor
);
510 public Future
<Void
> modifyNamespaceAsync(NamespaceDescriptor descriptor
) throws IOException
{
511 return admin
.modifyNamespace(descriptor
);
515 public Future
<Void
> deleteNamespaceAsync(String name
) throws IOException
{
516 return admin
.deleteNamespace(name
);
520 public NamespaceDescriptor
getNamespaceDescriptor(String name
)
521 throws NamespaceNotFoundException
, IOException
{
522 return get(admin
.getNamespaceDescriptor(name
));
526 public String
[] listNamespaces() throws IOException
{
527 return get(admin
.listNamespaces()).toArray(new String
[0]);
531 public NamespaceDescriptor
[] listNamespaceDescriptors() throws IOException
{
532 return get(admin
.listNamespaceDescriptors()).toArray(new NamespaceDescriptor
[0]);
536 public List
<TableDescriptor
> listTableDescriptorsByNamespace(byte[] name
) throws IOException
{
537 return get(admin
.listTableDescriptorsByNamespace(Bytes
.toString(name
)));
541 public TableName
[] listTableNamesByNamespace(String name
) throws IOException
{
542 return get(admin
.listTableNamesByNamespace(name
)).toArray(new TableName
[0]);
546 public List
<RegionInfo
> getRegions(TableName tableName
) throws IOException
{
547 return get(admin
.getRegions(tableName
));
551 public void close() {
552 // do nothing, AsyncAdmin is not a Closeable.
556 public List
<TableDescriptor
> listTableDescriptors(List
<TableName
> tableNames
) throws IOException
{
557 return get(admin
.listTableDescriptors(tableNames
));
561 public Future
<Boolean
> abortProcedureAsync(long procId
, boolean mayInterruptIfRunning
)
563 return admin
.abortProcedure(procId
, mayInterruptIfRunning
);
567 public String
getProcedures() throws IOException
{
568 return get(admin
.getProcedures());
572 public String
getLocks() throws IOException
{
573 return get(admin
.getLocks());
577 public void rollWALWriter(ServerName serverName
) throws IOException
, FailedLogCloseException
{
578 get(admin
.rollWALWriter(serverName
));
582 public CompactionState
getCompactionState(TableName tableName
) throws IOException
{
583 return get(admin
.getCompactionState(tableName
));
587 public CompactionState
getCompactionState(TableName tableName
, CompactType compactType
)
589 return get(admin
.getCompactionState(tableName
, compactType
));
593 public CompactionState
getCompactionStateForRegion(byte[] regionName
) throws IOException
{
594 return get(admin
.getCompactionStateForRegion(regionName
));
598 public long getLastMajorCompactionTimestamp(TableName tableName
) throws IOException
{
599 return get(admin
.getLastMajorCompactionTimestamp(tableName
)).orElse(0L);
603 public long getLastMajorCompactionTimestampForRegion(byte[] regionName
) throws IOException
{
604 return get(admin
.getLastMajorCompactionTimestampForRegion(regionName
)).orElse(0L);
608 public void snapshot(SnapshotDescription snapshot
)
609 throws IOException
, SnapshotCreationException
, IllegalArgumentException
{
610 get(admin
.snapshot(snapshot
));
614 public Future
<Void
> snapshotAsync(SnapshotDescription snapshot
)
615 throws IOException
, SnapshotCreationException
{
616 return admin
.snapshot(snapshot
);
620 public boolean isSnapshotFinished(SnapshotDescription snapshot
)
621 throws IOException
, HBaseSnapshotException
, UnknownSnapshotException
{
622 return get(admin
.isSnapshotFinished(snapshot
));
626 public void restoreSnapshot(String snapshotName
) throws IOException
, RestoreSnapshotException
{
627 get(admin
.restoreSnapshot(snapshotName
));
631 public void restoreSnapshot(String snapshotName
, boolean takeFailSafeSnapshot
, boolean restoreAcl
)
632 throws IOException
, RestoreSnapshotException
{
633 get(admin
.restoreSnapshot(snapshotName
, takeFailSafeSnapshot
, restoreAcl
));
637 public Future
<Void
> cloneSnapshotAsync(String snapshotName
, TableName tableName
,
638 boolean restoreAcl
) throws IOException
, TableExistsException
, RestoreSnapshotException
{
639 return admin
.cloneSnapshot(snapshotName
, tableName
, restoreAcl
);
643 public void execProcedure(String signature
, String instance
, Map
<String
, String
> props
)
645 get(admin
.execProcedure(signature
, instance
, props
));
649 public byte[] execProcedureWithReturn(String signature
, String instance
,
650 Map
<String
, String
> props
) throws IOException
{
651 return get(admin
.execProcedureWithReturn(signature
, instance
, props
));
655 public boolean isProcedureFinished(String signature
, String instance
, Map
<String
, String
> props
)
657 return get(admin
.isProcedureFinished(signature
, instance
, props
));
661 public List
<SnapshotDescription
> listSnapshots() throws IOException
{
662 return get(admin
.listSnapshots());
666 public List
<SnapshotDescription
> listSnapshots(Pattern pattern
) throws IOException
{
667 return get(admin
.listSnapshots(pattern
));
671 public List
<SnapshotDescription
> listTableSnapshots(Pattern tableNamePattern
,
672 Pattern snapshotNamePattern
) throws IOException
{
673 return get(admin
.listTableSnapshots(tableNamePattern
, snapshotNamePattern
));
677 public void deleteSnapshot(String snapshotName
) throws IOException
{
678 get(admin
.deleteSnapshot(snapshotName
));
682 public void deleteSnapshots(Pattern pattern
) throws IOException
{
683 get(admin
.deleteSnapshots(pattern
));
687 public void deleteTableSnapshots(Pattern tableNamePattern
, Pattern snapshotNamePattern
)
689 get(admin
.deleteTableSnapshots(tableNamePattern
, snapshotNamePattern
));
693 public void setQuota(QuotaSettings quota
) throws IOException
{
694 get(admin
.setQuota(quota
));
698 public List
<QuotaSettings
> getQuota(QuotaFilter filter
) throws IOException
{
699 return get(admin
.getQuota(filter
));
702 @SuppressWarnings("deprecation")
703 private static final class SyncCoprocessorRpcChannelOverAsync
implements CoprocessorRpcChannel
{
705 private final RpcChannel delegate
;
707 public SyncCoprocessorRpcChannelOverAsync(RpcChannel delegate
) {
708 this.delegate
= delegate
;
712 public void callMethod(MethodDescriptor method
, RpcController controller
, Message request
,
713 Message responsePrototype
, RpcCallback
<Message
> done
) {
714 ClientCoprocessorRpcController c
= new ClientCoprocessorRpcController();
715 CoprocessorBlockingRpcCallback
<Message
> callback
= new CoprocessorBlockingRpcCallback
<>();
716 delegate
.callMethod(method
, c
, request
, responsePrototype
, callback
);
719 ret
= callback
.get();
720 } catch (IOException e
) {
721 setCoprocessorError(controller
, e
);
725 setCoprocessorError(controller
, c
.getFailed());
731 public Message
callBlockingMethod(MethodDescriptor method
, RpcController controller
,
732 Message request
, Message responsePrototype
) throws ServiceException
{
733 ClientCoprocessorRpcController c
= new ClientCoprocessorRpcController();
734 CoprocessorBlockingRpcCallback
<Message
> done
= new CoprocessorBlockingRpcCallback
<>();
735 callMethod(method
, c
, request
, responsePrototype
, done
);
739 } catch (IOException e
) {
740 throw new ServiceException(e
);
743 setCoprocessorError(controller
, c
.getFailed());
744 throw new ServiceException(c
.getFailed());
750 @SuppressWarnings("deprecation")
752 public CoprocessorRpcChannel
coprocessorService() {
753 return new SyncCoprocessorRpcChannelOverAsync(
754 new MasterCoprocessorRpcChannelImpl(admin
.<Message
> newMasterCaller()));
757 @SuppressWarnings("deprecation")
759 public CoprocessorRpcChannel
coprocessorService(ServerName serverName
) {
760 return new SyncCoprocessorRpcChannelOverAsync(new RegionServerCoprocessorRpcChannelImpl(
761 admin
.<Message
> newServerCaller().serverName(serverName
)));
765 public void updateConfiguration(ServerName server
) throws IOException
{
766 get(admin
.updateConfiguration(server
));
770 public void updateConfiguration() throws IOException
{
771 get(admin
.updateConfiguration());
775 public List
<SecurityCapability
> getSecurityCapabilities() throws IOException
{
776 return get(admin
.getSecurityCapabilities());
780 public boolean splitSwitch(boolean enabled
, boolean synchronous
) throws IOException
{
781 return get(admin
.splitSwitch(enabled
, synchronous
));
785 public boolean mergeSwitch(boolean enabled
, boolean synchronous
) throws IOException
{
786 return get(admin
.mergeSwitch(enabled
, synchronous
));
790 public boolean isSplitEnabled() throws IOException
{
791 return get(admin
.isSplitEnabled());
795 public boolean isMergeEnabled() throws IOException
{
796 return get(admin
.isMergeEnabled());
800 public Future
<Void
> addReplicationPeerAsync(String peerId
, ReplicationPeerConfig peerConfig
,
801 boolean enabled
) throws IOException
{
802 return admin
.addReplicationPeer(peerId
, peerConfig
, enabled
);
806 public Future
<Void
> removeReplicationPeerAsync(String peerId
) throws IOException
{
807 return admin
.removeReplicationPeer(peerId
);
811 public Future
<Void
> enableReplicationPeerAsync(String peerId
) throws IOException
{
812 return admin
.enableReplicationPeer(peerId
);
816 public Future
<Void
> disableReplicationPeerAsync(String peerId
) throws IOException
{
817 return admin
.disableReplicationPeer(peerId
);
821 public ReplicationPeerConfig
getReplicationPeerConfig(String peerId
) throws IOException
{
822 return get(admin
.getReplicationPeerConfig(peerId
));
826 public Future
<Void
> updateReplicationPeerConfigAsync(String peerId
,
827 ReplicationPeerConfig peerConfig
) throws IOException
{
828 return admin
.updateReplicationPeerConfig(peerId
, peerConfig
);
832 public List
<ReplicationPeerDescription
> listReplicationPeers() throws IOException
{
833 return get(admin
.listReplicationPeers());
837 public List
<ReplicationPeerDescription
> listReplicationPeers(Pattern pattern
) throws IOException
{
838 return get(admin
.listReplicationPeers(pattern
));
842 public Future
<Void
> transitReplicationPeerSyncReplicationStateAsync(String peerId
,
843 SyncReplicationState state
) throws IOException
{
844 return admin
.transitReplicationPeerSyncReplicationState(peerId
, state
);
848 public void decommissionRegionServers(List
<ServerName
> servers
, boolean offload
)
850 get(admin
.decommissionRegionServers(servers
, offload
));
854 public List
<ServerName
> listDecommissionedRegionServers() throws IOException
{
855 return get(admin
.listDecommissionedRegionServers());
859 public void recommissionRegionServer(ServerName server
, List
<byte[]> encodedRegionNames
)
861 get(admin
.recommissionRegionServer(server
, encodedRegionNames
));
865 public List
<TableCFs
> listReplicatedTableCFs() throws IOException
{
866 return get(admin
.listReplicatedTableCFs());
870 public void enableTableReplication(TableName tableName
) throws IOException
{
871 get(admin
.enableTableReplication(tableName
));
875 public void disableTableReplication(TableName tableName
) throws IOException
{
876 get(admin
.disableTableReplication(tableName
));
880 public void clearCompactionQueues(ServerName serverName
, Set
<String
> queues
)
881 throws IOException
, InterruptedException
{
882 get(admin
.clearCompactionQueues(serverName
, queues
));
886 public List
<ServerName
> clearDeadServers(List
<ServerName
> servers
) throws IOException
{
887 return get(admin
.clearDeadServers(servers
));
891 public void cloneTableSchema(TableName tableName
, TableName newTableName
, boolean preserveSplits
)
893 get(admin
.cloneTableSchema(tableName
, newTableName
, preserveSplits
));
897 public boolean switchRpcThrottle(boolean enable
) throws IOException
{
898 return get(admin
.switchRpcThrottle(enable
));
902 public boolean isRpcThrottleEnabled() throws IOException
{
903 return get(admin
.isRpcThrottleEnabled());
907 public boolean exceedThrottleQuotaSwitch(boolean enable
) throws IOException
{
908 return get(admin
.exceedThrottleQuotaSwitch(enable
));
912 public Map
<TableName
, Long
> getSpaceQuotaTableSizes() throws IOException
{
913 return get(admin
.getSpaceQuotaTableSizes());
917 public Map
<TableName
, ?
extends SpaceQuotaSnapshotView
> getRegionServerSpaceQuotaSnapshots(
918 ServerName serverName
) throws IOException
{
919 return get(admin
.getRegionServerSpaceQuotaSnapshots(serverName
));
923 public SpaceQuotaSnapshotView
getCurrentSpaceQuotaSnapshot(String namespace
) throws IOException
{
924 return get(admin
.getCurrentSpaceQuotaSnapshot(namespace
));
928 public SpaceQuotaSnapshotView
getCurrentSpaceQuotaSnapshot(TableName tableName
)
930 return get(admin
.getCurrentSpaceQuotaSnapshot(tableName
));
934 public void grant(UserPermission userPermission
, boolean mergeExistingPermissions
)
936 get(admin
.grant(userPermission
, mergeExistingPermissions
));
940 public void revoke(UserPermission userPermission
) throws IOException
{
941 get(admin
.revoke(userPermission
));
945 public List
<UserPermission
> getUserPermissions(
946 GetUserPermissionsRequest getUserPermissionsRequest
) throws IOException
{
947 return get(admin
.getUserPermissions(getUserPermissionsRequest
));
951 public List
<Boolean
> hasUserPermissions(String userName
, List
<Permission
> permissions
)
953 return get(admin
.hasUserPermissions(userName
, permissions
));
957 public boolean snapshotCleanupSwitch(final boolean on
, final boolean synchronous
)
959 return get(admin
.snapshotCleanupSwitch(on
, synchronous
));
963 public boolean isSnapshotCleanupEnabled() throws IOException
{
964 return get(admin
.isSnapshotCleanupEnabled());
968 public List
<OnlineLogRecord
> getSlowLogResponses(final Set
<ServerName
> serverNames
,
969 final LogQueryFilter logQueryFilter
) throws IOException
{
970 return get(admin
.getSlowLogResponses(serverNames
, logQueryFilter
));
974 public List
<Boolean
> clearSlowLogResponses(final Set
<ServerName
> serverNames
)
976 return get(admin
.clearSlowLogResponses(serverNames
));
980 public RSGroupInfo
getRSGroup(String groupName
) throws IOException
{
981 return get(admin
.getRSGroup(groupName
));
985 public void moveServersToRSGroup(Set
<Address
> servers
, String groupName
) throws IOException
{
986 get(admin
.moveServersToRSGroup(servers
, groupName
));
990 public void addRSGroup(String groupName
) throws IOException
{
991 get(admin
.addRSGroup(groupName
));
995 public void removeRSGroup(String groupName
) throws IOException
{
996 get(admin
.removeRSGroup(groupName
));
1000 public boolean balanceRSGroup(String groupName
) throws IOException
{
1001 return get(admin
.balanceRSGroup(groupName
));
1005 public List
<RSGroupInfo
> listRSGroups() throws IOException
{
1006 return get(admin
.listRSGroups());
1010 public List
<TableName
> listTablesInRSGroup(String groupName
) throws IOException
{
1011 return get(admin
.listTablesInRSGroup(groupName
));
1015 public Pair
<List
<String
>, List
<TableName
>>
1016 getConfiguredNamespacesAndTablesInRSGroup(String groupName
) throws IOException
{
1017 return get(admin
.getConfiguredNamespacesAndTablesInRSGroup(groupName
));
1021 public RSGroupInfo
getRSGroup(Address hostPort
) throws IOException
{
1022 return get(admin
.getRSGroup(hostPort
));
1026 public void removeServersFromRSGroup(Set
<Address
> servers
) throws IOException
{
1027 get(admin
.removeServersFromRSGroup(servers
));
1031 public RSGroupInfo
getRSGroup(TableName tableName
) throws IOException
{
1032 return get(admin
.getRSGroup(tableName
));
1036 public void setRSGroup(Set
<TableName
> tables
, String groupName
) throws IOException
{
1037 get(admin
.setRSGroup(tables
, groupName
));