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