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