001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.client;
019
020import com.google.protobuf.RpcChannel;
021import java.util.EnumSet;
022import java.util.List;
023import java.util.Map;
024import java.util.Optional;
025import java.util.Set;
026import java.util.concurrent.CompletableFuture;
027import java.util.concurrent.ExecutorService;
028import java.util.function.Function;
029import java.util.regex.Pattern;
030import org.apache.hadoop.hbase.CacheEvictionStats;
031import org.apache.hadoop.hbase.ClusterMetrics;
032import org.apache.hadoop.hbase.ClusterMetrics.Option;
033import org.apache.hadoop.hbase.NamespaceDescriptor;
034import org.apache.hadoop.hbase.RegionMetrics;
035import org.apache.hadoop.hbase.ServerName;
036import org.apache.hadoop.hbase.TableName;
037import org.apache.hadoop.hbase.client.replication.TableCFs;
038import org.apache.hadoop.hbase.client.security.SecurityCapability;
039import org.apache.hadoop.hbase.quotas.QuotaFilter;
040import org.apache.hadoop.hbase.quotas.QuotaSettings;
041import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
042import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
043import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
044import org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest;
045import org.apache.hadoop.hbase.security.access.Permission;
046import org.apache.hadoop.hbase.security.access.UserPermission;
047import org.apache.hadoop.hbase.util.FutureUtils;
048import org.apache.yetus.audience.InterfaceAudience;
049
050/**
051 * Just a wrapper of {@link RawAsyncHBaseAdmin}. The difference is that users need to provide a
052 * thread pool when constructing this class, and the callback methods registered to the returned
053 * {@link CompletableFuture} will be executed in this thread pool. So usually it is safe for users
054 * to do anything they want in the callbacks without breaking the rpc framework.
055 * @since 2.0.0
056 * @see RawAsyncHBaseAdmin
057 * @see AsyncConnection#getAdmin(ExecutorService)
058 * @see AsyncConnection#getAdminBuilder(ExecutorService)
059 */
060@InterfaceAudience.Private
061class AsyncHBaseAdmin implements AsyncAdmin {
062
063  private final RawAsyncHBaseAdmin rawAdmin;
064
065  private final ExecutorService pool;
066
067  AsyncHBaseAdmin(RawAsyncHBaseAdmin rawAdmin, ExecutorService pool) {
068    this.rawAdmin = rawAdmin;
069    this.pool = pool;
070  }
071
072  private <T> CompletableFuture<T> wrap(CompletableFuture<T> future) {
073    return FutureUtils.wrapFuture(future, pool);
074  }
075
076  @Override
077  public CompletableFuture<Boolean> tableExists(TableName tableName) {
078    return wrap(rawAdmin.tableExists(tableName));
079  }
080
081  @Override
082  public CompletableFuture<List<TableDescriptor>> listTableDescriptors(boolean includeSysTables) {
083    return wrap(rawAdmin.listTableDescriptors(includeSysTables));
084  }
085
086  @Override
087  public CompletableFuture<List<TableDescriptor>> listTableDescriptors(Pattern pattern,
088    boolean includeSysTables) {
089    return wrap(rawAdmin.listTableDescriptors(pattern, includeSysTables));
090  }
091
092  @Override
093  public CompletableFuture<List<TableDescriptor>> listTableDescriptors(List<TableName> tableNames) {
094    return wrap(rawAdmin.listTableDescriptors(tableNames));
095  }
096
097  @Override
098  public CompletableFuture<List<TableDescriptor>> listTableDescriptorsByNamespace(String name) {
099    return wrap(rawAdmin.listTableDescriptorsByNamespace(name));
100  }
101
102  @Override
103  public CompletableFuture<List<TableName>> listTableNames(boolean includeSysTables) {
104    return wrap(rawAdmin.listTableNames(includeSysTables));
105  }
106
107  @Override
108  public CompletableFuture<List<TableName>> listTableNames(Pattern pattern,
109    boolean includeSysTables) {
110    return wrap(rawAdmin.listTableNames(pattern, includeSysTables));
111  }
112
113  @Override
114  public CompletableFuture<List<TableName>> listTableNamesByNamespace(String name) {
115    return wrap(rawAdmin.listTableNamesByNamespace(name));
116  }
117
118  @Override
119  public CompletableFuture<TableDescriptor> getDescriptor(TableName tableName) {
120    return wrap(rawAdmin.getDescriptor(tableName));
121  }
122
123  @Override
124  public CompletableFuture<Void> createTable(TableDescriptor desc) {
125    return wrap(rawAdmin.createTable(desc));
126  }
127
128  @Override
129  public CompletableFuture<Void> createTable(TableDescriptor desc, byte[] startKey, byte[] endKey,
130    int numRegions) {
131    return wrap(rawAdmin.createTable(desc, startKey, endKey, numRegions));
132  }
133
134  @Override
135  public CompletableFuture<Void> createTable(TableDescriptor desc, byte[][] splitKeys) {
136    return wrap(rawAdmin.createTable(desc, splitKeys));
137  }
138
139  @Override
140  public CompletableFuture<Void> modifyTable(TableDescriptor desc) {
141    return wrap(rawAdmin.modifyTable(desc));
142  }
143
144  @Override
145  public CompletableFuture<Void> modifyTableStoreFileTracker(TableName tableName, String dstSFT) {
146    return wrap(rawAdmin.modifyTableStoreFileTracker(tableName, dstSFT));
147  }
148
149  @Override
150  public CompletableFuture<Void> deleteTable(TableName tableName) {
151    return wrap(rawAdmin.deleteTable(tableName));
152  }
153
154  @Override
155  public CompletableFuture<Void> truncateTable(TableName tableName, boolean preserveSplits) {
156    return wrap(rawAdmin.truncateTable(tableName, preserveSplits));
157  }
158
159  @Override
160  public CompletableFuture<Void> enableTable(TableName tableName) {
161    return wrap(rawAdmin.enableTable(tableName));
162  }
163
164  @Override
165  public CompletableFuture<Void> disableTable(TableName tableName) {
166    return wrap(rawAdmin.disableTable(tableName));
167  }
168
169  @Override
170  public CompletableFuture<Boolean> isTableEnabled(TableName tableName) {
171    return wrap(rawAdmin.isTableEnabled(tableName));
172  }
173
174  @Override
175  public CompletableFuture<Boolean> isTableDisabled(TableName tableName) {
176    return wrap(rawAdmin.isTableDisabled(tableName));
177  }
178
179  @Override
180  public CompletableFuture<Boolean> isTableAvailable(TableName tableName) {
181    return wrap(rawAdmin.isTableAvailable(tableName));
182  }
183
184  @Override
185  public CompletableFuture<Boolean> isTableAvailable(TableName tableName, byte[][] splitKeys) {
186    return wrap(rawAdmin.isTableAvailable(tableName, splitKeys));
187  }
188
189  @Override
190  public CompletableFuture<Void> addColumnFamily(TableName tableName,
191    ColumnFamilyDescriptor columnFamily) {
192    return wrap(rawAdmin.addColumnFamily(tableName, columnFamily));
193  }
194
195  @Override
196  public CompletableFuture<Void> deleteColumnFamily(TableName tableName, byte[] columnFamily) {
197    return wrap(rawAdmin.deleteColumnFamily(tableName, columnFamily));
198  }
199
200  @Override
201  public CompletableFuture<Void> modifyColumnFamily(TableName tableName,
202    ColumnFamilyDescriptor columnFamily) {
203    return wrap(rawAdmin.modifyColumnFamily(tableName, columnFamily));
204  }
205
206  @Override
207  public CompletableFuture<Void> modifyColumnFamilyStoreFileTracker(TableName tableName,
208    byte[] family, String dstSFT) {
209    return wrap(rawAdmin.modifyColumnFamilyStoreFileTracker(tableName, family, dstSFT));
210  }
211
212  @Override
213  public CompletableFuture<Void> createNamespace(NamespaceDescriptor descriptor) {
214    return wrap(rawAdmin.createNamespace(descriptor));
215  }
216
217  @Override
218  public CompletableFuture<Void> modifyNamespace(NamespaceDescriptor descriptor) {
219    return wrap(rawAdmin.modifyNamespace(descriptor));
220  }
221
222  @Override
223  public CompletableFuture<Void> deleteNamespace(String name) {
224    return wrap(rawAdmin.deleteNamespace(name));
225  }
226
227  @Override
228  public CompletableFuture<NamespaceDescriptor> getNamespaceDescriptor(String name) {
229    return wrap(rawAdmin.getNamespaceDescriptor(name));
230  }
231
232  @Override
233  public CompletableFuture<List<String>> listNamespaces() {
234    return wrap(rawAdmin.listNamespaces());
235  }
236
237  @Override
238  public CompletableFuture<List<NamespaceDescriptor>> listNamespaceDescriptors() {
239    return wrap(rawAdmin.listNamespaceDescriptors());
240  }
241
242  @Override
243  public CompletableFuture<List<RegionInfo>> getRegions(ServerName serverName) {
244    return wrap(rawAdmin.getRegions(serverName));
245  }
246
247  @Override
248  public CompletableFuture<List<RegionInfo>> getRegions(TableName tableName) {
249    return wrap(rawAdmin.getRegions(tableName));
250  }
251
252  @Override
253  public CompletableFuture<Void> flush(TableName tableName) {
254    return wrap(rawAdmin.flush(tableName));
255  }
256
257  @Override
258  public CompletableFuture<Void> flush(TableName tableName, byte[] columnFamily) {
259    return wrap(rawAdmin.flush(tableName, columnFamily));
260  }
261
262  @Override
263  public CompletableFuture<Void> flushRegion(byte[] regionName) {
264    return wrap(rawAdmin.flushRegion(regionName));
265  }
266
267  @Override
268  public CompletableFuture<Void> flushRegion(byte[] regionName, byte[] columnFamily) {
269    return wrap(rawAdmin.flushRegion(regionName, columnFamily));
270  }
271
272  @Override
273  public CompletableFuture<Void> flushRegionServer(ServerName sn) {
274    return wrap(rawAdmin.flushRegionServer(sn));
275  }
276
277  @Override
278  public CompletableFuture<Void> compact(TableName tableName, CompactType compactType) {
279    return wrap(rawAdmin.compact(tableName, compactType));
280  }
281
282  @Override
283  public CompletableFuture<Void> compact(TableName tableName, byte[] columnFamily,
284    CompactType compactType) {
285    return wrap(rawAdmin.compact(tableName, columnFamily, compactType));
286  }
287
288  @Override
289  public CompletableFuture<Void> compactRegion(byte[] regionName) {
290    return wrap(rawAdmin.compactRegion(regionName));
291  }
292
293  @Override
294  public CompletableFuture<Void> compactRegion(byte[] regionName, byte[] columnFamily) {
295    return wrap(rawAdmin.compactRegion(regionName, columnFamily));
296  }
297
298  @Override
299  public CompletableFuture<Void> majorCompact(TableName tableName, CompactType compactType) {
300    return wrap(rawAdmin.majorCompact(tableName, compactType));
301  }
302
303  @Override
304  public CompletableFuture<Void> majorCompact(TableName tableName, byte[] columnFamily,
305    CompactType compactType) {
306    return wrap(rawAdmin.majorCompact(tableName, columnFamily, compactType));
307  }
308
309  @Override
310  public CompletableFuture<Void> majorCompactRegion(byte[] regionName) {
311    return wrap(rawAdmin.majorCompactRegion(regionName));
312  }
313
314  @Override
315  public CompletableFuture<Void> majorCompactRegion(byte[] regionName, byte[] columnFamily) {
316    return wrap(rawAdmin.majorCompactRegion(regionName, columnFamily));
317  }
318
319  @Override
320  public CompletableFuture<Void> compactRegionServer(ServerName serverName) {
321    return wrap(rawAdmin.compactRegionServer(serverName));
322  }
323
324  @Override
325  public CompletableFuture<Void> majorCompactRegionServer(ServerName serverName) {
326    return wrap(rawAdmin.majorCompactRegionServer(serverName));
327  }
328
329  @Override
330  public CompletableFuture<Boolean> mergeSwitch(boolean enabled, boolean drainMerges) {
331    return wrap(rawAdmin.mergeSwitch(enabled, drainMerges));
332  }
333
334  @Override
335  public CompletableFuture<Boolean> isMergeEnabled() {
336    return wrap(rawAdmin.isMergeEnabled());
337  }
338
339  @Override
340  public CompletableFuture<Boolean> splitSwitch(boolean enabled, boolean drainSplits) {
341    return wrap(rawAdmin.splitSwitch(enabled, drainSplits));
342  }
343
344  @Override
345  public CompletableFuture<Boolean> isSplitEnabled() {
346    return wrap(rawAdmin.isSplitEnabled());
347  }
348
349  @Override
350  public CompletableFuture<Void> mergeRegions(List<byte[]> nameOfRegionsToMerge, boolean forcible) {
351    return wrap(rawAdmin.mergeRegions(nameOfRegionsToMerge, forcible));
352  }
353
354  @Override
355  public CompletableFuture<Void> split(TableName tableName) {
356    return wrap(rawAdmin.split(tableName));
357  }
358
359  @Override
360  public CompletableFuture<Void> split(TableName tableName, byte[] splitPoint) {
361    return wrap(rawAdmin.split(tableName, splitPoint));
362  }
363
364  @Override
365  public CompletableFuture<Void> splitRegion(byte[] regionName) {
366    return wrap(rawAdmin.splitRegion(regionName));
367  }
368
369  @Override
370  public CompletableFuture<Void> splitRegion(byte[] regionName, byte[] splitPoint) {
371    return wrap(rawAdmin.splitRegion(regionName, splitPoint));
372  }
373
374  @Override
375  public CompletableFuture<Void> assign(byte[] regionName) {
376    return wrap(rawAdmin.assign(regionName));
377  }
378
379  @Override
380  public CompletableFuture<Void> unassign(byte[] regionName) {
381    return wrap(rawAdmin.unassign(regionName));
382  }
383
384  @Override
385  public CompletableFuture<Void> offline(byte[] regionName) {
386    return wrap(rawAdmin.offline(regionName));
387  }
388
389  @Override
390  public CompletableFuture<Void> move(byte[] regionName) {
391    return wrap(rawAdmin.move(regionName));
392  }
393
394  @Override
395  public CompletableFuture<Void> move(byte[] regionName, ServerName destServerName) {
396    return wrap(rawAdmin.move(regionName, destServerName));
397  }
398
399  @Override
400  public CompletableFuture<Void> setQuota(QuotaSettings quota) {
401    return wrap(rawAdmin.setQuota(quota));
402  }
403
404  @Override
405  public CompletableFuture<List<QuotaSettings>> getQuota(QuotaFilter filter) {
406    return wrap(rawAdmin.getQuota(filter));
407  }
408
409  @Override
410  public CompletableFuture<Void> addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig,
411    boolean enabled) {
412    return wrap(rawAdmin.addReplicationPeer(peerId, peerConfig, enabled));
413  }
414
415  @Override
416  public CompletableFuture<Void> removeReplicationPeer(String peerId) {
417    return wrap(rawAdmin.removeReplicationPeer(peerId));
418  }
419
420  @Override
421  public CompletableFuture<Void> enableReplicationPeer(String peerId) {
422    return wrap(rawAdmin.enableReplicationPeer(peerId));
423  }
424
425  @Override
426  public CompletableFuture<Void> disableReplicationPeer(String peerId) {
427    return wrap(rawAdmin.disableReplicationPeer(peerId));
428  }
429
430  @Override
431  public CompletableFuture<ReplicationPeerConfig> getReplicationPeerConfig(String peerId) {
432    return wrap(rawAdmin.getReplicationPeerConfig(peerId));
433  }
434
435  @Override
436  public CompletableFuture<Void> updateReplicationPeerConfig(String peerId,
437    ReplicationPeerConfig peerConfig) {
438    return wrap(rawAdmin.updateReplicationPeerConfig(peerId, peerConfig));
439  }
440
441  @Override
442  public CompletableFuture<Void> appendReplicationPeerTableCFs(String peerId,
443    Map<TableName, List<String>> tableCfs) {
444    return wrap(rawAdmin.appendReplicationPeerTableCFs(peerId, tableCfs));
445  }
446
447  @Override
448  public CompletableFuture<Void> removeReplicationPeerTableCFs(String peerId,
449    Map<TableName, List<String>> tableCfs) {
450    return wrap(rawAdmin.removeReplicationPeerTableCFs(peerId, tableCfs));
451  }
452
453  @Override
454  public CompletableFuture<List<ReplicationPeerDescription>> listReplicationPeers() {
455    return wrap(rawAdmin.listReplicationPeers());
456  }
457
458  @Override
459  public CompletableFuture<List<ReplicationPeerDescription>> listReplicationPeers(Pattern pattern) {
460    return wrap(rawAdmin.listReplicationPeers(pattern));
461  }
462
463  @Override
464  public CompletableFuture<List<TableCFs>> listReplicatedTableCFs() {
465    return wrap(rawAdmin.listReplicatedTableCFs());
466  }
467
468  @Override
469  public CompletableFuture<Void> enableTableReplication(TableName tableName) {
470    return wrap(rawAdmin.enableTableReplication(tableName));
471  }
472
473  @Override
474  public CompletableFuture<Void> disableTableReplication(TableName tableName) {
475    return wrap(rawAdmin.disableTableReplication(tableName));
476  }
477
478  @Override
479  public CompletableFuture<Void> snapshot(SnapshotDescription snapshot) {
480    return wrap(rawAdmin.snapshot(snapshot));
481  }
482
483  @Override
484  public CompletableFuture<Boolean> isSnapshotFinished(SnapshotDescription snapshot) {
485    return wrap(rawAdmin.isSnapshotFinished(snapshot));
486  }
487
488  @Override
489  public CompletableFuture<Void> restoreSnapshot(String snapshotName) {
490    return wrap(rawAdmin.restoreSnapshot(snapshotName));
491  }
492
493  @Override
494  public CompletableFuture<Void> restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot,
495    boolean restoreAcl) {
496    return wrap(rawAdmin.restoreSnapshot(snapshotName, takeFailSafeSnapshot, restoreAcl));
497  }
498
499  @Override
500  public CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName,
501    boolean restoreAcl, String customSFT) {
502    return wrap(rawAdmin.cloneSnapshot(snapshotName, tableName, restoreAcl, customSFT));
503  }
504
505  @Override
506  public CompletableFuture<List<SnapshotDescription>> listSnapshots() {
507    return wrap(rawAdmin.listSnapshots());
508  }
509
510  @Override
511  public CompletableFuture<List<SnapshotDescription>> listSnapshots(Pattern pattern) {
512    return wrap(rawAdmin.listSnapshots(pattern));
513  }
514
515  @Override
516  public CompletableFuture<List<SnapshotDescription>> listTableSnapshots(Pattern tableNamePattern) {
517    return wrap(rawAdmin.listTableSnapshots(tableNamePattern));
518  }
519
520  @Override
521  public CompletableFuture<List<SnapshotDescription>> listTableSnapshots(Pattern tableNamePattern,
522    Pattern snapshotNamePattern) {
523    return wrap(rawAdmin.listTableSnapshots(tableNamePattern, snapshotNamePattern));
524  }
525
526  @Override
527  public CompletableFuture<Void> deleteSnapshot(String snapshotName) {
528    return wrap(rawAdmin.deleteSnapshot(snapshotName));
529  }
530
531  @Override
532  public CompletableFuture<Void> deleteSnapshots() {
533    return wrap(rawAdmin.deleteSnapshots());
534  }
535
536  @Override
537  public CompletableFuture<Void> deleteSnapshots(Pattern pattern) {
538    return wrap(rawAdmin.deleteSnapshots(pattern));
539  }
540
541  @Override
542  public CompletableFuture<Void> deleteTableSnapshots(Pattern tableNamePattern) {
543    return wrap(rawAdmin.deleteTableSnapshots(tableNamePattern));
544  }
545
546  @Override
547  public CompletableFuture<Void> deleteTableSnapshots(Pattern tableNamePattern,
548    Pattern snapshotNamePattern) {
549    return wrap(rawAdmin.deleteTableSnapshots(tableNamePattern, snapshotNamePattern));
550  }
551
552  @Override
553  public CompletableFuture<Void> execProcedure(String signature, String instance,
554    Map<String, String> props) {
555    return wrap(rawAdmin.execProcedure(signature, instance, props));
556  }
557
558  @Override
559  public CompletableFuture<byte[]> execProcedureWithReturn(String signature, String instance,
560    Map<String, String> props) {
561    return wrap(rawAdmin.execProcedureWithReturn(signature, instance, props));
562  }
563
564  @Override
565  public CompletableFuture<Boolean> isProcedureFinished(String signature, String instance,
566    Map<String, String> props) {
567    return wrap(rawAdmin.isProcedureFinished(signature, instance, props));
568  }
569
570  @Override
571  public CompletableFuture<Boolean> abortProcedure(long procId, boolean mayInterruptIfRunning) {
572    return wrap(rawAdmin.abortProcedure(procId, mayInterruptIfRunning));
573  }
574
575  @Override
576  public CompletableFuture<String> getProcedures() {
577    return wrap(rawAdmin.getProcedures());
578  }
579
580  @Override
581  public CompletableFuture<String> getLocks() {
582    return wrap(rawAdmin.getLocks());
583  }
584
585  @Override
586  public CompletableFuture<Void> decommissionRegionServers(List<ServerName> servers,
587    boolean offload) {
588    return wrap(rawAdmin.decommissionRegionServers(servers, offload));
589  }
590
591  @Override
592  public CompletableFuture<List<ServerName>> listDecommissionedRegionServers() {
593    return wrap(rawAdmin.listDecommissionedRegionServers());
594  }
595
596  @Override
597  public CompletableFuture<Void> recommissionRegionServer(ServerName server,
598    List<byte[]> encodedRegionNames) {
599    return wrap(rawAdmin.recommissionRegionServer(server, encodedRegionNames));
600  }
601
602  @Override
603  public CompletableFuture<ClusterMetrics> getClusterMetrics() {
604    return getClusterMetrics(EnumSet.allOf(Option.class));
605  }
606
607  @Override
608  public CompletableFuture<ClusterMetrics> getClusterMetrics(EnumSet<Option> options) {
609    return wrap(rawAdmin.getClusterMetrics(options));
610  }
611
612  @Override
613  public CompletableFuture<Void> shutdown() {
614    return wrap(rawAdmin.shutdown());
615  }
616
617  @Override
618  public CompletableFuture<Void> stopMaster() {
619    return wrap(rawAdmin.stopMaster());
620  }
621
622  @Override
623  public CompletableFuture<Void> stopRegionServer(ServerName serverName) {
624    return wrap(rawAdmin.stopRegionServer(serverName));
625  }
626
627  @Override
628  public CompletableFuture<Void> updateConfiguration(ServerName serverName) {
629    return wrap(rawAdmin.updateConfiguration(serverName));
630  }
631
632  @Override
633  public CompletableFuture<Void> updateConfiguration() {
634    return wrap(rawAdmin.updateConfiguration());
635  }
636
637  @Override
638  public CompletableFuture<Void> rollWALWriter(ServerName serverName) {
639    return wrap(rawAdmin.rollWALWriter(serverName));
640  }
641
642  @Override
643  public CompletableFuture<Void> clearCompactionQueues(ServerName serverName, Set<String> queues) {
644    return wrap(rawAdmin.clearCompactionQueues(serverName, queues));
645  }
646
647  @Override
648  public CompletableFuture<List<SecurityCapability>> getSecurityCapabilities() {
649    return wrap(rawAdmin.getSecurityCapabilities());
650  }
651
652  @Override
653  public CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName) {
654    return wrap(rawAdmin.getRegionMetrics(serverName));
655  }
656
657  @Override
658  public CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName,
659    TableName tableName) {
660    return wrap(rawAdmin.getRegionMetrics(serverName, tableName));
661  }
662
663  @Override
664  public CompletableFuture<Boolean> isMasterInMaintenanceMode() {
665    return wrap(rawAdmin.isMasterInMaintenanceMode());
666  }
667
668  @Override
669  public CompletableFuture<CompactionState> getCompactionState(TableName tableName,
670    CompactType compactType) {
671    return wrap(rawAdmin.getCompactionState(tableName, compactType));
672  }
673
674  @Override
675  public CompletableFuture<CompactionState> getCompactionStateForRegion(byte[] regionName) {
676    return wrap(rawAdmin.getCompactionStateForRegion(regionName));
677  }
678
679  @Override
680  public CompletableFuture<Optional<Long>> getLastMajorCompactionTimestamp(TableName tableName) {
681    return wrap(rawAdmin.getLastMajorCompactionTimestamp(tableName));
682  }
683
684  @Override
685  public CompletableFuture<Optional<Long>>
686    getLastMajorCompactionTimestampForRegion(byte[] regionName) {
687    return wrap(rawAdmin.getLastMajorCompactionTimestampForRegion(regionName));
688  }
689
690  @Override
691  public CompletableFuture<Boolean> balancerSwitch(boolean on, boolean drainRITs) {
692    return wrap(rawAdmin.balancerSwitch(on, drainRITs));
693  }
694
695  @Override
696  public CompletableFuture<BalanceResponse> balance(BalanceRequest request) {
697    return wrap(rawAdmin.balance(request));
698  }
699
700  @Override
701  public CompletableFuture<Boolean> isBalancerEnabled() {
702    return wrap(rawAdmin.isBalancerEnabled());
703  }
704
705  @Override
706  public CompletableFuture<Boolean> normalizerSwitch(boolean on) {
707    return wrap(rawAdmin.normalizerSwitch(on));
708  }
709
710  @Override
711  public CompletableFuture<Boolean> isNormalizerEnabled() {
712    return wrap(rawAdmin.isNormalizerEnabled());
713  }
714
715  @Override
716  public CompletableFuture<Boolean> normalize(NormalizeTableFilterParams ntfp) {
717    return wrap(rawAdmin.normalize(ntfp));
718  }
719
720  @Override
721  public CompletableFuture<Boolean> cleanerChoreSwitch(boolean enabled) {
722    return wrap(rawAdmin.cleanerChoreSwitch(enabled));
723  }
724
725  @Override
726  public CompletableFuture<Boolean> isCleanerChoreEnabled() {
727    return wrap(rawAdmin.isCleanerChoreEnabled());
728  }
729
730  @Override
731  public CompletableFuture<Boolean> runCleanerChore() {
732    return wrap(rawAdmin.runCleanerChore());
733  }
734
735  @Override
736  public CompletableFuture<Boolean> catalogJanitorSwitch(boolean enabled) {
737    return wrap(rawAdmin.catalogJanitorSwitch(enabled));
738  }
739
740  @Override
741  public CompletableFuture<Boolean> isCatalogJanitorEnabled() {
742    return wrap(rawAdmin.isCatalogJanitorEnabled());
743  }
744
745  @Override
746  public CompletableFuture<Integer> runCatalogJanitor() {
747    return wrap(rawAdmin.runCatalogJanitor());
748  }
749
750  @Override
751  public <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
752    ServiceCaller<S, R> callable) {
753    return wrap(rawAdmin.coprocessorService(stubMaker, callable));
754  }
755
756  @Override
757  public <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
758    ServiceCaller<S, R> callable, ServerName serverName) {
759    return wrap(rawAdmin.coprocessorService(stubMaker, callable, serverName));
760  }
761
762  @Override
763  public CompletableFuture<List<ServerName>> listDeadServers() {
764    return wrap(rawAdmin.listDeadServers());
765  }
766
767  @Override
768  public CompletableFuture<List<ServerName>> listUnknownServers() {
769    return wrap(rawAdmin.listUnknownServers());
770  }
771
772  @Override
773  public CompletableFuture<List<ServerName>> clearDeadServers(List<ServerName> servers) {
774    return wrap(rawAdmin.clearDeadServers(servers));
775  }
776
777  @Override
778  public CompletableFuture<CacheEvictionStats> clearBlockCache(TableName tableName) {
779    return wrap(rawAdmin.clearBlockCache(tableName));
780  }
781
782  @Override
783  public CompletableFuture<Void> cloneTableSchema(TableName tableName, TableName newTableName,
784    boolean preserveSplits) {
785    return wrap(rawAdmin.cloneTableSchema(tableName, newTableName, preserveSplits));
786  }
787
788  @Override
789  public CompletableFuture<Map<ServerName, Boolean>> compactionSwitch(boolean switchState,
790    List<String> serverNamesList) {
791    return wrap(rawAdmin.compactionSwitch(switchState, serverNamesList));
792  }
793
794  @Override
795  public CompletableFuture<Boolean> switchRpcThrottle(boolean enable) {
796    return wrap(rawAdmin.switchRpcThrottle(enable));
797  }
798
799  @Override
800  public CompletableFuture<Boolean> isRpcThrottleEnabled() {
801    return wrap(rawAdmin.isRpcThrottleEnabled());
802  }
803
804  @Override
805  public CompletableFuture<Boolean> exceedThrottleQuotaSwitch(boolean enable) {
806    return wrap(rawAdmin.exceedThrottleQuotaSwitch(enable));
807  }
808
809  @Override
810  public CompletableFuture<Map<TableName, Long>> getSpaceQuotaTableSizes() {
811    return wrap(rawAdmin.getSpaceQuotaTableSizes());
812  }
813
814  @Override
815  public CompletableFuture<Map<TableName, SpaceQuotaSnapshot>>
816    getRegionServerSpaceQuotaSnapshots(ServerName serverName) {
817    return wrap(rawAdmin.getRegionServerSpaceQuotaSnapshots(serverName));
818  }
819
820  @Override
821  public CompletableFuture<SpaceQuotaSnapshot> getCurrentSpaceQuotaSnapshot(String namespace) {
822    return wrap(rawAdmin.getCurrentSpaceQuotaSnapshot(namespace));
823  }
824
825  @Override
826  public CompletableFuture<SpaceQuotaSnapshot> getCurrentSpaceQuotaSnapshot(TableName tableName) {
827    return wrap(rawAdmin.getCurrentSpaceQuotaSnapshot(tableName));
828  }
829
830  @Override
831  public CompletableFuture<Void> grant(UserPermission userPermission,
832    boolean mergeExistingPermissions) {
833    return wrap(rawAdmin.grant(userPermission, mergeExistingPermissions));
834  }
835
836  @Override
837  public CompletableFuture<Void> revoke(UserPermission userPermission) {
838    return wrap(rawAdmin.revoke(userPermission));
839  }
840
841  @Override
842  public CompletableFuture<List<UserPermission>>
843    getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest) {
844    return wrap(rawAdmin.getUserPermissions(getUserPermissionsRequest));
845  }
846
847  @Override
848  public CompletableFuture<List<Boolean>> hasUserPermissions(String userName,
849    List<Permission> permissions) {
850    return wrap(rawAdmin.hasUserPermissions(userName, permissions));
851  }
852
853  @Override
854  public CompletableFuture<Boolean> snapshotCleanupSwitch(final boolean on, final boolean sync) {
855    return wrap(rawAdmin.snapshotCleanupSwitch(on, sync));
856  }
857
858  @Override
859  public CompletableFuture<Boolean> isSnapshotCleanupEnabled() {
860    return wrap(rawAdmin.isSnapshotCleanupEnabled());
861  }
862
863  @Override
864  public CompletableFuture<List<Boolean>> clearSlowLogResponses(Set<ServerName> serverNames) {
865    return wrap(rawAdmin.clearSlowLogResponses(serverNames));
866  }
867
868  @Override
869  public CompletableFuture<List<LogEntry>> getLogEntries(Set<ServerName> serverNames,
870    String logType, ServerType serverType, int limit, Map<String, Object> filterParams) {
871    return wrap(rawAdmin.getLogEntries(serverNames, logType, serverType, limit, filterParams));
872  }
873
874  @Override
875  public CompletableFuture<Void> flushMasterStore() {
876    return wrap(rawAdmin.flushMasterStore());
877  }
878}