Interface Admin

All Superinterfaces:
Abortable, AutoCloseable, Closeable
All Known Implementing Classes:
HBaseAdmin, ThriftAdmin

@Public public interface Admin extends Abortable, Closeable
The administrative API for HBase. Obtain an instance from Connection.getAdmin() and call close() when done.

Admin can be used to create, drop, list, enable and disable and otherwise modify tables, as well as perform other administrative operations.

Since:
0.99.0
See Also:
  • Method Details

    • getOperationTimeout

      Return the operation timeout for a rpc call.
      See Also:
    • getSyncWaitTimeout

      Return the blocking wait time for an asynchronous operation. Can be configured by hbase.client.sync.wait.timeout.msec.

      For several operations, such as createTable, deleteTable, etc, the rpc call will finish right after we schedule a procedure at master side, so the timeout will not be controlled by the above getOperationTimeout(). And timeout value here tells you how much time we will wait until the procedure at master side is finished.

      In general, you can consider that the implementation for XXXX method is just a XXXXAsync().get(getSyncWaitTimeout(), TimeUnit.MILLISECONDS).

      See Also:
    • abort

      void abort(String why, Throwable e)
      Description copied from interface: Abortable
      Abort the server or client.
      Specified by:
      abort in interface Abortable
      Parameters:
      why - Why we're aborting.
      e - Throwable that caused abort. Can be null.
    • isAborted

      boolean isAborted()
      Description copied from interface: Abortable
      Check if the server or client was aborted.
      Specified by:
      isAborted in interface Abortable
      Returns:
      true if the server or client was aborted, false otherwise
    • getConnection

      Returns Connection used by this object.
    • tableExists

      boolean tableExists(TableName tableName) throws IOException
      Check if a table exists.
      Parameters:
      tableName - Table to check.
      Returns:
      true if table exists already.
      Throws:
      IOException - if a remote or network exception occurs
    • listTables

      Deprecated.
      since 2.0 version and will be removed in 3.0 version. Use listTableDescriptors().
      List all the userspace tables.
      Returns:
      an array of read-only HTableDescriptors
      Throws:
      IOException - if a remote or network exception occurs
      See Also:
    • listTableDescriptors

      List all the userspace tables.
      Returns:
      a list of TableDescriptors
      Throws:
      IOException - if a remote or network exception occurs
    • listTableDescriptorsByState

      List all enabled or disabled tables
      Parameters:
      isEnabled - is true means return enabled tables, false means return disabled tables
      Returns:
      a list of enabled or disabled tables
      Throws:
      IOException
    • listTables

      Deprecated.
      since 2.0 version and will be removed in 3.0 version. Use listTableDescriptors(java.util.regex.Pattern).
      List all the userspace tables that match the given pattern.
      Parameters:
      pattern - The compiled regular expression to match against
      Returns:
      an array of read-only HTableDescriptors
      Throws:
      IOException - if a remote or network exception occurs
      See Also:
    • listTableDescriptors

      List all the userspace tables that match the given pattern.
      Parameters:
      pattern - The compiled regular expression to match against
      Returns:
      a list of TableDescriptors
      Throws:
      IOException - if a remote or network exception occurs
      See Also:
    • listTables

      Deprecated.
      since 2.0 version and will be removed in 3.0 version. Use listTableDescriptors(Pattern) instead.
      List all the userspace tables matching the given regular expression.
      Parameters:
      regex - The regular expression to match against
      Returns:
      a list of read-only HTableDescriptors
      Throws:
      IOException - if a remote or network exception occurs
      See Also:
    • listTables

      @Deprecated HTableDescriptor[] listTables(Pattern pattern, boolean includeSysTables) throws IOException
      Deprecated.
      since 2.0 version and will be removed in 3.0 version. Use listTableDescriptors(java.util.regex.Pattern, boolean).
      List all the tables matching the given pattern.
      Parameters:
      pattern - The compiled regular expression to match against
      includeSysTables - false to match only against userspace tables
      Returns:
      an array of read-only HTableDescriptors
      Throws:
      IOException - if a remote or network exception occurs
      See Also:
    • listTableDescriptors

      List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables) throws IOException
      List all the tables matching the given pattern.
      Parameters:
      pattern - The compiled regular expression to match against
      includeSysTables - false to match only against userspace tables
      Returns:
      a list of TableDescriptors
      Throws:
      IOException - if a remote or network exception occurs
      See Also:
    • listTables

      @Deprecated HTableDescriptor[] listTables(String regex, boolean includeSysTables) throws IOException
      Deprecated.
      since 2.0 version and will be removed in 3.0 version. Use listTableDescriptors(Pattern, boolean).
      List all the tables matching the given pattern.
      Parameters:
      regex - The regular expression to match against
      includeSysTables - false to match only against userspace tables
      Returns:
      an array of read-only HTableDescriptors
      Throws:
      IOException - if a remote or network exception occurs
      See Also:
    • listTableNames

      List all of the names of userspace tables.
      Returns:
      TableName[] table names
      Throws:
      IOException - if a remote or network exception occurs
    • listTableNames

      default TableName[] listTableNames(Pattern pattern) throws IOException
      List all of the names of userspace tables.
      Parameters:
      pattern - The regular expression to match against
      Returns:
      array of table names
      Throws:
      IOException - if a remote or network exception occurs
    • listTableNames

      Deprecated.
      since 2.0 version and will be removed in 3.0 version. Use listTableNames(Pattern) instead.
      List all of the names of userspace tables.
      Parameters:
      regex - The regular expression to match against
      Returns:
      TableName[] table names
      Throws:
      IOException - if a remote or network exception occurs
    • listTableNames

      TableName[] listTableNames(Pattern pattern, boolean includeSysTables) throws IOException
      List all of the names of userspace tables.
      Parameters:
      pattern - The regular expression to match against
      includeSysTables - false to match only against userspace tables
      Returns:
      TableName[] table names
      Throws:
      IOException - if a remote or network exception occurs
    • listTableNames

      @Deprecated TableName[] listTableNames(String regex, boolean includeSysTables) throws IOException
      Deprecated.
      since 2.0 version and will be removed in 3.0 version. Use listTableNames(Pattern, boolean) instead.
      List all of the names of userspace tables.
      Parameters:
      regex - The regular expression to match against
      includeSysTables - false to match only against userspace tables
      Returns:
      TableName[] table names
      Throws:
      IOException - if a remote or network exception occurs
    • listTableNamesByState

      List<TableName> listTableNamesByState(boolean isEnabled) throws IOException
      List all enabled or disabled table names
      Parameters:
      isEnabled - is true means return enabled table names, false means return disabled table names
      Returns:
      a list of enabled or disabled table names
      Throws:
      IOException
    • getTableDescriptor

      Deprecated.
      since 2.0 version and will be removed in 3.0 version. Use getDescriptor(TableName).
      Get a table descriptor.
      Parameters:
      tableName - as a TableName
      Returns:
      the read-only tableDescriptor
      Throws:
      TableNotFoundException - if the table was not found
      IOException - if a remote or network exception occurs
    • getDescriptor

      Get a table descriptor.
      Parameters:
      tableName - as a TableName
      Returns:
      the tableDescriptor
      Throws:
      TableNotFoundException - if the table was not found
      IOException - if a remote or network exception occurs
    • createTable

      default void createTable(TableDescriptor desc) throws IOException
      Creates a new table. Synchronous operation.
      Parameters:
      desc - table descriptor for table
      Throws:
      IllegalArgumentException - if the table name is reserved
      MasterNotRunningException - if master is not running
      TableExistsException - if table already exists (If concurrent threads, the table may have been created between test-for-existence and attempt-at-creation).
      IOException - if a remote or network exception occurs
    • createTable

      void createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions) throws IOException
      Creates a new table with the specified number of regions. The start key specified will become the end key of the first region of the table, and the end key specified will become the start key of the last region of the table (the first region has a null start key and the last region has a null end key). BigInteger math will be used to divide the key range specified into enough segments to make the required number of total regions. Synchronous operation.
      Parameters:
      desc - table descriptor for table
      startKey - beginning of key range
      endKey - end of key range
      numRegions - the total number of regions to create
      Throws:
      IllegalArgumentException - if the table name is reserved
      IOException - if a remote or network exception occurs
      MasterNotRunningException - if master is not running
      TableExistsException - if table already exists (If concurrent threads, the table may have been created between test-for-existence and attempt-at-creation).
    • createTable

      default void createTable(TableDescriptor desc, byte[][] splitKeys) throws IOException
      Creates a new table with an initial set of empty regions defined by the specified split keys. The total number of regions created will be the number of split keys plus one. Synchronous operation. Note : Avoid passing empty split key.
      Parameters:
      desc - table descriptor for table
      splitKeys - array of split keys for the initial regions of the table
      Throws:
      IllegalArgumentException - if the table name is reserved, if the split keys are repeated and if the split key has empty byte array.
      MasterNotRunningException - if master is not running
      TableExistsException - if table already exists (If concurrent threads, the table may have been created between test-for-existence and attempt-at-creation).
      IOException - if a remote or network exception occurs
    • createTableAsync

      Creates a new table but does not block and wait for it to come online. You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete.

      Throws IllegalArgumentException Bad table name, if the split keys are repeated and if the split key has empty byte array.

      Parameters:
      desc - table descriptor for table
      Returns:
      the result of the async creation. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
      Throws:
      IOException - if a remote or network exception occurs
    • createTableAsync

      Future<Void> createTableAsync(TableDescriptor desc, byte[][] splitKeys) throws IOException
      Creates a new table but does not block and wait for it to come online. You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete.

      Throws IllegalArgumentException Bad table name, if the split keys are repeated and if the split key has empty byte array.

      Parameters:
      desc - table descriptor for table
      splitKeys - keys to check if the table has been created with all split keys
      Returns:
      the result of the async creation. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
      Throws:
      IOException - if a remote or network exception occurs
    • deleteTable

      default void deleteTable(TableName tableName) throws IOException
      Deletes a table. Synchronous operation.
      Parameters:
      tableName - name of table to delete
      Throws:
      IOException - if a remote or network exception occurs
    • deleteTableAsync

      Deletes the table but does not block and wait for it to be completely removed. You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete.
      Parameters:
      tableName - name of table to delete
      Returns:
      the result of the async delete. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
      Throws:
      IOException - if a remote or network exception occurs
    • deleteTables

      Deprecated.
      since 2.0 version and will be removed in 3.0 version This is just a trivial helper method without any magic. Consider using listTableDescriptors(Pattern) and deleteTable(TableName)
      Deletes tables matching the passed in pattern and wait on completion. Warning: Use this method carefully, there is no prompting and the effect is immediate. Consider using listTableDescriptors(Pattern) and deleteTable(org.apache.hadoop.hbase.TableName)
      Parameters:
      regex - The regular expression to match table names against
      Returns:
      Table descriptors for tables that couldn't be deleted. The return htds are read-only
      Throws:
      IOException - if a remote or network exception occurs
      See Also:
    • deleteTables

      Deprecated.
      since 2.0 version and will be removed in 3.0 version This is just a trivial helper method without any magic. Consider using listTableDescriptors(java.util.regex.Pattern) and deleteTable(TableName)
      Delete tables matching the passed in pattern and wait on completion. Warning: Use this method carefully, there is no prompting and the effect is immediate. Consider using listTableDescriptors(java.util.regex.Pattern) and deleteTable(org.apache.hadoop.hbase.TableName)
      Parameters:
      pattern - The pattern to match table names against
      Returns:
      Table descriptors for tables that couldn't be deleted The return htds are read-only
      Throws:
      IOException - if a remote or network exception occurs
    • truncateTable

      default void truncateTable(TableName tableName, boolean preserveSplits) throws IOException
      Truncate a table. Synchronous operation.
      Parameters:
      tableName - name of table to truncate
      preserveSplits - true if the splits should be preserved
      Throws:
      IOException - if a remote or network exception occurs
    • truncateTableAsync

      Future<Void> truncateTableAsync(TableName tableName, boolean preserveSplits) throws IOException
      Truncate the table but does not block and wait for it to be completely enabled. You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete.
      Parameters:
      tableName - name of table to delete
      preserveSplits - true if the splits should be preserved
      Returns:
      the result of the async truncate. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
      Throws:
      IOException - if a remote or network exception occurs
    • enableTable

      default void enableTable(TableName tableName) throws IOException
      Enable a table. May timeout. Use enableTableAsync(org.apache.hadoop.hbase.TableName) and isTableEnabled(org.apache.hadoop.hbase.TableName) instead. The table has to be in disabled state for it to be enabled.
      Parameters:
      tableName - name of the table
      Throws:
      IOException - if a remote or network exception occurs There could be couple types of IOException TableNotFoundException means the table doesn't exist. TableNotDisabledException means the table isn't in disabled state.
      See Also:
    • enableTableAsync

      Enable the table but does not block and wait for it to be completely enabled. You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete.
      Parameters:
      tableName - name of table to delete
      Returns:
      the result of the async enable. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
      Throws:
      IOException - if a remote or network exception occurs
    • enableTables

      Deprecated.
      since 2.0 version and will be removed in 3.0 version This is just a trivial helper method without any magic. Consider using listTableDescriptors(Pattern) and enableTable(org.apache.hadoop.hbase.TableName)
      Enable tables matching the passed in pattern and wait on completion. Warning: Use this method carefully, there is no prompting and the effect is immediate. Consider using listTableDescriptors(Pattern) and enableTable(org.apache.hadoop.hbase.TableName)
      Parameters:
      regex - The regular expression to match table names against
      Returns:
      Table descriptors for tables that couldn't be enabled. The return HTDs are read-only.
      Throws:
      IOException - if a remote or network exception occurs
      See Also:
    • enableTables

      Deprecated.
      since 2.0 version and will be removed in 3.0 version This is just a trivial helper method without any magic. Consider using listTableDescriptors(java.util.regex.Pattern) and enableTable(org.apache.hadoop.hbase.TableName)
      Enable tables matching the passed in pattern and wait on completion. Warning: Use this method carefully, there is no prompting and the effect is immediate. Consider using listTableDescriptors(java.util.regex.Pattern) and enableTable(org.apache.hadoop.hbase.TableName)
      Parameters:
      pattern - The pattern to match table names against
      Returns:
      Table descriptors for tables that couldn't be enabled. The return HTDs are read-only.
      Throws:
      IOException - if a remote or network exception occurs
    • disableTableAsync

      Disable the table but does not block and wait for it to be completely disabled. You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete.
      Parameters:
      tableName - name of table to delete
      Returns:
      the result of the async disable. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
      Throws:
      IOException - if a remote or network exception occurs
    • disableTable

      default void disableTable(TableName tableName) throws IOException
      Disable table and wait on completion. May timeout eventually. Use disableTableAsync(org.apache.hadoop.hbase.TableName) and isTableDisabled(org.apache.hadoop.hbase.TableName) instead. The table has to be in enabled state for it to be disabled.
      Throws:
      IOException - There could be couple types of IOException TableNotFoundException means the table doesn't exist. TableNotEnabledException means the table isn't in enabled state.
    • disableTables

      Deprecated.
      since 2.0 version and will be removed in 3.0 version This is just a trivial helper method without any magic. Consider using listTableDescriptors(Pattern) and disableTable(org.apache.hadoop.hbase.TableName)
      Disable tables matching the passed in pattern and wait on completion. Warning: Use this method carefully, there is no prompting and the effect is immediate. Consider using listTableDescriptors(Pattern) and disableTable(org.apache.hadoop.hbase.TableName)
      Parameters:
      regex - The regular expression to match table names against
      Returns:
      Table descriptors for tables that couldn't be disabled The return htds are read-only
      Throws:
      IOException - if a remote or network exception occurs
      See Also:
    • disableTables

      Deprecated.
      since 2.0 version and will be removed in 3.0 version This is just a trivial helper method without any magic. Consider using listTableDescriptors(java.util.regex.Pattern) and disableTable(org.apache.hadoop.hbase.TableName)
      Disable tables matching the passed in pattern and wait on completion. Warning: Use this method carefully, there is no prompting and the effect is immediate. Consider using listTableDescriptors(java.util.regex.Pattern) and disableTable(org.apache.hadoop.hbase.TableName)
      Parameters:
      pattern - The pattern to match table names against
      Returns:
      Table descriptors for tables that couldn't be disabled The return htds are read-only
      Throws:
      IOException - if a remote or network exception occurs
    • isTableEnabled

      boolean isTableEnabled(TableName tableName) throws IOException
      Check if a table is enabled.
      Parameters:
      tableName - name of table to check
      Returns:
      true if table is on-line
      Throws:
      IOException - if a remote or network exception occurs
    • isTableDisabled

      boolean isTableDisabled(TableName tableName) throws IOException
      Check if a table is disabled.
      Parameters:
      tableName - name of table to check
      Returns:
      true if table is off-line
      Throws:
      IOException - if a remote or network exception occurs
    • isTableAvailable

      boolean isTableAvailable(TableName tableName) throws IOException
      Check if a table is available.
      Parameters:
      tableName - name of table to check
      Returns:
      true if all regions of the table are available
      Throws:
      IOException - if a remote or network exception occurs
    • isTableAvailable

      @Deprecated boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws IOException
      Deprecated.
      Since 2.0.0. Will be removed in 3.0.0. Use isTableAvailable(TableName)
      Use this api to check if the table has been created with the specified number of splitkeys which was used while creating the given table. Note : If this api is used after a table's region gets splitted, the api may return false.
      Parameters:
      tableName - name of table to check
      splitKeys - keys to check if the table has been created with all split keys
      Throws:
      IOException - if a remote or network excpetion occurs
    • getAlterStatus

      Deprecated.
      Since 2.0.0. Will be removed in 3.0.0. No longer needed now you get a Future on an operation.
      Get the status of an alter (a.k.a modify) command - indicates how many regions have received the updated schema Asynchronous operation.
      Parameters:
      tableName - TableName instance
      Returns:
      Pair indicating the number of regions updated Pair.getFirst() is the regions that are yet to be updated Pair.getSecond() is the total number of regions of the table
      Throws:
      IOException - if a remote or network exception occurs
    • getAlterStatus

      Deprecated.
      Since 2.0.0. Will be removed in 3.0.0. No longer needed now you get a Future on an operation.
      Get the status of alter (a.k.a modify) command - indicates how many regions have received the updated schema Asynchronous operation.
      Parameters:
      tableName - name of the table to get the status of
      Returns:
      Pair indicating the number of regions updated Pair.getFirst() is the regions that are yet to be updated Pair.getSecond() is the total number of regions of the table
      Throws:
      IOException - if a remote or network exception occurs
    • addColumn

      @Deprecated default void addColumn(TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException
      Deprecated.
      As of release 2.0.0. This will be removed in HBase 3.0.0. Use addColumnFamily(TableName, ColumnFamilyDescriptor).
      Add a column family to an existing table. Synchronous operation. Use addColumnFamilyAsync(TableName, ColumnFamilyDescriptor) instead because it returns a Future from which you can learn whether success or failure.
      Parameters:
      tableName - name of the table to add column family to
      columnFamily - column family descriptor of column family to be added
      Throws:
      IOException - if a remote or network exception occurs
    • addColumnFamily

      default void addColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException
      Add a column family to an existing table. Synchronous operation. Use addColumnFamilyAsync(TableName, ColumnFamilyDescriptor) instead because it returns a Future from which you can learn whether success or failure.
      Parameters:
      tableName - name of the table to add column family to
      columnFamily - column family descriptor of column family to be added
      Throws:
      IOException - if a remote or network exception occurs
    • addColumnFamilyAsync

      Add a column family to an existing table. Asynchronous operation. You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete.
      Parameters:
      tableName - name of the table to add column family to
      columnFamily - column family descriptor of column family to be added
      Returns:
      the result of the async add column family. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
      Throws:
      IOException - if a remote or network exception occurs
    • deleteColumn

      @Deprecated void deleteColumn(TableName tableName, byte[] columnFamily) throws IOException
      Deprecated.
      As of release 2.0.0. This will be removed in HBase 3.0.0. Use deleteColumnFamily(TableName, byte[])}.
      Delete a column family from a table. Synchronous operation. Use deleteColumnFamily(TableName, byte[]) instead because it returns a Future from which you can learn whether success or failure.
      Parameters:
      tableName - name of table
      columnFamily - name of column family to be deleted
      Throws:
      IOException - if a remote or network exception occurs
    • deleteColumnFamily

      default void deleteColumnFamily(TableName tableName, byte[] columnFamily) throws IOException
      Delete a column family from a table. Synchronous operation. Use deleteColumnFamily(TableName, byte[]) instead because it returns a Future from which you can learn whether success or failure.
      Parameters:
      tableName - name of table
      columnFamily - name of column family to be deleted
      Throws:
      IOException - if a remote or network exception occurs
    • deleteColumnFamilyAsync

      Future<Void> deleteColumnFamilyAsync(TableName tableName, byte[] columnFamily) throws IOException
      Delete a column family from a table. Asynchronous operation. You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete.
      Parameters:
      tableName - name of table
      columnFamily - name of column family to be deleted
      Returns:
      the result of the async delete column family. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
      Throws:
      IOException - if a remote or network exception occurs
    • modifyColumn

      @Deprecated default void modifyColumn(TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException
      Deprecated.
      As of release 2.0.0. This will be removed in HBase 3.0.0. Use modifyColumnFamily(TableName, ColumnFamilyDescriptor).
      Modify an existing column family on a table. Synchronous operation. Use modifyColumnFamilyAsync(TableName, ColumnFamilyDescriptor) instead because it returns a Future from which you can learn whether success or failure.
      Parameters:
      tableName - name of table
      columnFamily - new column family descriptor to use
      Throws:
      IOException - if a remote or network exception occurs
    • modifyColumnFamily

      default void modifyColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException
      Modify an existing column family on a table. Synchronous operation. Use modifyColumnFamilyAsync(TableName, ColumnFamilyDescriptor) instead because it returns a Future from which you can learn whether success or failure.
      Parameters:
      tableName - name of table
      columnFamily - new column family descriptor to use
      Throws:
      IOException - if a remote or network exception occurs
    • modifyColumnFamilyAsync

      Modify an existing column family on a table. Asynchronous operation. You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete.
      Parameters:
      tableName - name of table
      columnFamily - new column family descriptor to use
      Returns:
      the result of the async modify column family. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
      Throws:
      IOException - if a remote or network exception occurs
    • modifyColumnFamilyStoreFileTracker

      default void modifyColumnFamilyStoreFileTracker(TableName tableName, byte[] family, String dstSFT) throws IOException
      Change the store file tracker of the given table's given family.
      Parameters:
      tableName - the table you want to change
      family - the family you want to change
      dstSFT - the destination store file tracker
      Throws:
      IOException - if a remote or network exception occurs
    • modifyColumnFamilyStoreFileTrackerAsync

      Future<Void> modifyColumnFamilyStoreFileTrackerAsync(TableName tableName, byte[] family, String dstSFT) throws IOException
      Change the store file tracker of the given table's given family.
      Parameters:
      tableName - the table you want to change
      family - the family you want to change
      dstSFT - the destination store file tracker
      Returns:
      the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the operation to complete
      Throws:
      IOException - if a remote or network exception occurs
    • closeRegion

      @Deprecated void closeRegion(String regionname, String serverName) throws IOException
      Deprecated.
      As of release 2.0.0, this will be removed in HBase 3.0.0. Use unassign(byte[], boolean).
      Uses unassign(byte[], boolean) to unassign the region. For expert-admins.
      Parameters:
      regionname - region name to close
      serverName - Deprecated. Not used.
      Throws:
      IOException - if a remote or network exception occurs
    • closeRegion

      @Deprecated void closeRegion(byte[] regionname, String serverName) throws IOException
      Deprecated.
      As of release 2.0.0, this will be removed in HBase 3.0.0. Use unassign(byte[], boolean).
      Uses unassign(byte[], boolean) to unassign the region. For expert-admins.
      Parameters:
      regionname - region name to close
      serverName - Deprecated. Not used.
      Throws:
      IOException - if a remote or network exception occurs
    • closeRegionWithEncodedRegionName

      @Deprecated boolean closeRegionWithEncodedRegionName(String encodedRegionName, String serverName) throws IOException
      Deprecated.
      As of release 2.0.0, this will be removed in HBase 3.0.0. Use unassign(byte[], boolean).
      Uses unassign(byte[], boolean) to unassign the region. For expert-admins.
      Parameters:
      encodedRegionName - The encoded region name; i.e. the hash that makes up the region name suffix: e.g. if regionname is TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396., then the encoded region name is: 527db22f95c8a9e0116f0cc13c680396.
      serverName - Deprecated. Not used.
      Returns:
      Deprecated. Returns true always.
      Throws:
      IOException - if a remote or network exception occurs
    • closeRegion

      Deprecated.
      As of release 2.0.0, this will be removed in HBase 3.0.0 (HBASE-18231). Use unassign(byte[], boolean).
      Used unassign(byte[], boolean) to unassign the region. For expert-admins.
      Parameters:
      sn - Deprecated. Not used.
      Throws:
      IOException - if a remote or network exception occurs
    • getOnlineRegions

      Deprecated.
      As of release 2.0.0, this will be removed in HBase 3.0.0 (HBASE-17980). Use getRegions(ServerName sn).
      Get all the online regions on a region server.
      Throws:
      IOException - if a remote or network exception occurs
    • getRegions

      Get all the online regions on a region server.
      Returns:
      List of RegionInfo
      Throws:
      IOException - if a remote or network exception occurs
    • flush

      void flush(TableName tableName) throws IOException
      Flush a table. Synchronous operation.
      Parameters:
      tableName - table to flush
      Throws:
      IOException - if a remote or network exception occurs
    • flush

      void flush(TableName tableName, byte[] columnFamily) throws IOException
      Flush the specified column family stores on all regions of the passed table. This runs as a synchronous operation.
      Parameters:
      tableName - table to flush
      columnFamily - column family within a table
      Throws:
      IOException - if a remote or network exception occurs
    • flush

      default void flush(TableName tableName, List<byte[]> columnFamilies) throws IOException
      Flush the specified column family stores on all regions of the passed table. This runs as a synchronous operation.
      Parameters:
      tableName - table to flush
      columnFamilies - column families within a table
      Throws:
      IOException - if a remote or network exception occurs
    • flushAsync

      Future<Void> flushAsync(TableName tableName, List<byte[]> columnFamilies) throws IOException
      Flush a table but does not block and wait for it to finish. You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete.
      Parameters:
      tableName - table to flush
      columnFamilies - column families within a table
      Returns:
      the result of the async creation. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
      Throws:
      IOException - if a remote or network exception occurs
    • flushRegion

      void flushRegion(byte[] regionName) throws IOException
      Flush an individual region. Synchronous operation.
      Parameters:
      regionName - region to flush
      Throws:
      IOException - if a remote or network exception occurs
    • flushRegion

      void flushRegion(byte[] regionName, byte[] columnFamily) throws IOException
      Flush a column family within a region. Synchronous operation.
      Parameters:
      regionName - region to flush
      columnFamily - column family within a region
      Throws:
      IOException - if a remote or network exception occurs
    • flushRegionServer

      void flushRegionServer(ServerName serverName) throws IOException
      Flush all regions on the region server. Synchronous operation.
      Parameters:
      serverName - the region server name to flush
      Throws:
      IOException - if a remote or network exception occurs
    • compact

      void compact(TableName tableName) throws IOException
      Compact a table. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).
      Parameters:
      tableName - table to compact
      Throws:
      IOException - if a remote or network exception occurs
    • compactRegion

      void compactRegion(byte[] regionName) throws IOException
      Compact an individual region. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).
      Parameters:
      regionName - region to compact
      Throws:
      IOException - if a remote or network exception occurs
    • compact

      void compact(TableName tableName, byte[] columnFamily) throws IOException
      Compact a column family within a table. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).
      Parameters:
      tableName - table to compact
      columnFamily - column family within a table
      Throws:
      IOException - if a remote or network exception occurs
    • compactRegion

      void compactRegion(byte[] regionName, byte[] columnFamily) throws IOException
      Compact a column family within a region. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).
      Parameters:
      regionName - region to compact
      columnFamily - column family within a region
      Throws:
      IOException - if a remote or network exception occurs
    • compact

      void compact(TableName tableName, CompactType compactType) throws IOException, InterruptedException
      Compact a table. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).
      Parameters:
      tableName - table to compact
      compactType - CompactType
      Throws:
      IOException - if a remote or network exception occurs
      InterruptedException
    • compact

      void compact(TableName tableName, byte[] columnFamily, CompactType compactType) throws IOException, InterruptedException
      Compact a column family within a table. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).
      Parameters:
      tableName - table to compact
      columnFamily - column family within a table
      compactType - CompactType
      Throws:
      IOException - if not a mob column family or if a remote or network exception occurs
      InterruptedException
    • majorCompact

      void majorCompact(TableName tableName) throws IOException
      Major compact a table. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).
      Parameters:
      tableName - table to major compact
      Throws:
      IOException - if a remote or network exception occurs
    • majorCompactRegion

      void majorCompactRegion(byte[] regionName) throws IOException
      Major compact a table or an individual region. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).
      Parameters:
      regionName - region to major compact
      Throws:
      IOException - if a remote or network exception occurs
    • majorCompact

      void majorCompact(TableName tableName, byte[] columnFamily) throws IOException
      Major compact a column family within a table. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).
      Parameters:
      tableName - table to major compact
      columnFamily - column family within a table
      Throws:
      IOException - if a remote or network exception occurs
    • majorCompactRegion

      void majorCompactRegion(byte[] regionName, byte[] columnFamily) throws IOException
      Major compact a column family within region. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).
      Parameters:
      regionName - egion to major compact
      columnFamily - column family within a region
      Throws:
      IOException - if a remote or network exception occurs
    • majorCompact

      void majorCompact(TableName tableName, CompactType compactType) throws IOException, InterruptedException
      Major compact a table. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).
      Parameters:
      tableName - table to compact
      compactType - CompactType
      Throws:
      IOException - if a remote or network exception occurs
      InterruptedException
    • majorCompact

      void majorCompact(TableName tableName, byte[] columnFamily, CompactType compactType) throws IOException, InterruptedException
      Major compact a column family within a table. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).
      Parameters:
      tableName - table to compact
      columnFamily - column family within a table
      compactType - CompactType
      Throws:
      IOException - if not a mob column family or if a remote or network exception occurs
      InterruptedException
    • compactRegionServer

      @Deprecated default void compactRegionServer(ServerName sn, boolean major) throws IOException, InterruptedException
      Deprecated.
      As of release 2.0.0, this will be removed in HBase 3.0.0. Use compactRegionServer(ServerName) or majorCompactRegionServer(ServerName).
      Compact all regions on the region server. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).
      Parameters:
      sn - the region server name
      major - if it's major compaction
      Throws:
      IOException - if a remote or network exception occurs
      InterruptedException
    • compactionSwitch

      Map<ServerName,Boolean> compactionSwitch(boolean switchState, List<String> serverNamesList) throws IOException
      Turn the compaction on or off. Disabling compactions will also interrupt any currently ongoing compactions. This state is ephemeral. The setting will be lost on restart. Compaction can also be enabled/disabled by modifying configuration hbase.regionserver.compaction.enabled in hbase-site.xml.
      Parameters:
      switchState - Set to true to enable, false to disable.
      serverNamesList - list of region servers.
      Returns:
      Previous compaction states for region servers
      Throws:
      IOException
    • compactRegionServer

      void compactRegionServer(ServerName serverName) throws IOException
      Compact all regions on the region server. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).
      Parameters:
      serverName - the region server name
      Throws:
      IOException - if a remote or network exception occurs
    • majorCompactRegionServer

      Major compact all regions on the region server. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).
      Parameters:
      serverName - the region server name
      Throws:
      IOException - if a remote or network exception occurs
    • move

      void move(byte[] encodedRegionName) throws IOException
      Move the region encodedRegionName to a random server.
      Parameters:
      encodedRegionName - The encoded region name; i.e. the hash that makes up the region name suffix: e.g. if regionname is TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396., then the encoded region name is: 527db22f95c8a9e0116f0cc13c680396.
      Throws:
      IOException - if we can't find a region named encodedRegionName
    • move

      @Deprecated default void move(byte[] encodedRegionName, byte[] destServerName) throws IOException
      Deprecated.
      since 2.2.0 and will be removed in 4.0.0. Use move(byte[], ServerName) instead. And if you want to move the region to a random server, please use move(byte[]).
      Move the region rencodedRegionName to destServerName.
      Parameters:
      encodedRegionName - The encoded region name; i.e. the hash that makes up the region name suffix: e.g. if regionname is TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396., then the encoded region name is: 527db22f95c8a9e0116f0cc13c680396.
      destServerName - The servername of the destination regionserver. If passed the empty byte array we'll assign to a random server. A server name is made of host, port and startcode. Here is an example: host187.example.com,60020,1289493121758
      Throws:
      IOException - if we can't find a region named encodedRegionName
      See Also:
    • move

      void move(byte[] encodedRegionName, ServerName destServerName) throws IOException
      Move the region rencodedRegionName to destServerName.
      Parameters:
      encodedRegionName - The encoded region name; i.e. the hash that makes up the region name suffix: e.g. if regionname is TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396., then the encoded region name is: 527db22f95c8a9e0116f0cc13c680396.
      destServerName - The servername of the destination regionserver. A server name is made of host, port and startcode. Here is an example: host187.example.com,60020,1289493121758
      Throws:
      IOException - if we can't find a region named encodedRegionName
    • assign

      void assign(byte[] regionName) throws IOException
      Assign a Region.
      Parameters:
      regionName - Region name to assign.
      Throws:
      IOException - if a remote or network exception occurs
    • unassign

      void unassign(byte[] regionName) throws IOException
      Unassign a Region.
      Parameters:
      regionName - Region name to unassign.
      Throws:
      IOException - if a remote or network exception occurs
    • unassign

      @Deprecated default void unassign(byte[] regionName, boolean force) throws IOException
      Deprecated.
      since 2.4.0 and will be removed in 4.0.0. Use unassign(byte[]) instead.
      Unassign a region from current hosting regionserver. Region will then be assigned to a regionserver chosen at random. Region could be reassigned back to the same server. Use move(byte[], ServerName) if you want to control the region movement.
      Parameters:
      regionName - Region to unassign. Will clear any existing RegionPlan if one found.
      force - If true, force unassign (Will remove region from regions-in-transition too if present. If results in double assignment use hbck -fix to resolve. To be used by experts).
      Throws:
      IOException - if a remote or network exception occurs
      See Also:
    • offline

      void offline(byte[] regionName) throws IOException
      Offline specified region from master's in-memory state. It will not attempt to reassign the region as in unassign. This API can be used when a region not served by any region server and still online as per Master's in memory state. If this API is incorrectly used on active region then master will loose track of that region. This is a special method that should be used by experts or hbck.
      Parameters:
      regionName - Region to offline.
      Throws:
      IOException - if a remote or network exception occurs
    • setBalancerRunning

      @Deprecated default boolean setBalancerRunning(boolean on, boolean synchronous) throws IOException
      Deprecated.
      Since 2.0.0. Will be removed in 3.0.0. Use balancerSwitch(boolean, boolean) instead.
      Turn the load balancer on or off.
      Parameters:
      synchronous - If true, it waits until current balance() call, if outstanding, to return.
      Returns:
      Previous balancer value
      Throws:
      IOException - if a remote or network exception occurs
    • balancerSwitch

      boolean balancerSwitch(boolean onOrOff, boolean synchronous) throws IOException
      Turn the load balancer on or off.
      Parameters:
      onOrOff - Set to true to enable, false to disable.
      synchronous - If true, it waits until current balance() call, if outstanding, to return.
      Returns:
      Previous balancer value
      Throws:
      IOException - if a remote or network exception occurs
    • balancer

      @Deprecated default boolean balancer() throws IOException
      Deprecated.
      Since 2.0.0. Will be removed in 3.0.0. Use balance() instead.
      Invoke the balancer. Will run the balancer and if regions to move, it will go ahead and do the reassignments. Can NOT run for various reasons. Check logs.
      Returns:
      true if balancer ran, false otherwise.
      Throws:
      IOException - if a remote or network exception occurs
    • balance

      default boolean balance() throws IOException
      Invoke the balancer. Will run the balancer and if regions to move, it will go ahead and do the reassignments. Can NOT run for various reasons. Check logs.
      Returns:
      true if balancer ran, false otherwise.
      Throws:
      IOException - if a remote or network exception occurs
    • balance

      Invoke the balancer with the given balance request. The BalanceRequest defines how the balancer will run. See BalanceRequest for more details.
      Parameters:
      request - defines how the balancer should run
      Returns:
      BalanceResponse with details about the results of the invocation.
      Throws:
      IOException - if a remote or network exception occurs
    • balancer

      @Deprecated default boolean balancer(boolean force) throws IOException
      Deprecated.
      Since 2.0.0. Will be removed in 3.0.0. Use balance(BalanceRequest) instead.
      Invoke the balancer. Will run the balancer and if regions to move, it will go ahead and do the reassignments. If there is region in transition, force parameter of true would still run balancer. Can *not* run for other reasons. Check logs.
      Parameters:
      force - whether we should force balance even if there is region in transition
      Returns:
      true if balancer ran, false otherwise.
      Throws:
      IOException - if a remote or network exception occurs
    • balance

      @Deprecated default boolean balance(boolean force) throws IOException
      Deprecated.
      Since 2.5.0. Will be removed in 4.0.0. Use balance(BalanceRequest) instead.
      Invoke the balancer. Will run the balancer and if regions to move, it will go ahead and do the reassignments. If there is region in transition, force parameter of true would still run balancer. Can *not* run for other reasons. Check logs.
      Parameters:
      force - whether we should force balance even if there is region in transition
      Returns:
      true if balancer ran, false otherwise.
      Throws:
      IOException - if a remote or network exception occurs
    • isBalancerEnabled

      boolean isBalancerEnabled() throws IOException
      Query the current state of the balancer.
      Returns:
      true if the balancer is enabled, false otherwise.
      Throws:
      IOException - if a remote or network exception occurs
    • clearBlockCache

      Clear all the blocks corresponding to this table from BlockCache. For expert-admins. Calling this API will drop all the cached blocks specific to a table from BlockCache. This can significantly impact the query performance as the subsequent queries will have to retrieve the blocks from underlying filesystem.
      Parameters:
      tableName - table to clear block cache
      Returns:
      CacheEvictionStats related to the eviction
      Throws:
      IOException - if a remote or network exception occurs
    • normalize

      default boolean normalize() throws IOException
      Invoke region normalizer. Can NOT run for various reasons. Check logs. This is a non-blocking invocation to region normalizer. If return value is true, it means the request was submitted successfully. We need to check logs for the details of which regions were split/merged.
      Returns:
      true if region normalizer ran, false otherwise.
      Throws:
      IOException - if a remote or network exception occurs
    • normalize

      Invoke region normalizer. Can NOT run for various reasons. Check logs. This is a non-blocking invocation to region normalizer. If return value is true, it means the request was submitted successfully. We need to check logs for the details of which regions were split/merged.
      Parameters:
      ntfp - limit to tables matching the specified filter.
      Returns:
      true if region normalizer ran, false otherwise.
      Throws:
      IOException - if a remote or network exception occurs
    • isNormalizerEnabled

      Query the current state of the region normalizer.
      Returns:
      true if region normalizer is enabled, false otherwise.
      Throws:
      IOException - if a remote or network exception occurs
    • setNormalizerRunning

      @Deprecated default boolean setNormalizerRunning(boolean on) throws IOException
      Deprecated.
      Since 2.0.0. Will be removed in 3.0.0. Use normalizerSwitch(boolean)} instead.
      Turn region normalizer on or off.
      Returns:
      Previous normalizer value
      Throws:
      IOException - if a remote or network exception occurs
    • normalizerSwitch

      boolean normalizerSwitch(boolean on) throws IOException
      Turn region normalizer on or off.
      Returns:
      Previous normalizer value
      Throws:
      IOException - if a remote or network exception occurs
    • enableCatalogJanitor

      @Deprecated default boolean enableCatalogJanitor(boolean enable) throws IOException
      Deprecated.
      Since 2.0.0. Will be removed in 3.0.0. Use catalogJanitorSwitch(boolean)} instead.
      Enable/Disable the catalog janitor.
      Parameters:
      enable - if true enables the catalog janitor
      Returns:
      the previous state
      Throws:
      IOException - if a remote or network exception occurs
    • catalogJanitorSwitch

      boolean catalogJanitorSwitch(boolean onOrOff) throws IOException
      Enable/Disable the catalog janitor/
      Parameters:
      onOrOff - if true enables the catalog janitor
      Returns:
      the previous state
      Throws:
      IOException - if a remote or network exception occurs
    • runCatalogScan

      @Deprecated default int runCatalogScan() throws IOException
      Deprecated.
      Since 2.0.0. Will be removed in 3.0.0. Use runCatalogJanitor()} instead.
      Ask for a scan of the catalog table.
      Returns:
      the number of entries cleaned. Returns -1 if previous run is in progress.
      Throws:
      IOException - if a remote or network exception occurs
    • runCatalogJanitor

      Ask for a scan of the catalog table.
      Returns:
      the number of entries cleaned
      Throws:
      IOException - if a remote or network exception occurs
    • isCatalogJanitorEnabled

      Query on the catalog janitor state (Enabled/Disabled?).
      Throws:
      IOException - if a remote or network exception occurs
    • setCleanerChoreRunning

      @Deprecated default boolean setCleanerChoreRunning(boolean on) throws IOException
      Deprecated.
      Since 2.0.0. Will be removed in 3.0.0. Use cleanerChoreSwitch(boolean)} instead.
      Enable/Disable the cleaner chore.
      Parameters:
      on - if true enables the cleaner chore
      Returns:
      the previous state
      Throws:
      IOException - if a remote or network exception occurs
    • cleanerChoreSwitch

      boolean cleanerChoreSwitch(boolean onOrOff) throws IOException
      Enable/Disable the cleaner chore.
      Parameters:
      onOrOff - if true enables the cleaner chore
      Returns:
      the previous state
      Throws:
      IOException - if a remote or network exception occurs
    • runCleanerChore

      boolean runCleanerChore() throws IOException
      Ask for cleaner chore to run.
      Returns:
      true if cleaner chore ran, false otherwise
      Throws:
      IOException - if a remote or network exception occurs
    • isCleanerChoreEnabled

      Query on the cleaner chore state (Enabled/Disabled?).
      Throws:
      IOException - if a remote or network exception occurs
    • mergeRegions

      @Deprecated void mergeRegions(byte[] nameOfRegionA, byte[] nameOfRegionB, boolean forcible) throws IOException
      Deprecated.
      Since 2.0. Will be removed in 3.0. Use mergeRegionsAsync(byte[], byte[], boolean) instead.
      Merge two regions. Asynchronous operation.
      Parameters:
      nameOfRegionA - encoded or full name of region a
      nameOfRegionB - encoded or full name of region b
      forcible - true if do a compulsory merge, otherwise we will only merge two adjacent regions
      Throws:
      IOException - if a remote or network exception occurs
    • mergeRegionsAsync

      @Deprecated default Future<Void> mergeRegionsAsync(byte[] nameOfRegionA, byte[] nameOfRegionB, boolean forcible) throws IOException
      Deprecated.
      since 2.3.0 and will be removed in 4.0.0. Multi-region merge feature is now supported. Use mergeRegionsAsync(byte[][], boolean) instead.
      Merge two regions. Asynchronous operation.
      Parameters:
      nameOfRegionA - encoded or full name of region a
      nameOfRegionB - encoded or full name of region b
      forcible - true if do a compulsory merge, otherwise we will only merge two adjacent regions
      Throws:
      IOException - if a remote or network exception occurs
    • mergeRegionsAsync

      Future<Void> mergeRegionsAsync(byte[][] nameofRegionsToMerge, boolean forcible) throws IOException
      Merge multiple regions (>=2). Asynchronous operation.
      Parameters:
      nameofRegionsToMerge - encoded or full name of daughter regions
      forcible - true if do a compulsory merge, otherwise we will only merge adjacent regions
      Throws:
      IOException - if a remote or network exception occurs
    • split

      void split(TableName tableName) throws IOException
      Split a table. The method will execute split action for each region in table. Asynchronous operation.
      Parameters:
      tableName - table to split
      Throws:
      IOException - if a remote or network exception occurs
    • splitRegion

      @Deprecated void splitRegion(byte[] regionName) throws IOException
      Deprecated.
      As of release 2.0.0, this will be removed in HBase 3.0.0. Use splitRegionAsync(byte[], byte[]).
      Split an individual region. Asynchronous operation.
      Parameters:
      regionName - region to split
      Throws:
      IOException - if a remote or network exception occurs
    • split

      void split(TableName tableName, byte[] splitPoint) throws IOException
      Split a table. Asynchronous operation.
      Parameters:
      tableName - table to split
      splitPoint - the explicit position to split on
      Throws:
      IOException - if a remote or network exception occurs
    • splitRegion

      @Deprecated void splitRegion(byte[] regionName, byte[] splitPoint) throws IOException
      Deprecated.
      As of release 2.0.0, this will be removed in HBase 3.0.0. Use splitRegionAsync(byte[], byte[]).
      Split an individual region. Asynchronous operation.
      Parameters:
      regionName - region to split
      splitPoint - the explicit position to split on
      Throws:
      IOException - if a remote or network exception occurs
    • splitRegionAsync

      Future<Void> splitRegionAsync(byte[] regionName) throws IOException
      Split an individual region. Asynchronous operation.
      Parameters:
      regionName - region to split
      Throws:
      IOException - if a remote or network exception occurs
    • splitRegionAsync

      Future<Void> splitRegionAsync(byte[] regionName, byte[] splitPoint) throws IOException
      Split an individual region. Asynchronous operation.
      Parameters:
      regionName - region to split
      splitPoint - the explicit position to split on
      Throws:
      IOException - if a remote or network exception occurs
    • modifyTable

      @Deprecated default void modifyTable(TableName tableName, TableDescriptor td) throws IOException
      Deprecated.
      since 2.0 version and will be removed in 3.0 version. use modifyTable(TableDescriptor)
      Modify an existing table, more IRB friendly version.
      Parameters:
      tableName - name of table.
      td - modified description of the table
      Throws:
      IOException - if a remote or network exception occurs
    • modifyTable

      default void modifyTable(TableDescriptor td, boolean reopenRegions) throws IOException
      Modify an existing table, more IRB friendly version.
      Parameters:
      td - modified description of the table
      reopenRegions - By default, 'modifyTable' reopens all regions, potentially causing a RIT(Region In Transition) storm in large tables. If set to 'false', regions will remain unaware of the modification until they are individually reopened. Please note that this may temporarily result in configuration inconsistencies among regions.
      Throws:
      IOException - if a remote or network exception occurs
    • modifyTable

      default void modifyTable(TableDescriptor td) throws IOException
      Modify an existing table, more IRB friendly version.
      Parameters:
      td - modified description of the table
      Throws:
      IOException - if a remote or network exception occurs
    • modifyTableAsync

      Deprecated.
      since 2.0 version and will be removed in 3.0 version. use modifyTableAsync(TableDescriptor, boolean)
      Modify an existing table, more IRB friendly version. Asynchronous operation. This means that it may be a while before your schema change is updated across all of the table. You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete.
      Parameters:
      tableName - name of table.
      td - modified description of the table
      Returns:
      the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the operation to complete
      Throws:
      IOException - if a remote or network exception occurs
    • modifyTableAsync

      Modify an existing table, more IRB (ruby) friendly version. Asynchronous operation. This means that it may be a while before your schema change is updated across all of the table. You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete.
      Parameters:
      td - description of the table
      Returns:
      the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the operation to complete
      Throws:
      IOException - if a remote or network exception occurs
    • modifyTableAsync

      Future<Void> modifyTableAsync(TableDescriptor td, boolean reopenRegions) throws IOException
      Modify an existing table, more IRB (ruby) friendly version. Asynchronous operation. This means that it may be a while before your schema change is updated across all of the table. You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete.
      Parameters:
      td - description of the table
      reopenRegions - By default, 'modifyTableAsync' reopens all regions, potentially causing a RIT(Region In Transition) storm in large tables. If set to 'false', regions will remain unaware of the modification until they are individually reopened. Please note that this may temporarily result in configuration inconsistencies among regions.
      Returns:
      the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the operation to complete
      Throws:
      IOException - if a remote or network exception occurs
    • modifyTableStoreFileTracker

      default void modifyTableStoreFileTracker(TableName tableName, String dstSFT) throws IOException
      Change the store file tracker of the given table.
      Parameters:
      tableName - the table you want to change
      dstSFT - the destination store file tracker
      Throws:
      IOException - if a remote or network exception occurs
    • modifyTableStoreFileTrackerAsync

      Change the store file tracker of the given table.
      Parameters:
      tableName - the table you want to change
      dstSFT - the destination store file tracker
      Returns:
      the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the operation to complete
      Throws:
      IOException - if a remote or network exception occurs
    • shutdown

      void shutdown() throws IOException
      Shuts down the HBase cluster.

      Notice that, a success shutdown call may ends with an error since the remote server has already been shutdown.

      Throws:
      IOException - if a remote or network exception occurs
    • stopMaster

      void stopMaster() throws IOException
      Shuts down the current HBase master only. Does not shutdown the cluster.

      Notice that, a success stopMaster call may ends with an error since the remote server has already been shutdown.

      Throws:
      IOException - if a remote or network exception occurs
      See Also:
    • isMasterInMaintenanceMode

      Check whether Master is in maintenance mode.
      Throws:
      IOException - if a remote or network exception occurs
    • stopRegionServer

      void stopRegionServer(String hostnamePort) throws IOException
      Stop the designated regionserver.
      Parameters:
      hostnamePort - Hostname and port delimited by a : as in example.org:1234
      Throws:
      IOException - if a remote or network exception occurs
    • getClusterStatus

      Deprecated.
      since 2.0 version and will be removed in 3.0 version. use getClusterMetrics()
      Get whole cluster status, containing status about:
       hbase version
       cluster id
       primary/backup master(s)
       master's coprocessors
       live/dead regionservers
       balancer
       regions in transition
       
      Returns:
      cluster status
      Throws:
      IOException - if a remote or network exception occurs
    • getClusterMetrics

      Get whole cluster metrics, containing status about:
       hbase version
       cluster id
       primary/backup master(s)
       master's coprocessors
       live/dead regionservers
       balancer
       regions in transition
       
      Returns:
      cluster metrics
      Throws:
      IOException - if a remote or network exception occurs
    • getClusterMetrics

      Get cluster status with a set of ClusterMetrics.Option to get desired status.
      Returns:
      cluster status
      Throws:
      IOException - if a remote or network exception occurs
    • getMaster

      default ServerName getMaster() throws IOException
      Get the current active master.
      Returns:
      current master server name
      Throws:
      IOException - if a remote or network exception occurs
    • getBackupMasters

      Get a list of current backup masters.
      Returns:
      current backup master list
      Throws:
      IOException - if a remote or network exception occurs
    • getRegionServers

      Get the live server list.
      Returns:
      current live region servers list
      Throws:
      IOException - if a remote or network exception occurs
    • getRegionServers

      default Collection<ServerName> getRegionServers(boolean excludeDecommissionedRS) throws IOException
      Retrieve all current live region servers including decommissioned if excludeDecommissionedRS is false, else non-decommissioned ones only
      Parameters:
      excludeDecommissionedRS - should we exclude decommissioned RS nodes
      Returns:
      all current live region servers including/excluding decommissioned hosts
      Throws:
      IOException - if a remote or network exception occurs
    • getRegionMetrics

      Get RegionMetrics of all regions hosted on a regionserver.
      Parameters:
      serverName - region server from which RegionMetrics is required.
      Returns:
      a RegionMetrics list of all regions hosted on a region server
      Throws:
      IOException - if a remote or network exception occurs
    • getRegionMetrics

      Get RegionMetrics of all regions hosted on a regionserver for a table.
      Parameters:
      serverName - region server from which RegionMetrics is required.
      tableName - get RegionMetrics of regions belonging to the table
      Returns:
      region metrics map of all regions of a table hosted on a region server
      Throws:
      IOException - if a remote or network exception occurs
    • getConfiguration

      org.apache.hadoop.conf.Configuration getConfiguration()
      Returns Configuration used by the instance.
    • createNamespace

      default void createNamespace(NamespaceDescriptor descriptor) throws IOException
      Create a new namespace. Blocks until namespace has been successfully created or an exception is thrown.
      Parameters:
      descriptor - descriptor which describes the new namespace.
      Throws:
      IOException - if a remote or network exception occurs
    • createNamespaceAsync

      Create a new namespace.
      Parameters:
      descriptor - descriptor which describes the new namespace
      Returns:
      the result of the async create namespace operation. Use Future.get(long, TimeUnit) to wait on the operation to complete.
      Throws:
      IOException - if a remote or network exception occurs
    • modifyNamespace

      default void modifyNamespace(NamespaceDescriptor descriptor) throws IOException
      Modify an existing namespace. Blocks until namespace has been successfully modified or an exception is thrown.
      Parameters:
      descriptor - descriptor which describes the new namespace
      Throws:
      IOException - if a remote or network exception occurs
    • modifyNamespaceAsync

      Modify an existing namespace.
      Parameters:
      descriptor - descriptor which describes the new namespace
      Returns:
      the result of the async modify namespace operation. Use Future.get(long, TimeUnit) to wait on the operation to complete.
      Throws:
      IOException - if a remote or network exception occurs
    • deleteNamespace

      default void deleteNamespace(String name) throws IOException
      Delete an existing namespace. Only empty namespaces (no tables) can be removed. Blocks until namespace has been successfully deleted or an exception is thrown.
      Parameters:
      name - namespace name
      Throws:
      IOException - if a remote or network exception occurs
    • deleteNamespaceAsync

      Delete an existing namespace. Only empty namespaces (no tables) can be removed.
      Parameters:
      name - namespace name
      Returns:
      the result of the async delete namespace operation. Use Future.get(long, TimeUnit) to wait on the operation to complete.
      Throws:
      IOException - if a remote or network exception occurs
    • getNamespaceDescriptor

      Get a namespace descriptor by name.
      Parameters:
      name - name of namespace descriptor
      Returns:
      A descriptor
      Throws:
      NamespaceNotFoundException - if the namespace was not found
      IOException - if a remote or network exception occurs
    • listNamespaces

      List available namespaces
      Returns:
      List of namespace names
      Throws:
      IOException - if a remote or network exception occurs
    • listNamespaceDescriptors

      List available namespace descriptors
      Returns:
      List of descriptors
      Throws:
      IOException - if a remote or network exception occurs
    • listTableDescriptorsByNamespace

      Deprecated.
      since 2.0 version and will be removed in 3.0 version. use listTableDescriptorsByNamespace(byte[])
      Get list of table descriptors by namespace.
      Parameters:
      name - namespace name
      Returns:
      HTD[] the read-only tableDescriptors
      Throws:
      IOException - if a remote or network exception occurs
    • listTableDescriptorsByNamespace

      Get list of table descriptors by namespace.
      Parameters:
      name - namespace name
      Returns:
      returns a list of TableDescriptors
      Throws:
      IOException - if a remote or network exception occurs
    • listTableNamesByNamespace

      Get list of table names by namespace.
      Parameters:
      name - namespace name
      Returns:
      The list of table names in the namespace
      Throws:
      IOException - if a remote or network exception occurs
    • getTableRegions

      Deprecated.
      As of release 2.0.0, this will be removed in HBase 3.0.0 (HBASE-17980). Use getRegions(TableName).
      Get the regions of a given table.
      Parameters:
      tableName - the name of the table
      Returns:
      List of HRegionInfo.
      Throws:
      IOException - if a remote or network exception occurs
    • getRegions

      Get the regions of a given table.
      Parameters:
      tableName - the name of the table
      Returns:
      List of RegionInfo.
      Throws:
      IOException - if a remote or network exception occurs
    • close

      void close() throws IOException
      Specified by:
      close in interface AutoCloseable
      Specified by:
      close in interface Closeable
      Throws:
      IOException
    • getTableDescriptorsByTableName

      Deprecated.
      since 2.0 version and will be removed in 3.0 version. use listTableDescriptors(List)
      Get tableDescriptors.
      Parameters:
      tableNames - List of table names
      Returns:
      HTD[] the read-only tableDescriptors
      Throws:
      IOException - if a remote or network exception occurs
    • listTableDescriptors

      Get tableDescriptors.
      Parameters:
      tableNames - List of table names
      Returns:
      returns a list of TableDescriptors
      Throws:
      IOException - if a remote or network exception occurs
    • getTableDescriptors

      Deprecated.
      since 2.0 version and will be removed in 3.0 version. use listTableDescriptors(List)
      Get tableDescriptors.
      Parameters:
      names - List of table names
      Returns:
      HTD[] the read-only tableDescriptors
      Throws:
      IOException - if a remote or network exception occurs
    • abortProcedure

      @Deprecated default boolean abortProcedure(long procId, boolean mayInterruptIfRunning) throws IOException
      Deprecated.
      since 2.1.1 and will be removed in 4.0.0.
      Abort a procedure.

      Do not use. Usually it is ignored but if not, it can do more damage than good. See hbck2.

      Parameters:
      procId - ID of the procedure to abort
      mayInterruptIfRunning - if the proc completed at least one step, should it be aborted?
      Returns:
      true if aborted, false if procedure already completed or does not exist
      Throws:
      IOException - if a remote or network exception occurs
      See Also:
    • abortProcedureAsync

      @Deprecated Future<Boolean> abortProcedureAsync(long procId, boolean mayInterruptIfRunning) throws IOException
      Deprecated.
      since 2.1.1 and will be removed in 4.0.0.
      Abort a procedure but does not block and wait for completion. You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete. Do not use. Usually it is ignored but if not, it can do more damage than good. See hbck2.
      Parameters:
      procId - ID of the procedure to abort
      mayInterruptIfRunning - if the proc completed at least one step, should it be aborted?
      Returns:
      true if aborted, false if procedure already completed or does not exist
      Throws:
      IOException - if a remote or network exception occurs
      See Also:
    • getProcedures

      Get procedures.
      Returns:
      procedure list in JSON
      Throws:
      IOException - if a remote or network exception occurs
    • getLocks

      Get locks.
      Returns:
      lock list in JSON
      Throws:
      IOException - if a remote or network exception occurs
    • rollWALWriter

      Roll the log writer. I.e. for filesystem based write ahead logs, start writing to a new file. Note that the actual rolling of the log writer is asynchronous and may not be complete when this method returns. As a side effect of this call, the named region server may schedule store flushes at the request of the wal.
      Parameters:
      serverName - The servername of the regionserver.
      Throws:
      IOException - if a remote or network exception occurs
      FailedLogCloseException - if we failed to close the WAL
    • getMasterCoprocessors

      Deprecated.
      since 2.0 version and will be removed in 3.0 version. use getMasterCoprocessorNames()
      Helper that delegates to getClusterMetrics().getMasterCoprocessorNames().
      Returns:
      an array of master coprocessors
      Throws:
      IOException - if a remote or network exception occurs
      See Also:
    • getMasterCoprocessorNames

      Helper that delegates to getClusterMetrics().getMasterCoprocessorNames().
      Returns:
      an array of master coprocessors
      Throws:
      IOException - if a remote or network exception occurs
      See Also:
    • getCompactionState

      Get the current compaction state of a table. It could be in a major compaction, a minor compaction, both, or none.
      Parameters:
      tableName - table to examine
      Returns:
      the current compaction state
      Throws:
      IOException - if a remote or network exception occurs
    • getCompactionState

      Get the current compaction state of a table. It could be in a compaction, or none.
      Parameters:
      tableName - table to examine
      compactType - CompactType
      Returns:
      the current compaction state
      Throws:
      IOException - if a remote or network exception occurs
    • getCompactionStateForRegion

      Get the current compaction state of region. It could be in a major compaction, a minor compaction, both, or none.
      Parameters:
      regionName - region to examine
      Returns:
      the current compaction state
      Throws:
      IOException - if a remote or network exception occurs
    • getLastMajorCompactionTimestamp

      Get the timestamp of the last major compaction for the passed table The timestamp of the oldest HFile resulting from a major compaction of that table, or 0 if no such HFile could be found.
      Parameters:
      tableName - table to examine
      Returns:
      the last major compaction timestamp or 0
      Throws:
      IOException - if a remote or network exception occurs
    • getLastMajorCompactionTimestampForRegion

      long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException
      Get the timestamp of the last major compaction for the passed region. The timestamp of the oldest HFile resulting from a major compaction of that region, or 0 if no such HFile could be found.
      Parameters:
      regionName - region to examine
      Returns:
      the last major compaction timestamp or 0
      Throws:
      IOException - if a remote or network exception occurs
    • snapshot

      default void snapshot(String snapshotName, TableName tableName) throws IOException, SnapshotCreationException, IllegalArgumentException
      Take a snapshot for the given table. If the table is enabled, a FLUSH-type snapshot will be taken. If the table is disabled, an offline snapshot is taken. Snapshots are considered unique based on the name of the snapshot. Attempts to take a snapshot with the same name (even a different type or with different parameters) will fail with a SnapshotCreationException indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in HBase. See TableName.isLegalFullyQualifiedTableName(byte[]).
      Parameters:
      snapshotName - name of the snapshot to be created
      tableName - name of the table for which snapshot is created
      Throws:
      IOException - if a remote or network exception occurs
      SnapshotCreationException - if snapshot creation failed
      IllegalArgumentException - if the snapshot request is formatted incorrectly
    • snapshot

      @Deprecated default void snapshot(byte[] snapshotName, TableName tableName) throws IOException, SnapshotCreationException, IllegalArgumentException
      Deprecated.
      since 2.3.0, will be removed in 3.0.0. Use snapshot(String, TableName) instead.
      Create a timestamp consistent snapshot for the given table. Snapshots are considered unique based on the name of the snapshot. Attempts to take a snapshot with the same name (even different type or with different parameters) will fail with a SnapshotCreationException indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in HBase.
      Parameters:
      snapshotName - name of the snapshot to be created
      tableName - name of the table for which snapshot is created
      Throws:
      IOException - if a remote or network exception occurs
      SnapshotCreationException - if snapshot creation failed
      IllegalArgumentException - if the snapshot request is formatted incorrectly
    • snapshot

      default void snapshot(String snapshotName, TableName tableName, SnapshotType type) throws IOException, SnapshotCreationException, IllegalArgumentException
      Create typed snapshot of the table. Snapshots are considered unique based on the name of the snapshot. Attempts to take a snapshot with the same name (even a different type or with different parameters) will fail with a SnapshotCreationException indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in HBase. See TableName.isLegalFullyQualifiedTableName(byte[]).
      Parameters:
      snapshotName - name to give the snapshot on the filesystem. Must be unique from all other snapshots stored on the cluster
      tableName - name of the table to snapshot
      type - type of snapshot to take
      Throws:
      IOException - we fail to reach the master
      SnapshotCreationException - if snapshot creation failed
      IllegalArgumentException - if the snapshot request is formatted incorrectly
    • snapshot

      default void snapshot(String snapshotName, TableName tableName, SnapshotType type, Map<String,Object> snapshotProps) throws IOException, SnapshotCreationException, IllegalArgumentException
      Create typed snapshot of the table. Snapshots are considered unique based on the name of the snapshot. Snapshots are taken sequentially even when requested concurrently, across all tables. Attempts to take a snapshot with the same name (even a different type or with different parameters) will fail with a SnapshotCreationException indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in HBase. See TableName.isLegalFullyQualifiedTableName(byte[]). Snapshot can live with ttl seconds.
      Parameters:
      snapshotName - name to give the snapshot on the filesystem. Must be unique from all other snapshots stored on the cluster
      tableName - name of the table to snapshot
      type - type of snapshot to take
      snapshotProps - snapshot additional properties e.g. TTL
      Throws:
      IOException - we fail to reach the master
      SnapshotCreationException - if snapshot creation failed
      IllegalArgumentException - if the snapshot request is formatted incorrectly
    • snapshot

      default void snapshot(String snapshotName, TableName tableName, Map<String,Object> snapshotProps) throws IOException, SnapshotCreationException, IllegalArgumentException
      Create typed snapshot of the table. Snapshots are considered unique based on the name of the snapshot. Snapshots are taken sequentially even when requested concurrently, across all tables. Attempts to take a snapshot with the same name (even a different type or with different parameters) will fail with a SnapshotCreationException indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in HBase. See TableName.isLegalFullyQualifiedTableName(byte[]). Snapshot can live with ttl seconds.
      Parameters:
      snapshotName - name to give the snapshot on the filesystem. Must be unique from all other snapshots stored on the cluster
      tableName - name of the table to snapshot
      snapshotProps - snapshot additional properties e.g. TTL
      Throws:
      IOException - we fail to reach the master
      SnapshotCreationException - if snapshot creation failed
      IllegalArgumentException - if the snapshot request is formatted incorrectly
    • snapshot

      Take a snapshot and wait for the server to complete that snapshot (blocking). Snapshots are considered unique based on the name of the snapshot. Snapshots are taken sequentially even when requested concurrently, across all tables. Attempts to take a snapshot with the same name (even a different type or with different parameters) will fail with a SnapshotCreationException indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in HBase. See TableName.isLegalFullyQualifiedTableName(byte[]). You should probably use snapshot(String, org.apache.hadoop.hbase.TableName) or snapshot(byte[], org.apache.hadoop.hbase.TableName) unless you are sure about the type of snapshot that you want to take.
      Parameters:
      snapshot - snapshot to take
      Throws:
      IOException - or we lose contact with the master.
      SnapshotCreationException - if snapshot failed to be taken
      IllegalArgumentException - if the snapshot request is formatted incorrectly
    • takeSnapshotAsync

      Deprecated.
      Since 2.0.0. Will be removed in 3.0.0. Use snapshotAsync(SnapshotDescription) instead.
      Take a snapshot without waiting for the server to complete that snapshot (asynchronous) Only a single snapshot should be taken at a time, or results may be undefined.
      Parameters:
      snapshot - snapshot to take
      Throws:
      IOException - if the snapshot did not succeed or we lose contact with the master.
      SnapshotCreationException - if snapshot creation failed
      IllegalArgumentException - if the snapshot request is formatted incorrectly
    • snapshotAsync

      Take a snapshot without waiting for the server to complete that snapshot (asynchronous) Only a single snapshot should be taken at a time, or results may be undefined.
      Parameters:
      snapshot - snapshot to take
      Throws:
      IOException - if the snapshot did not succeed or we lose contact with the master.
      SnapshotCreationException - if snapshot creation failed
      IllegalArgumentException - if the snapshot request is formatted incorrectly
    • isSnapshotFinished

      Check the current state of the passed snapshot. There are three possible states:
      1. running - returns false
      2. finished - returns true
      3. finished with error - throws the exception that caused the snapshot to fail
      The cluster only knows about the most recent snapshot. Therefore, if another snapshot has been run/started since the snapshot you are checking, you will receive an UnknownSnapshotException.
      Parameters:
      snapshot - description of the snapshot to check
      Returns:
      true if the snapshot is completed, false if the snapshot is still running
      Throws:
      IOException - if we have a network issue
      HBaseSnapshotException - if the snapshot failed
      UnknownSnapshotException - if the requested snapshot is unknown
    • restoreSnapshot

      @Deprecated default void restoreSnapshot(byte[] snapshotName) throws IOException, RestoreSnapshotException
      Deprecated.
      since 2.3.0, will be removed in 3.0.0. Use restoreSnapshot(String) instead.
      Restore the specified snapshot on the original table. (The table must be disabled) If the "hbase.snapshot.restore.take.failsafe.snapshot" configuration property is set to true, a snapshot of the current table is taken before executing the restore operation. In case of restore failure, the failsafe snapshot will be restored. If the restore completes without problem the failsafe snapshot is deleted.
      Parameters:
      snapshotName - name of the snapshot to restore
      Throws:
      IOException - if a remote or network exception occurs
      RestoreSnapshotException - if snapshot failed to be restored
      IllegalArgumentException - if the restore request is formatted incorrectly
    • restoreSnapshot

      Restore the specified snapshot on the original table. (The table must be disabled) If the "hbase.snapshot.restore.take.failsafe.snapshot" configuration property is set to true, a snapshot of the current table is taken before executing the restore operation. In case of restore failure, the failsafe snapshot will be restored. If the restore completes without problem the failsafe snapshot is deleted.
      Parameters:
      snapshotName - name of the snapshot to restore
      Throws:
      IOException - if a remote or network exception occurs
      RestoreSnapshotException - if snapshot failed to be restored
      IllegalArgumentException - if the restore request is formatted incorrectly
    • restoreSnapshotAsync

      Deprecated.
      since 2.3.0, will be removed in 3.0.0. The implementation does not take care of the failsafe property, so do not use it any more.
      Restore the specified snapshot on the original table. (The table must be disabled) If the "hbase.snapshot.restore.take.failsafe.snapshot" configuration property is set to true, a snapshot of the current table is taken before executing the restore operation. In case of restore failure, the failsafe snapshot will be restored. If the restore completes without problem the failsafe snapshot is deleted.
      Parameters:
      snapshotName - name of the snapshot to restore
      Returns:
      the result of the async restore snapshot. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
      Throws:
      IOException - if a remote or network exception occurs
      RestoreSnapshotException - if snapshot failed to be restored
    • restoreSnapshot

      @Deprecated default void restoreSnapshot(byte[] snapshotName, boolean takeFailSafeSnapshot) throws IOException, RestoreSnapshotException
      Deprecated.
      since 2.3.0, will be removed in 3.0.0. Use restoreSnapshot(String, boolean) instead.
      Restore the specified snapshot on the original table. (The table must be disabled) If 'takeFailSafeSnapshot' is set to true, a snapshot of the current table is taken before executing the restore operation. In case of restore failure, the failsafe snapshot will be restored. If the restore completes without problem the failsafe snapshot is deleted. The failsafe snapshot name is configurable by using the property "hbase.snapshot.restore.failsafe.name".
      Parameters:
      snapshotName - name of the snapshot to restore
      takeFailSafeSnapshot - true if the failsafe snapshot should be taken
      Throws:
      IOException - if a remote or network exception occurs
      RestoreSnapshotException - if snapshot failed to be restored
      IllegalArgumentException - if the restore request is formatted incorrectly
    • restoreSnapshot

      default void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot) throws IOException, RestoreSnapshotException
      Restore the specified snapshot on the original table. (The table must be disabled) If 'takeFailSafeSnapshot' is set to true, a snapshot of the current table is taken before executing the restore operation. In case of restore failure, the failsafe snapshot will be restored. If the restore completes without problem the failsafe snapshot is deleted. The failsafe snapshot name is configurable by using the property "hbase.snapshot.restore.failsafe.name".
      Parameters:
      snapshotName - name of the snapshot to restore
      takeFailSafeSnapshot - true if the failsafe snapshot should be taken
      Throws:
      IOException - if a remote or network exception occurs
      RestoreSnapshotException - if snapshot failed to be restored
      IllegalArgumentException - if the restore request is formatted incorrectly
    • restoreSnapshot

      void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl) throws IOException, RestoreSnapshotException
      Restore the specified snapshot on the original table. (The table must be disabled) If 'takeFailSafeSnapshot' is set to true, a snapshot of the current table is taken before executing the restore operation. In case of restore failure, the failsafe snapshot will be restored. If the restore completes without problem the failsafe snapshot is deleted. The failsafe snapshot name is configurable by using the property "hbase.snapshot.restore.failsafe.name".
      Parameters:
      snapshotName - name of the snapshot to restore
      takeFailSafeSnapshot - true if the failsafe snapshot should be taken
      restoreAcl - true to restore acl of snapshot
      Throws:
      IOException - if a remote or network exception occurs
      RestoreSnapshotException - if snapshot failed to be restored
      IllegalArgumentException - if the restore request is formatted incorrectly
    • cloneSnapshot

      @Deprecated default void cloneSnapshot(byte[] snapshotName, TableName tableName) throws IOException, TableExistsException, RestoreSnapshotException
      Deprecated.
      since 2.3.0, will be removed in 3.0.0. Use cloneSnapshot(String, TableName) instead.
      Create a new table by cloning the snapshot content.
      Parameters:
      snapshotName - name of the snapshot to be cloned
      tableName - name of the table where the snapshot will be restored
      Throws:
      IOException - if a remote or network exception occurs
      TableExistsException - if table to be created already exists
      RestoreSnapshotException - if snapshot failed to be cloned
      IllegalArgumentException - if the specified table has not a valid name
    • cloneSnapshot

      default void cloneSnapshot(String snapshotName, TableName tableName) throws IOException, TableExistsException, RestoreSnapshotException
      Create a new table by cloning the snapshot content.
      Parameters:
      snapshotName - name of the snapshot to be cloned
      tableName - name of the table where the snapshot will be restored
      Throws:
      IOException - if a remote or network exception occurs
      TableExistsException - if table to be created already exists
      RestoreSnapshotException - if snapshot failed to be cloned
      IllegalArgumentException - if the specified table has not a valid name
    • cloneSnapshot

      default void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) throws IOException, TableExistsException, RestoreSnapshotException
      Create a new table by cloning the snapshot content.
      Parameters:
      snapshotName - name of the snapshot to be cloned
      tableName - name of the table where the snapshot will be restored
      restoreAcl - true to clone acl into newly created table
      customSFT - specify the StoreFileTracker used for the table
      Throws:
      IOException - if a remote or network exception occurs
      TableExistsException - if table to be created already exists
      RestoreSnapshotException - if snapshot failed to be cloned
      IllegalArgumentException - if the specified table has not a valid name
    • cloneSnapshot

      default void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException
      Create a new table by cloning the snapshot content.
      Parameters:
      snapshotName - name of the snapshot to be cloned
      tableName - name of the table where the snapshot will be restored
      restoreAcl - true to clone acl into newly created table
      Throws:
      IOException - if a remote or network exception occurs
      TableExistsException - if table to be created already exists
      RestoreSnapshotException - if snapshot failed to be cloned
      IllegalArgumentException - if the specified table has not a valid name
    • cloneSnapshotAsync

      default Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName) throws IOException, TableExistsException
      Create a new table by cloning the snapshot content, but does not block and wait for it to be completely cloned. You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete.
      Parameters:
      snapshotName - name of the snapshot to be cloned
      tableName - name of the table where the snapshot will be restored
      Returns:
      the result of the async clone snapshot. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
      Throws:
      IOException - if a remote or network exception occurs
      TableExistsException - if table to be cloned already exists
    • cloneSnapshotAsync

      default Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName, boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException
      Create a new table by cloning the snapshot content.
      Parameters:
      snapshotName - name of the snapshot to be cloned
      tableName - name of the table where the snapshot will be restored
      restoreAcl - true to clone acl into newly created table
      Throws:
      IOException - if a remote or network exception occurs
      TableExistsException - if table to be created already exists
      RestoreSnapshotException - if snapshot failed to be cloned
      IllegalArgumentException - if the specified table has not a valid name
    • cloneSnapshotAsync

      Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) throws IOException, TableExistsException, RestoreSnapshotException
      Create a new table by cloning the snapshot content.
      Parameters:
      snapshotName - name of the snapshot to be cloned
      tableName - name of the table where the snapshot will be restored
      restoreAcl - true to clone acl into newly created table
      customSFT - specify the StroreFileTracker used for the table
      Throws:
      IOException - if a remote or network exception occurs
      TableExistsException - if table to be created already exists
      RestoreSnapshotException - if snapshot failed to be cloned
      IllegalArgumentException - if the specified table has not a valid name
    • execProcedure

      void execProcedure(String signature, String instance, Map<String,String> props) throws IOException
      Execute a distributed procedure on a cluster.
      Parameters:
      signature - A distributed procedure is uniquely identified by its signature (default the root ZK node name of the procedure).
      instance - The instance name of the procedure. For some procedures, this parameter is optional.
      props - Property/Value pairs of properties passing to the procedure
      Throws:
      IOException - if a remote or network exception occurs
    • execProcedureWithRet

      @Deprecated default byte[] execProcedureWithRet(String signature, String instance, Map<String,String> props) throws IOException
      Deprecated.
      Since 2.0.0. Will be removed in 3.0.0. Use execProcedureWithReturn(String, String, Map) } instead.
      Execute a distributed procedure on a cluster.
      Parameters:
      signature - A distributed procedure is uniquely identified by its signature (default the root ZK node name of the procedure).
      instance - The instance name of the procedure. For some procedures, this parameter is optional.
      props - Property/Value pairs of properties passing to the procedure
      Returns:
      data returned after procedure execution. null if no return data.
      Throws:
      IOException - if a remote or network exception occurs
    • execProcedureWithReturn

      byte[] execProcedureWithReturn(String signature, String instance, Map<String,String> props) throws IOException
      Execute a distributed procedure on a cluster.
      Parameters:
      signature - A distributed procedure is uniquely identified by its signature (default the root ZK node name of the procedure).
      instance - The instance name of the procedure. For some procedures, this parameter is optional.
      props - Property/Value pairs of properties passing to the procedure
      Returns:
      data returned after procedure execution. null if no return data.
      Throws:
      IOException - if a remote or network exception occurs
    • isProcedureFinished

      boolean isProcedureFinished(String signature, String instance, Map<String,String> props) throws IOException
      Check the current state of the specified procedure. There are three possible states:
      1. running - returns false
      2. finished - returns true
      3. finished with error - throws the exception that caused the procedure to fail
      Parameters:
      signature - The signature that uniquely identifies a procedure
      instance - The instance name of the procedure
      props - Property/Value pairs of properties passing to the procedure
      Returns:
      true if the specified procedure is finished successfully, false if it is still running
      Throws:
      IOException - if the specified procedure finished with error
    • listSnapshots

      List completed snapshots.
      Returns:
      a list of snapshot descriptors for completed snapshots
      Throws:
      IOException - if a network error occurs
    • listSnapshots

      Deprecated.
      since 2.0 version and will be removed in 3.0 version. Use listSnapshots(Pattern) instead.
      List all the completed snapshots matching the given regular expression.
      Parameters:
      regex - The regular expression to match against
      Returns:
      list of SnapshotDescription
      Throws:
      IOException - if a remote or network exception occurs
    • listSnapshots

      List all the completed snapshots matching the given pattern.
      Parameters:
      pattern - The compiled regular expression to match against
      Returns:
      list of SnapshotDescription
      Throws:
      IOException - if a remote or network exception occurs
    • listTableSnapshots

      @Deprecated List<SnapshotDescription> listTableSnapshots(String tableNameRegex, String snapshotNameRegex) throws IOException
      Deprecated.
      since 2.0 version and will be removed in 3.0 version. Use listTableSnapshots(Pattern, Pattern) instead.
      List all the completed snapshots matching the given table name regular expression and snapshot name regular expression.
      Parameters:
      tableNameRegex - The table name regular expression to match against
      snapshotNameRegex - The snapshot name regular expression to match against
      Returns:
      list of completed SnapshotDescription
      Throws:
      IOException - if a remote or network exception occurs
    • listTableSnapshots

      List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) throws IOException
      List all the completed snapshots matching the given table name regular expression and snapshot name regular expression.
      Parameters:
      tableNamePattern - The compiled table name regular expression to match against
      snapshotNamePattern - The compiled snapshot name regular expression to match against
      Returns:
      list of completed SnapshotDescription
      Throws:
      IOException - if a remote or network exception occurs
    • deleteSnapshot

      @Deprecated void deleteSnapshot(byte[] snapshotName) throws IOException
      Deprecated.
      Since 2.2.0. Will be removed in 3.0.0. Use deleteSnapshot(String) instead.
      Delete an existing snapshot.
      Parameters:
      snapshotName - name of the snapshot
      Throws:
      IOException - if a remote or network exception occurs
    • deleteSnapshot

      void deleteSnapshot(String snapshotName) throws IOException
      Delete an existing snapshot.
      Parameters:
      snapshotName - name of the snapshot
      Throws:
      IOException - if a remote or network exception occurs
    • deleteSnapshots

      Deprecated.
      since 2.0 version and will be removed in 3.0 version. Use deleteSnapshots(Pattern) instead.
      Delete existing snapshots whose names match the pattern passed.
      Parameters:
      regex - The regular expression to match against
      Throws:
      IOException - if a remote or network exception occurs
    • deleteSnapshots

      void deleteSnapshots(Pattern pattern) throws IOException
      Delete existing snapshots whose names match the pattern passed.
      Parameters:
      pattern - pattern for names of the snapshot to match
      Throws:
      IOException - if a remote or network exception occurs
    • deleteTableSnapshots

      @Deprecated void deleteTableSnapshots(String tableNameRegex, String snapshotNameRegex) throws IOException
      Deprecated.
      since 2.0 version and will be removed in 3.0 version. Use deleteTableSnapshots(Pattern, Pattern) instead.
      Delete all existing snapshots matching the given table name regular expression and snapshot name regular expression.
      Parameters:
      tableNameRegex - The table name regular expression to match against
      snapshotNameRegex - The snapshot name regular expression to match against
      Throws:
      IOException - if a remote or network exception occurs
    • deleteTableSnapshots

      void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) throws IOException
      Delete all existing snapshots matching the given table name regular expression and snapshot name regular expression.
      Parameters:
      tableNamePattern - The compiled table name regular expression to match against
      snapshotNamePattern - The compiled snapshot name regular expression to match against
      Throws:
      IOException - if a remote or network exception occurs
    • setQuota

      void setQuota(QuotaSettings quota) throws IOException
      Apply the new quota settings.
      Parameters:
      quota - the quota settings
      Throws:
      IOException - if a remote or network exception occurs
    • getQuotaRetriever

      Deprecated.
      Since 2.0.0. Will be removed in 3.0.0. Use getQuota(QuotaFilter).
      Return a QuotaRetriever to list the quotas based on the filter.
      Parameters:
      filter - the quota settings filter
      Returns:
      the quota retriever
      Throws:
      IOException - if a remote or network exception occurs
    • getQuota

      List the quotas based on the filter.
      Parameters:
      filter - the quota settings filter
      Returns:
      the QuotaSetting list
      Throws:
      IOException - if a remote or network exception occurs
    • coprocessorService

      Creates and returns a RpcChannel instance connected to the active master.

      The obtained RpcChannel instance can be used to access a published coprocessor Service using standard protobuf service invocations:

       CoprocessorRpcChannel channel = myAdmin.coprocessorService();
       MyService.BlockingInterface service = MyService.newBlockingStub(channel);
       MyCallRequest request = MyCallRequest.newBuilder()
           ...
           .build();
       MyCallResponse response = service.myCall(null, request);
       
      Returns:
      A MasterCoprocessorRpcChannel instance
    • coprocessorService

      Creates and returns a RpcChannel instance connected to the passed region server.

      The obtained RpcChannel instance can be used to access a published coprocessor Service using standard protobuf service invocations:

       CoprocessorRpcChannel channel = myAdmin.coprocessorService(serverName);
       MyService.BlockingInterface service = MyService.newBlockingStub(channel);
       MyCallRequest request = MyCallRequest.newBuilder()
           ...
           .build();
       MyCallResponse response = service.myCall(null, request);
       
      Parameters:
      serverName - the server name to which the endpoint call is made
      Returns:
      A RegionServerCoprocessorRpcChannel instance
    • updateConfiguration

      Update the configuration and trigger an online config change on the regionserver.
      Parameters:
      server - : The server whose config needs to be updated.
      Throws:
      IOException - if a remote or network exception occurs
    • updateConfiguration

      Update the configuration and trigger an online config change on all the regionservers.
      Throws:
      IOException - if a remote or network exception occurs
    • getMasterInfoPort

      default int getMasterInfoPort() throws IOException
      Get the info port of the current master if one is available.
      Returns:
      master info port
      Throws:
      IOException - if a remote or network exception occurs
    • getSecurityCapabilities

      Return the set of supported security capabilities.
      Throws:
      IOException - if a remote or network exception occurs
    • setSplitOrMergeEnabled

      @Deprecated default boolean[] setSplitOrMergeEnabled(boolean enabled, boolean synchronous, MasterSwitchType... switchTypes) throws IOException
      Deprecated.
      Since 2.0.0. Will be removed in 3.0.0. Use splitSwitch(boolean, boolean) or mergeSwitch(boolean, boolean) instead.
      Turn the Split or Merge switches on or off.
      Parameters:
      enabled - enabled or not
      synchronous - If true, it waits until current split() call, if outstanding, to return.
      switchTypes - switchType list MasterSwitchType
      Returns:
      Previous switch value array
      Throws:
      IOException - if a remote or network exception occurs
    • splitSwitch

      boolean splitSwitch(boolean enabled, boolean synchronous) throws IOException
      Turn the split switch on or off.
      Parameters:
      enabled - enabled or not
      synchronous - If true, it waits until current split() call, if outstanding, to return.
      Returns:
      Previous switch value
      Throws:
      IOException - if a remote or network exception occurs
    • mergeSwitch

      boolean mergeSwitch(boolean enabled, boolean synchronous) throws IOException
      Turn the merge switch on or off.
      Parameters:
      enabled - enabled or not
      synchronous - If true, it waits until current merge() call, if outstanding, to return.
      Returns:
      Previous switch value
      Throws:
      IOException - if a remote or network exception occurs
    • isSplitOrMergeEnabled

      @Deprecated default boolean isSplitOrMergeEnabled(MasterSwitchType switchType) throws IOException
      Deprecated.
      Since 2.0.0. Will be removed in 3.0.0. Use isSplitEnabled() or isMergeEnabled() instead.
      Query the current state of the switch.
      Returns:
      true if the switch is enabled, false otherwise.
      Throws:
      IOException - if a remote or network exception occurs
    • isSplitEnabled

      boolean isSplitEnabled() throws IOException
      Query the current state of the split switch.
      Returns:
      true if the switch is enabled, false otherwise.
      Throws:
      IOException - if a remote or network exception occurs
    • isMergeEnabled

      boolean isMergeEnabled() throws IOException
      Query the current state of the merge switch.
      Returns:
      true if the switch is enabled, false otherwise.
      Throws:
      IOException - if a remote or network exception occurs
    • addReplicationPeer

      default void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig) throws IOException
      Add a new replication peer for replicating data to slave cluster.
      Parameters:
      peerId - a short name that identifies the peer
      peerConfig - configuration for the replication peer
      Throws:
      IOException - if a remote or network exception occurs
    • addReplicationPeer

      default void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) throws IOException
      Add a new replication peer for replicating data to slave cluster.
      Parameters:
      peerId - a short name that identifies the peer
      peerConfig - configuration for the replication peer
      enabled - peer state, true if ENABLED and false if DISABLED
      Throws:
      IOException - if a remote or network exception occurs
    • addReplicationPeerAsync

      Add a new replication peer but does not block and wait for it.

      You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete.

      Parameters:
      peerId - a short name that identifies the peer
      peerConfig - configuration for the replication peer
      Returns:
      the result of the async operation
      Throws:
      IOException - IOException if a remote or network exception occurs
    • addReplicationPeerAsync

      Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) throws IOException
      Add a new replication peer but does not block and wait for it.

      You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete.

      Parameters:
      peerId - a short name that identifies the peer
      peerConfig - configuration for the replication peer
      enabled - peer state, true if ENABLED and false if DISABLED
      Returns:
      the result of the async operation
      Throws:
      IOException - IOException if a remote or network exception occurs
    • removeReplicationPeer

      default void removeReplicationPeer(String peerId) throws IOException
      Remove a peer and stop the replication.
      Parameters:
      peerId - a short name that identifies the peer
      Throws:
      IOException - if a remote or network exception occurs
    • removeReplicationPeerAsync

      Remove a replication peer but does not block and wait for it.

      You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete.

      Parameters:
      peerId - a short name that identifies the peer
      Returns:
      the result of the async operation
      Throws:
      IOException - IOException if a remote or network exception occurs
    • enableReplicationPeer

      default void enableReplicationPeer(String peerId) throws IOException
      Restart the replication stream to the specified peer.
      Parameters:
      peerId - a short name that identifies the peer
      Throws:
      IOException - if a remote or network exception occurs
    • enableReplicationPeerAsync

      Enable a replication peer but does not block and wait for it.

      You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete.

      Parameters:
      peerId - a short name that identifies the peer
      Returns:
      the result of the async operation
      Throws:
      IOException - IOException if a remote or network exception occurs
    • disableReplicationPeer

      default void disableReplicationPeer(String peerId) throws IOException
      Stop the replication stream to the specified peer.
      Parameters:
      peerId - a short name that identifies the peer
      Throws:
      IOException - if a remote or network exception occurs
    • disableReplicationPeerAsync

      Disable a replication peer but does not block and wait for it.

      You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete.

      Parameters:
      peerId - a short name that identifies the peer
      Returns:
      the result of the async operation
      Throws:
      IOException - IOException if a remote or network exception occurs
    • getReplicationPeerConfig

      Returns the configured ReplicationPeerConfig for the specified peer.
      Parameters:
      peerId - a short name that identifies the peer
      Returns:
      ReplicationPeerConfig for the peer
      Throws:
      IOException - if a remote or network exception occurs
    • updateReplicationPeerConfig

      default void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig) throws IOException
      Update the peerConfig for the specified peer.
      Parameters:
      peerId - a short name that identifies the peer
      peerConfig - new config for the replication peer
      Throws:
      IOException - if a remote or network exception occurs
    • updateReplicationPeerConfigAsync

      Update the peerConfig for the specified peer but does not block and wait for it.

      You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete.

      Parameters:
      peerId - a short name that identifies the peer
      peerConfig - new config for the replication peer
      Returns:
      the result of the async operation
      Throws:
      IOException - IOException if a remote or network exception occurs
    • appendReplicationPeerTableCFs

      Append the replicable table column family config from the specified peer.
      Parameters:
      id - a short that identifies the cluster
      tableCfs - A map from tableName to column family names
      Throws:
      ReplicationException - if tableCfs has conflict with existing config
      IOException - if a remote or network exception occurs
    • removeReplicationPeerTableCFs

      Remove some table-cfs from config of the specified peer.
      Parameters:
      id - a short name that identifies the cluster
      tableCfs - A map from tableName to column family names
      Throws:
      ReplicationException - if tableCfs has conflict with existing config
      IOException - if a remote or network exception occurs
    • listReplicationPeers

      Return a list of replication peers.
      Returns:
      a list of replication peers description
      Throws:
      IOException - if a remote or network exception occurs
    • listReplicationPeers

      Return a list of replication peers.
      Parameters:
      pattern - The compiled regular expression to match peer id
      Returns:
      a list of replication peers description
      Throws:
      IOException - if a remote or network exception occurs
    • replicationPeerModificationSwitch

      default boolean replicationPeerModificationSwitch(boolean on) throws IOException
      Enable or disable replication peer modification.

      This is especially useful when you want to change the replication peer storage.

      Parameters:
      on - true means enable, otherwise disable
      Returns:
      the previous enable/disable state
      Throws:
      IOException
    • replicationPeerModificationSwitch

      boolean replicationPeerModificationSwitch(boolean on, boolean drainProcedures) throws IOException
      Enable or disable replication peer modification.

      This is especially useful when you want to change the replication peer storage.

      Parameters:
      on - true means enable, otherwise disable
      drainProcedures - if true, will wait until all the running replication peer modification procedures finish
      Returns:
      the previous enable/disable state
      Throws:
      IOException
    • isReplicationPeerModificationEnabled

      Check whether replication peer modification is enabled.
      Returns:
      true if modification is enabled, otherwise false
      Throws:
      IOException
    • decommissionRegionServers

      void decommissionRegionServers(List<ServerName> servers, boolean offload) throws IOException
      Mark region server(s) as decommissioned to prevent additional regions from getting assigned to them. Optionally unload the regions on the servers. If there are multiple servers to be decommissioned, decommissioning them at the same time can prevent wasteful region movements. Region unloading is asynchronous.
      Parameters:
      servers - The list of servers to decommission.
      offload - True to offload the regions from the decommissioned servers
      Throws:
      IOException - if a remote or network exception occurs
    • listDecommissionedRegionServers

      List region servers marked as decommissioned, which can not be assigned regions.
      Returns:
      List of decommissioned region servers.
      Throws:
      IOException - if a remote or network exception occurs
    • recommissionRegionServer

      void recommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames) throws IOException
      Remove decommission marker from a region server to allow regions assignments. Load regions onto the server if a list of regions is given. Region loading is asynchronous.
      Parameters:
      server - The server to recommission.
      encodedRegionNames - Regions to load onto the server.
      Throws:
      IOException - if a remote or network exception occurs
    • listReplicatedTableCFs

      Find all table and column families that are replicated from this cluster
      Returns:
      the replicated table-cfs list of this cluster.
      Throws:
      IOException - if a remote or network exception occurs
    • enableTableReplication

      Enable a table's replication switch.
      Parameters:
      tableName - name of the table
      Throws:
      IOException - if a remote or network exception occurs
    • disableTableReplication

      Disable a table's replication switch.
      Parameters:
      tableName - name of the table
      Throws:
      IOException - if a remote or network exception occurs
    • isReplicationPeerEnabled

      Check if a replication peer is enabled.
      Parameters:
      peerId - id of replication peer to check
      Returns:
      true if replication peer is enabled
      Throws:
      IOException - if a remote or network exception occurs
    • clearCompactionQueues

      Clear compacting queues on a regionserver.
      Parameters:
      serverName - the region server name
      queues - the set of queue name
      Throws:
      IOException - if a remote or network exception occurs
      InterruptedException
    • listDeadServers

      List dead region servers.
      Returns:
      List of dead region servers.
      Throws:
      IOException - if a remote or network exception occurs
    • listUnknownServers

      List unknown region servers.
      Returns:
      List of unknown region servers.
      Throws:
      IOException
    • clearDeadServers

      Clear dead region servers from master.
      Parameters:
      servers - list of dead region servers.
      Returns:
      List of servers that are not cleared
      Throws:
      IOException - if a remote or network exception occurs
    • cloneTableSchema

      void cloneTableSchema(TableName tableName, TableName newTableName, boolean preserveSplits) throws IOException
      Create a new table by cloning the existent table schema.
      Parameters:
      tableName - name of the table to be cloned
      newTableName - name of the new table where the table will be created
      preserveSplits - True if the splits should be preserved
      Throws:
      IOException - if a remote or network exception occurs
    • switchRpcThrottle

      boolean switchRpcThrottle(boolean enable) throws IOException
      Switch the rpc throttle enable state.
      Parameters:
      enable - Set to true to enable, false to disable.
      Returns:
      Previous rpc throttle enabled value
      Throws:
      IOException - if a remote or network exception occurs
    • isRpcThrottleEnabled

      Get if the rpc throttle is enabled.
      Returns:
      True if rpc throttle is enabled
      Throws:
      IOException - if a remote or network exception occurs
    • exceedThrottleQuotaSwitch

      boolean exceedThrottleQuotaSwitch(boolean enable) throws IOException
      Switch the exceed throttle quota. If enabled, user/table/namespace throttle quota can be exceeded if region server has availble quota.
      Parameters:
      enable - Set to true to enable, false to disable.
      Returns:
      Previous exceed throttle enabled value
      Throws:
      IOException - if a remote or network exception occurs
    • getSpaceQuotaTableSizes

      Fetches the table sizes on the filesystem as tracked by the HBase Master.
      Throws:
      IOException - if a remote or network exception occurs
    • getRegionServerSpaceQuotaSnapshots

      Fetches the observed SpaceQuotaSnapshotViews observed by a RegionServer.
      Throws:
      IOException - if a remote or network exception occurs
    • getCurrentSpaceQuotaSnapshot

      Returns the Master's view of a quota on the given namespace or null if the Master has no quota information on that namespace.
      Throws:
      IOException - if a remote or network exception occurs
    • getCurrentSpaceQuotaSnapshot

      Returns the Master's view of a quota on the given tableName or null if the Master has no quota information on that table.
      Throws:
      IOException - if a remote or network exception occurs
    • grant

      void grant(UserPermission userPermission, boolean mergeExistingPermissions) throws IOException
      Grants user specific permissions
      Parameters:
      userPermission - user name and the specific permission
      mergeExistingPermissions - If set to false, later granted permissions will override previous granted permissions. otherwise, it'll merge with previous granted permissions.
      Throws:
      IOException - if a remote or network exception occurs
    • revoke

      void revoke(UserPermission userPermission) throws IOException
      Revokes user specific permissions
      Parameters:
      userPermission - user name and the specific permission
      Throws:
      IOException - if a remote or network exception occurs
    • getUserPermissions

      Get the global/namespace/table permissions for user
      Parameters:
      getUserPermissionsRequest - A request contains which user, global, namespace or table permissions needed
      Returns:
      The user and permission list
      Throws:
      IOException - if a remote or network exception occurs
    • hasUserPermissions

      List<Boolean> hasUserPermissions(String userName, List<Permission> permissions) throws IOException
      Check if the user has specific permissions
      Parameters:
      userName - the user name
      permissions - the specific permission list
      Returns:
      True if user has the specific permissions
      Throws:
      IOException - if a remote or network exception occurs
    • hasUserPermissions

      default List<Boolean> hasUserPermissions(List<Permission> permissions) throws IOException
      Check if call user has specific permissions
      Parameters:
      permissions - the specific permission list
      Returns:
      True if user has the specific permissions
      Throws:
      IOException - if a remote or network exception occurs
    • snapshotCleanupSwitch

      boolean snapshotCleanupSwitch(boolean on, boolean synchronous) throws IOException
      Turn on or off the auto snapshot cleanup based on TTL.
      Parameters:
      on - Set to true to enable, false to disable.
      synchronous - If true, it waits until current snapshot cleanup is completed, if outstanding.
      Returns:
      Previous auto snapshot cleanup value
      Throws:
      IOException - if a remote or network exception occurs
    • isSnapshotCleanupEnabled

      Query the current state of the auto snapshot cleanup based on TTL.
      Returns:
      true if the auto snapshot cleanup is enabled, false otherwise.
      Throws:
      IOException - if a remote or network exception occurs
    • getSlowLogResponses

      Deprecated.
      since 2.4.0 and will be removed in 4.0.0. Use getLogEntries(Set, String, ServerType, int, Map) instead.
      Retrieves online slow/large RPC logs from the provided list of RegionServers
      Parameters:
      serverNames - Server names to get slowlog responses from
      logQueryFilter - filter to be used if provided (determines slow / large RPC logs)
      Returns:
      online slowlog response list
      Throws:
      IOException - if a remote or network exception occurs
    • clearSlowLogResponses

      Clears online slow/large RPC logs from the provided list of RegionServers
      Parameters:
      serverNames - Set of Server names to clean slowlog responses from
      Returns:
      List of booleans representing if online slowlog response buffer is cleaned from each RegionServer
      Throws:
      IOException - if a remote or network exception occurs
    • getLogEntries

      List<LogEntry> getLogEntries(Set<ServerName> serverNames, String logType, ServerType serverType, int limit, Map<String,Object> filterParams) throws IOException
      Retrieve recent online records from HMaster / RegionServers. Examples include slow/large RPC logs, balancer decisions by master.
      Parameters:
      serverNames - servers to retrieve records from, useful in case of records maintained by RegionServer as we can select specific server. In case of servertype=MASTER, logs will only come from the currently active master.
      logType - string representing type of log records
      serverType - enum for server type: HMaster or RegionServer
      limit - put a limit to list of records that server should send in response
      filterParams - additional filter params
      Returns:
      Log entries representing online records from servers
      Throws:
      IOException - if a remote or network exception occurs
    • flushMasterStore

      Flush master local region
      Throws:
      IOException
    • truncateRegion

      void truncateRegion(byte[] regionName) throws IOException
      Truncate an individual region.
      Parameters:
      regionName - region to truncate
      Throws:
      IOException - if a remote or network exception occurs
    • truncateRegionAsync

      Future<Void> truncateRegionAsync(byte[] regionName) throws IOException
      Truncate an individual region. Asynchronous operation.
      Parameters:
      regionName - region to truncate
      Throws:
      IOException - if a remote or network exception occurs
    • getCachedFilesList

      Get the list of cached files
      Throws:
      IOException