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