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