HBASE-23937 : Support Online LargeLogs similar to SlowLogs APIs (#1346)
[hbase.git] / hbase-client / src / main / java / org / apache / hadoop / hbase / client / AdminOverAsyncAdmin.java
blobe3c888c10a57a29681f8c19ac80e4495e994d323
1 /**
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;
27 import java.util.Map;
28 import java.util.Set;
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;
75 /**
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) {
94 this.conn = conn;
95 this.admin = 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
102 @Override
103 public int getOperationTimeout() {
104 return operationTimeout;
107 @Override
108 public int getSyncWaitTimeout() {
109 return syncWaitTimeout;
112 @Override
113 public void abort(String why, Throwable e) {
114 LOG.warn("Aborting becasue of {}", why, e);
115 this.aborted = true;
118 @Override
119 public boolean isAborted() {
120 return aborted;
123 @Override
124 public Connection getConnection() {
125 return conn;
128 @Override
129 public boolean tableExists(TableName tableName) throws IOException {
130 return get(admin.tableExists(tableName));
133 @Override
134 public List<TableDescriptor> listTableDescriptors() throws IOException {
135 return get(admin.listTableDescriptors());
138 @Override
139 public List<TableDescriptor> listTableDescriptors(boolean includeSysTables)
140 throws IOException {
141 return get(admin.listTableDescriptors(includeSysTables));
144 @Override
145 public List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables)
146 throws IOException {
147 return get(admin.listTableDescriptors(pattern, includeSysTables));
150 @Override
151 public TableName[] listTableNames() throws IOException {
152 return get(admin.listTableNames()).toArray(new TableName[0]);
155 @Override
156 public TableName[] listTableNames(Pattern pattern, boolean includeSysTables) throws IOException {
157 return get(admin.listTableNames(pattern, includeSysTables)).toArray(new TableName[0]);
160 @Override
161 public TableDescriptor getDescriptor(TableName tableName)
162 throws TableNotFoundException, IOException {
163 return get(admin.getDescriptor(tableName));
166 @Override
167 public void createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions)
168 throws IOException {
169 get(admin.createTable(desc, startKey, endKey, numRegions));
172 @Override
173 public Future<Void> createTableAsync(TableDescriptor desc) throws IOException {
174 return admin.createTable(desc);
177 @Override
178 public Future<Void> createTableAsync(TableDescriptor desc, byte[][] splitKeys)
179 throws IOException {
180 return admin.createTable(desc, splitKeys);
183 @Override
184 public Future<Void> deleteTableAsync(TableName tableName) throws IOException {
185 return admin.deleteTable(tableName);
188 @Override
189 public Future<Void> truncateTableAsync(TableName tableName, boolean preserveSplits)
190 throws IOException {
191 return admin.truncateTable(tableName, preserveSplits);
194 @Override
195 public Future<Void> enableTableAsync(TableName tableName) throws IOException {
196 return admin.enableTable(tableName);
199 @Override
200 public Future<Void> disableTableAsync(TableName tableName) throws IOException {
201 return admin.disableTable(tableName);
204 @Override
205 public boolean isTableEnabled(TableName tableName) throws IOException {
206 return get(admin.isTableEnabled(tableName));
209 @Override
210 public boolean isTableDisabled(TableName tableName) throws IOException {
211 return get(admin.isTableDisabled(tableName));
214 @Override
215 public boolean isTableAvailable(TableName tableName) throws IOException {
216 return get(admin.isTableAvailable(tableName));
219 @Override
220 public Future<Void> addColumnFamilyAsync(TableName tableName, ColumnFamilyDescriptor columnFamily)
221 throws IOException {
222 return admin.addColumnFamily(tableName, columnFamily);
225 @Override
226 public Future<Void> deleteColumnFamilyAsync(TableName tableName, byte[] columnFamily)
227 throws IOException {
228 return admin.deleteColumnFamily(tableName, columnFamily);
231 @Override
232 public Future<Void> modifyColumnFamilyAsync(TableName tableName,
233 ColumnFamilyDescriptor columnFamily) throws IOException {
234 return admin.modifyColumnFamily(tableName, columnFamily);
237 @Override
238 public List<RegionInfo> getRegions(ServerName serverName) throws IOException {
239 return get(admin.getRegions(serverName));
242 @Override
243 public void flush(TableName tableName) throws IOException {
244 get(admin.flush(tableName));
247 @Override
248 public void flushRegion(byte[] regionName) throws IOException {
249 get(admin.flushRegion(regionName));
252 @Override
253 public void flushRegionServer(ServerName serverName) throws IOException {
254 get(admin.flushRegionServer(serverName));
257 @Override
258 public void compact(TableName tableName) throws IOException {
259 get(admin.compact(tableName));
262 @Override
263 public void compactRegion(byte[] regionName) throws IOException {
264 get(admin.compactRegion(regionName));
267 @Override
268 public void compact(TableName tableName, byte[] columnFamily) throws IOException {
269 get(admin.compact(tableName, columnFamily));
272 @Override
273 public void compactRegion(byte[] regionName, byte[] columnFamily) throws IOException {
274 get(admin.compactRegion(regionName, columnFamily));
277 @Override
278 public void compact(TableName tableName, CompactType compactType)
279 throws IOException, InterruptedException {
280 get(admin.compact(tableName, compactType));
283 @Override
284 public void compact(TableName tableName, byte[] columnFamily, CompactType compactType)
285 throws IOException, InterruptedException {
286 get(admin.compact(tableName, columnFamily, compactType));
289 @Override
290 public void majorCompact(TableName tableName) throws IOException {
291 get(admin.majorCompact(tableName));
294 @Override
295 public void majorCompactRegion(byte[] regionName) throws IOException {
296 get(admin.majorCompactRegion(regionName));
299 @Override
300 public void majorCompact(TableName tableName, byte[] columnFamily) throws IOException {
301 get(admin.majorCompact(tableName, columnFamily));
304 @Override
305 public void majorCompactRegion(byte[] regionName, byte[] columnFamily) throws IOException {
306 get(admin.majorCompactRegion(regionName, columnFamily));
309 @Override
310 public void majorCompact(TableName tableName, CompactType compactType)
311 throws IOException, InterruptedException {
312 get(admin.majorCompact(tableName, compactType));
315 @Override
316 public void majorCompact(TableName tableName, byte[] columnFamily, CompactType compactType)
317 throws IOException, InterruptedException {
318 get(admin.majorCompact(tableName, columnFamily, compactType));
321 @Override
322 public Map<ServerName, Boolean> compactionSwitch(boolean switchState,
323 List<String> serverNamesList) throws IOException {
324 return get(admin.compactionSwitch(switchState, serverNamesList));
327 @Override
328 public void compactRegionServer(ServerName serverName) throws IOException {
329 get(admin.compactRegionServer(serverName));
332 @Override
333 public void majorCompactRegionServer(ServerName serverName) throws IOException {
334 get(admin.majorCompactRegionServer(serverName));
337 @Override
338 public void move(byte[] encodedRegionName) throws IOException {
339 get(admin.move(encodedRegionName));
342 @Override
343 public void move(byte[] encodedRegionName, ServerName destServerName) throws IOException {
344 get(admin.move(encodedRegionName, destServerName));
347 @Override
348 public void assign(byte[] regionName) throws IOException {
349 get(admin.assign(regionName));
352 @Override
353 public void unassign(byte[] regionName, boolean force) throws IOException {
354 get(admin.unassign(regionName, force));
357 @Override
358 public void offline(byte[] regionName) throws IOException {
359 get(admin.offline(regionName));
362 @Override
363 public boolean balancerSwitch(boolean onOrOff, boolean synchronous) throws IOException {
364 return get(admin.balancerSwitch(onOrOff, synchronous));
367 @Override
368 public boolean balance() throws IOException {
369 return get(admin.balance());
372 @Override
373 public boolean balance(boolean force) throws IOException {
374 return get(admin.balance(force));
377 @Override
378 public boolean isBalancerEnabled() throws IOException {
379 return get(admin.isBalancerEnabled());
382 @Override
383 public CacheEvictionStats clearBlockCache(TableName tableName) throws IOException {
384 return get(admin.clearBlockCache(tableName));
387 @Override
388 public boolean normalize() throws IOException {
389 return get(admin.normalize());
392 @Override
393 public boolean isNormalizerEnabled() throws IOException {
394 return get(admin.isNormalizerEnabled());
397 @Override
398 public boolean normalizerSwitch(boolean on) throws IOException {
399 return get(admin.normalizerSwitch(on));
402 @Override
403 public boolean catalogJanitorSwitch(boolean onOrOff) throws IOException {
404 return get(admin.catalogJanitorSwitch(onOrOff));
407 @Override
408 public int runCatalogJanitor() throws IOException {
409 return get(admin.runCatalogJanitor());
412 @Override
413 public boolean isCatalogJanitorEnabled() throws IOException {
414 return get(admin.isCatalogJanitorEnabled());
417 @Override
418 public boolean cleanerChoreSwitch(boolean onOrOff) throws IOException {
419 return get(admin.cleanerChoreSwitch(onOrOff));
422 @Override
423 public boolean runCleanerChore() throws IOException {
424 return get(admin.runCleanerChore());
427 @Override
428 public boolean isCleanerChoreEnabled() throws IOException {
429 return get(admin.isCleanerChoreEnabled());
432 @Override
433 public Future<Void> mergeRegionsAsync(byte[][] nameOfRegionsToMerge, boolean forcible)
434 throws IOException {
435 return admin.mergeRegions(Arrays.asList(nameOfRegionsToMerge), forcible);
438 @Override
439 public void split(TableName tableName) throws IOException {
440 get(admin.split(tableName));
443 @Override
444 public void split(TableName tableName, byte[] splitPoint) throws IOException {
445 get(admin.split(tableName, splitPoint));
448 @Override
449 public Future<Void> splitRegionAsync(byte[] regionName) throws IOException {
450 return admin.splitRegion(regionName);
453 @Override
454 public Future<Void> splitRegionAsync(byte[] regionName, byte[] splitPoint) throws IOException {
455 return admin.splitRegion(regionName, splitPoint);
458 @Override
459 public Future<Void> modifyTableAsync(TableDescriptor td) throws IOException {
460 return admin.modifyTable(td);
463 @Override
464 public void shutdown() throws IOException {
465 get(admin.shutdown());
468 @Override
469 public void stopMaster() throws IOException {
470 get(admin.stopMaster());
473 @Override
474 public boolean isMasterInMaintenanceMode() throws IOException {
475 return get(admin.isMasterInMaintenanceMode());
478 @Override
479 public void stopRegionServer(String hostnamePort) throws IOException {
480 get(admin.stopRegionServer(ServerName.valueOf(hostnamePort, 0)));
483 @Override
484 public ClusterMetrics getClusterMetrics(EnumSet<Option> options) throws IOException {
485 return get(admin.getClusterMetrics(options));
488 @Override
489 public List<RegionMetrics> getRegionMetrics(ServerName serverName) throws IOException {
490 return get(admin.getRegionMetrics(serverName));
493 @Override
494 public List<RegionMetrics> getRegionMetrics(ServerName serverName, TableName tableName)
495 throws IOException {
496 return get(admin.getRegionMetrics(serverName, tableName));
499 @Override
500 public Configuration getConfiguration() {
501 return conn.getConfiguration();
504 @Override
505 public Future<Void> createNamespaceAsync(NamespaceDescriptor descriptor) throws IOException {
506 return admin.createNamespace(descriptor);
509 @Override
510 public Future<Void> modifyNamespaceAsync(NamespaceDescriptor descriptor) throws IOException {
511 return admin.modifyNamespace(descriptor);
514 @Override
515 public Future<Void> deleteNamespaceAsync(String name) throws IOException {
516 return admin.deleteNamespace(name);
519 @Override
520 public NamespaceDescriptor getNamespaceDescriptor(String name)
521 throws NamespaceNotFoundException, IOException {
522 return get(admin.getNamespaceDescriptor(name));
525 @Override
526 public String[] listNamespaces() throws IOException {
527 return get(admin.listNamespaces()).toArray(new String[0]);
530 @Override
531 public NamespaceDescriptor[] listNamespaceDescriptors() throws IOException {
532 return get(admin.listNamespaceDescriptors()).toArray(new NamespaceDescriptor[0]);
535 @Override
536 public List<TableDescriptor> listTableDescriptorsByNamespace(byte[] name) throws IOException {
537 return get(admin.listTableDescriptorsByNamespace(Bytes.toString(name)));
540 @Override
541 public TableName[] listTableNamesByNamespace(String name) throws IOException {
542 return get(admin.listTableNamesByNamespace(name)).toArray(new TableName[0]);
545 @Override
546 public List<RegionInfo> getRegions(TableName tableName) throws IOException {
547 return get(admin.getRegions(tableName));
550 @Override
551 public void close() {
552 // do nothing, AsyncAdmin is not a Closeable.
555 @Override
556 public List<TableDescriptor> listTableDescriptors(List<TableName> tableNames) throws IOException {
557 return get(admin.listTableDescriptors(tableNames));
560 @Override
561 public Future<Boolean> abortProcedureAsync(long procId, boolean mayInterruptIfRunning)
562 throws IOException {
563 return admin.abortProcedure(procId, mayInterruptIfRunning);
566 @Override
567 public String getProcedures() throws IOException {
568 return get(admin.getProcedures());
571 @Override
572 public String getLocks() throws IOException {
573 return get(admin.getLocks());
576 @Override
577 public void rollWALWriter(ServerName serverName) throws IOException, FailedLogCloseException {
578 get(admin.rollWALWriter(serverName));
581 @Override
582 public CompactionState getCompactionState(TableName tableName) throws IOException {
583 return get(admin.getCompactionState(tableName));
586 @Override
587 public CompactionState getCompactionState(TableName tableName, CompactType compactType)
588 throws IOException {
589 return get(admin.getCompactionState(tableName, compactType));
592 @Override
593 public CompactionState getCompactionStateForRegion(byte[] regionName) throws IOException {
594 return get(admin.getCompactionStateForRegion(regionName));
597 @Override
598 public long getLastMajorCompactionTimestamp(TableName tableName) throws IOException {
599 return get(admin.getLastMajorCompactionTimestamp(tableName)).orElse(0L);
602 @Override
603 public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException {
604 return get(admin.getLastMajorCompactionTimestampForRegion(regionName)).orElse(0L);
607 @Override
608 public void snapshot(SnapshotDescription snapshot)
609 throws IOException, SnapshotCreationException, IllegalArgumentException {
610 get(admin.snapshot(snapshot));
613 @Override
614 public Future<Void> snapshotAsync(SnapshotDescription snapshot)
615 throws IOException, SnapshotCreationException {
616 return admin.snapshot(snapshot);
619 @Override
620 public boolean isSnapshotFinished(SnapshotDescription snapshot)
621 throws IOException, HBaseSnapshotException, UnknownSnapshotException {
622 return get(admin.isSnapshotFinished(snapshot));
625 @Override
626 public void restoreSnapshot(String snapshotName) throws IOException, RestoreSnapshotException {
627 get(admin.restoreSnapshot(snapshotName));
630 @Override
631 public void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl)
632 throws IOException, RestoreSnapshotException {
633 get(admin.restoreSnapshot(snapshotName, takeFailSafeSnapshot, restoreAcl));
636 @Override
637 public Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName,
638 boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException {
639 return admin.cloneSnapshot(snapshotName, tableName, restoreAcl);
642 @Override
643 public void execProcedure(String signature, String instance, Map<String, String> props)
644 throws IOException {
645 get(admin.execProcedure(signature, instance, props));
648 @Override
649 public byte[] execProcedureWithReturn(String signature, String instance,
650 Map<String, String> props) throws IOException {
651 return get(admin.execProcedureWithReturn(signature, instance, props));
654 @Override
655 public boolean isProcedureFinished(String signature, String instance, Map<String, String> props)
656 throws IOException {
657 return get(admin.isProcedureFinished(signature, instance, props));
660 @Override
661 public List<SnapshotDescription> listSnapshots() throws IOException {
662 return get(admin.listSnapshots());
665 @Override
666 public List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException {
667 return get(admin.listSnapshots(pattern));
670 @Override
671 public List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern,
672 Pattern snapshotNamePattern) throws IOException {
673 return get(admin.listTableSnapshots(tableNamePattern, snapshotNamePattern));
676 @Override
677 public void deleteSnapshot(String snapshotName) throws IOException {
678 get(admin.deleteSnapshot(snapshotName));
681 @Override
682 public void deleteSnapshots(Pattern pattern) throws IOException {
683 get(admin.deleteSnapshots(pattern));
686 @Override
687 public void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern)
688 throws IOException {
689 get(admin.deleteTableSnapshots(tableNamePattern, snapshotNamePattern));
692 @Override
693 public void setQuota(QuotaSettings quota) throws IOException {
694 get(admin.setQuota(quota));
697 @Override
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;
711 @Override
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);
717 Message ret;
718 try {
719 ret = callback.get();
720 } catch (IOException e) {
721 setCoprocessorError(controller, e);
722 return;
724 if (c.failed()) {
725 setCoprocessorError(controller, c.getFailed());
727 done.run(ret);
730 @Override
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);
736 Message ret;
737 try {
738 ret = done.get();
739 } catch (IOException e) {
740 throw new ServiceException(e);
742 if (c.failed()) {
743 setCoprocessorError(controller, c.getFailed());
744 throw new ServiceException(c.getFailed());
746 return ret;
750 @SuppressWarnings("deprecation")
751 @Override
752 public CoprocessorRpcChannel coprocessorService() {
753 return new SyncCoprocessorRpcChannelOverAsync(
754 new MasterCoprocessorRpcChannelImpl(admin.<Message> newMasterCaller()));
757 @SuppressWarnings("deprecation")
758 @Override
759 public CoprocessorRpcChannel coprocessorService(ServerName serverName) {
760 return new SyncCoprocessorRpcChannelOverAsync(new RegionServerCoprocessorRpcChannelImpl(
761 admin.<Message> newServerCaller().serverName(serverName)));
764 @Override
765 public void updateConfiguration(ServerName server) throws IOException {
766 get(admin.updateConfiguration(server));
769 @Override
770 public void updateConfiguration() throws IOException {
771 get(admin.updateConfiguration());
774 @Override
775 public List<SecurityCapability> getSecurityCapabilities() throws IOException {
776 return get(admin.getSecurityCapabilities());
779 @Override
780 public boolean splitSwitch(boolean enabled, boolean synchronous) throws IOException {
781 return get(admin.splitSwitch(enabled, synchronous));
784 @Override
785 public boolean mergeSwitch(boolean enabled, boolean synchronous) throws IOException {
786 return get(admin.mergeSwitch(enabled, synchronous));
789 @Override
790 public boolean isSplitEnabled() throws IOException {
791 return get(admin.isSplitEnabled());
794 @Override
795 public boolean isMergeEnabled() throws IOException {
796 return get(admin.isMergeEnabled());
799 @Override
800 public Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig,
801 boolean enabled) throws IOException {
802 return admin.addReplicationPeer(peerId, peerConfig, enabled);
805 @Override
806 public Future<Void> removeReplicationPeerAsync(String peerId) throws IOException {
807 return admin.removeReplicationPeer(peerId);
810 @Override
811 public Future<Void> enableReplicationPeerAsync(String peerId) throws IOException {
812 return admin.enableReplicationPeer(peerId);
815 @Override
816 public Future<Void> disableReplicationPeerAsync(String peerId) throws IOException {
817 return admin.disableReplicationPeer(peerId);
820 @Override
821 public ReplicationPeerConfig getReplicationPeerConfig(String peerId) throws IOException {
822 return get(admin.getReplicationPeerConfig(peerId));
825 @Override
826 public Future<Void> updateReplicationPeerConfigAsync(String peerId,
827 ReplicationPeerConfig peerConfig) throws IOException {
828 return admin.updateReplicationPeerConfig(peerId, peerConfig);
831 @Override
832 public List<ReplicationPeerDescription> listReplicationPeers() throws IOException {
833 return get(admin.listReplicationPeers());
836 @Override
837 public List<ReplicationPeerDescription> listReplicationPeers(Pattern pattern) throws IOException {
838 return get(admin.listReplicationPeers(pattern));
841 @Override
842 public Future<Void> transitReplicationPeerSyncReplicationStateAsync(String peerId,
843 SyncReplicationState state) throws IOException {
844 return admin.transitReplicationPeerSyncReplicationState(peerId, state);
847 @Override
848 public void decommissionRegionServers(List<ServerName> servers, boolean offload)
849 throws IOException {
850 get(admin.decommissionRegionServers(servers, offload));
853 @Override
854 public List<ServerName> listDecommissionedRegionServers() throws IOException {
855 return get(admin.listDecommissionedRegionServers());
858 @Override
859 public void recommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames)
860 throws IOException {
861 get(admin.recommissionRegionServer(server, encodedRegionNames));
864 @Override
865 public List<TableCFs> listReplicatedTableCFs() throws IOException {
866 return get(admin.listReplicatedTableCFs());
869 @Override
870 public void enableTableReplication(TableName tableName) throws IOException {
871 get(admin.enableTableReplication(tableName));
874 @Override
875 public void disableTableReplication(TableName tableName) throws IOException {
876 get(admin.disableTableReplication(tableName));
879 @Override
880 public void clearCompactionQueues(ServerName serverName, Set<String> queues)
881 throws IOException, InterruptedException {
882 get(admin.clearCompactionQueues(serverName, queues));
885 @Override
886 public List<ServerName> clearDeadServers(List<ServerName> servers) throws IOException {
887 return get(admin.clearDeadServers(servers));
890 @Override
891 public void cloneTableSchema(TableName tableName, TableName newTableName, boolean preserveSplits)
892 throws IOException {
893 get(admin.cloneTableSchema(tableName, newTableName, preserveSplits));
896 @Override
897 public boolean switchRpcThrottle(boolean enable) throws IOException {
898 return get(admin.switchRpcThrottle(enable));
901 @Override
902 public boolean isRpcThrottleEnabled() throws IOException {
903 return get(admin.isRpcThrottleEnabled());
906 @Override
907 public boolean exceedThrottleQuotaSwitch(boolean enable) throws IOException {
908 return get(admin.exceedThrottleQuotaSwitch(enable));
911 @Override
912 public Map<TableName, Long> getSpaceQuotaTableSizes() throws IOException {
913 return get(admin.getSpaceQuotaTableSizes());
916 @Override
917 public Map<TableName, ? extends SpaceQuotaSnapshotView> getRegionServerSpaceQuotaSnapshots(
918 ServerName serverName) throws IOException {
919 return get(admin.getRegionServerSpaceQuotaSnapshots(serverName));
922 @Override
923 public SpaceQuotaSnapshotView getCurrentSpaceQuotaSnapshot(String namespace) throws IOException {
924 return get(admin.getCurrentSpaceQuotaSnapshot(namespace));
927 @Override
928 public SpaceQuotaSnapshotView getCurrentSpaceQuotaSnapshot(TableName tableName)
929 throws IOException {
930 return get(admin.getCurrentSpaceQuotaSnapshot(tableName));
933 @Override
934 public void grant(UserPermission userPermission, boolean mergeExistingPermissions)
935 throws IOException {
936 get(admin.grant(userPermission, mergeExistingPermissions));
939 @Override
940 public void revoke(UserPermission userPermission) throws IOException {
941 get(admin.revoke(userPermission));
944 @Override
945 public List<UserPermission> getUserPermissions(
946 GetUserPermissionsRequest getUserPermissionsRequest) throws IOException {
947 return get(admin.getUserPermissions(getUserPermissionsRequest));
950 @Override
951 public List<Boolean> hasUserPermissions(String userName, List<Permission> permissions)
952 throws IOException {
953 return get(admin.hasUserPermissions(userName, permissions));
956 @Override
957 public boolean snapshotCleanupSwitch(final boolean on, final boolean synchronous)
958 throws IOException {
959 return get(admin.snapshotCleanupSwitch(on, synchronous));
962 @Override
963 public boolean isSnapshotCleanupEnabled() throws IOException {
964 return get(admin.isSnapshotCleanupEnabled());
967 @Override
968 public List<OnlineLogRecord> getSlowLogResponses(final Set<ServerName> serverNames,
969 final LogQueryFilter logQueryFilter) throws IOException {
970 return get(admin.getSlowLogResponses(serverNames, logQueryFilter));
973 @Override
974 public List<Boolean> clearSlowLogResponses(final Set<ServerName> serverNames)
975 throws IOException {
976 return get(admin.clearSlowLogResponses(serverNames));
979 @Override
980 public RSGroupInfo getRSGroup(String groupName) throws IOException {
981 return get(admin.getRSGroup(groupName));
984 @Override
985 public void moveServersToRSGroup(Set<Address> servers, String groupName) throws IOException {
986 get(admin.moveServersToRSGroup(servers, groupName));
989 @Override
990 public void addRSGroup(String groupName) throws IOException {
991 get(admin.addRSGroup(groupName));
994 @Override
995 public void removeRSGroup(String groupName) throws IOException {
996 get(admin.removeRSGroup(groupName));
999 @Override
1000 public boolean balanceRSGroup(String groupName) throws IOException {
1001 return get(admin.balanceRSGroup(groupName));
1004 @Override
1005 public List<RSGroupInfo> listRSGroups() throws IOException {
1006 return get(admin.listRSGroups());
1009 @Override
1010 public List<TableName> listTablesInRSGroup(String groupName) throws IOException {
1011 return get(admin.listTablesInRSGroup(groupName));
1014 @Override
1015 public Pair<List<String>, List<TableName>>
1016 getConfiguredNamespacesAndTablesInRSGroup(String groupName) throws IOException {
1017 return get(admin.getConfiguredNamespacesAndTablesInRSGroup(groupName));
1020 @Override
1021 public RSGroupInfo getRSGroup(Address hostPort) throws IOException {
1022 return get(admin.getRSGroup(hostPort));
1025 @Override
1026 public void removeServersFromRSGroup(Set<Address> servers) throws IOException {
1027 get(admin.removeServersFromRSGroup(servers));
1030 @Override
1031 public RSGroupInfo getRSGroup(TableName tableName) throws IOException {
1032 return get(admin.getRSGroup(tableName));
1035 @Override
1036 public void setRSGroup(Set<TableName> tables, String groupName) throws IOException {
1037 get(admin.setRSGroup(tables, groupName));