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