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.util.FutureUtils.get;
021
022import java.io.Closeable;
023import java.io.IOException;
024import java.util.Collection;
025import java.util.EnumSet;
026import java.util.HashMap;
027import java.util.List;
028import java.util.Map;
029import java.util.Set;
030import java.util.concurrent.Future;
031import java.util.concurrent.TimeUnit;
032import java.util.regex.Pattern;
033import java.util.stream.Collectors;
034import org.apache.hadoop.conf.Configuration;
035import org.apache.hadoop.hbase.Abortable;
036import org.apache.hadoop.hbase.CacheEvictionStats;
037import org.apache.hadoop.hbase.ClusterMetrics;
038import org.apache.hadoop.hbase.ClusterMetrics.Option;
039import org.apache.hadoop.hbase.NamespaceDescriptor;
040import org.apache.hadoop.hbase.NamespaceNotFoundException;
041import org.apache.hadoop.hbase.RegionMetrics;
042import org.apache.hadoop.hbase.ServerName;
043import org.apache.hadoop.hbase.TableExistsException;
044import org.apache.hadoop.hbase.TableName;
045import org.apache.hadoop.hbase.TableNotFoundException;
046import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
047import org.apache.hadoop.hbase.client.replication.TableCFs;
048import org.apache.hadoop.hbase.client.security.SecurityCapability;
049import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
050import org.apache.hadoop.hbase.net.Address;
051import org.apache.hadoop.hbase.quotas.QuotaFilter;
052import org.apache.hadoop.hbase.quotas.QuotaSettings;
053import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotView;
054import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
055import org.apache.hadoop.hbase.replication.ReplicationException;
056import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
057import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
058import org.apache.hadoop.hbase.replication.SyncReplicationState;
059import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
060import org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest;
061import org.apache.hadoop.hbase.security.access.Permission;
062import org.apache.hadoop.hbase.security.access.UserPermission;
063import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
064import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
065import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
066import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
067import org.apache.hadoop.hbase.util.Bytes;
068import org.apache.hadoop.hbase.util.Pair;
069import org.apache.yetus.audience.InterfaceAudience;
070
071import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
072
073/**
074 * The administrative API for HBase. Obtain an instance from {@link Connection#getAdmin()} and call
075 * {@link #close()} when done.
076 * <p>
077 * Admin can be used to create, drop, list, enable and disable and otherwise modify tables, as well
078 * as perform other administrative operations.
079 * @see ConnectionFactory
080 * @see Connection
081 * @see Table
082 * @since 0.99.0
083 */
084@InterfaceAudience.Public
085public interface Admin extends Abortable, Closeable {
086
087  /**
088   * Return the operation timeout for a rpc call.
089   * @see #getSyncWaitTimeout()
090   */
091  int getOperationTimeout();
092
093  /**
094   * Return the blocking wait time for an asynchronous operation. Can be configured by
095   * {@code hbase.client.sync.wait.timeout.msec}.
096   * <p/>
097   * For several operations, such as createTable, deleteTable, etc, the rpc call will finish right
098   * after we schedule a procedure at master side, so the timeout will not be controlled by the
099   * above {@link #getOperationTimeout()}. And timeout value here tells you how much time we will
100   * wait until the procedure at master side is finished.
101   * <p/>
102   * In general, you can consider that the implementation for XXXX method is just a
103   * XXXXAsync().get(getSyncWaitTimeout(), TimeUnit.MILLISECONDS).
104   * @see #getOperationTimeout()
105   */
106  int getSyncWaitTimeout();
107
108  @Override
109  void abort(String why, Throwable e);
110
111  @Override
112  boolean isAborted();
113
114  /** Returns Connection used by this object. */
115  Connection getConnection();
116
117  /**
118   * Check if a table exists.
119   * @param tableName Table to check.
120   * @return <code>true</code> if table exists already.
121   * @throws IOException if a remote or network exception occurs
122   */
123  boolean tableExists(TableName tableName) throws IOException;
124
125  /**
126   * List all the userspace tables.
127   * @return a list of TableDescriptors
128   * @throws IOException if a remote or network exception occurs
129   */
130  List<TableDescriptor> listTableDescriptors() throws IOException;
131
132  /**
133   * List all userspace tables and whether or not include system tables.
134   * @return a list of TableDescriptors
135   * @throws IOException if a remote or network exception occurs
136   */
137  List<TableDescriptor> listTableDescriptors(boolean includeSysTables) throws IOException;
138
139  /**
140   * List all the userspace tables that match the given pattern.
141   * @param pattern The compiled regular expression to match against
142   * @return a list of TableDescriptors
143   * @throws IOException if a remote or network exception occurs
144   * @see #listTableDescriptors()
145   */
146  default List<TableDescriptor> listTableDescriptors(Pattern pattern) throws IOException {
147    return listTableDescriptors(pattern, false);
148  }
149
150  /**
151   * List all the tables matching the given pattern.
152   * @param pattern          The compiled regular expression to match against
153   * @param includeSysTables <code>false</code> to match only against userspace tables
154   * @return a list of TableDescriptors
155   * @throws IOException if a remote or network exception occurs
156   * @see #listTableDescriptors()
157   */
158  List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables)
159    throws IOException;
160
161  /**
162   * List all enabled or disabled tables
163   * @param isEnabled is true means return enabled tables, false means return disabled tables
164   * @return a list of enabled or disabled tables
165   */
166  List<TableDescriptor> listTableDescriptorsByState(boolean isEnabled) throws IOException;
167
168  /**
169   * List all of the names of userspace tables.
170   * @return TableName[] table names
171   * @throws IOException if a remote or network exception occurs
172   */
173  TableName[] listTableNames() throws IOException;
174
175  /**
176   * List all of the names of userspace tables.
177   * @param pattern The regular expression to match against
178   * @return array of table names
179   * @throws IOException if a remote or network exception occurs
180   */
181  default TableName[] listTableNames(Pattern pattern) throws IOException {
182    return listTableNames(pattern, false);
183  }
184
185  /**
186   * List all of the names of userspace tables.
187   * @param pattern          The regular expression to match against
188   * @param includeSysTables <code>false</code> to match only against userspace tables
189   * @return TableName[] table names
190   * @throws IOException if a remote or network exception occurs
191   */
192  TableName[] listTableNames(Pattern pattern, boolean includeSysTables) throws IOException;
193
194  /**
195   * List all enabled or disabled table names
196   * @param isEnabled is true means return enabled table names, false means return disabled table
197   *                  names
198   * @return a list of enabled or disabled table names
199   */
200  List<TableName> listTableNamesByState(boolean isEnabled) throws IOException;
201
202  /**
203   * Get a table descriptor.
204   * @param tableName as a {@link TableName}
205   * @return the tableDescriptor
206   * @throws TableNotFoundException if the table was not found
207   * @throws IOException            if a remote or network exception occurs
208   */
209  TableDescriptor getDescriptor(TableName tableName) throws TableNotFoundException, IOException;
210
211  /**
212   * Creates a new table. Synchronous operation.
213   * @param desc table descriptor for table
214   * @throws IllegalArgumentException                          if the table name is reserved
215   * @throws org.apache.hadoop.hbase.MasterNotRunningException if master is not running
216   * @throws TableExistsException                              if table already exists (If
217   *                                                           concurrent threads, the table may
218   *                                                           have been created between
219   *                                                           test-for-existence and
220   *                                                           attempt-at-creation).
221   * @throws IOException                                       if a remote or network exception
222   *                                                           occurs
223   */
224  default void createTable(TableDescriptor desc) throws IOException {
225    get(createTableAsync(desc), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
226  }
227
228  /**
229   * Creates a new table with the specified number of regions. The start key specified will become
230   * the end key of the first region of the table, and the end key specified will become the start
231   * key of the last region of the table (the first region has a null start key and the last region
232   * has a null end key). BigInteger math will be used to divide the key range specified into enough
233   * segments to make the required number of total regions. Synchronous operation.
234   * @param desc       table descriptor for table
235   * @param startKey   beginning of key range
236   * @param endKey     end of key range
237   * @param numRegions the total number of regions to create
238   * @throws IOException                                       if a remote or network exception
239   *                                                           occurs
240   * @throws IllegalArgumentException                          if the table name is reserved
241   * @throws org.apache.hadoop.hbase.MasterNotRunningException if master is not running
242   * @throws TableExistsException                              if table already exists (If
243   *                                                           concurrent threads, the table may
244   *                                                           have been created between
245   *                                                           test-for-existence and
246   *                                                           attempt-at-creation).
247   */
248  void createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions)
249    throws IOException;
250
251  /**
252   * Creates a new table with an initial set of empty regions defined by the specified split keys.
253   * The total number of regions created will be the number of split keys plus one. Synchronous
254   * operation. Note : Avoid passing empty split key.
255   * @param desc      table descriptor for table
256   * @param splitKeys array of split keys for the initial regions of the table
257   * @throws IllegalArgumentException                          if the table name is reserved, if the
258   *                                                           split keys are repeated and if the
259   *                                                           split key has empty byte array.
260   * @throws org.apache.hadoop.hbase.MasterNotRunningException if master is not running
261   * @throws TableExistsException                              if table already exists (If
262   *                                                           concurrent threads, the table may
263   *                                                           have been created between
264   *                                                           test-for-existence and
265   *                                                           attempt-at-creation).
266   * @throws IOException                                       if a remote or network exception
267   *                                                           occurs
268   */
269  default void createTable(TableDescriptor desc, byte[][] splitKeys) throws IOException {
270    get(createTableAsync(desc, splitKeys), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
271  }
272
273  /**
274   * Creates a new table but does not block and wait for it to come online. You can use
275   * Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
276   * ExecutionException if there was an error while executing the operation or TimeoutException in
277   * case the wait timeout was not long enough to allow the operation to complete.
278   * <p/>
279   * Throws IllegalArgumentException Bad table name, if the split keys are repeated and if the split
280   * key has empty byte array.
281   * @param desc table descriptor for table
282   * @throws IOException if a remote or network exception occurs
283   * @return the result of the async creation. You can use Future.get(long, TimeUnit) to wait on the
284   *         operation to complete.
285   */
286  Future<Void> createTableAsync(TableDescriptor desc) throws IOException;
287
288  /**
289   * Creates a new table but does not block and wait for it to come online. You can use
290   * Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
291   * ExecutionException if there was an error while executing the operation or TimeoutException in
292   * case the wait timeout was not long enough to allow the operation to complete. Throws
293   * IllegalArgumentException Bad table name, if the split keys are repeated and if the split key
294   * has empty byte array.
295   * @param desc      table descriptor for table
296   * @param splitKeys keys to check if the table has been created with all split keys
297   * @throws IOException if a remote or network exception occurs
298   * @return the result of the async creation. You can use Future.get(long, TimeUnit) to wait on the
299   *         operation to complete.
300   */
301  Future<Void> createTableAsync(TableDescriptor desc, byte[][] splitKeys) throws IOException;
302
303  /**
304   * Deletes a table. Synchronous operation.
305   * @param tableName name of table to delete
306   * @throws IOException if a remote or network exception occurs
307   */
308  default void deleteTable(TableName tableName) throws IOException {
309    get(deleteTableAsync(tableName), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
310  }
311
312  /**
313   * Deletes the table but does not block and wait for it to be completely removed. You can use
314   * Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
315   * ExecutionException if there was an error while executing the operation or TimeoutException in
316   * case the wait timeout was not long enough to allow the operation to complete.
317   * @param tableName name of table to delete
318   * @throws IOException if a remote or network exception occurs
319   * @return the result of the async delete. You can use Future.get(long, TimeUnit) to wait on the
320   *         operation to complete.
321   */
322  Future<Void> deleteTableAsync(TableName tableName) throws IOException;
323
324  /**
325   * Truncate a table. Synchronous operation.
326   * @param tableName      name of table to truncate
327   * @param preserveSplits <code>true</code> if the splits should be preserved
328   * @throws IOException if a remote or network exception occurs
329   */
330  default void truncateTable(TableName tableName, boolean preserveSplits) throws IOException {
331    get(truncateTableAsync(tableName, preserveSplits), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
332  }
333
334  /**
335   * Truncate the table but does not block and wait for it to be completely enabled. You can use
336   * Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
337   * ExecutionException if there was an error while executing the operation or TimeoutException in
338   * case the wait timeout was not long enough to allow the operation to complete.
339   * @param tableName      name of table to delete
340   * @param preserveSplits <code>true</code> if the splits should be preserved
341   * @throws IOException if a remote or network exception occurs
342   * @return the result of the async truncate. You can use Future.get(long, TimeUnit) to wait on the
343   *         operation to complete.
344   */
345  Future<Void> truncateTableAsync(TableName tableName, boolean preserveSplits) throws IOException;
346
347  /**
348   * Enable a table. May timeout. Use {@link #enableTableAsync(org.apache.hadoop.hbase.TableName)}
349   * and {@link #isTableEnabled(org.apache.hadoop.hbase.TableName)} instead. The table has to be in
350   * disabled state for it to be enabled.
351   * @param tableName name of the table
352   * @throws IOException There could be couple types of IOException TableNotFoundException means the
353   *                     table doesn't exist. TableNotDisabledException means the table isn't in
354   *                     disabled state.
355   * @see #isTableEnabled(org.apache.hadoop.hbase.TableName)
356   * @see #disableTable(org.apache.hadoop.hbase.TableName)
357   * @see #enableTableAsync(org.apache.hadoop.hbase.TableName)
358   */
359  default void enableTable(TableName tableName) throws IOException {
360    get(enableTableAsync(tableName), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
361  }
362
363  /**
364   * Enable the table but does not block and wait for it to be completely enabled. You can use
365   * Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
366   * ExecutionException if there was an error while executing the operation or TimeoutException in
367   * case the wait timeout was not long enough to allow the operation to complete.
368   * @param tableName name of table to delete
369   * @throws IOException if a remote or network exception occurs
370   * @return the result of the async enable. You can use Future.get(long, TimeUnit) to wait on the
371   *         operation to complete.
372   */
373  Future<Void> enableTableAsync(TableName tableName) throws IOException;
374
375  /**
376   * Disable the table but does not block and wait for it to be completely disabled. You can use
377   * Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
378   * ExecutionException if there was an error while executing the operation or TimeoutException in
379   * case the wait timeout was not long enough to allow the operation to complete.
380   * @param tableName name of table to delete
381   * @throws IOException if a remote or network exception occurs
382   * @return the result of the async disable. You can use Future.get(long, TimeUnit) to wait on the
383   *         operation to complete.
384   */
385  Future<Void> disableTableAsync(TableName tableName) throws IOException;
386
387  /**
388   * Disable table and wait on completion. May timeout eventually. Use
389   * {@link #disableTableAsync(org.apache.hadoop.hbase.TableName)} and
390   * {@link #isTableDisabled(org.apache.hadoop.hbase.TableName)} instead. The table has to be in
391   * enabled state for it to be disabled.
392   * @throws IOException There could be couple types of IOException TableNotFoundException means the
393   *                     table doesn't exist. TableNotEnabledException means the table isn't in
394   *                     enabled state.
395   */
396  default void disableTable(TableName tableName) throws IOException {
397    get(disableTableAsync(tableName), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
398  }
399
400  /**
401   * Check if a table is enabled.
402   * @param tableName name of table to check
403   * @return <code>true</code> if table is on-line
404   * @throws IOException if a remote or network exception occurs
405   */
406  boolean isTableEnabled(TableName tableName) throws IOException;
407
408  /**
409   * Check if a table is disabled.
410   * @param tableName name of table to check
411   * @return <code>true</code> if table is off-line
412   * @throws IOException if a remote or network exception occurs
413   */
414  boolean isTableDisabled(TableName tableName) throws IOException;
415
416  /**
417   * Check if a table is available.
418   * @param tableName name of table to check
419   * @return <code>true</code> if all regions of the table are available
420   * @throws IOException if a remote or network exception occurs
421   */
422  boolean isTableAvailable(TableName tableName) throws IOException;
423
424  /**
425   * Add a column family to an existing table. Synchronous operation. Use
426   * {@link #addColumnFamilyAsync(TableName, ColumnFamilyDescriptor)} instead because it returns a
427   * {@link Future} from which you can learn whether success or failure.
428   * @param tableName    name of the table to add column family to
429   * @param columnFamily column family descriptor of column family to be added
430   * @throws IOException if a remote or network exception occurs
431   */
432  default void addColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily)
433    throws IOException {
434    get(addColumnFamilyAsync(tableName, columnFamily), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
435  }
436
437  /**
438   * Add a column family to an existing table. Asynchronous operation. You can use Future.get(long,
439   * TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an
440   * error while executing the operation or TimeoutException in case the wait timeout was not long
441   * enough to allow the operation to complete.
442   * @param tableName    name of the table to add column family to
443   * @param columnFamily column family descriptor of column family to be added
444   * @throws IOException if a remote or network exception occurs
445   * @return the result of the async add column family. You can use Future.get(long, TimeUnit) to
446   *         wait on the operation to complete.
447   */
448  Future<Void> addColumnFamilyAsync(TableName tableName, ColumnFamilyDescriptor columnFamily)
449    throws IOException;
450
451  /**
452   * Delete a column family from a table. Synchronous operation. Use
453   * {@link #deleteColumnFamily(TableName, byte[])} instead because it returns a {@link Future} from
454   * which you can learn whether success or failure.
455   * @param tableName    name of table
456   * @param columnFamily name of column family to be deleted
457   * @throws IOException if a remote or network exception occurs
458   */
459  default void deleteColumnFamily(TableName tableName, byte[] columnFamily) throws IOException {
460    get(deleteColumnFamilyAsync(tableName, columnFamily), getSyncWaitTimeout(),
461      TimeUnit.MILLISECONDS);
462  }
463
464  /**
465   * Delete a column family from a table. Asynchronous operation. You can use Future.get(long,
466   * TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an
467   * error while executing the operation or TimeoutException in case the wait timeout was not long
468   * enough to allow the operation to complete.
469   * @param tableName    name of table
470   * @param columnFamily name of column family to be deleted
471   * @throws IOException if a remote or network exception occurs
472   * @return the result of the async delete column family. You can use Future.get(long, TimeUnit) to
473   *         wait on the operation to complete.
474   */
475  Future<Void> deleteColumnFamilyAsync(TableName tableName, byte[] columnFamily) throws IOException;
476
477  /**
478   * Modify an existing column family on a table. Synchronous operation. Use
479   * {@link #modifyColumnFamilyAsync(TableName, ColumnFamilyDescriptor)} instead because it returns
480   * a {@link Future} from which you can learn whether success or failure.
481   * @param tableName    name of table
482   * @param columnFamily new column family descriptor to use
483   * @throws IOException if a remote or network exception occurs
484   */
485  default void modifyColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily)
486    throws IOException {
487    get(modifyColumnFamilyAsync(tableName, columnFamily), getSyncWaitTimeout(),
488      TimeUnit.MILLISECONDS);
489  }
490
491  /**
492   * Modify an existing column family on a table. Asynchronous operation. You can use
493   * Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
494   * ExecutionException if there was an error while executing the operation or TimeoutException in
495   * case the wait timeout was not long enough to allow the operation to complete.
496   * @param tableName    name of table
497   * @param columnFamily new column family descriptor to use
498   * @throws IOException if a remote or network exception occurs
499   * @return the result of the async modify column family. You can use Future.get(long, TimeUnit) to
500   *         wait on the operation to complete.
501   */
502  Future<Void> modifyColumnFamilyAsync(TableName tableName, ColumnFamilyDescriptor columnFamily)
503    throws IOException;
504
505  /**
506   * Change the store file tracker of the given table's given family.
507   * @param tableName the table you want to change
508   * @param family    the family you want to change
509   * @param dstSFT    the destination store file tracker
510   * @throws IOException if a remote or network exception occurs
511   */
512  default void modifyColumnFamilyStoreFileTracker(TableName tableName, byte[] family, String dstSFT)
513    throws IOException {
514    get(modifyColumnFamilyStoreFileTrackerAsync(tableName, family, dstSFT), getSyncWaitTimeout(),
515      TimeUnit.MILLISECONDS);
516  }
517
518  /**
519   * Change the store file tracker of the given table's given family.
520   * @param tableName the table you want to change
521   * @param family    the family you want to change
522   * @param dstSFT    the destination store file tracker
523   * @return the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the
524   *         operation to complete
525   * @throws IOException if a remote or network exception occurs
526   */
527  Future<Void> modifyColumnFamilyStoreFileTrackerAsync(TableName tableName, byte[] family,
528    String dstSFT) throws IOException;
529
530  /**
531   * Get all the online regions on a region server.
532   * @return List of {@link RegionInfo}
533   * @throws IOException if a remote or network exception occurs
534   */
535  List<RegionInfo> getRegions(ServerName serverName) throws IOException;
536
537  /**
538   * Flush a table. Synchronous operation.
539   * @param tableName table to flush
540   * @throws IOException if a remote or network exception occurs
541   */
542  void flush(TableName tableName) throws IOException;
543
544  /**
545   * Flush the specified column family stores on all regions of the passed table. This runs as a
546   * synchronous operation.
547   * @param tableName    table to flush
548   * @param columnFamily column family within a table
549   * @throws IOException if a remote or network exception occurs
550   */
551  void flush(TableName tableName, byte[] columnFamily) throws IOException;
552
553  /**
554   * Flush the specified column family stores on all regions of the passed table. This runs as a
555   * synchronous operation.
556   * @param tableName      table to flush
557   * @param columnFamilies column families within a table
558   * @throws IOException if a remote or network exception occurs
559   */
560  void flush(TableName tableName, List<byte[]> columnFamilies) throws IOException;
561
562  /**
563   * Flush an individual region. Synchronous operation.
564   * @param regionName region to flush
565   * @throws IOException if a remote or network exception occurs
566   */
567  void flushRegion(byte[] regionName) throws IOException;
568
569  /**
570   * Flush a column family within a region. Synchronous operation.
571   * @param regionName   region to flush
572   * @param columnFamily column family within a region
573   * @throws IOException if a remote or network exception occurs
574   */
575  void flushRegion(byte[] regionName, byte[] columnFamily) throws IOException;
576
577  /**
578   * Flush all regions on the region server. Synchronous operation.
579   * @param serverName the region server name to flush
580   * @throws IOException if a remote or network exception occurs
581   */
582  void flushRegionServer(ServerName serverName) throws IOException;
583
584  /**
585   * Compact a table. Asynchronous operation in that this method requests that a Compaction run and
586   * then it returns. It does not wait on the completion of Compaction (it can take a while).
587   * @param tableName table to compact
588   * @throws IOException if a remote or network exception occurs
589   */
590  void compact(TableName tableName) throws IOException;
591
592  /**
593   * Compact an individual region. Asynchronous operation in that this method requests that a
594   * Compaction run and then it returns. It does not wait on the completion of Compaction (it can
595   * take a while).
596   * @param regionName region to compact
597   * @throws IOException if a remote or network exception occurs
598   */
599  void compactRegion(byte[] regionName) throws IOException;
600
601  /**
602   * Compact a column family within a table. Asynchronous operation in that this method requests
603   * that a Compaction run and then it returns. It does not wait on the completion of Compaction (it
604   * can take a while).
605   * @param tableName    table to compact
606   * @param columnFamily column family within a table
607   * @throws IOException if a remote or network exception occurs
608   */
609  void compact(TableName tableName, byte[] columnFamily) throws IOException;
610
611  /**
612   * Compact a column family within a region. Asynchronous operation in that this method requests
613   * that a Compaction run and then it returns. It does not wait on the completion of Compaction (it
614   * can take a while).
615   * @param regionName   region to compact
616   * @param columnFamily column family within a region
617   * @throws IOException if a remote or network exception occurs
618   */
619  void compactRegion(byte[] regionName, byte[] columnFamily) throws IOException;
620
621  /**
622   * Compact a table. Asynchronous operation in that this method requests that a Compaction run and
623   * then it returns. It does not wait on the completion of Compaction (it can take a while).
624   * @param tableName   table to compact
625   * @param compactType {@link org.apache.hadoop.hbase.client.CompactType}
626   * @throws IOException if a remote or network exception occurs
627   */
628  void compact(TableName tableName, CompactType compactType)
629    throws IOException, InterruptedException;
630
631  /**
632   * Compact a column family within a table. Asynchronous operation in that this method requests
633   * that a Compaction run and then it returns. It does not wait on the completion of Compaction (it
634   * can take a while).
635   * @param tableName    table to compact
636   * @param columnFamily column family within a table
637   * @param compactType  {@link org.apache.hadoop.hbase.client.CompactType}
638   * @throws IOException if not a mob column family or if a remote or network exception occurs
639   */
640  void compact(TableName tableName, byte[] columnFamily, CompactType compactType)
641    throws IOException, InterruptedException;
642
643  /**
644   * Major compact a table. Asynchronous operation in that this method requests that a Compaction
645   * run and then it returns. It does not wait on the completion of Compaction (it can take a
646   * while).
647   * @param tableName table to major compact
648   * @throws IOException if a remote or network exception occurs
649   */
650  void majorCompact(TableName tableName) throws IOException;
651
652  /**
653   * Major compact a table or an individual region. Asynchronous operation in that this method
654   * requests that a Compaction run and then it returns. It does not wait on the completion of
655   * Compaction (it can take a while).
656   * @param regionName region to major compact
657   * @throws IOException if a remote or network exception occurs
658   */
659  void majorCompactRegion(byte[] regionName) throws IOException;
660
661  /**
662   * Major compact a column family within a table. Asynchronous operation in that this method
663   * requests that a Compaction run and then it returns. It does not wait on the completion of
664   * Compaction (it can take a while).
665   * @param tableName    table to major compact
666   * @param columnFamily column family within a table
667   * @throws IOException if a remote or network exception occurs
668   */
669  void majorCompact(TableName tableName, byte[] columnFamily) throws IOException;
670
671  /**
672   * Major compact a column family within region. Asynchronous operation in that this method
673   * requests that a Compaction run and then it returns. It does not wait on the completion of
674   * Compaction (it can take a while).
675   * @param regionName   egion to major compact
676   * @param columnFamily column family within a region
677   * @throws IOException if a remote or network exception occurs
678   */
679  void majorCompactRegion(byte[] regionName, byte[] columnFamily) throws IOException;
680
681  /**
682   * Major compact a table. Asynchronous operation in that this method requests that a Compaction
683   * run and then it returns. It does not wait on the completion of Compaction (it can take a
684   * while).
685   * @param tableName   table to compact
686   * @param compactType {@link org.apache.hadoop.hbase.client.CompactType}
687   * @throws IOException if a remote or network exception occurs
688   */
689  void majorCompact(TableName tableName, CompactType compactType)
690    throws IOException, InterruptedException;
691
692  /**
693   * Major compact a column family within a table. Asynchronous operation in that this method
694   * requests that a Compaction run and then it returns. It does not wait on the completion of
695   * Compaction (it can take a while).
696   * @param tableName    table to compact
697   * @param columnFamily column family within a table
698   * @param compactType  {@link org.apache.hadoop.hbase.client.CompactType}
699   * @throws IOException if not a mob column family or if a remote or network exception occurs
700   */
701  void majorCompact(TableName tableName, byte[] columnFamily, CompactType compactType)
702    throws IOException, InterruptedException;
703
704  /**
705   * Turn the compaction on or off. Disabling compactions will also interrupt any currently ongoing
706   * compactions. This state is ephemeral. The setting will be lost on restart. Compaction can also
707   * be enabled/disabled by modifying configuration hbase.regionserver.compaction.enabled in
708   * hbase-site.xml.
709   * @param switchState     Set to <code>true</code> to enable, <code>false</code> to disable.
710   * @param serverNamesList list of region servers.
711   * @return Previous compaction states for region servers
712   * @throws IOException if a remote or network exception occurs
713   */
714  Map<ServerName, Boolean> compactionSwitch(boolean switchState, List<String> serverNamesList)
715    throws IOException;
716
717  /**
718   * Compact all regions on the region server. Asynchronous operation in that this method requests
719   * that a Compaction run and then it returns. It does not wait on the completion of Compaction (it
720   * can take a while).
721   * @param serverName the region server name
722   * @throws IOException if a remote or network exception occurs
723   */
724  void compactRegionServer(ServerName serverName) throws IOException;
725
726  /**
727   * Major compact all regions on the region server. Asynchronous operation in that this method
728   * requests that a Compaction run and then it returns. It does not wait on the completion of
729   * Compaction (it can take a while).
730   * @param serverName the region server name
731   * @throws IOException if a remote or network exception occurs
732   */
733  void majorCompactRegionServer(ServerName serverName) throws IOException;
734
735  /**
736   * Move the region <code>encodedRegionName</code> to a random server.
737   * @param encodedRegionName The encoded region name; i.e. the hash that makes up the region name
738   *                          suffix: e.g. if regionname is
739   *                          <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>,
740   *                          then the encoded region name is:
741   *                          <code>527db22f95c8a9e0116f0cc13c680396</code>.
742   * @throws IOException if we can't find a region named <code>encodedRegionName</code>
743   */
744  void move(byte[] encodedRegionName) throws IOException;
745
746  /**
747   * Move the region <code>rencodedRegionName</code> to <code>destServerName</code>.
748   * @param encodedRegionName The encoded region name; i.e. the hash that makes up the region name
749   *                          suffix: e.g. if regionname is
750   *                          <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>,
751   *                          then the encoded region name is:
752   *                          <code>527db22f95c8a9e0116f0cc13c680396</code>.
753   * @param destServerName    The servername of the destination regionserver. If passed the empty
754   *                          byte array we'll assign to a random server. A server name is made of
755   *                          host, port and startcode. Here is an example:
756   *                          <code> host187.example.com,60020,1289493121758</code>
757   * @throws IOException if we can't find a region named <code>encodedRegionName</code>
758   * @deprecated since 2.2.0 and will be removed in 4.0.0. Use {@link #move(byte[], ServerName)}
759   *             instead. And if you want to move the region to a random server, please use
760   *             {@link #move(byte[])}.
761   * @see <a href="https://issues.apache.org/jira/browse/HBASE-22108">HBASE-22108</a>
762   */
763  @Deprecated
764  default void move(byte[] encodedRegionName, byte[] destServerName) throws IOException {
765    if (destServerName == null || destServerName.length == 0) {
766      move(encodedRegionName);
767    } else {
768      move(encodedRegionName, ServerName.valueOf(Bytes.toString(destServerName)));
769    }
770  }
771
772  /**
773   * Move the region <code>encodedRegionName</code> to <code>destServerName</code>.
774   * @param encodedRegionName The encoded region name; i.e. the hash that makes up the region name
775   *                          suffix: e.g. if regionname is
776   *                          <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>,
777   *                          then the encoded region name is:
778   *                          <code>527db22f95c8a9e0116f0cc13c680396</code>.
779   * @param destServerName    The servername of the destination regionserver. A server name is made
780   *                          of host, port and startcode. Here is an example:
781   *                          <code> host187.example.com,60020,1289493121758</code>
782   * @throws IOException if we can't find a region named <code>encodedRegionName</code>
783   */
784  void move(byte[] encodedRegionName, ServerName destServerName) throws IOException;
785
786  /**
787   * Assign a Region.
788   * @param regionName Region name to assign.
789   * @throws IOException if a remote or network exception occurs
790   */
791  void assign(byte[] regionName) throws IOException;
792
793  /**
794   * Unassign a Region.
795   * @param regionName Region name to unassign.
796   * @throws IOException if a remote or network exception occurs
797   */
798  void unassign(byte[] regionName) throws IOException;
799
800  /**
801   * Unassign a region from current hosting regionserver. Region will then be assigned to a
802   * regionserver chosen at random. Region could be reassigned back to the same server. Use
803   * {@link #move(byte[], ServerName)} if you want to control the region movement.
804   * @param regionName Region to unassign. Will clear any existing RegionPlan if one found.
805   * @param force      If <code>true</code>, force unassign (Will remove region from
806   *                   regions-in-transition too if present. If results in double assignment use
807   *                   hbck -fix to resolve. To be used by experts).
808   * @throws IOException if a remote or network exception occurs
809   * @deprecated since 2.4.0 and will be removed in 4.0.0. Use {@link #unassign(byte[])} instead.
810   * @see <a href="https://issues.apache.org/jira/browse/HBASE-24875">HBASE-24875</a>
811   */
812  @Deprecated
813  default void unassign(byte[] regionName, boolean force) throws IOException {
814    unassign(regionName);
815  }
816
817  /**
818   * Offline specified region from master's in-memory state. It will not attempt to reassign the
819   * region as in unassign. This API can be used when a region not served by any region server and
820   * still online as per Master's in memory state. If this API is incorrectly used on active region
821   * then master will loose track of that region. This is a special method that should be used by
822   * experts or hbck.
823   * @param regionName Region to offline.
824   * @throws IOException if a remote or network exception occurs
825   */
826  void offline(byte[] regionName) throws IOException;
827
828  /**
829   * Turn the load balancer on or off.
830   * @param onOrOff     Set to <code>true</code> to enable, <code>false</code> to disable.
831   * @param synchronous If <code>true</code>, it waits until current balance() call, if outstanding,
832   *                    to return.
833   * @return Previous balancer value
834   * @throws IOException if a remote or network exception occurs
835   */
836  boolean balancerSwitch(boolean onOrOff, boolean synchronous) throws IOException;
837
838  /**
839   * Invoke the balancer. Will run the balancer and if regions to move, it will go ahead and do the
840   * reassignments. Can NOT run for various reasons. Check logs.
841   * @return <code>true</code> if balancer ran, <code>false</code> otherwise.
842   * @throws IOException if a remote or network exception occurs
843   */
844  default boolean balance() throws IOException {
845    return balance(BalanceRequest.defaultInstance()).isBalancerRan();
846  }
847
848  /**
849   * Invoke the balancer with the given balance request. The BalanceRequest defines how the balancer
850   * will run. See {@link BalanceRequest} for more details.
851   * @param request defines how the balancer should run
852   * @return {@link BalanceResponse} with details about the results of the invocation.
853   * @throws IOException if a remote or network exception occurs
854   */
855  BalanceResponse balance(BalanceRequest request) throws IOException;
856
857  /**
858   * Invoke the balancer. Will run the balancer and if regions to move, it will go ahead and do the
859   * reassignments. If there is region in transition, force parameter of true would still run
860   * balancer. Can *not* run for other reasons. Check logs.
861   * @param force whether we should force balance even if there is region in transition
862   * @return <code>true</code> if balancer ran, <code>false</code> otherwise.
863   * @throws IOException if a remote or network exception occurs
864   * @deprecated Since 2.5.0. Will be removed in 4.0.0. Use {@link #balance(BalanceRequest)}
865   *             instead.
866   */
867  @Deprecated
868  default boolean balance(boolean force) throws IOException {
869    return balance(BalanceRequest.newBuilder().setIgnoreRegionsInTransition(force).build())
870      .isBalancerRan();
871  }
872
873  /**
874   * Query the current state of the balancer.
875   * @return <code>true</code> if the balancer is enabled, <code>false</code> otherwise.
876   * @throws IOException if a remote or network exception occurs
877   */
878  boolean isBalancerEnabled() throws IOException;
879
880  /**
881   * Clear all the blocks corresponding to this table from BlockCache. For expert-admins. Calling
882   * this API will drop all the cached blocks specific to a table from BlockCache. This can
883   * significantly impact the query performance as the subsequent queries will have to retrieve the
884   * blocks from underlying filesystem.
885   * @param tableName table to clear block cache
886   * @return CacheEvictionStats related to the eviction
887   * @throws IOException if a remote or network exception occurs
888   */
889  CacheEvictionStats clearBlockCache(final TableName tableName) throws IOException;
890
891  /**
892   * Invoke region normalizer. Can NOT run for various reasons. Check logs. This is a non-blocking
893   * invocation to region normalizer. If return value is true, it means the request was submitted
894   * successfully. We need to check logs for the details of which regions were split/merged.
895   * @return {@code true} if region normalizer ran, {@code false} otherwise.
896   * @throws IOException if a remote or network exception occurs
897   */
898  default boolean normalize() throws IOException {
899    return normalize(new NormalizeTableFilterParams.Builder().build());
900  }
901
902  /**
903   * Invoke region normalizer. Can NOT run for various reasons. Check logs. This is a non-blocking
904   * invocation to region normalizer. If return value is true, it means the request was submitted
905   * successfully. We need to check logs for the details of which regions were split/merged.
906   * @param ntfp limit to tables matching the specified filter.
907   * @return {@code true} if region normalizer ran, {@code false} otherwise.
908   * @throws IOException if a remote or network exception occurs
909   */
910  boolean normalize(NormalizeTableFilterParams ntfp) throws IOException;
911
912  /**
913   * Query the current state of the region normalizer.
914   * @return <code>true</code> if region normalizer is enabled, <code>false</code> otherwise.
915   * @throws IOException if a remote or network exception occurs
916   */
917  boolean isNormalizerEnabled() throws IOException;
918
919  /**
920   * Turn region normalizer on or off.
921   * @return Previous normalizer value
922   * @throws IOException if a remote or network exception occurs
923   */
924  boolean normalizerSwitch(boolean on) throws IOException;
925
926  /**
927   * Enable/Disable the catalog janitor/
928   * @param onOrOff if <code>true</code> enables the catalog janitor
929   * @return the previous state
930   * @throws IOException if a remote or network exception occurs
931   */
932  boolean catalogJanitorSwitch(boolean onOrOff) throws IOException;
933
934  /**
935   * Ask for a scan of the catalog table.
936   * @return the number of entries cleaned. Returns -1 if previous run is in progress.
937   * @throws IOException if a remote or network exception occurs
938   */
939  int runCatalogJanitor() throws IOException;
940
941  /**
942   * Query on the catalog janitor state (Enabled/Disabled?).
943   * @throws IOException if a remote or network exception occurs
944   */
945  boolean isCatalogJanitorEnabled() throws IOException;
946
947  /**
948   * Enable/Disable the cleaner chore.
949   * @param onOrOff if <code>true</code> enables the cleaner chore
950   * @return the previous state
951   * @throws IOException if a remote or network exception occurs
952   */
953  boolean cleanerChoreSwitch(boolean onOrOff) throws IOException;
954
955  /**
956   * Ask for cleaner chore to run.
957   * @return <code>true</code> if cleaner chore ran, <code>false</code> otherwise
958   * @throws IOException if a remote or network exception occurs
959   */
960  boolean runCleanerChore() throws IOException;
961
962  /**
963   * Query on the cleaner chore state (Enabled/Disabled?).
964   * @throws IOException if a remote or network exception occurs
965   */
966  boolean isCleanerChoreEnabled() throws IOException;
967
968  /**
969   * Merge two regions. Asynchronous operation.
970   * @param nameOfRegionA encoded or full name of region a
971   * @param nameOfRegionB encoded or full name of region b
972   * @param forcible      <code>true</code> if do a compulsory merge, otherwise we will only merge
973   *                      two adjacent regions
974   * @throws IOException if a remote or network exception occurs
975   * @deprecated since 2.3.0 and will be removed in 4.0.0. Multi-region merge feature is now
976   *             supported. Use {@link #mergeRegionsAsync(byte[][], boolean)} instead.
977   */
978  @Deprecated
979  default Future<Void> mergeRegionsAsync(byte[] nameOfRegionA, byte[] nameOfRegionB,
980    boolean forcible) throws IOException {
981    byte[][] nameofRegionsToMerge = new byte[2][];
982    nameofRegionsToMerge[0] = nameOfRegionA;
983    nameofRegionsToMerge[1] = nameOfRegionB;
984    return mergeRegionsAsync(nameofRegionsToMerge, forcible);
985  }
986
987  /**
988   * Merge multiple regions (>=2). Asynchronous operation.
989   * @param nameofRegionsToMerge encoded or full name of daughter regions
990   * @param forcible             <code>true</code> if do a compulsory merge, otherwise we will only
991   *                             merge adjacent regions
992   * @throws IOException if a remote or network exception occurs
993   */
994  Future<Void> mergeRegionsAsync(byte[][] nameofRegionsToMerge, boolean forcible)
995    throws IOException;
996
997  /**
998   * Split a table. The method will execute split action for each region in table.
999   * @param tableName table to split
1000   * @throws IOException if a remote or network exception occurs
1001   */
1002  void split(TableName tableName) throws IOException;
1003
1004  /**
1005   * Split a table.
1006   * @param tableName  table to split
1007   * @param splitPoint the explicit position to split on
1008   * @throws IOException if a remote or network exception occurs
1009   */
1010  void split(TableName tableName, byte[] splitPoint) throws IOException;
1011
1012  /**
1013   * Split an individual region. Asynchronous operation.
1014   * @param regionName region to split
1015   * @throws IOException if a remote or network exception occurs
1016   */
1017  Future<Void> splitRegionAsync(byte[] regionName) throws IOException;
1018
1019  /**
1020   * Split an individual region. Asynchronous operation.
1021   * @param regionName region to split
1022   * @param splitPoint the explicit position to split on
1023   * @throws IOException if a remote or network exception occurs
1024   */
1025  Future<Void> splitRegionAsync(byte[] regionName, byte[] splitPoint) throws IOException;
1026
1027  /**
1028   * Modify an existing table, more IRB friendly version.
1029   * @param td modified description of the table
1030   * @throws IOException if a remote or network exception occurs
1031   */
1032  default void modifyTable(TableDescriptor td) throws IOException {
1033    get(modifyTableAsync(td), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
1034  }
1035
1036  /**
1037   * Truncate an individual region.
1038   * @param regionName region to truncate
1039   * @throws IOException if a remote or network exception occurs
1040   */
1041  void truncateRegion(byte[] regionName) throws IOException;
1042
1043  /**
1044   * Truncate an individual region. Asynchronous operation.
1045   * @param regionName region to truncate
1046   * @throws IOException if a remote or network exception occurs
1047   */
1048  Future<Void> truncateRegionAsync(byte[] regionName) throws IOException;
1049
1050  /**
1051   * Modify an existing table, more IRB (ruby) friendly version. Asynchronous operation. This means
1052   * that it may be a while before your schema change is updated across all of the table. You can
1053   * use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
1054   * ExecutionException if there was an error while executing the operation or TimeoutException in
1055   * case the wait timeout was not long enough to allow the operation to complete.
1056   * @param td description of the table
1057   * @throws IOException if a remote or network exception occurs
1058   * @return the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the
1059   *         operation to complete
1060   */
1061  default Future<Void> modifyTableAsync(TableDescriptor td) throws IOException {
1062    return modifyTableAsync(td, true);
1063  }
1064
1065  /**
1066   * The same as {@link #modifyTableAsync(TableDescriptor td)}, except for the reopenRegions
1067   * parameter, which controls whether the process of modifying the table should reopen all regions.
1068   * @param td            description of the table
1069   * @param reopenRegions By default, 'modifyTable' reopens all regions, potentially causing a RIT
1070   *                      (Region In Transition) storm in large tables. If set to 'false', regions
1071   *                      will remain unaware of the modification until they are individually
1072   *                      reopened. Please note that this may temporarily result in configuration
1073   *                      inconsistencies among regions.
1074   * @return the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the
1075   *         operation to complete
1076   * @throws IOException if a remote or network exception occurs
1077   */
1078  Future<Void> modifyTableAsync(TableDescriptor td, boolean reopenRegions) throws IOException;
1079
1080  /**
1081   * Change the store file tracker of the given table.
1082   * @param tableName the table you want to change
1083   * @param dstSFT    the destination store file tracker
1084   * @throws IOException if a remote or network exception occurs
1085   */
1086  default void modifyTableStoreFileTracker(TableName tableName, String dstSFT) throws IOException {
1087    get(modifyTableStoreFileTrackerAsync(tableName, dstSFT), getSyncWaitTimeout(),
1088      TimeUnit.MILLISECONDS);
1089  }
1090
1091  /**
1092   * Change the store file tracker of the given table.
1093   * @param tableName the table you want to change
1094   * @param dstSFT    the destination store file tracker
1095   * @return the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the
1096   *         operation to complete
1097   * @throws IOException if a remote or network exception occurs
1098   */
1099  Future<Void> modifyTableStoreFileTrackerAsync(TableName tableName, String dstSFT)
1100    throws IOException;
1101
1102  /**
1103   * Shuts down the HBase cluster.
1104   * <p/>
1105   * Notice that, a success shutdown call may ends with an error since the remote server has already
1106   * been shutdown.
1107   * @throws IOException if a remote or network exception occurs
1108   */
1109  void shutdown() throws IOException;
1110
1111  /**
1112   * Shuts down the current HBase master only. Does not shutdown the cluster.
1113   * <p/>
1114   * Notice that, a success stopMaster call may ends with an error since the remote server has
1115   * already been shutdown.
1116   * @throws IOException if a remote or network exception occurs
1117   * @see #shutdown()
1118   */
1119  void stopMaster() throws IOException;
1120
1121  /**
1122   * Check whether Master is in maintenance mode.
1123   * @throws IOException if a remote or network exception occurs
1124   */
1125  boolean isMasterInMaintenanceMode() throws IOException;
1126
1127  /**
1128   * Stop the designated regionserver.
1129   * @param hostnamePort Hostname and port delimited by a <code>:</code> as in
1130   *                     <code>example.org:1234</code>
1131   * @throws IOException if a remote or network exception occurs
1132   */
1133  void stopRegionServer(String hostnamePort) throws IOException;
1134
1135  /**
1136   * Get whole cluster metrics, containing status about:
1137   *
1138   * <pre>
1139   * hbase version
1140   * cluster id
1141   * primary/backup master(s)
1142   * master's coprocessors
1143   * live/dead regionservers
1144   * balancer
1145   * regions in transition
1146   * </pre>
1147   *
1148   * @return cluster metrics
1149   * @throws IOException if a remote or network exception occurs
1150   */
1151  default ClusterMetrics getClusterMetrics() throws IOException {
1152    return getClusterMetrics(EnumSet.allOf(ClusterMetrics.Option.class));
1153  }
1154
1155  /**
1156   * Get cluster status with a set of {@link Option} to get desired status.
1157   * @return cluster status
1158   * @throws IOException if a remote or network exception occurs
1159   */
1160  ClusterMetrics getClusterMetrics(EnumSet<Option> options) throws IOException;
1161
1162  /**
1163   * Get the current active master.
1164   * @return current master server name
1165   * @throws IOException if a remote or network exception occurs
1166   */
1167  default ServerName getMaster() throws IOException {
1168    return getClusterMetrics(EnumSet.of(Option.MASTER)).getMasterName();
1169  }
1170
1171  /**
1172   * Get a list of current backup masters.
1173   * @return current backup master list
1174   * @throws IOException if a remote or network exception occurs
1175   */
1176  default Collection<ServerName> getBackupMasters() throws IOException {
1177    return getClusterMetrics(EnumSet.of(Option.BACKUP_MASTERS)).getBackupMasterNames();
1178  }
1179
1180  /**
1181   * Get the live server list.
1182   * @return current live region servers list
1183   * @throws IOException if a remote or network exception occurs
1184   */
1185  default Collection<ServerName> getRegionServers() throws IOException {
1186    return getRegionServers(false);
1187  }
1188
1189  /**
1190   * Retrieve all current live region servers including decommissioned if excludeDecommissionedRS is
1191   * false, else non-decommissioned ones only
1192   * @param excludeDecommissionedRS should we exclude decommissioned RS nodes
1193   * @return all current live region servers including/excluding decommissioned hosts
1194   * @throws IOException if a remote or network exception occurs
1195   */
1196  default Collection<ServerName> getRegionServers(boolean excludeDecommissionedRS)
1197    throws IOException {
1198    List<ServerName> allServers =
1199      getClusterMetrics(EnumSet.of(Option.SERVERS_NAME)).getServersName();
1200    if (!excludeDecommissionedRS) {
1201      return allServers;
1202    }
1203    List<ServerName> decommissionedRegionServers = listDecommissionedRegionServers();
1204    return allServers.stream().filter(s -> !decommissionedRegionServers.contains(s))
1205      .collect(ImmutableList.toImmutableList());
1206  }
1207
1208  /**
1209   * Get {@link RegionMetrics} of all regions hosted on a regionserver.
1210   * @param serverName region server from which {@link RegionMetrics} is required.
1211   * @return a {@link RegionMetrics} list of all regions hosted on a region server
1212   * @throws IOException if a remote or network exception occurs
1213   */
1214  List<RegionMetrics> getRegionMetrics(ServerName serverName) throws IOException;
1215
1216  /**
1217   * Get {@link RegionMetrics} of all regions hosted on a regionserver for a table.
1218   * @param serverName region server from which {@link RegionMetrics} is required.
1219   * @param tableName  get {@link RegionMetrics} of regions belonging to the table
1220   * @return region metrics map of all regions of a table hosted on a region server
1221   * @throws IOException if a remote or network exception occurs
1222   */
1223  List<RegionMetrics> getRegionMetrics(ServerName serverName, TableName tableName)
1224    throws IOException;
1225
1226  /** Returns Configuration used by the instance. */
1227  Configuration getConfiguration();
1228
1229  /**
1230   * Create a new namespace. Blocks until namespace has been successfully created or an exception is
1231   * thrown.
1232   * @param descriptor descriptor which describes the new namespace.
1233   * @throws IOException if a remote or network exception occurs
1234   */
1235  default void createNamespace(NamespaceDescriptor descriptor) throws IOException {
1236    get(createNamespaceAsync(descriptor), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
1237  }
1238
1239  /**
1240   * Create a new namespace.
1241   * @param descriptor descriptor which describes the new namespace
1242   * @return the result of the async create namespace operation. Use Future.get(long, TimeUnit) to
1243   *         wait on the operation to complete.
1244   * @throws IOException if a remote or network exception occurs
1245   */
1246  Future<Void> createNamespaceAsync(NamespaceDescriptor descriptor) throws IOException;
1247
1248  /**
1249   * Modify an existing namespace. Blocks until namespace has been successfully modified or an
1250   * exception is thrown.
1251   * @param descriptor descriptor which describes the new namespace
1252   * @throws IOException if a remote or network exception occurs
1253   */
1254  default void modifyNamespace(NamespaceDescriptor descriptor) throws IOException {
1255    get(modifyNamespaceAsync(descriptor), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
1256  }
1257
1258  /**
1259   * Modify an existing namespace.
1260   * @param descriptor descriptor which describes the new namespace
1261   * @return the result of the async modify namespace operation. Use Future.get(long, TimeUnit) to
1262   *         wait on the operation to complete.
1263   * @throws IOException if a remote or network exception occurs
1264   */
1265  Future<Void> modifyNamespaceAsync(NamespaceDescriptor descriptor) throws IOException;
1266
1267  /**
1268   * Delete an existing namespace. Only empty namespaces (no tables) can be removed. Blocks until
1269   * namespace has been successfully deleted or an exception is thrown.
1270   * @param name namespace name
1271   * @throws IOException if a remote or network exception occurs
1272   */
1273  default void deleteNamespace(String name) throws IOException {
1274    get(deleteNamespaceAsync(name), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
1275  }
1276
1277  /**
1278   * Delete an existing namespace. Only empty namespaces (no tables) can be removed.
1279   * @param name namespace name
1280   * @return the result of the async delete namespace operation. Use Future.get(long, TimeUnit) to
1281   *         wait on the operation to complete.
1282   * @throws IOException if a remote or network exception occurs
1283   */
1284  Future<Void> deleteNamespaceAsync(String name) throws IOException;
1285
1286  /**
1287   * Get a namespace descriptor by name.
1288   * @param name name of namespace descriptor
1289   * @return A descriptor
1290   * @throws org.apache.hadoop.hbase.NamespaceNotFoundException if the namespace was not found
1291   * @throws IOException                                        if a remote or network exception
1292   *                                                            occurs
1293   */
1294  NamespaceDescriptor getNamespaceDescriptor(String name)
1295    throws NamespaceNotFoundException, IOException;
1296
1297  /**
1298   * List available namespaces
1299   * @return List of namespace names
1300   * @throws IOException if a remote or network exception occurs
1301   */
1302  String[] listNamespaces() throws IOException;
1303
1304  /**
1305   * List available namespace descriptors
1306   * @return List of descriptors
1307   * @throws IOException if a remote or network exception occurs
1308   */
1309  NamespaceDescriptor[] listNamespaceDescriptors() throws IOException;
1310
1311  /**
1312   * Get list of table descriptors by namespace.
1313   * @param name namespace name
1314   * @return returns a list of TableDescriptors
1315   * @throws IOException if a remote or network exception occurs
1316   */
1317  List<TableDescriptor> listTableDescriptorsByNamespace(byte[] name) throws IOException;
1318
1319  /**
1320   * Get list of table names by namespace.
1321   * @param name namespace name
1322   * @return The list of table names in the namespace
1323   * @throws IOException if a remote or network exception occurs
1324   */
1325  TableName[] listTableNamesByNamespace(String name) throws IOException;
1326
1327  /**
1328   * Get the regions of a given table.
1329   * @param tableName the name of the table
1330   * @return List of {@link RegionInfo}.
1331   * @throws IOException if a remote or network exception occurs
1332   */
1333  List<RegionInfo> getRegions(TableName tableName) throws IOException;
1334
1335  @Override
1336  void close();
1337
1338  /**
1339   * Get tableDescriptors.
1340   * @param tableNames List of table names
1341   * @return returns a list of TableDescriptors
1342   * @throws IOException if a remote or network exception occurs
1343   */
1344  List<TableDescriptor> listTableDescriptors(List<TableName> tableNames) throws IOException;
1345
1346  /**
1347   * Abort a procedure.
1348   * <p/>
1349   * Do not use. Usually it is ignored but if not, it can do more damage than good. See hbck2.
1350   * @param procId                ID of the procedure to abort
1351   * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted?
1352   * @return <code>true</code> if aborted, <code>false</code> if procedure already completed or does
1353   *         not exist
1354   * @throws IOException if a remote or network exception occurs
1355   * @deprecated since 2.1.1 and will be removed in 4.0.0.
1356   * @see <a href="https://issues.apache.org/jira/browse/HBASE-21223">HBASE-21223</a>
1357   */
1358  @Deprecated
1359  default boolean abortProcedure(long procId, boolean mayInterruptIfRunning) throws IOException {
1360    return get(abortProcedureAsync(procId, mayInterruptIfRunning), getSyncWaitTimeout(),
1361      TimeUnit.MILLISECONDS);
1362  }
1363
1364  /**
1365   * Abort a procedure but does not block and wait for completion. You can use Future.get(long,
1366   * TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an
1367   * error while executing the operation or TimeoutException in case the wait timeout was not long
1368   * enough to allow the operation to complete. Do not use. Usually it is ignored but if not, it can
1369   * do more damage than good. See hbck2.
1370   * @param procId                ID of the procedure to abort
1371   * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted?
1372   * @return <code>true</code> if aborted, <code>false</code> if procedure already completed or does
1373   *         not exist
1374   * @throws IOException if a remote or network exception occurs
1375   * @deprecated since 2.1.1 and will be removed in 4.0.0.
1376   * @see <a href="https://issues.apache.org/jira/browse/HBASE-21223">HBASE-21223</a>
1377   */
1378  @Deprecated
1379  Future<Boolean> abortProcedureAsync(long procId, boolean mayInterruptIfRunning)
1380    throws IOException;
1381
1382  /**
1383   * Get procedures.
1384   * @return procedure list in JSON
1385   * @throws IOException if a remote or network exception occurs
1386   */
1387  String getProcedures() throws IOException;
1388
1389  /**
1390   * Get locks.
1391   * @return lock list in JSON
1392   * @throws IOException if a remote or network exception occurs
1393   */
1394  String getLocks() throws IOException;
1395
1396  /**
1397   * Roll the log writer. I.e. for filesystem based write ahead logs, start writing to a new file.
1398   * Note that the actual rolling of the log writer is asynchronous and may not be complete when
1399   * this method returns. As a side effect of this call, the named region server may schedule store
1400   * flushes at the request of the wal.
1401   * @param serverName The servername of the regionserver.
1402   * @throws IOException             if a remote or network exception occurs
1403   * @throws FailedLogCloseException if we failed to close the WAL
1404   */
1405  void rollWALWriter(ServerName serverName) throws IOException, FailedLogCloseException;
1406
1407  /**
1408   * Helper that delegates to getClusterMetrics().getMasterCoprocessorNames().
1409   * @return an array of master coprocessors
1410   * @see org.apache.hadoop.hbase.ClusterMetrics#getMasterCoprocessorNames()
1411   */
1412  default List<String> getMasterCoprocessorNames() throws IOException {
1413    return getClusterMetrics(EnumSet.of(Option.MASTER_COPROCESSORS)).getMasterCoprocessorNames();
1414  }
1415
1416  /**
1417   * Get the current compaction state of a table. It could be in a major compaction, a minor
1418   * compaction, both, or none.
1419   * @param tableName table to examine
1420   * @return the current compaction state
1421   * @throws IOException if a remote or network exception occurs
1422   */
1423  CompactionState getCompactionState(TableName tableName) throws IOException;
1424
1425  /**
1426   * Get the current compaction state of a table. It could be in a compaction, or none.
1427   * @param tableName   table to examine
1428   * @param compactType {@link org.apache.hadoop.hbase.client.CompactType}
1429   * @return the current compaction state
1430   * @throws IOException if a remote or network exception occurs
1431   */
1432  CompactionState getCompactionState(TableName tableName, CompactType compactType)
1433    throws IOException;
1434
1435  /**
1436   * Get the current compaction state of region. It could be in a major compaction, a minor
1437   * compaction, both, or none.
1438   * @param regionName region to examine
1439   * @return the current compaction state
1440   * @throws IOException if a remote or network exception occurs
1441   */
1442  CompactionState getCompactionStateForRegion(byte[] regionName) throws IOException;
1443
1444  /**
1445   * Get the timestamp of the last major compaction for the passed table The timestamp of the oldest
1446   * HFile resulting from a major compaction of that table, or 0 if no such HFile could be found.
1447   * @param tableName table to examine
1448   * @return the last major compaction timestamp or 0
1449   * @throws IOException if a remote or network exception occurs
1450   */
1451  long getLastMajorCompactionTimestamp(TableName tableName) throws IOException;
1452
1453  /**
1454   * Get the timestamp of the last major compaction for the passed region. The timestamp of the
1455   * oldest HFile resulting from a major compaction of that region, or 0 if no such HFile could be
1456   * found.
1457   * @param regionName region to examine
1458   * @return the last major compaction timestamp or 0
1459   * @throws IOException if a remote or network exception occurs
1460   */
1461  long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException;
1462
1463  /**
1464   * Take a snapshot for the given table. If the table is enabled, a FLUSH-type snapshot will be
1465   * taken. If the table is disabled, an offline snapshot is taken. Snapshots are taken sequentially
1466   * even when requested concurrently, across all tables. Snapshots are considered unique based on
1467   * <b>the name of the snapshot</b>. Attempts to take a snapshot with the same name (even a
1468   * different type or with different parameters) will fail with a
1469   * {@link org.apache.hadoop.hbase.snapshot.SnapshotCreationException} indicating the duplicate
1470   * naming. Snapshot names follow the same naming constraints as tables in HBase. See
1471   * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
1472   * @param snapshotName name of the snapshot to be created
1473   * @param tableName    name of the table for which snapshot is created
1474   * @throws IOException                                                if a remote or network
1475   *                                                                    exception occurs
1476   * @throws org.apache.hadoop.hbase.snapshot.SnapshotCreationException if snapshot creation failed
1477   * @throws IllegalArgumentException                                   if the snapshot request is
1478   *                                                                    formatted incorrectly
1479   */
1480  default void snapshot(String snapshotName, TableName tableName)
1481    throws IOException, SnapshotCreationException, IllegalArgumentException {
1482    snapshot(snapshotName, tableName, SnapshotType.FLUSH);
1483  }
1484
1485  /**
1486   * Create typed snapshot of the table. Snapshots are considered unique based on <b>the name of the
1487   * snapshot</b>. Snapshots are taken sequentially even when requested concurrently, across all
1488   * tables. Attempts to take a snapshot with the same name (even a different type or with different
1489   * parameters) will fail with a {@link SnapshotCreationException} indicating the duplicate naming.
1490   * Snapshot names follow the same naming constraints as tables in HBase. See
1491   * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
1492   * @param snapshotName name to give the snapshot on the filesystem. Must be unique from all other
1493   *                     snapshots stored on the cluster
1494   * @param tableName    name of the table to snapshot
1495   * @param type         type of snapshot to take
1496   * @throws IOException               we fail to reach the master
1497   * @throws SnapshotCreationException if snapshot creation failed
1498   * @throws IllegalArgumentException  if the snapshot request is formatted incorrectly
1499   */
1500  default void snapshot(String snapshotName, TableName tableName, SnapshotType type)
1501    throws IOException, SnapshotCreationException, IllegalArgumentException {
1502    snapshot(new SnapshotDescription(snapshotName, tableName, type));
1503  }
1504
1505  /**
1506   * Create typed snapshot of the table. Snapshots are considered unique based on <b>the name of the
1507   * snapshot</b>. Snapshots are taken sequentially even when requested concurrently, across all
1508   * tables. Attempts to take a snapshot with the same name (even a different type or with different
1509   * parameters) will fail with a {@link SnapshotCreationException} indicating the duplicate naming.
1510   * Snapshot names follow the same naming constraints as tables in HBase. See
1511   * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}. Snapshot can
1512   * live with ttl seconds.
1513   * @param snapshotName  name to give the snapshot on the filesystem. Must be unique from all other
1514   *                      snapshots stored on the cluster
1515   * @param tableName     name of the table to snapshot
1516   * @param type          type of snapshot to take
1517   * @param snapshotProps snapshot additional properties e.g. TTL
1518   * @throws IOException               we fail to reach the master
1519   * @throws SnapshotCreationException if snapshot creation failed
1520   * @throws IllegalArgumentException  if the snapshot request is formatted incorrectly
1521   */
1522  default void snapshot(String snapshotName, TableName tableName, SnapshotType type,
1523    Map<String, Object> snapshotProps)
1524    throws IOException, SnapshotCreationException, IllegalArgumentException {
1525    snapshot(new SnapshotDescription(snapshotName, tableName, type, snapshotProps));
1526  }
1527
1528  /**
1529   * Create typed snapshot of the table. Snapshots are considered unique based on <b>the name of the
1530   * snapshot</b>. Snapshots are taken sequentially even when requested concurrently, across all
1531   * tables. Attempts to take a snapshot with the same name (even a different type or with different
1532   * parameters) will fail with a {@link SnapshotCreationException} indicating the duplicate naming.
1533   * Snapshot names follow the same naming constraints as tables in HBase. See
1534   * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}. Snapshot can
1535   * live with ttl seconds.
1536   * @param snapshotName  name to give the snapshot on the filesystem. Must be unique from all other
1537   *                      snapshots stored on the cluster
1538   * @param tableName     name of the table to snapshot
1539   * @param snapshotProps snapshot additional properties e.g. TTL
1540   * @throws IOException               we fail to reach the master
1541   * @throws SnapshotCreationException if snapshot creation failed
1542   * @throws IllegalArgumentException  if the snapshot request is formatted incorrectly
1543   */
1544  default void snapshot(String snapshotName, TableName tableName, Map<String, Object> snapshotProps)
1545    throws IOException, SnapshotCreationException, IllegalArgumentException {
1546    snapshot(new SnapshotDescription(snapshotName, tableName, SnapshotType.FLUSH, snapshotProps));
1547  }
1548
1549  /**
1550   * Take a snapshot and wait for the server to complete that snapshot (blocking). Snapshots are
1551   * considered unique based on <b>the name of the snapshot</b>. Snapshots are taken sequentially
1552   * even when requested concurrently, across all tables. Attempts to take a snapshot with the same
1553   * name (even a different type or with different parameters) will fail with a
1554   * {@link SnapshotCreationException} indicating the duplicate naming. Snapshot names follow the
1555   * same naming constraints as tables in HBase. See
1556   * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}. You should
1557   * probably use {@link #snapshot(String, org.apache.hadoop.hbase.TableName)} unless you are sure
1558   * about the type of snapshot that you want to take.
1559   * @param snapshot snapshot to take
1560   * @throws IOException               or we lose contact with the master.
1561   * @throws SnapshotCreationException if snapshot failed to be taken
1562   * @throws IllegalArgumentException  if the snapshot request is formatted incorrectly
1563   */
1564  void snapshot(SnapshotDescription snapshot)
1565    throws IOException, SnapshotCreationException, IllegalArgumentException;
1566
1567  /**
1568   * Take a snapshot without waiting for the server to complete that snapshot (asynchronous).
1569   * Snapshots are considered unique based on <b>the name of the snapshot</b>. Snapshots are taken
1570   * sequentially even when requested concurrently, across all tables.
1571   * @param snapshot snapshot to take
1572   * @throws IOException               if the snapshot did not succeed or we lose contact with the
1573   *                                   master.
1574   * @throws SnapshotCreationException if snapshot creation failed
1575   * @throws IllegalArgumentException  if the snapshot request is formatted incorrectly
1576   */
1577  Future<Void> snapshotAsync(SnapshotDescription snapshot)
1578    throws IOException, SnapshotCreationException;
1579
1580  /**
1581   * Check the current state of the passed snapshot. There are three possible states:
1582   * <ol>
1583   * <li>running - returns <tt>false</tt></li>
1584   * <li>finished - returns <tt>true</tt></li>
1585   * <li>finished with error - throws the exception that caused the snapshot to fail</li>
1586   * </ol>
1587   * The cluster only knows about the most recent snapshot. Therefore, if another snapshot has been
1588   * run/started since the snapshot you are checking, you will receive an
1589   * {@link org.apache.hadoop.hbase.snapshot.UnknownSnapshotException}.
1590   * @param snapshot description of the snapshot to check
1591   * @return <tt>true</tt> if the snapshot is completed, <tt>false</tt> if the snapshot is still
1592   *         running
1593   * @throws IOException                                               if we have a network issue
1594   * @throws org.apache.hadoop.hbase.snapshot.HBaseSnapshotException   if the snapshot failed
1595   * @throws org.apache.hadoop.hbase.snapshot.UnknownSnapshotException if the requested snapshot is
1596   *                                                                   unknown
1597   */
1598  boolean isSnapshotFinished(SnapshotDescription snapshot)
1599    throws IOException, HBaseSnapshotException, UnknownSnapshotException;
1600
1601  /**
1602   * Restore the specified snapshot on the original table. (The table must be disabled) If the
1603   * "hbase.snapshot.restore.take.failsafe.snapshot" configuration property is set to
1604   * <code>true</code>, a snapshot of the current table is taken before executing the restore
1605   * operation. In case of restore failure, the failsafe snapshot will be restored. If the restore
1606   * completes without problem the failsafe snapshot is deleted.
1607   * @param snapshotName name of the snapshot to restore
1608   * @throws IOException              if a remote or network exception occurs
1609   * @throws RestoreSnapshotException if snapshot failed to be restored
1610   * @throws IllegalArgumentException if the restore request is formatted incorrectly
1611   */
1612  void restoreSnapshot(String snapshotName) throws IOException, RestoreSnapshotException;
1613
1614  /**
1615   * Restore the specified snapshot on the original table. (The table must be disabled) If
1616   * 'takeFailSafeSnapshot' is set to <code>true</code>, a snapshot of the current table is taken
1617   * before executing the restore operation. In case of restore failure, the failsafe snapshot will
1618   * be restored. If the restore completes without problem the failsafe snapshot is deleted. The
1619   * failsafe snapshot name is configurable by using the property
1620   * "hbase.snapshot.restore.failsafe.name".
1621   * @param snapshotName         name of the snapshot to restore
1622   * @param takeFailSafeSnapshot <code>true</code> if the failsafe snapshot should be taken
1623   * @throws IOException              if a remote or network exception occurs
1624   * @throws RestoreSnapshotException if snapshot failed to be restored
1625   * @throws IllegalArgumentException if the restore request is formatted incorrectly
1626   */
1627  default void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot)
1628    throws IOException, RestoreSnapshotException {
1629    restoreSnapshot(snapshotName, takeFailSafeSnapshot, false);
1630  }
1631
1632  /**
1633   * Restore the specified snapshot on the original table. (The table must be disabled) If
1634   * 'takeFailSafeSnapshot' is set to <code>true</code>, a snapshot of the current table is taken
1635   * before executing the restore operation. In case of restore failure, the failsafe snapshot will
1636   * be restored. If the restore completes without problem the failsafe snapshot is deleted. The
1637   * failsafe snapshot name is configurable by using the property
1638   * "hbase.snapshot.restore.failsafe.name".
1639   * @param snapshotName         name of the snapshot to restore
1640   * @param takeFailSafeSnapshot <code>true</code> if the failsafe snapshot should be taken
1641   * @param restoreAcl           <code>true</code> to restore acl of snapshot
1642   * @throws IOException              if a remote or network exception occurs
1643   * @throws RestoreSnapshotException if snapshot failed to be restored
1644   * @throws IllegalArgumentException if the restore request is formatted incorrectly
1645   */
1646  void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl)
1647    throws IOException, RestoreSnapshotException;
1648
1649  /**
1650   * Create a new table by cloning the snapshot content.
1651   * @param snapshotName name of the snapshot to be cloned
1652   * @param tableName    name of the table where the snapshot will be restored
1653   * @throws IOException              if a remote or network exception occurs
1654   * @throws TableExistsException     if table to be created already exists
1655   * @throws RestoreSnapshotException if snapshot failed to be cloned
1656   * @throws IllegalArgumentException if the specified table has not a valid name
1657   */
1658  default void cloneSnapshot(String snapshotName, TableName tableName)
1659    throws IOException, TableExistsException, RestoreSnapshotException {
1660    cloneSnapshot(snapshotName, tableName, false, null);
1661  }
1662
1663  /**
1664   * Create a new table by cloning the snapshot content.
1665   * @param snapshotName name of the snapshot to be cloned
1666   * @param tableName    name of the table where the snapshot will be restored
1667   * @param restoreAcl   <code>true</code> to clone acl into newly created table
1668   * @param customSFT    specify the StoreFileTracker used for the table
1669   * @throws IOException              if a remote or network exception occurs
1670   * @throws TableExistsException     if table to be created already exists
1671   * @throws RestoreSnapshotException if snapshot failed to be cloned
1672   * @throws IllegalArgumentException if the specified table has not a valid name
1673   */
1674  default void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl,
1675    String customSFT) throws IOException, TableExistsException, RestoreSnapshotException {
1676    get(cloneSnapshotAsync(snapshotName, tableName, restoreAcl, customSFT), getSyncWaitTimeout(),
1677      TimeUnit.MILLISECONDS);
1678  }
1679
1680  /**
1681   * Create a new table by cloning the snapshot content.
1682   * @param snapshotName name of the snapshot to be cloned
1683   * @param tableName    name of the table where the snapshot will be restored
1684   * @param restoreAcl   <code>true</code> to clone acl into newly created table
1685   * @throws IOException              if a remote or network exception occurs
1686   * @throws TableExistsException     if table to be created already exists
1687   * @throws RestoreSnapshotException if snapshot failed to be cloned
1688   * @throws IllegalArgumentException if the specified table has not a valid name
1689   */
1690  default void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl)
1691    throws IOException, TableExistsException, RestoreSnapshotException {
1692    get(cloneSnapshotAsync(snapshotName, tableName, restoreAcl), getSyncWaitTimeout(),
1693      TimeUnit.MILLISECONDS);
1694  }
1695
1696  /**
1697   * Create a new table by cloning the snapshot content, but does not block and wait for it to be
1698   * completely cloned. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
1699   * It may throw ExecutionException if there was an error while executing the operation or
1700   * TimeoutException in case the wait timeout was not long enough to allow the operation to
1701   * complete.
1702   * @param snapshotName name of the snapshot to be cloned
1703   * @param tableName    name of the table where the snapshot will be restored
1704   * @throws IOException          if a remote or network exception occurs
1705   * @throws TableExistsException if table to be cloned already exists
1706   * @return the result of the async clone snapshot. You can use Future.get(long, TimeUnit) to wait
1707   *         on the operation to complete.
1708   */
1709  default Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName)
1710    throws IOException, TableExistsException {
1711    return cloneSnapshotAsync(snapshotName, tableName, false);
1712  }
1713
1714  /**
1715   * Create a new table by cloning the snapshot content.
1716   * @param snapshotName name of the snapshot to be cloned
1717   * @param tableName    name of the table where the snapshot will be restored
1718   * @param restoreAcl   <code>true</code> to clone acl into newly created table
1719   * @throws IOException              if a remote or network exception occurs
1720   * @throws TableExistsException     if table to be created already exists
1721   * @throws RestoreSnapshotException if snapshot failed to be cloned
1722   * @throws IllegalArgumentException if the specified table has not a valid name
1723   */
1724  default Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName,
1725    boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException {
1726    return cloneSnapshotAsync(snapshotName, tableName, restoreAcl, null);
1727  }
1728
1729  /**
1730   * Create a new table by cloning the snapshot content.
1731   * @param snapshotName name of the snapshot to be cloned
1732   * @param tableName    name of the table where the snapshot will be restored
1733   * @param restoreAcl   <code>true</code> to clone acl into newly created table
1734   * @param customSFT    specify the StroreFileTracker used for the table
1735   * @throws IOException              if a remote or network exception occurs
1736   * @throws TableExistsException     if table to be created already exists
1737   * @throws RestoreSnapshotException if snapshot failed to be cloned
1738   * @throws IllegalArgumentException if the specified table has not a valid name
1739   */
1740  Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName, boolean restoreAcl,
1741    String customSFT) throws IOException, TableExistsException, RestoreSnapshotException;
1742
1743  /**
1744   * Execute a distributed procedure on a cluster.
1745   * @param signature A distributed procedure is uniquely identified by its signature (default the
1746   *                  root ZK node name of the procedure).
1747   * @param instance  The instance name of the procedure. For some procedures, this parameter is
1748   *                  optional.
1749   * @param props     Property/Value pairs of properties passing to the procedure
1750   * @throws IOException if a remote or network exception occurs
1751   */
1752  void execProcedure(String signature, String instance, Map<String, String> props)
1753    throws IOException;
1754
1755  /**
1756   * Execute a distributed procedure on a cluster.
1757   * @param signature A distributed procedure is uniquely identified by its signature (default the
1758   *                  root ZK node name of the procedure).
1759   * @param instance  The instance name of the procedure. For some procedures, this parameter is
1760   *                  optional.
1761   * @param props     Property/Value pairs of properties passing to the procedure
1762   * @return data returned after procedure execution. null if no return data.
1763   * @throws IOException if a remote or network exception occurs
1764   */
1765  byte[] execProcedureWithReturn(String signature, String instance, Map<String, String> props)
1766    throws IOException;
1767
1768  /**
1769   * Check the current state of the specified procedure. There are three possible states:
1770   * <ol>
1771   * <li>running - returns <tt>false</tt></li>
1772   * <li>finished - returns <tt>true</tt></li>
1773   * <li>finished with error - throws the exception that caused the procedure to fail</li>
1774   * </ol>
1775   * @param signature The signature that uniquely identifies a procedure
1776   * @param instance  The instance name of the procedure
1777   * @param props     Property/Value pairs of properties passing to the procedure
1778   * @return <code>true</code> if the specified procedure is finished successfully,
1779   *         <code>false</code> if it is still running
1780   * @throws IOException if the specified procedure finished with error
1781   */
1782  boolean isProcedureFinished(String signature, String instance, Map<String, String> props)
1783    throws IOException;
1784
1785  /**
1786   * List completed snapshots.
1787   * @return a list of snapshot descriptors for completed snapshots
1788   * @throws IOException if a network error occurs
1789   */
1790  List<SnapshotDescription> listSnapshots() throws IOException;
1791
1792  /**
1793   * List all the completed snapshots matching the given pattern.
1794   * @param pattern The compiled regular expression to match against
1795   * @return list of SnapshotDescription
1796   * @throws IOException if a remote or network exception occurs
1797   */
1798  List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException;
1799
1800  /**
1801   * List all the completed snapshots matching the given table name regular expression and snapshot
1802   * name regular expression.
1803   * @param tableNamePattern    The compiled table name regular expression to match against
1804   * @param snapshotNamePattern The compiled snapshot name regular expression to match against
1805   * @return list of completed SnapshotDescription
1806   * @throws IOException if a remote or network exception occurs
1807   */
1808  List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern,
1809    Pattern snapshotNamePattern) throws IOException;
1810
1811  /**
1812   * Delete an existing snapshot.
1813   * @param snapshotName name of the snapshot
1814   * @throws IOException if a remote or network exception occurs
1815   */
1816  void deleteSnapshot(String snapshotName) throws IOException;
1817
1818  /**
1819   * Delete existing snapshots whose names match the pattern passed.
1820   * @param pattern pattern for names of the snapshot to match
1821   * @throws IOException if a remote or network exception occurs
1822   */
1823  void deleteSnapshots(Pattern pattern) throws IOException;
1824
1825  /**
1826   * Delete all existing snapshots matching the given table name regular expression and snapshot
1827   * name regular expression.
1828   * @param tableNamePattern    The compiled table name regular expression to match against
1829   * @param snapshotNamePattern The compiled snapshot name regular expression to match against
1830   * @throws IOException if a remote or network exception occurs
1831   */
1832  void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern)
1833    throws IOException;
1834
1835  /**
1836   * Apply the new quota settings.
1837   * @param quota the quota settings
1838   * @throws IOException if a remote or network exception occurs
1839   */
1840  void setQuota(QuotaSettings quota) throws IOException;
1841
1842  /**
1843   * List the quotas based on the filter.
1844   * @param filter the quota settings filter
1845   * @return the QuotaSetting list
1846   * @throws IOException if a remote or network exception occurs
1847   */
1848  List<QuotaSettings> getQuota(QuotaFilter filter) throws IOException;
1849
1850  /**
1851   * Creates and returns a {@link org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel}
1852   * instance connected to the active master.
1853   * <p/>
1854   * The obtained {@link org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel} instance can be
1855   * used to access a published coprocessor
1856   * {@link org.apache.hbase.thirdparty.com.google.protobuf.Service} using standard protobuf service
1857   * invocations:
1858   * <p/>
1859   * <div style="background-color: #cccccc; padding: 2px"> <blockquote>
1860   *
1861   * <pre>
1862   * CoprocessorRpcChannel channel = myAdmin.coprocessorService();
1863   * MyService.BlockingInterface service = MyService.newBlockingStub(channel);
1864   * MyCallRequest request = MyCallRequest.newBuilder()
1865   *     ...
1866   *     .build();
1867   * MyCallResponse response = service.myCall(null, request);
1868   * </pre>
1869   *
1870   * </blockquote> </div>
1871   * @return A MasterCoprocessorRpcChannel instance
1872   * @deprecated since 3.0.0, will removed in 4.0.0. This is too low level, please stop using it any
1873   *             more. Use the coprocessorService methods in {@link AsyncAdmin} instead.
1874   */
1875  @Deprecated
1876  CoprocessorRpcChannel coprocessorService();
1877
1878  /**
1879   * Creates and returns a {@link org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel}
1880   * instance connected to the passed region server.
1881   * <p/>
1882   * The obtained {@link org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel} instance can be
1883   * used to access a published coprocessor
1884   * {@link org.apache.hbase.thirdparty.com.google.protobuf.Service} using standard protobuf service
1885   * invocations:
1886   * <p/>
1887   * <div style="background-color: #cccccc; padding: 2px"> <blockquote>
1888   *
1889   * <pre>
1890   * CoprocessorRpcChannel channel = myAdmin.coprocessorService(serverName);
1891   * MyService.BlockingInterface service = MyService.newBlockingStub(channel);
1892   * MyCallRequest request = MyCallRequest.newBuilder()
1893   *     ...
1894   *     .build();
1895   * MyCallResponse response = service.myCall(null, request);
1896   * </pre>
1897   *
1898   * </blockquote> </div>
1899   * @param serverName the server name to which the endpoint call is made
1900   * @return A RegionServerCoprocessorRpcChannel instance
1901   * @deprecated since 3.0.0, will removed in 4.0.0. This is too low level, please stop using it any
1902   *             more. Use the coprocessorService methods in {@link AsyncAdmin} instead.
1903   */
1904  @Deprecated
1905  CoprocessorRpcChannel coprocessorService(ServerName serverName);
1906
1907  /**
1908   * Update the configuration and trigger an online config change on the regionserver.
1909   * @param server : The server whose config needs to be updated.
1910   * @throws IOException if a remote or network exception occurs
1911   */
1912  void updateConfiguration(ServerName server) throws IOException;
1913
1914  /**
1915   * Update the configuration and trigger an online config change on all the regionservers.
1916   * @throws IOException if a remote or network exception occurs
1917   */
1918  void updateConfiguration() throws IOException;
1919
1920  /**
1921   * Update the configuration and trigger an online config change on all the regionservers in the
1922   * RSGroup.
1923   * @param groupName the group name
1924   * @throws IOException if a remote or network exception occurs
1925   */
1926  void updateConfiguration(String groupName) throws IOException;
1927
1928  /**
1929   * Get the info port of the current master if one is available.
1930   * @return master info port
1931   * @throws IOException if a remote or network exception occurs
1932   */
1933  default int getMasterInfoPort() throws IOException {
1934    return getClusterMetrics(EnumSet.of(Option.MASTER_INFO_PORT)).getMasterInfoPort();
1935  }
1936
1937  /**
1938   * Return the set of supported security capabilities.
1939   * @throws IOException if a remote or network exception occurs
1940   */
1941  List<SecurityCapability> getSecurityCapabilities() throws IOException;
1942
1943  /**
1944   * Turn the split switch on or off.
1945   * @param enabled     enabled or not
1946   * @param synchronous If <code>true</code>, it waits until current split() call, if outstanding,
1947   *                    to return.
1948   * @return Previous switch value
1949   * @throws IOException if a remote or network exception occurs
1950   */
1951  boolean splitSwitch(boolean enabled, boolean synchronous) throws IOException;
1952
1953  /**
1954   * Turn the merge switch on or off.
1955   * @param enabled     enabled or not
1956   * @param synchronous If <code>true</code>, it waits until current merge() call, if outstanding,
1957   *                    to return.
1958   * @return Previous switch value
1959   * @throws IOException if a remote or network exception occurs
1960   */
1961  boolean mergeSwitch(boolean enabled, boolean synchronous) throws IOException;
1962
1963  /**
1964   * Query the current state of the split switch.
1965   * @return <code>true</code> if the switch is enabled, <code>false</code> otherwise.
1966   * @throws IOException if a remote or network exception occurs
1967   */
1968  boolean isSplitEnabled() throws IOException;
1969
1970  /**
1971   * Query the current state of the merge switch.
1972   * @return <code>true</code> if the switch is enabled, <code>false</code> otherwise.
1973   * @throws IOException if a remote or network exception occurs
1974   */
1975  boolean isMergeEnabled() throws IOException;
1976
1977  /**
1978   * Add a new replication peer for replicating data to slave cluster.
1979   * @param peerId     a short name that identifies the peer
1980   * @param peerConfig configuration for the replication peer
1981   * @throws IOException if a remote or network exception occurs
1982   */
1983  default void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig)
1984    throws IOException {
1985    addReplicationPeer(peerId, peerConfig, true);
1986  }
1987
1988  /**
1989   * Add a new replication peer for replicating data to slave cluster.
1990   * @param peerId     a short name that identifies the peer
1991   * @param peerConfig configuration for the replication peer
1992   * @param enabled    peer state, true if ENABLED and false if DISABLED
1993   * @throws IOException if a remote or network exception occurs
1994   */
1995  default void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
1996    throws IOException {
1997    get(addReplicationPeerAsync(peerId, peerConfig, enabled), getSyncWaitTimeout(),
1998      TimeUnit.MILLISECONDS);
1999  }
2000
2001  /**
2002   * Add a new replication peer but does not block and wait for it.
2003   * <p/>
2004   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
2005   * ExecutionException if there was an error while executing the operation or TimeoutException in
2006   * case the wait timeout was not long enough to allow the operation to complete.
2007   * @param peerId     a short name that identifies the peer
2008   * @param peerConfig configuration for the replication peer
2009   * @return the result of the async operation
2010   * @throws IOException IOException if a remote or network exception occurs
2011   */
2012  default Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig)
2013    throws IOException {
2014    return addReplicationPeerAsync(peerId, peerConfig, true);
2015  }
2016
2017  /**
2018   * Add a new replication peer but does not block and wait for it.
2019   * <p>
2020   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
2021   * ExecutionException if there was an error while executing the operation or TimeoutException in
2022   * case the wait timeout was not long enough to allow the operation to complete.
2023   * @param peerId     a short name that identifies the peer
2024   * @param peerConfig configuration for the replication peer
2025   * @param enabled    peer state, true if ENABLED and false if DISABLED
2026   * @return the result of the async operation
2027   * @throws IOException IOException if a remote or network exception occurs
2028   */
2029  Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig,
2030    boolean enabled) throws IOException;
2031
2032  /**
2033   * Remove a peer and stop the replication.
2034   * @param peerId a short name that identifies the peer
2035   * @throws IOException if a remote or network exception occurs
2036   */
2037  default void removeReplicationPeer(String peerId) throws IOException {
2038    get(removeReplicationPeerAsync(peerId), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
2039  }
2040
2041  /**
2042   * Remove a replication peer but does not block and wait for it.
2043   * <p>
2044   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
2045   * ExecutionException if there was an error while executing the operation or TimeoutException in
2046   * case the wait timeout was not long enough to allow the operation to complete.
2047   * @param peerId a short name that identifies the peer
2048   * @return the result of the async operation
2049   * @throws IOException IOException if a remote or network exception occurs
2050   */
2051  Future<Void> removeReplicationPeerAsync(String peerId) throws IOException;
2052
2053  /**
2054   * Restart the replication stream to the specified peer.
2055   * @param peerId a short name that identifies the peer
2056   * @throws IOException if a remote or network exception occurs
2057   */
2058  default void enableReplicationPeer(String peerId) throws IOException {
2059    get(enableReplicationPeerAsync(peerId), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
2060  }
2061
2062  /**
2063   * Enable a replication peer but does not block and wait for it.
2064   * <p>
2065   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
2066   * ExecutionException if there was an error while executing the operation or TimeoutException in
2067   * case the wait timeout was not long enough to allow the operation to complete.
2068   * @param peerId a short name that identifies the peer
2069   * @return the result of the async operation
2070   * @throws IOException IOException if a remote or network exception occurs
2071   */
2072  Future<Void> enableReplicationPeerAsync(String peerId) throws IOException;
2073
2074  /**
2075   * Stop the replication stream to the specified peer.
2076   * @param peerId a short name that identifies the peer
2077   * @throws IOException if a remote or network exception occurs
2078   */
2079  default void disableReplicationPeer(String peerId) throws IOException {
2080    get(disableReplicationPeerAsync(peerId), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
2081  }
2082
2083  /**
2084   * Disable a replication peer but does not block and wait for it.
2085   * <p/>
2086   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
2087   * ExecutionException if there was an error while executing the operation or TimeoutException in
2088   * case the wait timeout was not long enough to allow the operation to complete.
2089   * @param peerId a short name that identifies the peer
2090   * @return the result of the async operation
2091   * @throws IOException IOException if a remote or network exception occurs
2092   */
2093  Future<Void> disableReplicationPeerAsync(String peerId) throws IOException;
2094
2095  /**
2096   * Returns the configured ReplicationPeerConfig for the specified peer.
2097   * @param peerId a short name that identifies the peer
2098   * @return ReplicationPeerConfig for the peer
2099   * @throws IOException if a remote or network exception occurs
2100   */
2101  ReplicationPeerConfig getReplicationPeerConfig(String peerId) throws IOException;
2102
2103  /**
2104   * Update the peerConfig for the specified peer.
2105   * @param peerId     a short name that identifies the peer
2106   * @param peerConfig new config for the replication peer
2107   * @throws IOException if a remote or network exception occurs
2108   */
2109  default void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
2110    throws IOException {
2111    get(updateReplicationPeerConfigAsync(peerId, peerConfig), getSyncWaitTimeout(),
2112      TimeUnit.MILLISECONDS);
2113  }
2114
2115  /**
2116   * Update the peerConfig for the specified peer but does not block and wait for it.
2117   * <p/>
2118   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
2119   * ExecutionException if there was an error while executing the operation or TimeoutException in
2120   * case the wait timeout was not long enough to allow the operation to complete.
2121   * @param peerId     a short name that identifies the peer
2122   * @param peerConfig new config for the replication peer
2123   * @return the result of the async operation
2124   * @throws IOException IOException if a remote or network exception occurs
2125   */
2126  Future<Void> updateReplicationPeerConfigAsync(String peerId, ReplicationPeerConfig peerConfig)
2127    throws IOException;
2128
2129  /**
2130   * Append the replicable table column family config from the specified peer.
2131   * @param id       a short that identifies the cluster
2132   * @param tableCfs A map from tableName to column family names
2133   * @throws ReplicationException if tableCfs has conflict with existing config
2134   * @throws IOException          if a remote or network exception occurs
2135   */
2136  default void appendReplicationPeerTableCFs(String id, Map<TableName, List<String>> tableCfs)
2137    throws ReplicationException, IOException {
2138    if (tableCfs == null) {
2139      throw new ReplicationException("tableCfs is null");
2140    }
2141    ReplicationPeerConfig peerConfig = getReplicationPeerConfig(id);
2142    ReplicationPeerConfig newPeerConfig =
2143      ReplicationPeerConfigUtil.appendTableCFsToReplicationPeerConfig(tableCfs, peerConfig);
2144    updateReplicationPeerConfig(id, newPeerConfig);
2145  }
2146
2147  /**
2148   * Remove some table-cfs from config of the specified peer.
2149   * @param id       a short name that identifies the cluster
2150   * @param tableCfs A map from tableName to column family names
2151   * @throws ReplicationException if tableCfs has conflict with existing config
2152   * @throws IOException          if a remote or network exception occurs
2153   */
2154  default void removeReplicationPeerTableCFs(String id, Map<TableName, List<String>> tableCfs)
2155    throws ReplicationException, IOException {
2156    if (tableCfs == null) {
2157      throw new ReplicationException("tableCfs is null");
2158    }
2159    ReplicationPeerConfig peerConfig = getReplicationPeerConfig(id);
2160    ReplicationPeerConfig newPeerConfig =
2161      ReplicationPeerConfigUtil.removeTableCFsFromReplicationPeerConfig(tableCfs, peerConfig, id);
2162    updateReplicationPeerConfig(id, newPeerConfig);
2163  }
2164
2165  /**
2166   * Return a list of replication peers.
2167   * @return a list of replication peers description
2168   * @throws IOException if a remote or network exception occurs
2169   */
2170  List<ReplicationPeerDescription> listReplicationPeers() throws IOException;
2171
2172  /**
2173   * Return a list of replication peers.
2174   * @param pattern The compiled regular expression to match peer id
2175   * @return a list of replication peers description
2176   * @throws IOException if a remote or network exception occurs
2177   */
2178  List<ReplicationPeerDescription> listReplicationPeers(Pattern pattern) throws IOException;
2179
2180  /**
2181   * Transit current cluster to a new state in a synchronous replication peer.
2182   * @param peerId a short name that identifies the peer
2183   * @param state  a new state of current cluster
2184   * @throws IOException if a remote or network exception occurs
2185   */
2186  default void transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState state)
2187    throws IOException {
2188    get(transitReplicationPeerSyncReplicationStateAsync(peerId, state), getSyncWaitTimeout(),
2189      TimeUnit.MILLISECONDS);
2190  }
2191
2192  /**
2193   * Transit current cluster to a new state in a synchronous replication peer. But does not block
2194   * and wait for it.
2195   * <p>
2196   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
2197   * ExecutionException if there was an error while executing the operation or TimeoutException in
2198   * case the wait timeout was not long enough to allow the operation to complete.
2199   * @param peerId a short name that identifies the peer
2200   * @param state  a new state of current cluster
2201   * @throws IOException if a remote or network exception occurs
2202   */
2203  Future<Void> transitReplicationPeerSyncReplicationStateAsync(String peerId,
2204    SyncReplicationState state) throws IOException;
2205
2206  /**
2207   * Get the current cluster state in a synchronous replication peer.
2208   * @param peerId a short name that identifies the peer
2209   * @return the current cluster state
2210   * @throws IOException if a remote or network exception occurs
2211   */
2212  default SyncReplicationState getReplicationPeerSyncReplicationState(String peerId)
2213    throws IOException {
2214    List<ReplicationPeerDescription> peers = listReplicationPeers(Pattern.compile(peerId));
2215    if (peers.isEmpty() || !peers.get(0).getPeerId().equals(peerId)) {
2216      throw new IOException("Replication peer " + peerId + " does not exist");
2217    }
2218    return peers.get(0).getSyncReplicationState();
2219  }
2220
2221  /**
2222   * Check if a replication peer is enabled.
2223   * @param peerId id of replication peer to check
2224   * @return <code>true</code> if replication peer is enabled
2225   * @throws IOException if a remote or network exception occurs
2226   */
2227  boolean isReplicationPeerEnabled(String peerId) throws IOException;
2228
2229  /**
2230   * Enable or disable replication peer modification.
2231   * <p/>
2232   * This is especially useful when you want to change the replication peer storage.
2233   * @param on {@code true} means enable, otherwise disable
2234   * @return the previous enable/disable state
2235   */
2236  default boolean replicationPeerModificationSwitch(boolean on) throws IOException {
2237    return replicationPeerModificationSwitch(on, false);
2238  }
2239
2240  /**
2241   * Enable or disable replication peer modification.
2242   * <p/>
2243   * This is especially useful when you want to change the replication peer storage.
2244   * @param on              {@code true} means enable, otherwise disable
2245   * @param drainProcedures if {@code true}, will wait until all the running replication peer
2246   *                        modification procedures finish
2247   * @return the previous enable/disable state
2248   */
2249  boolean replicationPeerModificationSwitch(boolean on, boolean drainProcedures) throws IOException;
2250
2251  /**
2252   * Check whether replication peer modification is enabled.
2253   * @return {@code true} if modification is enabled, otherwise {@code false}
2254   */
2255  boolean isReplicationPeerModificationEnabled() throws IOException;
2256
2257  /**
2258   * Mark region server(s) as decommissioned to prevent additional regions from getting assigned to
2259   * them. Optionally unload the regions on the servers. If there are multiple servers to be
2260   * decommissioned, decommissioning them at the same time can prevent wasteful region movements.
2261   * Region unloading is asynchronous.
2262   * @param servers The list of servers to decommission.
2263   * @param offload True to offload the regions from the decommissioned servers
2264   * @throws IOException if a remote or network exception occurs
2265   */
2266  void decommissionRegionServers(List<ServerName> servers, boolean offload) throws IOException;
2267
2268  /**
2269   * List region servers marked as decommissioned, which can not be assigned regions.
2270   * @return List of decommissioned region servers.
2271   * @throws IOException if a remote or network exception occurs
2272   */
2273  List<ServerName> listDecommissionedRegionServers() throws IOException;
2274
2275  /**
2276   * Remove decommission marker from a region server to allow regions assignments. Load regions onto
2277   * the server if a list of regions is given. Region loading is asynchronous.
2278   * @param server             The server to recommission.
2279   * @param encodedRegionNames Regions to load onto the server.
2280   * @throws IOException if a remote or network exception occurs
2281   */
2282  void recommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames)
2283    throws IOException;
2284
2285  /**
2286   * Find all table and column families that are replicated from this cluster
2287   * @return the replicated table-cfs list of this cluster.
2288   * @throws IOException if a remote or network exception occurs
2289   */
2290  List<TableCFs> listReplicatedTableCFs() throws IOException;
2291
2292  /**
2293   * Enable a table's replication switch.
2294   * @param tableName name of the table
2295   * @throws IOException if a remote or network exception occurs
2296   */
2297  void enableTableReplication(TableName tableName) throws IOException;
2298
2299  /**
2300   * Disable a table's replication switch.
2301   * @param tableName name of the table
2302   * @throws IOException if a remote or network exception occurs
2303   */
2304  void disableTableReplication(TableName tableName) throws IOException;
2305
2306  /**
2307   * Clear compacting queues on a regionserver.
2308   * @param serverName the region server name
2309   * @param queues     the set of queue name
2310   * @throws IOException if a remote or network exception occurs
2311   */
2312  void clearCompactionQueues(ServerName serverName, Set<String> queues)
2313    throws IOException, InterruptedException;
2314
2315  /**
2316   * List dead region servers.
2317   * @return List of dead region servers.
2318   */
2319  default List<ServerName> listDeadServers() throws IOException {
2320    return getClusterMetrics(EnumSet.of(Option.DEAD_SERVERS)).getDeadServerNames();
2321  }
2322
2323  /**
2324   * List unknown region servers.
2325   * @return List of unknown region servers.
2326   */
2327  default List<ServerName> listUnknownServers() throws IOException {
2328    return getClusterMetrics(EnumSet.of(Option.UNKNOWN_SERVERS)).getUnknownServerNames();
2329  }
2330
2331  /**
2332   * Clear dead region servers from master.
2333   * @param servers list of dead region servers.
2334   * @throws IOException if a remote or network exception occurs
2335   * @return List of servers that are not cleared
2336   */
2337  List<ServerName> clearDeadServers(List<ServerName> servers) throws IOException;
2338
2339  /**
2340   * Create a new table by cloning the existent table schema.
2341   * @param tableName      name of the table to be cloned
2342   * @param newTableName   name of the new table where the table will be created
2343   * @param preserveSplits True if the splits should be preserved
2344   * @throws IOException if a remote or network exception occurs
2345   */
2346  void cloneTableSchema(TableName tableName, TableName newTableName, boolean preserveSplits)
2347    throws IOException;
2348
2349  /**
2350   * Switch the rpc throttle enable state.
2351   * @param enable Set to <code>true</code> to enable, <code>false</code> to disable.
2352   * @return Previous rpc throttle enabled value
2353   * @throws IOException if a remote or network exception occurs
2354   */
2355  boolean switchRpcThrottle(boolean enable) throws IOException;
2356
2357  /**
2358   * Get if the rpc throttle is enabled.
2359   * @return True if rpc throttle is enabled
2360   * @throws IOException if a remote or network exception occurs
2361   */
2362  boolean isRpcThrottleEnabled() throws IOException;
2363
2364  /**
2365   * Switch the exceed throttle quota. If enabled, user/table/namespace throttle quota can be
2366   * exceeded if region server has availble quota.
2367   * @param enable Set to <code>true</code> to enable, <code>false</code> to disable.
2368   * @return Previous exceed throttle enabled value
2369   * @throws IOException if a remote or network exception occurs
2370   */
2371  boolean exceedThrottleQuotaSwitch(final boolean enable) throws IOException;
2372
2373  /**
2374   * Fetches the table sizes on the filesystem as tracked by the HBase Master.
2375   * @throws IOException if a remote or network exception occurs
2376   */
2377  Map<TableName, Long> getSpaceQuotaTableSizes() throws IOException;
2378
2379  /**
2380   * Fetches the observed {@link SpaceQuotaSnapshotView}s observed by a RegionServer.
2381   * @throws IOException if a remote or network exception occurs
2382   */
2383  Map<TableName, ? extends SpaceQuotaSnapshotView>
2384    getRegionServerSpaceQuotaSnapshots(ServerName serverName) throws IOException;
2385
2386  /**
2387   * Returns the Master's view of a quota on the given {@code namespace} or null if the Master has
2388   * no quota information on that namespace.
2389   * @throws IOException if a remote or network exception occurs
2390   */
2391  SpaceQuotaSnapshotView getCurrentSpaceQuotaSnapshot(String namespace) throws IOException;
2392
2393  /**
2394   * Returns the Master's view of a quota on the given {@code tableName} or null if the Master has
2395   * no quota information on that table.
2396   * @throws IOException if a remote or network exception occurs
2397   */
2398  SpaceQuotaSnapshotView getCurrentSpaceQuotaSnapshot(TableName tableName) throws IOException;
2399
2400  /**
2401   * Grants user specific permissions
2402   * @param userPermission           user name and the specific permission
2403   * @param mergeExistingPermissions If set to false, later granted permissions will override
2404   *                                 previous granted permissions. otherwise, it'll merge with
2405   *                                 previous granted permissions.
2406   * @throws IOException if a remote or network exception occurs
2407   */
2408  void grant(UserPermission userPermission, boolean mergeExistingPermissions) throws IOException;
2409
2410  /**
2411   * Revokes user specific permissions
2412   * @param userPermission user name and the specific permission
2413   * @throws IOException if a remote or network exception occurs
2414   */
2415  void revoke(UserPermission userPermission) throws IOException;
2416
2417  /**
2418   * Get the global/namespace/table permissions for user
2419   * @param getUserPermissionsRequest A request contains which user, global, namespace or table
2420   *                                  permissions needed
2421   * @return The user and permission list
2422   * @throws IOException if a remote or network exception occurs
2423   */
2424  List<UserPermission> getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest)
2425    throws IOException;
2426
2427  /**
2428   * Check if the user has specific permissions
2429   * @param userName    the user name
2430   * @param permissions the specific permission list
2431   * @return True if user has the specific permissions
2432   * @throws IOException if a remote or network exception occurs
2433   */
2434  List<Boolean> hasUserPermissions(String userName, List<Permission> permissions)
2435    throws IOException;
2436
2437  /**
2438   * Check if call user has specific permissions
2439   * @param permissions the specific permission list
2440   * @return True if user has the specific permissions
2441   * @throws IOException if a remote or network exception occurs
2442   */
2443  default List<Boolean> hasUserPermissions(List<Permission> permissions) throws IOException {
2444    return hasUserPermissions(null, permissions);
2445  }
2446
2447  /**
2448   * Turn on or off the auto snapshot cleanup based on TTL.
2449   * @param on          Set to <code>true</code> to enable, <code>false</code> to disable.
2450   * @param synchronous If <code>true</code>, it waits until current snapshot cleanup is completed,
2451   *                    if outstanding.
2452   * @return Previous auto snapshot cleanup value
2453   * @throws IOException if a remote or network exception occurs
2454   */
2455  boolean snapshotCleanupSwitch(final boolean on, final boolean synchronous) throws IOException;
2456
2457  /**
2458   * Query the current state of the auto snapshot cleanup based on TTL.
2459   * @return <code>true</code> if the auto snapshot cleanup is enabled, <code>false</code>
2460   *         otherwise.
2461   * @throws IOException if a remote or network exception occurs
2462   */
2463  boolean isSnapshotCleanupEnabled() throws IOException;
2464
2465  /**
2466   * Retrieves online slow/large RPC logs from the provided list of RegionServers
2467   * @param serverNames    Server names to get slowlog responses from
2468   * @param logQueryFilter filter to be used if provided (determines slow / large RPC logs)
2469   * @return online slowlog response list
2470   * @throws IOException if a remote or network exception occurs
2471   * @deprecated since 2.4.0 and will be removed in 4.0.0. Use
2472   *             {@link #getLogEntries(Set, String, ServerType, int, Map)} instead.
2473   */
2474  @Deprecated
2475  default List<OnlineLogRecord> getSlowLogResponses(final Set<ServerName> serverNames,
2476    final LogQueryFilter logQueryFilter) throws IOException {
2477    String logType;
2478    if (LogQueryFilter.Type.LARGE_LOG.equals(logQueryFilter.getType())) {
2479      logType = "LARGE_LOG";
2480    } else {
2481      logType = "SLOW_LOG";
2482    }
2483    Map<String, Object> filterParams = new HashMap<>();
2484    filterParams.put("regionName", logQueryFilter.getRegionName());
2485    filterParams.put("clientAddress", logQueryFilter.getClientAddress());
2486    filterParams.put("tableName", logQueryFilter.getTableName());
2487    filterParams.put("userName", logQueryFilter.getUserName());
2488    filterParams.put("filterByOperator", logQueryFilter.getFilterByOperator().toString());
2489    List<LogEntry> logEntries = getLogEntries(serverNames, logType, ServerType.REGION_SERVER,
2490      logQueryFilter.getLimit(), filterParams);
2491    return logEntries.stream().map(logEntry -> (OnlineLogRecord) logEntry)
2492      .collect(Collectors.toList());
2493  }
2494
2495  /**
2496   * Clears online slow/large RPC logs from the provided list of RegionServers
2497   * @param serverNames Set of Server names to clean slowlog responses from
2498   * @return List of booleans representing if online slowlog response buffer is cleaned from each
2499   *         RegionServer
2500   * @throws IOException if a remote or network exception occurs
2501   */
2502  List<Boolean> clearSlowLogResponses(final Set<ServerName> serverNames) throws IOException;
2503
2504  /**
2505   * Creates a new RegionServer group with the given name
2506   * @param groupName the name of the group
2507   * @throws IOException if a remote or network exception occurs
2508   */
2509  void addRSGroup(String groupName) throws IOException;
2510
2511  /**
2512   * Get group info for the given group name
2513   * @param groupName the group name
2514   * @return group info
2515   * @throws IOException if a remote or network exception occurs
2516   */
2517  RSGroupInfo getRSGroup(String groupName) throws IOException;
2518
2519  /**
2520   * Get group info for the given hostPort
2521   * @param hostPort HostPort to get RSGroupInfo for
2522   * @throws IOException if a remote or network exception occurs
2523   */
2524  RSGroupInfo getRSGroup(Address hostPort) throws IOException;
2525
2526  /**
2527   * Get group info for the given table
2528   * @param tableName table name to get RSGroupInfo for
2529   * @throws IOException if a remote or network exception occurs
2530   */
2531  RSGroupInfo getRSGroup(TableName tableName) throws IOException;
2532
2533  /**
2534   * Lists current set of RegionServer groups
2535   * @throws IOException if a remote or network exception occurs
2536   */
2537  List<RSGroupInfo> listRSGroups() throws IOException;
2538
2539  /**
2540   * Get all tables in this RegionServer group.
2541   * @param groupName the group name
2542   * @throws IOException if a remote or network exception occurs
2543   * @see #getConfiguredNamespacesAndTablesInRSGroup(String)
2544   */
2545  List<TableName> listTablesInRSGroup(String groupName) throws IOException;
2546
2547  /**
2548   * Get the namespaces and tables which have this RegionServer group in descriptor.
2549   * <p/>
2550   * The difference between this method and {@link #listTablesInRSGroup(String)} is that, this
2551   * method will not include the table which is actually in this RegionServr group but without the
2552   * RegionServer group configuration in its {@link TableDescriptor}. For example, we have a group
2553   * 'A', and we make namespace 'nsA' in this group, then all the tables under this namespace will
2554   * in the group 'A', but this method will not return these tables but only the namespace 'nsA',
2555   * while the {@link #listTablesInRSGroup(String)} will return all these tables.
2556   * @param groupName the group name
2557   * @throws IOException if a remote or network exception occurs
2558   * @see #listTablesInRSGroup(String)
2559   */
2560  Pair<List<String>, List<TableName>> getConfiguredNamespacesAndTablesInRSGroup(String groupName)
2561    throws IOException;
2562
2563  /**
2564   * Remove RegionServer group associated with the given name
2565   * @param groupName the group name
2566   * @throws IOException if a remote or network exception occurs
2567   */
2568  void removeRSGroup(String groupName) throws IOException;
2569
2570  /**
2571   * Remove decommissioned servers from group 1. Sometimes we may find the server aborted due to
2572   * some hardware failure and we must offline the server for repairing. Or we need to move some
2573   * servers to join other clusters. So we need to remove these servers from the group. 2.
2574   * Dead/recovering/live servers will be disallowed.
2575   * @param servers set of servers to remove
2576   * @throws IOException if a remote or network exception occurs
2577   */
2578  void removeServersFromRSGroup(Set<Address> servers) throws IOException;
2579
2580  /**
2581   * Move given set of servers to the specified target RegionServer group
2582   * @param servers     set of servers to move
2583   * @param targetGroup the group to move servers to
2584   * @throws IOException if a remote or network exception occurs
2585   */
2586  void moveServersToRSGroup(Set<Address> servers, String targetGroup) throws IOException;
2587
2588  /**
2589   * Set the RegionServer group for tables
2590   * @param tables    tables to set group for
2591   * @param groupName group name for tables
2592   * @throws IOException if a remote or network exception occurs
2593   */
2594  void setRSGroup(Set<TableName> tables, String groupName) throws IOException;
2595
2596  /**
2597   * Balance regions in the given RegionServer group
2598   * @param groupName the group name
2599   * @return BalanceResponse details about the balancer run
2600   * @throws IOException if a remote or network exception occurs
2601   */
2602  default BalanceResponse balanceRSGroup(String groupName) throws IOException {
2603    return balanceRSGroup(groupName, BalanceRequest.defaultInstance());
2604  }
2605
2606  /**
2607   * Balance regions in the given RegionServer group, running based on the given
2608   * {@link BalanceRequest}.
2609   * @return BalanceResponse details about the balancer run
2610   */
2611  BalanceResponse balanceRSGroup(String groupName, BalanceRequest request) throws IOException;
2612
2613  /**
2614   * Rename rsgroup
2615   * @param oldName old rsgroup name
2616   * @param newName new rsgroup name
2617   * @throws IOException if a remote or network exception occurs
2618   */
2619  void renameRSGroup(String oldName, String newName) throws IOException;
2620
2621  /**
2622   * Update RSGroup configuration
2623   * @param groupName     the group name
2624   * @param configuration new configuration of the group name to be set
2625   * @throws IOException if a remote or network exception occurs
2626   */
2627  void updateRSGroupConfig(String groupName, Map<String, String> configuration) throws IOException;
2628
2629  /**
2630   * Retrieve recent online records from HMaster / RegionServers. Examples include slow/large RPC
2631   * logs, balancer decisions by master.
2632   * @param serverNames  servers to retrieve records from, useful in case of records maintained by
2633   *                     RegionServer as we can select specific server. In case of
2634   *                     servertype=MASTER, logs will only come from the currently active master.
2635   * @param logType      string representing type of log records
2636   * @param serverType   enum for server type: HMaster or RegionServer
2637   * @param limit        put a limit to list of records that server should send in response
2638   * @param filterParams additional filter params
2639   * @return Log entries representing online records from servers
2640   * @throws IOException if a remote or network exception occurs
2641   */
2642  List<LogEntry> getLogEntries(Set<ServerName> serverNames, String logType, ServerType serverType,
2643    int limit, Map<String, Object> filterParams) throws IOException;
2644
2645  /**
2646   * Flush master local region
2647   */
2648  void flushMasterStore() throws IOException;
2649
2650  /**
2651   * Get the list of cached files
2652   */
2653  List<String> getCachedFilesList(ServerName serverName) throws IOException;
2654}