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