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 static org.apache.hadoop.hbase.client.ConnectionUtils.setCoprocessorError;
021import static org.apache.hadoop.hbase.util.FutureUtils.get;
022
023import java.io.IOException;
024import java.util.Arrays;
025import java.util.EnumSet;
026import java.util.List;
027import java.util.Map;
028import java.util.Set;
029import java.util.concurrent.Future;
030import java.util.regex.Pattern;
031import org.apache.hadoop.conf.Configuration;
032import org.apache.hadoop.hbase.CacheEvictionStats;
033import org.apache.hadoop.hbase.ClusterMetrics;
034import org.apache.hadoop.hbase.ClusterMetrics.Option;
035import org.apache.hadoop.hbase.HConstants;
036import org.apache.hadoop.hbase.NamespaceDescriptor;
037import org.apache.hadoop.hbase.NamespaceNotFoundException;
038import org.apache.hadoop.hbase.RegionMetrics;
039import org.apache.hadoop.hbase.ServerName;
040import org.apache.hadoop.hbase.TableExistsException;
041import org.apache.hadoop.hbase.TableName;
042import org.apache.hadoop.hbase.TableNotFoundException;
043import org.apache.hadoop.hbase.client.replication.TableCFs;
044import org.apache.hadoop.hbase.client.security.SecurityCapability;
045import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
046import org.apache.hadoop.hbase.net.Address;
047import org.apache.hadoop.hbase.quotas.QuotaFilter;
048import org.apache.hadoop.hbase.quotas.QuotaSettings;
049import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotView;
050import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
051import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
052import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
053import org.apache.hadoop.hbase.replication.SyncReplicationState;
054import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
055import org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest;
056import org.apache.hadoop.hbase.security.access.Permission;
057import org.apache.hadoop.hbase.security.access.UserPermission;
058import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
059import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
060import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
061import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
062import org.apache.hadoop.hbase.util.Bytes;
063import org.apache.hadoop.hbase.util.Pair;
064import org.apache.yetus.audience.InterfaceAudience;
065import org.slf4j.Logger;
066import org.slf4j.LoggerFactory;
067
068import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
069import org.apache.hbase.thirdparty.com.google.protobuf.Message;
070import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
071import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
072import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
073import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
074
075/**
076 * The {@link Admin} implementation which is based on an {@link AsyncAdmin}.
077 */
078@InterfaceAudience.Private
079class AdminOverAsyncAdmin implements Admin {
080
081  private static final Logger LOG = LoggerFactory.getLogger(AdminOverAsyncAdmin.class);
082
083  private volatile boolean aborted = false;
084
085  private final Connection conn;
086
087  private final RawAsyncHBaseAdmin admin;
088
089  private final int operationTimeout;
090
091  private final int syncWaitTimeout;
092
093  public AdminOverAsyncAdmin(Connection conn, RawAsyncHBaseAdmin admin) {
094    this.conn = conn;
095    this.admin = admin;
096    this.operationTimeout = conn.getConfiguration().getInt(
097      HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
098    this.syncWaitTimeout =
099      conn.getConfiguration().getInt("hbase.client.sync.wait.timeout.msec", 10 * 60000); // 10min
100  }
101
102  @Override
103  public int getOperationTimeout() {
104    return operationTimeout;
105  }
106
107  @Override
108  public int getSyncWaitTimeout() {
109    return syncWaitTimeout;
110  }
111
112  @Override
113  public void abort(String why, Throwable e) {
114    LOG.warn("Aborting becasue of {}", why, e);
115    this.aborted = true;
116  }
117
118  @Override
119  public boolean isAborted() {
120    return aborted;
121  }
122
123  @Override
124  public Connection getConnection() {
125    return conn;
126  }
127
128  @Override
129  public boolean tableExists(TableName tableName) throws IOException {
130    return get(admin.tableExists(tableName));
131  }
132
133  @Override
134  public List<TableDescriptor> listTableDescriptors() throws IOException {
135    return get(admin.listTableDescriptors());
136  }
137
138  @Override
139  public List<TableDescriptor> listTableDescriptors(boolean includeSysTables) throws IOException {
140    return get(admin.listTableDescriptors(includeSysTables));
141  }
142
143  @Override
144  public List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables)
145    throws IOException {
146    return get(admin.listTableDescriptors(pattern, includeSysTables));
147  }
148
149  @Override
150  public List<TableDescriptor> listTableDescriptorsByState(boolean isEnabled) throws IOException {
151    return get(admin.listTableDescriptorsByState(isEnabled));
152  }
153
154  @Override
155  public TableName[] listTableNames() throws IOException {
156    return get(admin.listTableNames()).toArray(new TableName[0]);
157  }
158
159  @Override
160  public TableName[] listTableNames(Pattern pattern, boolean includeSysTables) throws IOException {
161    return get(admin.listTableNames(pattern, includeSysTables)).toArray(new TableName[0]);
162  }
163
164  @Override
165  public List<TableName> listTableNamesByState(boolean isEnabled) throws IOException {
166    return get(admin.listTableNamesByState(isEnabled));
167  }
168
169  @Override
170  public TableDescriptor getDescriptor(TableName tableName)
171    throws TableNotFoundException, IOException {
172    return get(admin.getDescriptor(tableName));
173  }
174
175  @Override
176  public void createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions)
177    throws IOException {
178    get(admin.createTable(desc, startKey, endKey, numRegions));
179  }
180
181  @Override
182  public Future<Void> createTableAsync(TableDescriptor desc) throws IOException {
183    return admin.createTable(desc);
184  }
185
186  @Override
187  public Future<Void> createTableAsync(TableDescriptor desc, byte[][] splitKeys)
188    throws IOException {
189    return admin.createTable(desc, splitKeys);
190  }
191
192  @Override
193  public Future<Void> deleteTableAsync(TableName tableName) throws IOException {
194    return admin.deleteTable(tableName);
195  }
196
197  @Override
198  public Future<Void> truncateTableAsync(TableName tableName, boolean preserveSplits)
199    throws IOException {
200    return admin.truncateTable(tableName, preserveSplits);
201  }
202
203  @Override
204  public Future<Void> enableTableAsync(TableName tableName) throws IOException {
205    return admin.enableTable(tableName);
206  }
207
208  @Override
209  public Future<Void> disableTableAsync(TableName tableName) throws IOException {
210    return admin.disableTable(tableName);
211  }
212
213  @Override
214  public boolean isTableEnabled(TableName tableName) throws IOException {
215    return get(admin.isTableEnabled(tableName));
216  }
217
218  @Override
219  public boolean isTableDisabled(TableName tableName) throws IOException {
220    return get(admin.isTableDisabled(tableName));
221  }
222
223  @Override
224  public boolean isTableAvailable(TableName tableName) throws IOException {
225    return get(admin.isTableAvailable(tableName));
226  }
227
228  @Override
229  public Future<Void> addColumnFamilyAsync(TableName tableName, ColumnFamilyDescriptor columnFamily)
230    throws IOException {
231    return admin.addColumnFamily(tableName, columnFamily);
232  }
233
234  @Override
235  public Future<Void> deleteColumnFamilyAsync(TableName tableName, byte[] columnFamily)
236    throws IOException {
237    return admin.deleteColumnFamily(tableName, columnFamily);
238  }
239
240  @Override
241  public Future<Void> modifyColumnFamilyAsync(TableName tableName,
242    ColumnFamilyDescriptor columnFamily) throws IOException {
243    return admin.modifyColumnFamily(tableName, columnFamily);
244  }
245
246  @Override
247  public Future<Void> modifyColumnFamilyStoreFileTrackerAsync(TableName tableName, byte[] family,
248    String dstSFT) throws IOException {
249    return admin.modifyColumnFamilyStoreFileTracker(tableName, family, dstSFT);
250  }
251
252  @Override
253  public List<RegionInfo> getRegions(ServerName serverName) throws IOException {
254    return get(admin.getRegions(serverName));
255  }
256
257  @Override
258  public void flush(TableName tableName) throws IOException {
259    get(admin.flush(tableName));
260  }
261
262  @Override
263  public void flush(TableName tableName, byte[] columnFamily) throws IOException {
264    get(admin.flush(tableName, columnFamily));
265  }
266
267  @Override
268  public void flush(TableName tableName, List<byte[]> columnFamilies) throws IOException {
269    get(admin.flush(tableName, columnFamilies));
270  }
271
272  @Override
273  public void flushRegion(byte[] regionName) throws IOException {
274    get(admin.flushRegion(regionName));
275  }
276
277  @Override
278  public void flushRegion(byte[] regionName, byte[] columnFamily) throws IOException {
279    get(admin.flushRegion(regionName, columnFamily));
280  }
281
282  @Override
283  public void flushRegionServer(ServerName serverName) throws IOException {
284    get(admin.flushRegionServer(serverName));
285  }
286
287  @Override
288  public void compact(TableName tableName) throws IOException {
289    get(admin.compact(tableName));
290  }
291
292  @Override
293  public void compactRegion(byte[] regionName) throws IOException {
294    get(admin.compactRegion(regionName));
295  }
296
297  @Override
298  public void compact(TableName tableName, byte[] columnFamily) throws IOException {
299    get(admin.compact(tableName, columnFamily));
300  }
301
302  @Override
303  public void compactRegion(byte[] regionName, byte[] columnFamily) throws IOException {
304    get(admin.compactRegion(regionName, columnFamily));
305  }
306
307  @Override
308  public void compact(TableName tableName, CompactType compactType)
309    throws IOException, InterruptedException {
310    get(admin.compact(tableName, compactType));
311  }
312
313  @Override
314  public void compact(TableName tableName, byte[] columnFamily, CompactType compactType)
315    throws IOException, InterruptedException {
316    get(admin.compact(tableName, columnFamily, compactType));
317  }
318
319  @Override
320  public void majorCompact(TableName tableName) throws IOException {
321    get(admin.majorCompact(tableName));
322  }
323
324  @Override
325  public void majorCompactRegion(byte[] regionName) throws IOException {
326    get(admin.majorCompactRegion(regionName));
327  }
328
329  @Override
330  public void majorCompact(TableName tableName, byte[] columnFamily) throws IOException {
331    get(admin.majorCompact(tableName, columnFamily));
332  }
333
334  @Override
335  public void majorCompactRegion(byte[] regionName, byte[] columnFamily) throws IOException {
336    get(admin.majorCompactRegion(regionName, columnFamily));
337  }
338
339  @Override
340  public void majorCompact(TableName tableName, CompactType compactType)
341    throws IOException, InterruptedException {
342    get(admin.majorCompact(tableName, compactType));
343  }
344
345  @Override
346  public void majorCompact(TableName tableName, byte[] columnFamily, CompactType compactType)
347    throws IOException, InterruptedException {
348    get(admin.majorCompact(tableName, columnFamily, compactType));
349  }
350
351  @Override
352  public Map<ServerName, Boolean> compactionSwitch(boolean switchState,
353    List<String> serverNamesList) throws IOException {
354    return get(admin.compactionSwitch(switchState, serverNamesList));
355  }
356
357  @Override
358  public void compactRegionServer(ServerName serverName) throws IOException {
359    get(admin.compactRegionServer(serverName));
360  }
361
362  @Override
363  public void majorCompactRegionServer(ServerName serverName) throws IOException {
364    get(admin.majorCompactRegionServer(serverName));
365  }
366
367  @Override
368  public void move(byte[] encodedRegionName) throws IOException {
369    get(admin.move(encodedRegionName));
370  }
371
372  @Override
373  public void move(byte[] encodedRegionName, ServerName destServerName) throws IOException {
374    get(admin.move(encodedRegionName, destServerName));
375  }
376
377  @Override
378  public void assign(byte[] regionName) throws IOException {
379    get(admin.assign(regionName));
380  }
381
382  @Override
383  public void unassign(byte[] regionName) throws IOException {
384    get(admin.unassign(regionName));
385  }
386
387  @Override
388  public void offline(byte[] regionName) throws IOException {
389    get(admin.offline(regionName));
390  }
391
392  @Override
393  public boolean balancerSwitch(boolean onOrOff, boolean synchronous) throws IOException {
394    return get(admin.balancerSwitch(onOrOff, synchronous));
395  }
396
397  @Override
398  public BalanceResponse balance(BalanceRequest request) throws IOException {
399    return get(admin.balance(request));
400  }
401
402  @Override
403  public boolean balance() throws IOException {
404    return get(admin.balance());
405  }
406
407  @Override
408  public boolean balance(boolean force) throws IOException {
409    return get(admin.balance(force));
410  }
411
412  @Override
413  public boolean isBalancerEnabled() throws IOException {
414    return get(admin.isBalancerEnabled());
415  }
416
417  @Override
418  public CacheEvictionStats clearBlockCache(TableName tableName) throws IOException {
419    return get(admin.clearBlockCache(tableName));
420  }
421
422  @Override
423  public boolean normalize(NormalizeTableFilterParams ntfp) throws IOException {
424    return get(admin.normalize(ntfp));
425  }
426
427  @Override
428  public boolean isNormalizerEnabled() throws IOException {
429    return get(admin.isNormalizerEnabled());
430  }
431
432  @Override
433  public boolean normalizerSwitch(boolean on) throws IOException {
434    return get(admin.normalizerSwitch(on));
435  }
436
437  @Override
438  public boolean catalogJanitorSwitch(boolean onOrOff) throws IOException {
439    return get(admin.catalogJanitorSwitch(onOrOff));
440  }
441
442  @Override
443  public int runCatalogJanitor() throws IOException {
444    return get(admin.runCatalogJanitor());
445  }
446
447  @Override
448  public boolean isCatalogJanitorEnabled() throws IOException {
449    return get(admin.isCatalogJanitorEnabled());
450  }
451
452  @Override
453  public boolean cleanerChoreSwitch(boolean onOrOff) throws IOException {
454    return get(admin.cleanerChoreSwitch(onOrOff));
455  }
456
457  @Override
458  public boolean runCleanerChore() throws IOException {
459    return get(admin.runCleanerChore());
460  }
461
462  @Override
463  public boolean isCleanerChoreEnabled() throws IOException {
464    return get(admin.isCleanerChoreEnabled());
465  }
466
467  @Override
468  public Future<Void> mergeRegionsAsync(byte[][] nameOfRegionsToMerge, boolean forcible)
469    throws IOException {
470    return admin.mergeRegions(Arrays.asList(nameOfRegionsToMerge), forcible);
471  }
472
473  @Override
474  public void split(TableName tableName) throws IOException {
475    get(admin.split(tableName));
476  }
477
478  @Override
479  public void split(TableName tableName, byte[] splitPoint) throws IOException {
480    get(admin.split(tableName, splitPoint));
481  }
482
483  @Override
484  public Future<Void> splitRegionAsync(byte[] regionName) throws IOException {
485    return admin.splitRegion(regionName);
486  }
487
488  @Override
489  public Future<Void> splitRegionAsync(byte[] regionName, byte[] splitPoint) throws IOException {
490    return admin.splitRegion(regionName, splitPoint);
491  }
492
493  @Override
494  public void truncateRegion(byte[] regionName) throws IOException {
495    get(admin.truncateRegion(regionName));
496  }
497
498  @Override
499  public Future<Void> truncateRegionAsync(byte[] regionName) {
500    return admin.truncateRegion(regionName);
501  }
502
503  @Override
504  public Future<Void> modifyTableAsync(TableDescriptor td) throws IOException {
505    return modifyTableAsync(td, true);
506  }
507
508  @Override
509  public Future<Void> modifyTableAsync(TableDescriptor td, boolean reopenRegions)
510    throws IOException {
511    return admin.modifyTable(td, reopenRegions);
512  }
513
514  @Override
515  public Future<Void> reopenTableRegionsAsync(TableName tableName) throws IOException {
516    return admin.reopenTableRegions(tableName).toCompletableFuture();
517  }
518
519  @Override
520  public Future<Void> reopenTableRegionsAsync(TableName tableName, List<RegionInfo> regions)
521    throws IOException {
522    return admin.reopenTableRegions(tableName, regions).toCompletableFuture();
523  }
524
525  @Override
526  public Future<Void> modifyTableStoreFileTrackerAsync(TableName tableName, String dstSFT)
527    throws IOException {
528    return admin.modifyTableStoreFileTracker(tableName, dstSFT);
529  }
530
531  @Override
532  public void shutdown() throws IOException {
533    get(admin.shutdown());
534  }
535
536  @Override
537  public void stopMaster() throws IOException {
538    get(admin.stopMaster());
539  }
540
541  @Override
542  public boolean isMasterInMaintenanceMode() throws IOException {
543    return get(admin.isMasterInMaintenanceMode());
544  }
545
546  @Override
547  public void stopRegionServer(String hostnamePort) throws IOException {
548    get(admin.stopRegionServer(ServerName.valueOf(hostnamePort, 0)));
549  }
550
551  @Override
552  public ClusterMetrics getClusterMetrics(EnumSet<Option> options) throws IOException {
553    return get(admin.getClusterMetrics(options));
554  }
555
556  @Override
557  public List<RegionMetrics> getRegionMetrics(ServerName serverName) throws IOException {
558    return get(admin.getRegionMetrics(serverName));
559  }
560
561  @Override
562  public List<RegionMetrics> getRegionMetrics(ServerName serverName, TableName tableName)
563    throws IOException {
564    return get(admin.getRegionMetrics(serverName, tableName));
565  }
566
567  @Override
568  public Configuration getConfiguration() {
569    return conn.getConfiguration();
570  }
571
572  @Override
573  public Future<Void> createNamespaceAsync(NamespaceDescriptor descriptor) throws IOException {
574    return admin.createNamespace(descriptor);
575  }
576
577  @Override
578  public Future<Void> modifyNamespaceAsync(NamespaceDescriptor descriptor) throws IOException {
579    return admin.modifyNamespace(descriptor);
580  }
581
582  @Override
583  public Future<Void> deleteNamespaceAsync(String name) throws IOException {
584    return admin.deleteNamespace(name);
585  }
586
587  @Override
588  public NamespaceDescriptor getNamespaceDescriptor(String name)
589    throws NamespaceNotFoundException, IOException {
590    return get(admin.getNamespaceDescriptor(name));
591  }
592
593  @Override
594  public String[] listNamespaces() throws IOException {
595    return get(admin.listNamespaces()).toArray(new String[0]);
596  }
597
598  @Override
599  public NamespaceDescriptor[] listNamespaceDescriptors() throws IOException {
600    return get(admin.listNamespaceDescriptors()).toArray(new NamespaceDescriptor[0]);
601  }
602
603  @Override
604  public List<TableDescriptor> listTableDescriptorsByNamespace(byte[] name) throws IOException {
605    return get(admin.listTableDescriptorsByNamespace(Bytes.toString(name)));
606  }
607
608  @Override
609  public TableName[] listTableNamesByNamespace(String name) throws IOException {
610    return get(admin.listTableNamesByNamespace(name)).toArray(new TableName[0]);
611  }
612
613  @Override
614  public List<RegionInfo> getRegions(TableName tableName) throws IOException {
615    return get(admin.getRegions(tableName));
616  }
617
618  @Override
619  public void close() {
620    // do nothing, AsyncAdmin is not a Closeable.
621  }
622
623  @Override
624  public List<TableDescriptor> listTableDescriptors(List<TableName> tableNames) throws IOException {
625    return get(admin.listTableDescriptors(tableNames));
626  }
627
628  @Override
629  public Future<Boolean> abortProcedureAsync(long procId, boolean mayInterruptIfRunning)
630    throws IOException {
631    return admin.abortProcedure(procId, mayInterruptIfRunning);
632  }
633
634  @Override
635  public String getProcedures() throws IOException {
636    return get(admin.getProcedures());
637  }
638
639  @Override
640  public String getLocks() throws IOException {
641    return get(admin.getLocks());
642  }
643
644  @Override
645  public void rollWALWriter(ServerName serverName) throws IOException, FailedLogCloseException {
646    get(admin.rollWALWriter(serverName));
647  }
648
649  @Override
650  public Map<ServerName, Long> rollAllWALWriters() throws IOException {
651    return get(admin.rollAllWALWriters());
652  }
653
654  @Override
655  public CompactionState getCompactionState(TableName tableName) throws IOException {
656    return get(admin.getCompactionState(tableName));
657  }
658
659  @Override
660  public CompactionState getCompactionState(TableName tableName, CompactType compactType)
661    throws IOException {
662    return get(admin.getCompactionState(tableName, compactType));
663  }
664
665  @Override
666  public CompactionState getCompactionStateForRegion(byte[] regionName) throws IOException {
667    return get(admin.getCompactionStateForRegion(regionName));
668  }
669
670  @Override
671  public long getLastMajorCompactionTimestamp(TableName tableName) throws IOException {
672    return get(admin.getLastMajorCompactionTimestamp(tableName)).orElse(0L);
673  }
674
675  @Override
676  public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException {
677    return get(admin.getLastMajorCompactionTimestampForRegion(regionName)).orElse(0L);
678  }
679
680  @Override
681  public void snapshot(SnapshotDescription snapshot)
682    throws IOException, SnapshotCreationException, IllegalArgumentException {
683    get(admin.snapshot(snapshot));
684  }
685
686  @Override
687  public Future<Void> snapshotAsync(SnapshotDescription snapshot)
688    throws IOException, SnapshotCreationException {
689    return admin.snapshot(snapshot);
690  }
691
692  @Override
693  public boolean isSnapshotFinished(SnapshotDescription snapshot)
694    throws IOException, HBaseSnapshotException, UnknownSnapshotException {
695    return get(admin.isSnapshotFinished(snapshot));
696  }
697
698  @Override
699  public void restoreSnapshot(String snapshotName) throws IOException, RestoreSnapshotException {
700    get(admin.restoreSnapshot(snapshotName));
701  }
702
703  @Override
704  public void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl)
705    throws IOException, RestoreSnapshotException {
706    get(admin.restoreSnapshot(snapshotName, takeFailSafeSnapshot, restoreAcl));
707  }
708
709  @Override
710  public Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName,
711    boolean restoreAcl, String customSFT)
712    throws IOException, TableExistsException, RestoreSnapshotException {
713    return admin.cloneSnapshot(snapshotName, tableName, restoreAcl, customSFT);
714  }
715
716  @Override
717  public void execProcedure(String signature, String instance, Map<String, String> props)
718    throws IOException {
719    get(admin.execProcedure(signature, instance, props));
720  }
721
722  @Override
723  public byte[] execProcedureWithReturn(String signature, String instance,
724    Map<String, String> props) throws IOException {
725    return get(admin.execProcedureWithReturn(signature, instance, props));
726  }
727
728  @Override
729  public boolean isProcedureFinished(String signature, String instance, Map<String, String> props)
730    throws IOException {
731    return get(admin.isProcedureFinished(signature, instance, props));
732  }
733
734  @Override
735  public List<SnapshotDescription> listSnapshots() throws IOException {
736    return get(admin.listSnapshots());
737  }
738
739  @Override
740  public List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException {
741    return get(admin.listSnapshots(pattern));
742  }
743
744  @Override
745  public List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern,
746    Pattern snapshotNamePattern) throws IOException {
747    return get(admin.listTableSnapshots(tableNamePattern, snapshotNamePattern));
748  }
749
750  @Override
751  public void deleteSnapshot(String snapshotName) throws IOException {
752    get(admin.deleteSnapshot(snapshotName));
753  }
754
755  @Override
756  public void deleteSnapshots(Pattern pattern) throws IOException {
757    get(admin.deleteSnapshots(pattern));
758  }
759
760  @Override
761  public void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern)
762    throws IOException {
763    get(admin.deleteTableSnapshots(tableNamePattern, snapshotNamePattern));
764  }
765
766  @Override
767  public void setQuota(QuotaSettings quota) throws IOException {
768    get(admin.setQuota(quota));
769  }
770
771  @Override
772  public List<QuotaSettings> getQuota(QuotaFilter filter) throws IOException {
773    return get(admin.getQuota(filter));
774  }
775
776  @SuppressWarnings("deprecation")
777  private static final class SyncCoprocessorRpcChannelOverAsync implements CoprocessorRpcChannel {
778
779    private final RpcChannel delegate;
780
781    public SyncCoprocessorRpcChannelOverAsync(RpcChannel delegate) {
782      this.delegate = delegate;
783    }
784
785    @Override
786    public void callMethod(MethodDescriptor method, RpcController controller, Message request,
787      Message responsePrototype, RpcCallback<Message> done) {
788      ClientCoprocessorRpcController c = new ClientCoprocessorRpcController();
789      CoprocessorBlockingRpcCallback<Message> callback = new CoprocessorBlockingRpcCallback<>();
790      delegate.callMethod(method, c, request, responsePrototype, callback);
791      Message ret;
792      try {
793        ret = callback.get();
794      } catch (IOException e) {
795        setCoprocessorError(controller, e);
796        return;
797      }
798      if (c.failed()) {
799        setCoprocessorError(controller, c.getFailed());
800      }
801      done.run(ret);
802    }
803
804    @Override
805    public Message callBlockingMethod(MethodDescriptor method, RpcController controller,
806      Message request, Message responsePrototype) throws ServiceException {
807      ClientCoprocessorRpcController c = new ClientCoprocessorRpcController();
808      CoprocessorBlockingRpcCallback<Message> done = new CoprocessorBlockingRpcCallback<>();
809      callMethod(method, c, request, responsePrototype, done);
810      Message ret;
811      try {
812        ret = done.get();
813      } catch (IOException e) {
814        throw new ServiceException(e);
815      }
816      if (c.failed()) {
817        setCoprocessorError(controller, c.getFailed());
818        throw new ServiceException(c.getFailed());
819      }
820      return ret;
821    }
822  }
823
824  @SuppressWarnings("deprecation")
825  @Override
826  public CoprocessorRpcChannel coprocessorService() {
827    return new SyncCoprocessorRpcChannelOverAsync(
828      new MasterCoprocessorRpcChannelImpl(admin.<Message> newMasterCaller()));
829  }
830
831  @SuppressWarnings("deprecation")
832  @Override
833  public CoprocessorRpcChannel coprocessorService(ServerName serverName) {
834    return new SyncCoprocessorRpcChannelOverAsync(new RegionServerCoprocessorRpcChannelImpl(
835      admin.<Message> newServerCaller().serverName(serverName)));
836  }
837
838  @Override
839  public void updateConfiguration(ServerName server) throws IOException {
840    get(admin.updateConfiguration(server));
841  }
842
843  @Override
844  public void updateConfiguration() throws IOException {
845    get(admin.updateConfiguration());
846  }
847
848  @Override
849  public void updateConfiguration(String groupName) throws IOException {
850    get(admin.updateConfiguration(groupName));
851  }
852
853  @Override
854  public List<SecurityCapability> getSecurityCapabilities() throws IOException {
855    return get(admin.getSecurityCapabilities());
856  }
857
858  @Override
859  public boolean splitSwitch(boolean enabled, boolean synchronous) throws IOException {
860    return get(admin.splitSwitch(enabled, synchronous));
861  }
862
863  @Override
864  public boolean mergeSwitch(boolean enabled, boolean synchronous) throws IOException {
865    return get(admin.mergeSwitch(enabled, synchronous));
866  }
867
868  @Override
869  public boolean isSplitEnabled() throws IOException {
870    return get(admin.isSplitEnabled());
871  }
872
873  @Override
874  public boolean isMergeEnabled() throws IOException {
875    return get(admin.isMergeEnabled());
876  }
877
878  @Override
879  public Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig,
880    boolean enabled) throws IOException {
881    return admin.addReplicationPeer(peerId, peerConfig, enabled);
882  }
883
884  @Override
885  public Future<Void> removeReplicationPeerAsync(String peerId) throws IOException {
886    return admin.removeReplicationPeer(peerId);
887  }
888
889  @Override
890  public Future<Void> enableReplicationPeerAsync(String peerId) throws IOException {
891    return admin.enableReplicationPeer(peerId);
892  }
893
894  @Override
895  public Future<Void> disableReplicationPeerAsync(String peerId) throws IOException {
896    return admin.disableReplicationPeer(peerId);
897  }
898
899  @Override
900  public ReplicationPeerConfig getReplicationPeerConfig(String peerId) throws IOException {
901    return get(admin.getReplicationPeerConfig(peerId));
902  }
903
904  @Override
905  public Future<Void> updateReplicationPeerConfigAsync(String peerId,
906    ReplicationPeerConfig peerConfig) throws IOException {
907    return admin.updateReplicationPeerConfig(peerId, peerConfig);
908  }
909
910  @Override
911  public List<ReplicationPeerDescription> listReplicationPeers() throws IOException {
912    return get(admin.listReplicationPeers());
913  }
914
915  @Override
916  public List<ReplicationPeerDescription> listReplicationPeers(Pattern pattern) throws IOException {
917    return get(admin.listReplicationPeers(pattern));
918  }
919
920  @Override
921  public Future<Void> transitReplicationPeerSyncReplicationStateAsync(String peerId,
922    SyncReplicationState state) throws IOException {
923    return admin.transitReplicationPeerSyncReplicationState(peerId, state);
924  }
925
926  @Override
927  public boolean isReplicationPeerEnabled(String peerId) throws IOException {
928    return get(admin.isReplicationPeerEnabled(peerId));
929  }
930
931  @Override
932  public boolean replicationPeerModificationSwitch(boolean on, boolean drainProcedures)
933    throws IOException {
934    return get(admin.replicationPeerModificationSwitch(on, drainProcedures));
935  }
936
937  @Override
938  public boolean isReplicationPeerModificationEnabled() throws IOException {
939    return get(admin.isReplicationPeerModificationEnabled());
940  }
941
942  @Override
943  public void decommissionRegionServers(List<ServerName> servers, boolean offload)
944    throws IOException {
945    get(admin.decommissionRegionServers(servers, offload));
946  }
947
948  @Override
949  public List<ServerName> listDecommissionedRegionServers() throws IOException {
950    return get(admin.listDecommissionedRegionServers());
951  }
952
953  @Override
954  public void recommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames)
955    throws IOException {
956    get(admin.recommissionRegionServer(server, encodedRegionNames));
957  }
958
959  @Override
960  public List<TableCFs> listReplicatedTableCFs() throws IOException {
961    return get(admin.listReplicatedTableCFs());
962  }
963
964  @Override
965  public void enableTableReplication(TableName tableName) throws IOException {
966    get(admin.enableTableReplication(tableName));
967  }
968
969  @Override
970  public void disableTableReplication(TableName tableName) throws IOException {
971    get(admin.disableTableReplication(tableName));
972  }
973
974  @Override
975  public void clearCompactionQueues(ServerName serverName, Set<String> queues)
976    throws IOException, InterruptedException {
977    get(admin.clearCompactionQueues(serverName, queues));
978  }
979
980  @Override
981  public List<ServerName> clearDeadServers(List<ServerName> servers) throws IOException {
982    return get(admin.clearDeadServers(servers));
983  }
984
985  @Override
986  public void cloneTableSchema(TableName tableName, TableName newTableName, boolean preserveSplits)
987    throws IOException {
988    get(admin.cloneTableSchema(tableName, newTableName, preserveSplits));
989  }
990
991  @Override
992  public boolean switchRpcThrottle(boolean enable) throws IOException {
993    return get(admin.switchRpcThrottle(enable));
994  }
995
996  @Override
997  public boolean isRpcThrottleEnabled() throws IOException {
998    return get(admin.isRpcThrottleEnabled());
999  }
1000
1001  @Override
1002  public boolean exceedThrottleQuotaSwitch(boolean enable) throws IOException {
1003    return get(admin.exceedThrottleQuotaSwitch(enable));
1004  }
1005
1006  @Override
1007  public Map<TableName, Long> getSpaceQuotaTableSizes() throws IOException {
1008    return get(admin.getSpaceQuotaTableSizes());
1009  }
1010
1011  @Override
1012  public Map<TableName, ? extends SpaceQuotaSnapshotView>
1013    getRegionServerSpaceQuotaSnapshots(ServerName serverName) throws IOException {
1014    return get(admin.getRegionServerSpaceQuotaSnapshots(serverName));
1015  }
1016
1017  @Override
1018  public SpaceQuotaSnapshotView getCurrentSpaceQuotaSnapshot(String namespace) throws IOException {
1019    return get(admin.getCurrentSpaceQuotaSnapshot(namespace));
1020  }
1021
1022  @Override
1023  public SpaceQuotaSnapshotView getCurrentSpaceQuotaSnapshot(TableName tableName)
1024    throws IOException {
1025    return get(admin.getCurrentSpaceQuotaSnapshot(tableName));
1026  }
1027
1028  @Override
1029  public void grant(UserPermission userPermission, boolean mergeExistingPermissions)
1030    throws IOException {
1031    get(admin.grant(userPermission, mergeExistingPermissions));
1032  }
1033
1034  @Override
1035  public void revoke(UserPermission userPermission) throws IOException {
1036    get(admin.revoke(userPermission));
1037  }
1038
1039  @Override
1040  public List<UserPermission>
1041    getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest) throws IOException {
1042    return get(admin.getUserPermissions(getUserPermissionsRequest));
1043  }
1044
1045  @Override
1046  public List<Boolean> hasUserPermissions(String userName, List<Permission> permissions)
1047    throws IOException {
1048    return get(admin.hasUserPermissions(userName, permissions));
1049  }
1050
1051  @Override
1052  public boolean snapshotCleanupSwitch(final boolean on, final boolean synchronous)
1053    throws IOException {
1054    return get(admin.snapshotCleanupSwitch(on, synchronous));
1055  }
1056
1057  @Override
1058  public boolean isSnapshotCleanupEnabled() throws IOException {
1059    return get(admin.isSnapshotCleanupEnabled());
1060  }
1061
1062  @Override
1063  public List<Boolean> clearSlowLogResponses(final Set<ServerName> serverNames) throws IOException {
1064    return get(admin.clearSlowLogResponses(serverNames));
1065  }
1066
1067  @Override
1068  public RSGroupInfo getRSGroup(String groupName) throws IOException {
1069    return get(admin.getRSGroup(groupName));
1070  }
1071
1072  @Override
1073  public void moveServersToRSGroup(Set<Address> servers, String groupName) throws IOException {
1074    get(admin.moveServersToRSGroup(servers, groupName));
1075  }
1076
1077  @Override
1078  public void addRSGroup(String groupName) throws IOException {
1079    get(admin.addRSGroup(groupName));
1080  }
1081
1082  @Override
1083  public void removeRSGroup(String groupName) throws IOException {
1084    get(admin.removeRSGroup(groupName));
1085  }
1086
1087  @Override
1088  public BalanceResponse balanceRSGroup(String groupName, BalanceRequest request)
1089    throws IOException {
1090    return get(admin.balanceRSGroup(groupName, request));
1091  }
1092
1093  @Override
1094  public List<RSGroupInfo> listRSGroups() throws IOException {
1095    return get(admin.listRSGroups());
1096  }
1097
1098  @Override
1099  public List<TableName> listTablesInRSGroup(String groupName) throws IOException {
1100    return get(admin.listTablesInRSGroup(groupName));
1101  }
1102
1103  @Override
1104  public Pair<List<String>, List<TableName>>
1105    getConfiguredNamespacesAndTablesInRSGroup(String groupName) throws IOException {
1106    return get(admin.getConfiguredNamespacesAndTablesInRSGroup(groupName));
1107  }
1108
1109  @Override
1110  public RSGroupInfo getRSGroup(Address hostPort) throws IOException {
1111    return get(admin.getRSGroup(hostPort));
1112  }
1113
1114  @Override
1115  public void removeServersFromRSGroup(Set<Address> servers) throws IOException {
1116    get(admin.removeServersFromRSGroup(servers));
1117  }
1118
1119  @Override
1120  public RSGroupInfo getRSGroup(TableName tableName) throws IOException {
1121    return get(admin.getRSGroup(tableName));
1122  }
1123
1124  @Override
1125  public void setRSGroup(Set<TableName> tables, String groupName) throws IOException {
1126    get(admin.setRSGroup(tables, groupName));
1127  }
1128
1129  @Override
1130  public void renameRSGroup(String oldName, String newName) throws IOException {
1131    get(admin.renameRSGroup(oldName, newName));
1132  }
1133
1134  @Override
1135  public void updateRSGroupConfig(String groupName, Map<String, String> configuration)
1136    throws IOException {
1137    get(admin.updateRSGroupConfig(groupName, configuration));
1138  }
1139
1140  @Override
1141  public List<LogEntry> getLogEntries(Set<ServerName> serverNames, String logType,
1142    ServerType serverType, int limit, Map<String, Object> filterParams) throws IOException {
1143    return get(admin.getLogEntries(serverNames, logType, serverType, limit, filterParams));
1144  }
1145
1146  @Override
1147  public void flushMasterStore() throws IOException {
1148    get(admin.flushMasterStore());
1149  }
1150
1151  @Override
1152  public List<String> getCachedFilesList(ServerName serverName) throws IOException {
1153    return get(admin.getCachedFilesList(serverName));
1154  }
1155
1156  @Override
1157  public void restoreBackupSystemTable(String snapshotName) throws IOException {
1158    get(admin.restoreBackupSystemTable(snapshotName));
1159  }
1160}