HBASE-22001 Polish the Admin interface

Signed-off-by: stack <stack@apache.org>
This commit is contained in:
zhangduo 2019-03-09 07:52:58 +08:00 committed by Duo Zhang
parent 1ad2175ded
commit 31b3716e84
6 changed files with 391 additions and 556 deletions

View File

@ -17,6 +17,8 @@
*/ */
package org.apache.hadoop.hbase.client; package org.apache.hadoop.hbase.client;
import static org.apache.hadoop.hbase.util.FutureUtils.get;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.util.Collection; import java.util.Collection;
@ -25,6 +27,7 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.Future; import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.regex.Pattern; import java.util.regex.Pattern;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Abortable;
@ -41,6 +44,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
import org.apache.hadoop.hbase.client.replication.TableCFs; import org.apache.hadoop.hbase.client.replication.TableCFs;
import org.apache.hadoop.hbase.client.security.SecurityCapability; import org.apache.hadoop.hbase.client.security.SecurityCapability;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
@ -57,6 +61,7 @@ import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException; import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
import org.apache.hadoop.hbase.snapshot.SnapshotCreationException; import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException; import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceAudience;
@ -73,8 +78,28 @@ import org.apache.yetus.audience.InterfaceAudience;
*/ */
@InterfaceAudience.Public @InterfaceAudience.Public
public interface Admin extends Abortable, Closeable { public interface Admin extends Abortable, Closeable {
/**
* Return the operation timeout for a rpc call.
* @see #getSyncWaitTimeout()
*/
int getOperationTimeout(); int getOperationTimeout();
/**
* Return the blocking wait time for an asynchronous operation. Can be configured by
* {@code hbase.client.sync.wait.timeout.msec}.
* <p/>
* 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 {@link #getOperationTimeout()}. And timeout value here tells you how much time we will
* wait until the procedure at master side is finished.
* <p/>
* In general, you can consider that the implementation for XXXX method is just a
* XXXXAsync().get(getSyncWaitTimeout(), TimeUnit.MILLISECONDS).
* @see #getOperationTimeout()
*/
int getSyncWaitTimeout();
@Override @Override
void abort(String why, Throwable e); void abort(String why, Throwable e);
@ -135,7 +160,9 @@ public interface Admin extends Abortable, Closeable {
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
* @see #listTables() * @see #listTables()
*/ */
List<TableDescriptor> listTableDescriptors(Pattern pattern) throws IOException; default List<TableDescriptor> listTableDescriptors(Pattern pattern) throws IOException {
return listTableDescriptors(pattern, false);
}
/** /**
* List all the userspace tables matching the given regular expression. * List all the userspace tables matching the given regular expression.
@ -207,7 +234,9 @@ public interface Admin extends Abortable, Closeable {
* @return array of table names * @return array of table names
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
TableName[] listTableNames(Pattern pattern) throws IOException; default TableName[] listTableNames(Pattern pattern) throws IOException {
return listTableNames(pattern, false);
}
/** /**
* List all of the names of userspace tables. * List all of the names of userspace tables.
@ -314,7 +343,9 @@ public interface Admin extends Abortable, Closeable {
* threads, the table may have been created between test-for-existence and attempt-at-creation). * threads, the table may have been created between test-for-existence and attempt-at-creation).
* @throws IOException * @throws IOException
*/ */
void createTable(TableDescriptor desc, byte[][] splitKeys) throws IOException; default void createTable(TableDescriptor desc, byte[][] splitKeys) throws IOException {
get(createTableAsync(desc, splitKeys), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
}
/** /**
* Creates a new table but does not block and wait for it to come online. * Creates a new table but does not block and wait for it to come online.
@ -336,11 +367,12 @@ public interface Admin extends Abortable, Closeable {
/** /**
* Deletes a table. Synchronous operation. * Deletes a table. Synchronous operation.
*
* @param tableName name of table to delete * @param tableName name of table to delete
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
void deleteTable(TableName tableName) throws IOException; default void deleteTable(TableName tableName) throws IOException {
get(deleteTableAsync(tableName), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
}
/** /**
* Deletes the table but does not block and wait for it to be completely removed. * Deletes the table but does not block and wait for it to be completely removed.
@ -402,8 +434,9 @@ public interface Admin extends Abortable, Closeable {
* @param preserveSplits <code>true</code> if the splits should be preserved * @param preserveSplits <code>true</code> if the splits should be preserved
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
void truncateTable(TableName tableName, boolean preserveSplits) default void truncateTable(TableName tableName, boolean preserveSplits) throws IOException {
throws IOException; get(truncateTableAsync(tableName, preserveSplits), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
}
/** /**
* Truncate the table but does not block and wait for it to be completely enabled. You can use * Truncate the table but does not block and wait for it to be completely enabled. You can use
@ -423,16 +456,17 @@ public interface Admin extends Abortable, Closeable {
* Enable a table. May timeout. Use {@link #enableTableAsync(org.apache.hadoop.hbase.TableName)} * Enable a table. May timeout. Use {@link #enableTableAsync(org.apache.hadoop.hbase.TableName)}
* and {@link #isTableEnabled(org.apache.hadoop.hbase.TableName)} instead. The table has to be in * and {@link #isTableEnabled(org.apache.hadoop.hbase.TableName)} instead. The table has to be in
* disabled state for it to be enabled. * disabled state for it to be enabled.
*
* @param tableName name of the table * @param tableName name of the table
* @throws IOException if a remote or network exception occurs There could be couple types of * @throws IOException if a remote or network exception occurs There could be couple types of
* IOException TableNotFoundException means the table doesn't exist. TableNotDisabledException * IOException TableNotFoundException means the table doesn't exist.
* means the table isn't in disabled state. * TableNotDisabledException means the table isn't in disabled state.
* @see #isTableEnabled(org.apache.hadoop.hbase.TableName) * @see #isTableEnabled(org.apache.hadoop.hbase.TableName)
* @see #disableTable(org.apache.hadoop.hbase.TableName) * @see #disableTable(org.apache.hadoop.hbase.TableName)
* @see #enableTableAsync(org.apache.hadoop.hbase.TableName) * @see #enableTableAsync(org.apache.hadoop.hbase.TableName)
*/ */
void enableTable(TableName tableName) throws IOException; default void enableTable(TableName tableName) throws IOException {
get(enableTableAsync(tableName), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
}
/** /**
* Enable the table but does not block and wait for it to be completely enabled. * Enable the table but does not block and wait for it to be completely enabled.
@ -500,16 +534,17 @@ public interface Admin extends Abortable, Closeable {
Future<Void> disableTableAsync(TableName tableName) throws IOException; Future<Void> disableTableAsync(TableName tableName) throws IOException;
/** /**
* Disable table and wait on completion. May timeout eventually. Use {@link * Disable table and wait on completion. May timeout eventually. Use
* #disableTableAsync(org.apache.hadoop.hbase.TableName)} and * {@link #disableTableAsync(org.apache.hadoop.hbase.TableName)} and
* {@link #isTableDisabled(org.apache.hadoop.hbase.TableName)} instead. The table has to be in * {@link #isTableDisabled(org.apache.hadoop.hbase.TableName)} instead. The table has to be in
* enabled state for it to be disabled. * enabled state for it to be disabled.
*
* @param tableName * @param tableName
* @throws IOException There could be couple types of IOException TableNotFoundException means the * @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. * table doesn't exist. TableNotEnabledException means the table isn't in enabled state.
*/ */
void disableTable(TableName tableName) throws IOException; default void disableTable(TableName tableName) throws IOException {
get(disableTableAsync(tableName), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
}
/** /**
* Disable tables matching the passed in pattern and wait on completion. Warning: Use this method * Disable tables matching the passed in pattern and wait on completion. Warning: Use this method
@ -637,8 +672,10 @@ public interface Admin extends Abortable, Closeable {
* @param columnFamily column family descriptor of column family to be added * @param columnFamily column family descriptor of column family to be added
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
void addColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily) default void addColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily)
throws IOException; throws IOException {
get(addColumnFamilyAsync(tableName, columnFamily), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
}
/** /**
* Add a column family to an existing table. Asynchronous operation. * Add a column family to an existing table. Asynchronous operation.
@ -679,7 +716,10 @@ public interface Admin extends Abortable, Closeable {
* @param columnFamily name of column family to be deleted * @param columnFamily name of column family to be deleted
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
void deleteColumnFamily(TableName tableName, byte[] columnFamily) throws IOException; default void deleteColumnFamily(TableName tableName, byte[] columnFamily) throws IOException {
get(deleteColumnFamilyAsync(tableName, columnFamily), getSyncWaitTimeout(),
TimeUnit.MILLISECONDS);
}
/** /**
* Delete a column family from a table. Asynchronous operation. * Delete a column family from a table. Asynchronous operation.
@ -698,9 +738,9 @@ public interface Admin extends Abortable, Closeable {
throws IOException; throws IOException;
/** /**
* Modify an existing column family on a table. Synchronous operation. * Modify an existing column family on a table. Synchronous operation. Use
* Use {@link #modifyColumnFamilyAsync(TableName, ColumnFamilyDescriptor)} instead because it * {@link #modifyColumnFamilyAsync(TableName, ColumnFamilyDescriptor)} instead because it returns
* returns a {@link Future} from which you can learn whether success or failure. * a {@link Future} from which you can learn whether success or failure.
* @param tableName name of table * @param tableName name of table
* @param columnFamily new column family descriptor to use * @param columnFamily new column family descriptor to use
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
@ -722,8 +762,11 @@ public interface Admin extends Abortable, Closeable {
* @param columnFamily new column family descriptor to use * @param columnFamily new column family descriptor to use
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
void modifyColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily) default void modifyColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily)
throws IOException; throws IOException {
get(modifyColumnFamilyAsync(tableName, columnFamily), getSyncWaitTimeout(),
TimeUnit.MILLISECONDS);
}
/** /**
* Modify an existing column family on a table. Asynchronous operation. * Modify an existing column family on a table. Asynchronous operation.
@ -1347,8 +1390,7 @@ public interface Admin extends Abortable, Closeable {
* @param splitPoint the explicit position to split on * @param splitPoint the explicit position to split on
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
void split(TableName tableName, byte[] splitPoint) void split(TableName tableName, byte[] splitPoint) throws IOException;
throws IOException;
/** /**
* Split an individual region. Asynchronous operation. * Split an individual region. Asynchronous operation.
@ -1369,28 +1411,33 @@ public interface Admin extends Abortable, Closeable {
* @param splitPoint the explicit position to split on * @param splitPoint the explicit position to split on
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
Future<Void> splitRegionAsync(byte[] regionName, byte[] splitPoint) Future<Void> splitRegionAsync(byte[] regionName, byte[] splitPoint) throws IOException;
throws IOException;
/** /**
* Modify an existing table, more IRB friendly version. * Modify an existing table, more IRB friendly version.
*
* @param tableName name of table. * @param tableName name of table.
* @param td modified description of the table * @param td modified description of the table
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
* @deprecated since 2.0 version and will be removed in 3.0 version. * @deprecated since 2.0 version and will be removed in 3.0 version. use
* use {@link #modifyTable(TableDescriptor)} * {@link #modifyTable(TableDescriptor)}
*/ */
@Deprecated @Deprecated
void modifyTable(TableName tableName, TableDescriptor td) default void modifyTable(TableName tableName, TableDescriptor td) throws IOException {
throws IOException; if (!tableName.equals(td.getTableName())) {
throw new IllegalArgumentException("the specified table name '" + tableName +
"' doesn't match with the HTD one: " + td.getTableName());
}
modifyTable(td);
}
/** /**
* Modify an existing table, more IRB friendly version. * Modify an existing table, more IRB friendly version.
* @param td modified description of the table * @param td modified description of the table
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
void modifyTable(TableDescriptor td) throws IOException; default void modifyTable(TableDescriptor td) throws IOException {
get(modifyTableAsync(td), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
}
/** /**
* Modify an existing table, more IRB friendly version. Asynchronous operation. This means that * Modify an existing table, more IRB friendly version. Asynchronous operation. This means that
@ -1409,8 +1456,14 @@ public interface Admin extends Abortable, Closeable {
* use {@link #modifyTableAsync(TableDescriptor)} * use {@link #modifyTableAsync(TableDescriptor)}
*/ */
@Deprecated @Deprecated
Future<Void> modifyTableAsync(TableName tableName, TableDescriptor td) default Future<Void> modifyTableAsync(TableName tableName, TableDescriptor td)
throws IOException; throws IOException {
if (!tableName.equals(td.getTableName())) {
throw new IllegalArgumentException("the specified table name '" + tableName +
"' doesn't match with the HTD one: " + td.getTableName());
}
return modifyTableAsync(td);
}
/** /**
* Modify an existing table, more IRB (ruby) friendly version. Asynchronous operation. This means that * Modify an existing table, more IRB (ruby) friendly version. Asynchronous operation. This means that
@ -1425,29 +1478,22 @@ public interface Admin extends Abortable, Closeable {
* @return the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the * @return the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the
* operation to complete * operation to complete
*/ */
Future<Void> modifyTableAsync(TableDescriptor td) Future<Void> modifyTableAsync(TableDescriptor td) throws IOException;
throws IOException;
/** /**
* <p>
* Shuts down the HBase cluster. * Shuts down the HBase cluster.
* </p> * <p/>
* <p>
* Notice that, a success shutdown call may ends with an error since the remote server has already * Notice that, a success shutdown call may ends with an error since the remote server has already
* been shutdown. * been shutdown.
* </p>
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
void shutdown() throws IOException; void shutdown() throws IOException;
/** /**
* <p>
* Shuts down the current HBase master only. Does not shutdown the cluster. * Shuts down the current HBase master only. Does not shutdown the cluster.
* </p> * <p/>
* <p>
* Notice that, a success stopMaster call may ends with an error since the remote server has * Notice that, a success stopMaster call may ends with an error since the remote server has
* already been shutdown. * already been shutdown.
* </p>
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
* @see #shutdown() * @see #shutdown()
*/ */
@ -1567,55 +1613,50 @@ public interface Admin extends Abortable, Closeable {
Configuration getConfiguration(); Configuration getConfiguration();
/** /**
* Create a new namespace. Blocks until namespace has been successfully created or an exception * Create a new namespace. Blocks until namespace has been successfully created or an exception is
* is thrown. * thrown.
*
* @param descriptor descriptor which describes the new namespace. * @param descriptor descriptor which describes the new namespace.
*/ */
void createNamespace(NamespaceDescriptor descriptor) default void createNamespace(NamespaceDescriptor descriptor) throws IOException {
throws IOException; get(createNamespaceAsync(descriptor), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
}
/** /**
* Create a new namespace. * Create a new namespace.
*
* @param descriptor descriptor which describes the new namespace * @param descriptor descriptor which describes the new namespace
* @return the result of the async create namespace operation. Use Future.get(long, TimeUnit) to * @return the result of the async create namespace operation. Use Future.get(long, TimeUnit) to
* wait on the operation to complete. * wait on the operation to complete.
*/ */
Future<Void> createNamespaceAsync(NamespaceDescriptor descriptor) Future<Void> createNamespaceAsync(NamespaceDescriptor descriptor) throws IOException;
throws IOException;
/** /**
* Modify an existing namespace. Blocks until namespace has been successfully modified or an * Modify an existing namespace. Blocks until namespace has been successfully modified or an
* exception is thrown. * exception is thrown.
*
* @param descriptor descriptor which describes the new namespace * @param descriptor descriptor which describes the new namespace
*/ */
void modifyNamespace(NamespaceDescriptor descriptor) default void modifyNamespace(NamespaceDescriptor descriptor) throws IOException {
throws IOException; get(modifyNamespaceAsync(descriptor), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
}
/** /**
* Modify an existing namespace. * Modify an existing namespace.
*
* @param descriptor descriptor which describes the new namespace * @param descriptor descriptor which describes the new namespace
* @return the result of the async modify namespace operation. Use Future.get(long, TimeUnit) to * @return the result of the async modify namespace operation. Use Future.get(long, TimeUnit) to
* wait on the operation to complete. * wait on the operation to complete.
*/ */
Future<Void> modifyNamespaceAsync(NamespaceDescriptor descriptor) Future<Void> modifyNamespaceAsync(NamespaceDescriptor descriptor) throws IOException;
throws IOException;
/** /**
* Delete an existing namespace. Only empty namespaces (no tables) can be removed. * Delete an existing namespace. Only empty namespaces (no tables) can be removed. Blocks until
* Blocks until namespace has been successfully deleted or an * namespace has been successfully deleted or an exception is thrown.
* exception is thrown.
*
* @param name namespace name * @param name namespace name
*/ */
void deleteNamespace(String name) throws IOException; default void deleteNamespace(String name) throws IOException {
get(deleteNamespaceAsync(name), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
}
/** /**
* Delete an existing namespace. Only empty namespaces (no tables) can be removed. * Delete an existing namespace. Only empty namespaces (no tables) can be removed.
*
* @param name namespace name * @param name namespace name
* @return the result of the async delete namespace operation. Use Future.get(long, TimeUnit) to * @return the result of the async delete namespace operation. Use Future.get(long, TimeUnit) to
* wait on the operation to complete. * wait on the operation to complete.
@ -1624,7 +1665,6 @@ public interface Admin extends Abortable, Closeable {
/** /**
* Get a namespace descriptor by name. * Get a namespace descriptor by name.
*
* @param name name of namespace descriptor * @param name name of namespace descriptor
* @return A descriptor * @return A descriptor
* @throws org.apache.hadoop.hbase.NamespaceNotFoundException * @throws org.apache.hadoop.hbase.NamespaceNotFoundException
@ -1656,23 +1696,17 @@ public interface Admin extends Abortable, Closeable {
/** /**
* Get list of table descriptors by namespace. * Get list of table descriptors by namespace.
*
* @param name namespace name * @param name namespace name
* @return returns a list of TableDescriptors * @return returns a list of TableDescriptors
* @throws IOException
*/ */
List<TableDescriptor> listTableDescriptorsByNamespace(byte[] name) List<TableDescriptor> listTableDescriptorsByNamespace(byte[] name) throws IOException;
throws IOException;
/** /**
* Get list of table names by namespace. * Get list of table names by namespace.
*
* @param name namespace name * @param name namespace name
* @return The list of table names in the namespace * @return The list of table names in the namespace
* @throws IOException
*/ */
TableName[] listTableNamesByNamespace(String name) TableName[] listTableNamesByNamespace(String name) throws IOException;
throws IOException;
/** /**
* Get the regions of a given table. * Get the regions of a given table.
@ -1738,17 +1772,20 @@ public interface Admin extends Abortable, Closeable {
/** /**
* Abort a procedure. * Abort a procedure.
* <p/>
* Do not use. Usually it is ignored but if not, it can do more damage than good. See hbck2. * Do not use. Usually it is ignored but if not, it can do more damage than good. See hbck2.
* @param procId ID of the procedure to abort * @param procId ID of the procedure to abort
* @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted? * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted?
* @return <code>true</code> if aborted, <code>false</code> if procedure already completed or does not exist * @return <code>true</code> if aborted, <code>false</code> if procedure already completed or does
* not exist
* @throws IOException * @throws IOException
* @deprecated Since 2.1.1 -- to be removed. * @deprecated Since 2.1.1 -- to be removed.
*/ */
@Deprecated @Deprecated
boolean abortProcedure( default boolean abortProcedure(long procId, boolean mayInterruptIfRunning) throws IOException {
long procId, return get(abortProcedureAsync(procId, mayInterruptIfRunning), getSyncWaitTimeout(),
boolean mayInterruptIfRunning) throws IOException; TimeUnit.MILLISECONDS);
}
/** /**
* Abort a procedure but does not block and wait for completion. * Abort a procedure but does not block and wait for completion.
@ -1877,19 +1914,20 @@ public interface Admin extends Abortable, Closeable {
* Take a snapshot for the given table. If the table is enabled, a FLUSH-type snapshot will be * 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 * taken. If the table is disabled, an offline snapshot is taken. Snapshots are considered unique
* based on <b>the name of the snapshot</b>. Attempts to take a snapshot with the same name (even * based on <b>the name of the snapshot</b>. Attempts to take a snapshot with the same name (even
* a different type or with different parameters) will fail with a {@link * a different type or with different parameters) will fail with a
* org.apache.hadoop.hbase.snapshot.SnapshotCreationException} indicating the duplicate naming. * {@link org.apache.hadoop.hbase.snapshot.SnapshotCreationException} indicating the duplicate
* Snapshot names follow the same naming constraints as tables in HBase. See {@link * naming. Snapshot names follow the same naming constraints as tables in HBase. See
* org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}. * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
*
* @param snapshotName name of the snapshot to be created * @param snapshotName name of the snapshot to be created
* @param tableName name of the table for which snapshot is created * @param tableName name of the table for which snapshot is created
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
* @throws org.apache.hadoop.hbase.snapshot.SnapshotCreationException if snapshot creation failed * @throws org.apache.hadoop.hbase.snapshot.SnapshotCreationException if snapshot creation failed
* @throws IllegalArgumentException if the snapshot request is formatted incorrectly * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
*/ */
void snapshot(String snapshotName, TableName tableName) default void snapshot(String snapshotName, TableName tableName)
throws IOException, SnapshotCreationException, IllegalArgumentException; throws IOException, SnapshotCreationException, IllegalArgumentException {
snapshot(snapshotName, tableName, SnapshotType.FLUSH);
}
/** /**
* Create a timestamp consistent snapshot for the given table. Snapshots are considered unique * Create a timestamp consistent snapshot for the given table. Snapshots are considered unique
@ -1897,15 +1935,19 @@ public interface Admin extends Abortable, Closeable {
* different type or with different parameters) will fail with a {@link SnapshotCreationException} * different type or with different parameters) will fail with a {@link SnapshotCreationException}
* indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in * indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in
* HBase. * HBase.
*
* @param snapshotName name of the snapshot to be created * @param snapshotName name of the snapshot to be created
* @param tableName name of the table for which snapshot is created * @param tableName name of the table for which snapshot is created
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
* @throws SnapshotCreationException if snapshot creation failed * @throws SnapshotCreationException if snapshot creation failed
* @throws IllegalArgumentException if the snapshot request is formatted incorrectly * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
* @deprecated since 2.3.0, will be removed in 3.0.0. Use {@link #snapshot(String, TableName)}
* instead.
*/ */
void snapshot(byte[] snapshotName, TableName tableName) @Deprecated
throws IOException, SnapshotCreationException, IllegalArgumentException; default void snapshot(byte[] snapshotName, TableName tableName)
throws IOException, SnapshotCreationException, IllegalArgumentException {
snapshot(Bytes.toString(snapshotName), tableName);
}
/** /**
* Create typed snapshot of the table. Snapshots are considered unique based on <b>the name of the * Create typed snapshot of the table. Snapshots are considered unique based on <b>the name of the
@ -1913,7 +1955,6 @@ public interface Admin extends Abortable, Closeable {
* different parameters) will fail with a {@link SnapshotCreationException} indicating the * different parameters) will fail with a {@link SnapshotCreationException} indicating the
* duplicate naming. Snapshot names follow the same naming constraints as tables in HBase. See * duplicate naming. Snapshot names follow the same naming constraints as tables in HBase. See
* {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}. * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
*
* @param snapshotName name to give the snapshot on the filesystem. Must be unique from all other * @param snapshotName name to give the snapshot on the filesystem. Must be unique from all other
* snapshots stored on the cluster * snapshots stored on the cluster
* @param tableName name of the table to snapshot * @param tableName name of the table to snapshot
@ -1922,10 +1963,10 @@ public interface Admin extends Abortable, Closeable {
* @throws SnapshotCreationException if snapshot creation failed * @throws SnapshotCreationException if snapshot creation failed
* @throws IllegalArgumentException if the snapshot request is formatted incorrectly * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
*/ */
void snapshot(String snapshotName, default void snapshot(String snapshotName, TableName tableName, SnapshotType type)
TableName tableName, throws IOException, SnapshotCreationException, IllegalArgumentException {
SnapshotType type) throws IOException, SnapshotCreationException, snapshot(new SnapshotDescription(snapshotName, tableName, type));
IllegalArgumentException; }
/** /**
* Take a snapshot and wait for the server to complete that snapshot (blocking). Only a single * Take a snapshot and wait for the server to complete that snapshot (blocking). Only a single
@ -1934,12 +1975,11 @@ public interface Admin extends Abortable, Closeable {
* single cluster). Snapshots are considered unique based on <b>the name of the snapshot</b>. * single cluster). Snapshots are considered unique based on <b>the name of the snapshot</b>.
* Attempts to take a snapshot with the same name (even a different type or with different * Attempts to take a snapshot with the same name (even a different type or with different
* parameters) will fail with a {@link SnapshotCreationException} indicating the duplicate naming. * parameters) will fail with a {@link SnapshotCreationException} indicating the duplicate naming.
* Snapshot names follow the same naming constraints as tables in HBase. See {@link * Snapshot names follow the same naming constraints as tables in HBase. See
* org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}. You should probably * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}. You should
* use {@link #snapshot(String, org.apache.hadoop.hbase.TableName)} or * probably use {@link #snapshot(String, org.apache.hadoop.hbase.TableName)} or
* {@link #snapshot(byte[], org.apache.hadoop.hbase.TableName)} unless you are sure about the type * {@link #snapshot(byte[], org.apache.hadoop.hbase.TableName)} unless you are sure about the type
* of snapshot that you want to take. * of snapshot that you want to take.
*
* @param snapshot snapshot to take * @param snapshot snapshot to take
* @throws IOException or we lose contact with the master. * @throws IOException or we lose contact with the master.
* @throws SnapshotCreationException if snapshot failed to be taken * @throws SnapshotCreationException if snapshot failed to be taken
@ -1960,6 +2000,7 @@ public interface Admin extends Abortable, Closeable {
* {@link #snapshotAsync(SnapshotDescription)} instead. * {@link #snapshotAsync(SnapshotDescription)} instead.
*/ */
@Deprecated @Deprecated
@SuppressWarnings("FutureReturnValueIgnored")
default void takeSnapshotAsync(SnapshotDescription snapshot) default void takeSnapshotAsync(SnapshotDescription snapshot)
throws IOException, SnapshotCreationException { throws IOException, SnapshotCreationException {
snapshotAsync(snapshot); snapshotAsync(snapshot);
@ -1968,13 +2009,13 @@ public interface Admin extends Abortable, Closeable {
/** /**
* Take a snapshot without waiting for the server to complete that snapshot (asynchronous) Only a * 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. * single snapshot should be taken at a time, or results may be undefined.
*
* @param snapshot snapshot to take * @param snapshot snapshot to take
* @throws IOException if the snapshot did not succeed or we lose contact with the master. * @throws IOException if the snapshot did not succeed or we lose contact with the master.
* @throws SnapshotCreationException if snapshot creation failed * @throws SnapshotCreationException if snapshot creation failed
* @throws IllegalArgumentException if the snapshot request is formatted incorrectly * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
*/ */
void snapshotAsync(SnapshotDescription snapshot) throws IOException, SnapshotCreationException; Future<Void> snapshotAsync(SnapshotDescription snapshot)
throws IOException, SnapshotCreationException;
/** /**
* Check the current state of the passed snapshot. There are three possible states: <ol> * Check the current state of the passed snapshot. There are three possible states: <ol>
@ -1997,26 +2038,29 @@ public interface Admin extends Abortable, Closeable {
/** /**
* Restore the specified snapshot on the original table. (The table must be disabled) If the * 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 <code>true</code>, a * "hbase.snapshot.restore.take.failsafe.snapshot" configuration property is set to
* snapshot of the current table is taken before executing the restore operation. In case of * <code>true</code>, a snapshot of the current table is taken before executing the restore
* restore failure, the failsafe snapshot will be restored. If the restore completes without * operation. In case of restore failure, the failsafe snapshot will be restored. If the restore
* problem the failsafe snapshot is deleted. * completes without problem the failsafe snapshot is deleted.
*
* @param snapshotName name of the snapshot to restore * @param snapshotName name of the snapshot to restore
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
* @throws org.apache.hadoop.hbase.snapshot.RestoreSnapshotException if snapshot failed to be * @throws org.apache.hadoop.hbase.snapshot.RestoreSnapshotException if snapshot failed to be
* restored * restored
* @throws IllegalArgumentException if the restore request is formatted incorrectly * @throws IllegalArgumentException if the restore request is formatted incorrectly
* @deprecated since 2.3.0, will be removed in 3.0.0. Use {@link #restoreSnapshot(String)}
* instead.
*/ */
void restoreSnapshot(byte[] snapshotName) throws IOException, RestoreSnapshotException; @Deprecated
default void restoreSnapshot(byte[] snapshotName) throws IOException, RestoreSnapshotException {
restoreSnapshot(Bytes.toString(snapshotName));
}
/** /**
* Restore the specified snapshot on the original table. (The table must be disabled) If the * 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 <code>true</code>, a * "hbase.snapshot.restore.take.failsafe.snapshot" configuration property is set to
* snapshot of the current table is taken before executing the restore operation. In case of * <code>true</code>, a snapshot of the current table is taken before executing the restore
* restore failure, the failsafe snapshot will be restored. If the restore completes without * operation. In case of restore failure, the failsafe snapshot will be restored. If the restore
* problem the failsafe snapshot is deleted. * completes without problem the failsafe snapshot is deleted.
*
* @param snapshotName name of the snapshot to restore * @param snapshotName name of the snapshot to restore
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
* @throws RestoreSnapshotException if snapshot failed to be restored * @throws RestoreSnapshotException if snapshot failed to be restored
@ -2026,59 +2070,66 @@ public interface Admin extends Abortable, Closeable {
/** /**
* Restore the specified snapshot on the original table. (The table must be disabled) If the * 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 <code>true</code>, a * "hbase.snapshot.restore.take.failsafe.snapshot" configuration property is set to
* snapshot of the current table is taken before executing the restore operation. In case of * <code>true</code>, a snapshot of the current table is taken before executing the restore
* restore failure, the failsafe snapshot will be restored. If the restore completes without * operation. In case of restore failure, the failsafe snapshot will be restored. If the restore
* problem the failsafe snapshot is deleted. * completes without problem the failsafe snapshot is deleted.
*
* @param snapshotName name of the snapshot to restore * @param snapshotName name of the snapshot to restore
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
* @throws RestoreSnapshotException if snapshot failed to be restored * @throws RestoreSnapshotException if snapshot failed to be restored
* @return the result of the async restore snapshot. You can use Future.get(long, TimeUnit) * @return the result of the async restore snapshot. You can use Future.get(long, TimeUnit) to
* to wait on the operation to complete. * wait on the operation to complete.
* @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.
*/ */
@Deprecated
Future<Void> restoreSnapshotAsync(String snapshotName) Future<Void> restoreSnapshotAsync(String snapshotName)
throws IOException, RestoreSnapshotException; throws IOException, RestoreSnapshotException;
/** /**
* Restore the specified snapshot on the original table. (The table must be disabled) If * Restore the specified snapshot on the original table. (The table must be disabled) If
* 'takeFailSafeSnapshot' is set to <code>true</code>, a snapshot of the current table is taken before * 'takeFailSafeSnapshot' is set to <code>true</code>, a snapshot of the current table is taken
* executing the restore operation. In case of restore failure, the failsafe snapshot will be * before executing the restore operation. In case of restore failure, the failsafe snapshot will
* restored. If the restore completes without problem the failsafe snapshot is deleted. The * 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".
* @param snapshotName name of the snapshot to restore
* @param takeFailSafeSnapshot <code>true</code> if the failsafe snapshot should be taken
* @throws IOException if a remote or network exception occurs
* @throws RestoreSnapshotException if snapshot failed to be restored
* @throws IllegalArgumentException if the restore request is formatted incorrectly
* @deprecated since 2.3.0, will be removed in 3.0.0. Use
* {@link #restoreSnapshot(String, boolean)} instead.
*/
@Deprecated
default void restoreSnapshot(byte[] snapshotName, boolean takeFailSafeSnapshot)
throws IOException, RestoreSnapshotException {
restoreSnapshot(Bytes.toString(snapshotName), takeFailSafeSnapshot);
}
/**
* Restore the specified snapshot on the original table. (The table must be disabled) If
* 'takeFailSafeSnapshot' is set to <code>true</code>, 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 * failsafe snapshot name is configurable by using the property
* "hbase.snapshot.restore.failsafe.name". * "hbase.snapshot.restore.failsafe.name".
*
* @param snapshotName name of the snapshot to restore * @param snapshotName name of the snapshot to restore
* @param takeFailSafeSnapshot <code>true</code> if the failsafe snapshot should be taken * @param takeFailSafeSnapshot <code>true</code> if the failsafe snapshot should be taken
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
* @throws RestoreSnapshotException if snapshot failed to be restored * @throws RestoreSnapshotException if snapshot failed to be restored
* @throws IllegalArgumentException if the restore request is formatted incorrectly * @throws IllegalArgumentException if the restore request is formatted incorrectly
*/ */
void restoreSnapshot(byte[] snapshotName, boolean takeFailSafeSnapshot) default void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot)
throws IOException, RestoreSnapshotException; throws IOException, RestoreSnapshotException {
restoreSnapshot(snapshotName, takeFailSafeSnapshot, false);
}
/** /**
* Restore the specified snapshot on the original table. (The table must be disabled) If * Restore the specified snapshot on the original table. (The table must be disabled) If
* 'takeFailSafeSnapshot' is set to <code>true</code>, a snapshot of the current table is taken before * 'takeFailSafeSnapshot' is set to <code>true</code>, a snapshot of the current table is taken
* executing the restore operation. In case of restore failure, the failsafe snapshot will be * before executing the restore operation. In case of restore failure, the failsafe snapshot will
* restored. If the restore completes without problem the failsafe snapshot is deleted. The * 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".
*
* @param snapshotName name of the snapshot to restore
* @param takeFailSafeSnapshot <code>true</code> if the failsafe snapshot should be taken
* @throws IOException if a remote or network exception occurs
* @throws RestoreSnapshotException if snapshot failed to be restored
* @throws IllegalArgumentException if the restore request is formatted incorrectly
*/
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 <code>true</code>, 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 * failsafe snapshot name is configurable by using the property
* "hbase.snapshot.restore.failsafe.name". * "hbase.snapshot.restore.failsafe.name".
* @param snapshotName name of the snapshot to restore * @param snapshotName name of the snapshot to restore
@ -2093,7 +2144,23 @@ public interface Admin extends Abortable, Closeable {
/** /**
* Create a new table by cloning the snapshot content. * Create a new table by cloning the snapshot content.
* * @param snapshotName name of the snapshot to be cloned
* @param tableName name of the table where the snapshot will be restored
* @throws IOException if a remote or network exception occurs
* @throws TableExistsException if table to be created already exists
* @throws RestoreSnapshotException if snapshot failed to be cloned
* @throws IllegalArgumentException if the specified table has not a valid name
* @deprecated since 2.3.0, will be removed in 3.0.0. Use
* {@link #cloneSnapshot(String, TableName)} instead.
*/
@Deprecated
default void cloneSnapshot(byte[] snapshotName, TableName tableName)
throws IOException, TableExistsException, RestoreSnapshotException {
cloneSnapshot(Bytes.toString(snapshotName), tableName);
}
/**
* Create a new table by cloning the snapshot content.
* @param snapshotName name of the snapshot to be cloned * @param snapshotName name of the snapshot to be cloned
* @param tableName name of the table where the snapshot will be restored * @param tableName name of the table where the snapshot will be restored
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
@ -2101,8 +2168,10 @@ public interface Admin extends Abortable, Closeable {
* @throws RestoreSnapshotException if snapshot failed to be cloned * @throws RestoreSnapshotException if snapshot failed to be cloned
* @throws IllegalArgumentException if the specified table has not a valid name * @throws IllegalArgumentException if the specified table has not a valid name
*/ */
void cloneSnapshot(byte[] snapshotName, TableName tableName) default void cloneSnapshot(String snapshotName, TableName tableName)
throws IOException, TableExistsException, RestoreSnapshotException; throws IOException, TableExistsException, RestoreSnapshotException {
cloneSnapshot(snapshotName, tableName, false);
}
/** /**
* Create a new table by cloning the snapshot content. * Create a new table by cloning the snapshot content.
@ -2114,40 +2183,43 @@ public interface Admin extends Abortable, Closeable {
* @throws RestoreSnapshotException if snapshot failed to be cloned * @throws RestoreSnapshotException if snapshot failed to be cloned
* @throws IllegalArgumentException if the specified table has not a valid name * @throws IllegalArgumentException if the specified table has not a valid name
*/ */
void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl) default void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl)
throws IOException, TableExistsException, RestoreSnapshotException; throws IOException, TableExistsException, RestoreSnapshotException {
get(cloneSnapshotAsync(snapshotName, tableName, restoreAcl), getSyncWaitTimeout(),
TimeUnit.MILLISECONDS);
}
/**
* 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.
* @param snapshotName name of the snapshot to be cloned
* @param tableName name of the table where the snapshot will be restored
* @throws IOException if a remote or network exception occurs
* @throws TableExistsException if table to be cloned already exists
* @return the result of the async clone snapshot. You can use Future.get(long, TimeUnit) to wait
* on the operation to complete.
*/
default Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName)
throws IOException, TableExistsException {
return cloneSnapshotAsync(snapshotName, tableName, false);
}
/** /**
* Create a new table by cloning the snapshot content. * Create a new table by cloning the snapshot content.
*
* @param snapshotName name of the snapshot to be cloned * @param snapshotName name of the snapshot to be cloned
* @param tableName name of the table where the snapshot will be restored * @param tableName name of the table where the snapshot will be restored
* @param restoreAcl <code>true</code> to clone acl into newly created table
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
* @throws TableExistsException if table to be created already exists * @throws TableExistsException if table to be created already exists
* @throws RestoreSnapshotException if snapshot failed to be cloned * @throws RestoreSnapshotException if snapshot failed to be cloned
* @throws IllegalArgumentException if the specified table has not a valid name * @throws IllegalArgumentException if the specified table has not a valid name
*/ */
void cloneSnapshot(String snapshotName, TableName tableName) Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName, boolean restoreAcl)
throws IOException, TableExistsException, RestoreSnapshotException; throws IOException, TableExistsException, RestoreSnapshotException;
/**
* 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.
*
* @param snapshotName name of the snapshot to be cloned
* @param tableName name of the table where the snapshot will be restored
* @throws IOException if a remote or network exception occurs
* @throws TableExistsException if table to be cloned already exists
* @return the result of the async clone snapshot. You can use Future.get(long, TimeUnit)
* to wait on the operation to complete.
*/
Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName)
throws IOException, TableExistsException;
/** /**
* Execute a distributed procedure on a cluster. * Execute a distributed procedure on a cluster.
* *
@ -2182,7 +2254,6 @@ public interface Admin extends Abortable, Closeable {
/** /**
* Execute a distributed procedure on a cluster. * Execute a distributed procedure on a cluster.
*
* @param signature A distributed procedure is uniquely identified by its signature (default the * @param signature A distributed procedure is uniquely identified by its signature (default the
* root ZK node name of the procedure). * root ZK node name of the procedure).
* @param instance The instance name of the procedure. For some procedures, this parameter is * @param instance The instance name of the procedure. For some procedures, this parameter is
@ -2518,12 +2589,15 @@ public interface Admin extends Abortable, Closeable {
* @param enabled peer state, true if ENABLED and false if DISABLED * @param enabled peer state, true if ENABLED and false if DISABLED
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) default void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
throws IOException; throws IOException {
get(addReplicationPeerAsync(peerId, peerConfig, enabled), getSyncWaitTimeout(),
TimeUnit.MILLISECONDS);
}
/** /**
* Add a new replication peer but does not block and wait for it. * Add a new replication peer but does not block and wait for it.
* <p> * <p/>
* You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw * 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 * 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. * case the wait timeout was not long enough to allow the operation to complete.
@ -2557,7 +2631,10 @@ public interface Admin extends Abortable, Closeable {
* @param peerId a short name that identifies the peer * @param peerId a short name that identifies the peer
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
void removeReplicationPeer(String peerId) throws IOException; default void removeReplicationPeer(String peerId) throws IOException {
get(removeReplicationPeerAsync(peerId), getSyncWaitTimeout(),
TimeUnit.MILLISECONDS);
}
/** /**
* Remove a replication peer but does not block and wait for it. * Remove a replication peer but does not block and wait for it.
@ -2576,7 +2653,9 @@ public interface Admin extends Abortable, Closeable {
* @param peerId a short name that identifies the peer * @param peerId a short name that identifies the peer
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
void enableReplicationPeer(String peerId) throws IOException; default void enableReplicationPeer(String peerId) throws IOException {
get(enableReplicationPeerAsync(peerId), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
}
/** /**
* Enable a replication peer but does not block and wait for it. * Enable a replication peer but does not block and wait for it.
@ -2595,11 +2674,13 @@ public interface Admin extends Abortable, Closeable {
* @param peerId a short name that identifies the peer * @param peerId a short name that identifies the peer
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
void disableReplicationPeer(String peerId) throws IOException; default void disableReplicationPeer(String peerId) throws IOException {
get(disableReplicationPeerAsync(peerId), getSyncWaitTimeout(), TimeUnit.MILLISECONDS);
}
/** /**
* Disable a replication peer but does not block and wait for it. * Disable a replication peer but does not block and wait for it.
* <p> * <p/>
* You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw * 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 * 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. * case the wait timeout was not long enough to allow the operation to complete.
@ -2623,12 +2704,15 @@ public interface Admin extends Abortable, Closeable {
* @param peerConfig new config for the replication peer * @param peerConfig new config for the replication peer
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
void updateReplicationPeerConfig(String peerId, default void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
ReplicationPeerConfig peerConfig) throws IOException; throws IOException {
get(updateReplicationPeerConfigAsync(peerId, peerConfig), getSyncWaitTimeout(),
TimeUnit.MILLISECONDS);
}
/** /**
* Update the peerConfig for the specified peer but does not block and wait for it. * Update the peerConfig for the specified peer but does not block and wait for it.
* <p> * <p/>
* You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw * 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 * 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. * case the wait timeout was not long enough to allow the operation to complete.
@ -2647,9 +2731,16 @@ public interface Admin extends Abortable, Closeable {
* @throws ReplicationException if tableCfs has conflict with existing config * @throws ReplicationException if tableCfs has conflict with existing config
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
void appendReplicationPeerTableCFs(String id, default void appendReplicationPeerTableCFs(String id, Map<TableName, List<String>> tableCfs)
Map<TableName, List<String>> tableCfs) throws ReplicationException, IOException {
throws ReplicationException, IOException; if (tableCfs == null) {
throw new ReplicationException("tableCfs is null");
}
ReplicationPeerConfig peerConfig = getReplicationPeerConfig(id);
ReplicationPeerConfig newPeerConfig =
ReplicationPeerConfigUtil.appendTableCFsToReplicationPeerConfig(tableCfs, peerConfig);
updateReplicationPeerConfig(id, newPeerConfig);
}
/** /**
* Remove some table-cfs from config of the specified peer. * Remove some table-cfs from config of the specified peer.
@ -2658,9 +2749,16 @@ public interface Admin extends Abortable, Closeable {
* @throws ReplicationException if tableCfs has conflict with existing config * @throws ReplicationException if tableCfs has conflict with existing config
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
void removeReplicationPeerTableCFs(String id, default void removeReplicationPeerTableCFs(String id, Map<TableName, List<String>> tableCfs)
Map<TableName, List<String>> tableCfs) throws ReplicationException, IOException {
throws ReplicationException, IOException; if (tableCfs == null) {
throw new ReplicationException("tableCfs is null");
}
ReplicationPeerConfig peerConfig = getReplicationPeerConfig(id);
ReplicationPeerConfig newPeerConfig =
ReplicationPeerConfigUtil.removeTableCFsFromReplicationPeerConfig(tableCfs, peerConfig, id);
updateReplicationPeerConfig(id, newPeerConfig);
}
/** /**
* Return a list of replication peers. * Return a list of replication peers.
@ -2747,25 +2845,24 @@ public interface Admin extends Abortable, Closeable {
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
* @return List of servers that are not cleared * @return List of servers that are not cleared
*/ */
List<ServerName> clearDeadServers(final List<ServerName> servers) throws IOException; List<ServerName> clearDeadServers(List<ServerName> servers) throws IOException;
/** /**
* Create a new table by cloning the existent table schema. * Create a new table by cloning the existent table schema.
*
* @param tableName name of the table to be cloned * @param tableName name of the table to be cloned
* @param newTableName name of the new table where the table will be created * @param newTableName name of the new table where the table will be created
* @param preserveSplits True if the splits should be preserved * @param preserveSplits True if the splits should be preserved
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
void cloneTableSchema(final TableName tableName, final TableName newTableName, void cloneTableSchema(TableName tableName, TableName newTableName, boolean preserveSplits)
final boolean preserveSplits) throws IOException; throws IOException;
/** /**
* Switch the rpc throttle enable state. * Switch the rpc throttle enable state.
* @param enable Set to <code>true</code> to enable, <code>false</code> to disable. * @param enable Set to <code>true</code> to enable, <code>false</code> to disable.
* @return Previous rpc throttle enabled value * @return Previous rpc throttle enabled value
*/ */
boolean switchRpcThrottle(final boolean enable) throws IOException; boolean switchRpcThrottle(boolean enable) throws IOException;
/** /**
* Get if the rpc throttle is enabled. * Get if the rpc throttle is enabled.

View File

@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.client;
import com.google.protobuf.Descriptors; import com.google.protobuf.Descriptors;
import com.google.protobuf.Message; import com.google.protobuf.Message;
import com.google.protobuf.RpcController; import com.google.protobuf.RpcController;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.io.InterruptedIOException; import java.io.InterruptedIOException;
@ -45,7 +44,6 @@ import java.util.function.Supplier;
import java.util.regex.Pattern; import java.util.regex.Pattern;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import java.util.stream.Stream; import java.util.stream.Stream;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.CacheEvictionStats; import org.apache.hadoop.hbase.CacheEvictionStats;
@ -87,7 +85,6 @@ import org.apache.hadoop.hbase.quotas.QuotaRetriever;
import org.apache.hadoop.hbase.quotas.QuotaSettings; import org.apache.hadoop.hbase.quotas.QuotaSettings;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot; import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException; import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.hadoop.hbase.security.access.Permission; import org.apache.hadoop.hbase.security.access.Permission;
@ -268,6 +265,11 @@ public class HBaseAdmin implements Admin {
return operationTimeout; return operationTimeout;
} }
@Override
public int getSyncWaitTimeout() {
return syncWaitTimeout;
}
HBaseAdmin(ClusterConnection connection) throws IOException { HBaseAdmin(ClusterConnection connection) throws IOException {
this.conf = connection.getConfiguration(); this.conf = connection.getConfiguration();
this.connection = connection; this.connection = connection;
@ -332,11 +334,6 @@ public class HBaseAdmin implements Admin {
return listTableDescriptors((Pattern)null, false); return listTableDescriptors((Pattern)null, false);
} }
@Override
public List<TableDescriptor> listTableDescriptors(Pattern pattern) throws IOException {
return listTableDescriptors(pattern, false);
}
@Override @Override
public List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables) public List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables)
throws IOException { throws IOException {
@ -359,11 +356,6 @@ public class HBaseAdmin implements Admin {
operationTimeout, rpcTimeout); operationTimeout, rpcTimeout);
} }
@Override
public void modifyTable(TableDescriptor td) throws IOException {
get(modifyTableAsync(td), syncWaitTimeout, TimeUnit.MILLISECONDS);
}
@Override @Override
public Future<Void> modifyTableAsync(TableDescriptor td) throws IOException { public Future<Void> modifyTableAsync(TableDescriptor td) throws IOException {
ModifyTableResponse response = executeCallable( ModifyTableResponse response = executeCallable(
@ -508,11 +500,6 @@ public class HBaseAdmin implements Admin {
return listTableNames((Pattern)null, false); return listTableNames((Pattern)null, false);
} }
@Override
public TableName[] listTableNames(Pattern pattern) throws IOException {
return listTableNames(pattern, false);
}
@Override @Override
public TableName[] listTableNames(String regex) throws IOException { public TableName[] listTableNames(String regex) throws IOException {
return listTableNames(Pattern.compile(regex), false); return listTableNames(Pattern.compile(regex), false);
@ -609,37 +596,29 @@ public class HBaseAdmin implements Admin {
} }
@Override @Override
public void createTable(TableDescriptor desc) public void createTable(TableDescriptor desc) throws IOException {
throws IOException {
createTable(desc, null); createTable(desc, null);
} }
@Override @Override
public void createTable(TableDescriptor desc, byte [] startKey, public void createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions)
byte [] endKey, int numRegions)
throws IOException { throws IOException {
if(numRegions < 3) { if (numRegions < 3) {
throw new IllegalArgumentException("Must create at least three regions"); throw new IllegalArgumentException("Must create at least three regions");
} else if(Bytes.compareTo(startKey, endKey) >= 0) { } else if (Bytes.compareTo(startKey, endKey) >= 0) {
throw new IllegalArgumentException("Start key must be smaller than end key"); throw new IllegalArgumentException("Start key must be smaller than end key");
} }
if (numRegions == 3) { if (numRegions == 3) {
createTable(desc, new byte[][]{startKey, endKey}); createTable(desc, new byte[][] { startKey, endKey });
return; return;
} }
byte [][] splitKeys = Bytes.split(startKey, endKey, numRegions - 3); byte[][] splitKeys = Bytes.split(startKey, endKey, numRegions - 3);
if(splitKeys == null || splitKeys.length != numRegions - 1) { if (splitKeys == null || splitKeys.length != numRegions - 1) {
throw new IllegalArgumentException("Unable to split key range into enough regions"); throw new IllegalArgumentException("Unable to split key range into enough regions");
} }
createTable(desc, splitKeys); createTable(desc, splitKeys);
} }
@Override
public void createTable(final TableDescriptor desc, byte [][] splitKeys)
throws IOException {
get(createTableAsync(desc, splitKeys), syncWaitTimeout, TimeUnit.MILLISECONDS);
}
@Override @Override
public Future<Void> createTableAsync(final TableDescriptor desc, final byte[][] splitKeys) public Future<Void> createTableAsync(final TableDescriptor desc, final byte[][] splitKeys)
throws IOException { throws IOException {
@ -709,11 +688,6 @@ public class HBaseAdmin implements Admin {
} }
} }
@Override
public void deleteTable(final TableName tableName) throws IOException {
get(deleteTableAsync(tableName), syncWaitTimeout, TimeUnit.MILLISECONDS);
}
@Override @Override
public Future<Void> deleteTableAsync(final TableName tableName) throws IOException { public Future<Void> deleteTableAsync(final TableName tableName) throws IOException {
DeleteTableResponse response = executeCallable( DeleteTableResponse response = executeCallable(
@ -789,12 +763,6 @@ public class HBaseAdmin implements Admin {
return failed.toArray(new HTableDescriptor[failed.size()]); return failed.toArray(new HTableDescriptor[failed.size()]);
} }
@Override
public void truncateTable(final TableName tableName, final boolean preserveSplits)
throws IOException {
get(truncateTableAsync(tableName, preserveSplits), syncWaitTimeout, TimeUnit.MILLISECONDS);
}
@Override @Override
public Future<Void> truncateTableAsync(final TableName tableName, final boolean preserveSplits) public Future<Void> truncateTableAsync(final TableName tableName, final boolean preserveSplits)
throws IOException { throws IOException {
@ -855,12 +823,6 @@ public class HBaseAdmin implements Admin {
return splits; return splits;
} }
@Override
public void enableTable(final TableName tableName)
throws IOException {
get(enableTableAsync(tableName), syncWaitTimeout, TimeUnit.MILLISECONDS);
}
@Override @Override
public Future<Void> enableTableAsync(final TableName tableName) throws IOException { public Future<Void> enableTableAsync(final TableName tableName) throws IOException {
TableName.isLegalFullyQualifiedTableName(tableName.getName()); TableName.isLegalFullyQualifiedTableName(tableName.getName());
@ -920,12 +882,6 @@ public class HBaseAdmin implements Admin {
return failed.toArray(new HTableDescriptor[failed.size()]); return failed.toArray(new HTableDescriptor[failed.size()]);
} }
@Override
public void disableTable(final TableName tableName)
throws IOException {
get(disableTableAsync(tableName), syncWaitTimeout, TimeUnit.MILLISECONDS);
}
@Override @Override
public Future<Void> disableTableAsync(final TableName tableName) throws IOException { public Future<Void> disableTableAsync(final TableName tableName) throws IOException {
TableName.isLegalFullyQualifiedTableName(tableName.getName()); TableName.isLegalFullyQualifiedTableName(tableName.getName());
@ -1039,12 +995,6 @@ public class HBaseAdmin implements Admin {
return getAlterStatus(TableName.valueOf(tableName)); return getAlterStatus(TableName.valueOf(tableName));
} }
@Override
public void addColumnFamily(final TableName tableName, final ColumnFamilyDescriptor columnFamily)
throws IOException {
get(addColumnFamilyAsync(tableName, columnFamily), syncWaitTimeout, TimeUnit.MILLISECONDS);
}
@Override @Override
public Future<Void> addColumnFamilyAsync(final TableName tableName, public Future<Void> addColumnFamilyAsync(final TableName tableName,
final ColumnFamilyDescriptor columnFamily) throws IOException { final ColumnFamilyDescriptor columnFamily) throws IOException {
@ -1089,12 +1039,6 @@ public class HBaseAdmin implements Admin {
deleteColumnFamily(tableName, columnFamily); deleteColumnFamily(tableName, columnFamily);
} }
@Override
public void deleteColumnFamily(final TableName tableName, final byte[] columnFamily)
throws IOException {
get(deleteColumnFamilyAsync(tableName, columnFamily), syncWaitTimeout, TimeUnit.MILLISECONDS);
}
@Override @Override
public Future<Void> deleteColumnFamilyAsync(final TableName tableName, final byte[] columnFamily) public Future<Void> deleteColumnFamilyAsync(final TableName tableName, final byte[] columnFamily)
throws IOException { throws IOException {
@ -1128,12 +1072,6 @@ public class HBaseAdmin implements Admin {
} }
} }
@Override
public void modifyColumnFamily(final TableName tableName,
final ColumnFamilyDescriptor columnFamily) throws IOException {
get(modifyColumnFamilyAsync(tableName, columnFamily), syncWaitTimeout, TimeUnit.MILLISECONDS);
}
@Override @Override
public Future<Void> modifyColumnFamilyAsync(final TableName tableName, public Future<Void> modifyColumnFamilyAsync(final TableName tableName,
final ColumnFamilyDescriptor columnFamily) throws IOException { final ColumnFamilyDescriptor columnFamily) throws IOException {
@ -1981,22 +1919,6 @@ public class HBaseAdmin implements Admin {
splitRegionAsync(regionServerPair.getFirst(), splitPoint); splitRegionAsync(regionServerPair.getFirst(), splitPoint);
} }
@Override
public void modifyTable(final TableName tableName, final TableDescriptor td)
throws IOException {
get(modifyTableAsync(tableName, td), syncWaitTimeout, TimeUnit.MILLISECONDS);
}
@Override
public Future<Void> modifyTableAsync(final TableName tableName, final TableDescriptor td)
throws IOException {
if (!tableName.equals(td.getTableName())) {
throw new IllegalArgumentException("the specified table name '" + tableName +
"' doesn't match with the HTD one: " + td.getTableName());
}
return modifyTableAsync(td);
}
private static class ModifyTableFuture extends TableFuture<Void> { private static class ModifyTableFuture extends TableFuture<Void> {
public ModifyTableFuture(final HBaseAdmin admin, final TableName tableName, public ModifyTableFuture(final HBaseAdmin admin, final TableName tableName,
final ModifyTableResponse response) { final ModifyTableResponse response) {
@ -2226,12 +2148,6 @@ public class HBaseAdmin implements Admin {
} }
} }
@Override
public void createNamespace(final NamespaceDescriptor descriptor)
throws IOException {
get(createNamespaceAsync(descriptor), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
}
@Override @Override
public Future<Void> createNamespaceAsync(final NamespaceDescriptor descriptor) public Future<Void> createNamespaceAsync(final NamespaceDescriptor descriptor)
throws IOException { throws IOException {
@ -2253,12 +2169,6 @@ public class HBaseAdmin implements Admin {
}; };
} }
@Override
public void modifyNamespace(final NamespaceDescriptor descriptor)
throws IOException {
get(modifyNamespaceAsync(descriptor), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
}
@Override @Override
public Future<Void> modifyNamespaceAsync(final NamespaceDescriptor descriptor) public Future<Void> modifyNamespaceAsync(final NamespaceDescriptor descriptor)
throws IOException { throws IOException {
@ -2280,12 +2190,6 @@ public class HBaseAdmin implements Admin {
}; };
} }
@Override
public void deleteNamespace(final String name)
throws IOException {
get(deleteNamespaceAsync(name), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
}
@Override @Override
public Future<Void> deleteNamespaceAsync(final String name) public Future<Void> deleteNamespaceAsync(final String name)
throws IOException { throws IOException {
@ -2487,44 +2391,6 @@ public class HBaseAdmin implements Admin {
} }
} }
/**
* Roll the log writer. I.e. when using a file system based write ahead log,
* start writing log messages to a new file.
*
* Note that when talking to a version 1.0+ HBase deployment, the rolling is asynchronous.
* This method will return as soon as the roll is requested and the return value will
* always be null. Additionally, the named region server may schedule store flushes at the
* request of the wal handling the roll request.
*
* When talking to a 0.98 or older HBase deployment, the rolling is synchronous and the
* return value may be either null or a list of encoded region names.
*
* @param serverName
* The servername of the regionserver. A server name is made of host,
* port and startcode. This is mandatory. Here is an example:
* <code> host187.example.com,60020,1289493121758</code>
* @return a set of {@link HRegionInfo#getEncodedName()} that would allow the wal to
* clean up some underlying files. null if there's nothing to flush.
* @throws IOException if a remote or network exception occurs
* @throws FailedLogCloseException
* @deprecated use {@link #rollWALWriter(ServerName)}
*/
@Deprecated
public synchronized byte[][] rollHLogWriter(String serverName)
throws IOException, FailedLogCloseException {
ServerName sn = ServerName.valueOf(serverName);
final RollWALWriterResponse response = rollWALWriterImpl(sn);
int regionCount = response.getRegionToFlushCount();
if (0 == regionCount) {
return null;
}
byte[][] regionsToFlush = new byte[regionCount][];
for (int i = 0; i < regionCount; i++) {
regionsToFlush[i] = ProtobufUtil.toBytes(response.getRegionToFlush(i));
}
return regionsToFlush;
}
@Override @Override
public synchronized void rollWALWriter(ServerName serverName) public synchronized void rollWALWriter(ServerName serverName)
throws IOException, FailedLogCloseException { throws IOException, FailedLogCloseException {
@ -2565,26 +2431,6 @@ public class HBaseAdmin implements Admin {
return null; return null;
} }
@Override
public void snapshot(final String snapshotName,
final TableName tableName) throws IOException,
SnapshotCreationException, IllegalArgumentException {
snapshot(snapshotName, tableName, SnapshotType.FLUSH);
}
@Override
public void snapshot(final byte[] snapshotName, final TableName tableName)
throws IOException, SnapshotCreationException, IllegalArgumentException {
snapshot(Bytes.toString(snapshotName), tableName, SnapshotType.FLUSH);
}
@Override
public void snapshot(final String snapshotName, final TableName tableName,
SnapshotType type)
throws IOException, SnapshotCreationException, IllegalArgumentException {
snapshot(new SnapshotDescription(snapshotName, tableName, type));
}
@Override @Override
public void snapshot(SnapshotDescription snapshotDesc) public void snapshot(SnapshotDescription snapshotDesc)
throws IOException, SnapshotCreationException, IllegalArgumentException { throws IOException, SnapshotCreationException, IllegalArgumentException {
@ -2630,9 +2476,35 @@ public class HBaseAdmin implements Admin {
} }
@Override @Override
public void snapshotAsync(SnapshotDescription snapshotDesc) throws IOException, public Future<Void> snapshotAsync(SnapshotDescription snapshotDesc)
SnapshotCreationException { throws IOException, SnapshotCreationException {
asyncSnapshot(ProtobufUtil.createHBaseProtosSnapshotDesc(snapshotDesc)); asyncSnapshot(ProtobufUtil.createHBaseProtosSnapshotDesc(snapshotDesc));
return new ProcedureFuture<Void>(this, null) {
@Override
protected Void waitOperationResult(long deadlineTs) throws IOException, TimeoutException {
waitForState(deadlineTs, new WaitForStateCallable() {
@Override
public void throwInterruptedException() throws InterruptedIOException {
throw new InterruptedIOException(
"Interrupted while waiting for taking snapshot" + snapshotDesc);
}
@Override
public void throwTimeoutException(long elapsedTime) throws TimeoutException {
throw new TimeoutException("Snapshot '" + snapshotDesc.getName() +
"' wasn't completed in expectedTime:" + elapsedTime + " ms");
}
@Override
public boolean checkState(int tries) throws IOException {
return isSnapshotFinished(snapshotDesc);
}
});
return null;
}
};
} }
private SnapshotResponse asyncSnapshot(SnapshotProtos.SnapshotDescription snapshot) private SnapshotResponse asyncSnapshot(SnapshotProtos.SnapshotDescription snapshot)
@ -2686,7 +2558,7 @@ public class HBaseAdmin implements Admin {
restoreSnapshot(Bytes.toString(snapshotName), takeFailSafeSnapshot); restoreSnapshot(Bytes.toString(snapshotName), takeFailSafeSnapshot);
} }
/* /**
* Check whether the snapshot exists and contains disabled table * Check whether the snapshot exists and contains disabled table
* *
* @param snapshotName name of the snapshot to restore * @param snapshotName name of the snapshot to restore
@ -2804,36 +2676,12 @@ public class HBaseAdmin implements Admin {
} }
@Override @Override
public void cloneSnapshot(final byte[] snapshotName, final TableName tableName) public Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName,
throws IOException, TableExistsException, RestoreSnapshotException { boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException {
cloneSnapshot(Bytes.toString(snapshotName), tableName);
}
@Override
public void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl)
throws IOException, TableExistsException, RestoreSnapshotException {
if (tableExists(tableName)) { if (tableExists(tableName)) {
throw new TableExistsException(tableName); throw new TableExistsException(tableName);
} }
get( return internalRestoreSnapshotAsync(snapshotName, tableName, restoreAcl);
internalRestoreSnapshotAsync(snapshotName, tableName, restoreAcl),
Integer.MAX_VALUE,
TimeUnit.MILLISECONDS);
}
@Override
public void cloneSnapshot(final String snapshotName, final TableName tableName)
throws IOException, TableExistsException, RestoreSnapshotException {
cloneSnapshot(snapshotName, tableName, false);
}
@Override
public Future<Void> cloneSnapshotAsync(final String snapshotName, final TableName tableName)
throws IOException, TableExistsException {
if (tableExists(tableName)) {
throw new TableExistsException(tableName);
}
return internalRestoreSnapshotAsync(snapshotName, tableName, false);
} }
@Override @Override
@ -3974,13 +3822,6 @@ public class HBaseAdmin implements Admin {
return this.rpcControllerFactory; return this.rpcControllerFactory;
} }
@Override
public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
throws IOException {
get(addReplicationPeerAsync(peerId, peerConfig, enabled), this.syncWaitTimeout,
TimeUnit.MILLISECONDS);
}
@Override @Override
public Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig, public Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig,
boolean enabled) throws IOException { boolean enabled) throws IOException {
@ -3995,11 +3836,6 @@ public class HBaseAdmin implements Admin {
return new ReplicationFuture(this, peerId, response.getProcId(), () -> "ADD_REPLICATION_PEER"); return new ReplicationFuture(this, peerId, response.getProcId(), () -> "ADD_REPLICATION_PEER");
} }
@Override
public void removeReplicationPeer(String peerId) throws IOException {
get(removeReplicationPeerAsync(peerId), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
}
@Override @Override
public Future<Void> removeReplicationPeerAsync(String peerId) throws IOException { public Future<Void> removeReplicationPeerAsync(String peerId) throws IOException {
RemoveReplicationPeerResponse response = RemoveReplicationPeerResponse response =
@ -4015,11 +3851,6 @@ public class HBaseAdmin implements Admin {
() -> "REMOVE_REPLICATION_PEER"); () -> "REMOVE_REPLICATION_PEER");
} }
@Override
public void enableReplicationPeer(final String peerId) throws IOException {
get(enableReplicationPeerAsync(peerId), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
}
@Override @Override
public Future<Void> enableReplicationPeerAsync(final String peerId) throws IOException { public Future<Void> enableReplicationPeerAsync(final String peerId) throws IOException {
EnableReplicationPeerResponse response = EnableReplicationPeerResponse response =
@ -4035,11 +3866,6 @@ public class HBaseAdmin implements Admin {
() -> "ENABLE_REPLICATION_PEER"); () -> "ENABLE_REPLICATION_PEER");
} }
@Override
public void disableReplicationPeer(final String peerId) throws IOException {
get(disableReplicationPeerAsync(peerId), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
}
@Override @Override
public Future<Void> disableReplicationPeerAsync(final String peerId) throws IOException { public Future<Void> disableReplicationPeerAsync(final String peerId) throws IOException {
DisableReplicationPeerResponse response = DisableReplicationPeerResponse response =
@ -4068,13 +3894,6 @@ public class HBaseAdmin implements Admin {
}); });
} }
@Override
public void updateReplicationPeerConfig(final String peerId,
final ReplicationPeerConfig peerConfig) throws IOException {
get(updateReplicationPeerConfigAsync(peerId, peerConfig), this.syncWaitTimeout,
TimeUnit.MILLISECONDS);
}
@Override @Override
public Future<Void> updateReplicationPeerConfigAsync(final String peerId, public Future<Void> updateReplicationPeerConfigAsync(final String peerId,
final ReplicationPeerConfig peerConfig) throws IOException { final ReplicationPeerConfig peerConfig) throws IOException {
@ -4091,32 +3910,6 @@ public class HBaseAdmin implements Admin {
() -> "UPDATE_REPLICATION_PEER_CONFIG"); () -> "UPDATE_REPLICATION_PEER_CONFIG");
} }
@Override
public void appendReplicationPeerTableCFs(String id,
Map<TableName, List<String>> tableCfs)
throws ReplicationException, IOException {
if (tableCfs == null) {
throw new ReplicationException("tableCfs is null");
}
ReplicationPeerConfig peerConfig = getReplicationPeerConfig(id);
ReplicationPeerConfig newPeerConfig =
ReplicationPeerConfigUtil.appendTableCFsToReplicationPeerConfig(tableCfs, peerConfig);
updateReplicationPeerConfig(id, newPeerConfig);
}
@Override
public void removeReplicationPeerTableCFs(String id,
Map<TableName, List<String>> tableCfs)
throws ReplicationException, IOException {
if (tableCfs == null) {
throw new ReplicationException("tableCfs is null");
}
ReplicationPeerConfig peerConfig = getReplicationPeerConfig(id);
ReplicationPeerConfig newPeerConfig =
ReplicationPeerConfigUtil.removeTableCFsFromReplicationPeerConfig(tableCfs, peerConfig, id);
updateReplicationPeerConfig(id, newPeerConfig);
}
@Override @Override
public List<ReplicationPeerDescription> listReplicationPeers() throws IOException { public List<ReplicationPeerDescription> listReplicationPeers() throws IOException {
return listReplicationPeers((Pattern)null); return listReplicationPeers((Pattern)null);

View File

@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue;
import java.io.Closeable; import java.io.Closeable;
import java.lang.annotation.Annotation; import java.lang.annotation.Annotation;
import java.lang.reflect.Method; import java.lang.reflect.Method;
import java.lang.reflect.Modifier;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -32,8 +33,6 @@ import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@Category({ ClientTests.class, SmallTests.class }) @Category({ ClientTests.class, SmallTests.class })
public class TestInterfaceAlign { public class TestInterfaceAlign {
@ -42,8 +41,6 @@ public class TestInterfaceAlign {
public static final HBaseClassTestRule CLASS_RULE = public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestInterfaceAlign.class); HBaseClassTestRule.forClass(TestInterfaceAlign.class);
private static final Logger LOG = LoggerFactory.getLogger(TestInterfaceAlign.class);
/** /**
* Test methods name match up * Test methods name match up
*/ */
@ -54,6 +51,7 @@ public class TestInterfaceAlign {
// Remove some special methods // Remove some special methods
adminMethodNames.remove("getOperationTimeout"); adminMethodNames.remove("getOperationTimeout");
adminMethodNames.remove("getSyncWaitTimeout");
adminMethodNames.remove("getConnection"); adminMethodNames.remove("getConnection");
adminMethodNames.remove("getConfiguration"); adminMethodNames.remove("getConfiguration");
adminMethodNames.removeAll(getMethodNames(Abortable.class)); adminMethodNames.removeAll(getMethodNames(Abortable.class));
@ -78,7 +76,8 @@ public class TestInterfaceAlign {
private <T> List<String> getMethodNames(Class<T> c) { private <T> List<String> getMethodNames(Class<T> c) {
// DON'T use the getDeclaredMethods as we want to check the Public APIs only. // DON'T use the getDeclaredMethods as we want to check the Public APIs only.
return Arrays.asList(c.getMethods()).stream().filter(m -> !isDeprecated(m)) return Arrays.asList(c.getMethods()).stream().filter(m -> !isDeprecated(m))
.map(Method::getName).distinct().collect(Collectors.toList()); .filter(m -> !Modifier.isStatic(m.getModifiers())).map(Method::getName).distinct()
.collect(Collectors.toList());
} }
private boolean isDeprecated(Method method) { private boolean isDeprecated(Method method) {

View File

@ -24,7 +24,10 @@ import java.util.concurrent.CompletionException;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executor; import java.util.concurrent.Executor;
import java.util.concurrent.Future; import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.function.BiConsumer; import java.util.function.BiConsumer;
import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -135,6 +138,24 @@ public final class FutureUtils {
} }
} }
/**
* A helper class for getting the result of a Future, and convert the error to an
* {@link IOException}.
*/
public static <T> T get(Future<T> future, long timeout, TimeUnit unit) throws IOException {
try {
return future.get(timeout, unit);
} catch (InterruptedException e) {
throw (IOException) new InterruptedIOException().initCause(e);
} catch (ExecutionException e) {
Throwable cause = e.getCause();
Throwables.propagateIfPossible(cause, IOException.class);
throw new IOException(cause);
} catch (TimeoutException e) {
throw new TimeoutIOException(e);
}
}
/** /**
* Returns a CompletableFuture that is already completed exceptionally with the given exception. * Returns a CompletableFuture that is already completed exceptionally with the given exception.
*/ */

View File

@ -26,6 +26,7 @@ import java.io.IOException;
import java.util.Collection; import java.util.Collection;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import java.util.concurrent.Future;
import java.util.regex.Pattern; import java.util.regex.Pattern;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -313,7 +314,7 @@ public class TestSnapshotFromMaster {
// take a snapshot of the table // take a snapshot of the table
String snapshotName = "snapshot"; String snapshotName = "snapshot";
byte[] snapshotNameBytes = Bytes.toBytes(snapshotName); byte[] snapshotNameBytes = Bytes.toBytes(snapshotName);
admin.snapshot(snapshotNameBytes, TABLE_NAME); admin.snapshot(snapshotName, TABLE_NAME);
LOG.info("After snapshot File-System state"); LOG.info("After snapshot File-System state");
FSUtils.logFileSystemState(fs, rootDir, LOG); FSUtils.logFileSystemState(fs, rootDir, LOG);
@ -436,12 +437,13 @@ public class TestSnapshotFromMaster {
table.put(put); table.put(put);
} }
String snapshotName = "testAsyncSnapshotWillNotBlockSnapshotHFileCleaner01"; String snapshotName = "testAsyncSnapshotWillNotBlockSnapshotHFileCleaner01";
Future<Void> future =
UTIL.getAdmin().snapshotAsync(new org.apache.hadoop.hbase.client.SnapshotDescription( UTIL.getAdmin().snapshotAsync(new org.apache.hadoop.hbase.client.SnapshotDescription(
snapshotName, TABLE_NAME, SnapshotType.FLUSH)); snapshotName, TABLE_NAME, SnapshotType.FLUSH));
Waiter.waitFor(UTIL.getConfiguration(), 10 * 1000L, 200L, Waiter.waitFor(UTIL.getConfiguration(), 10 * 1000L, 200L,
() -> UTIL.getAdmin().listSnapshots(Pattern.compile(snapshotName)).size() == 1); () -> UTIL.getAdmin().listSnapshots(Pattern.compile(snapshotName)).size() == 1);
assertTrue(master.getSnapshotManager().isTakingAnySnapshot()); assertTrue(master.getSnapshotManager().isTakingAnySnapshot());
Thread.sleep(11 * 1000L); future.get();
assertFalse(master.getSnapshotManager().isTakingAnySnapshot()); assertFalse(master.getSnapshotManager().isTakingAnySnapshot());
} }
} }

View File

@ -26,7 +26,6 @@ import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.Future; import java.util.concurrent.Future;
import java.util.regex.Pattern; import java.util.regex.Pattern;
import org.apache.commons.lang3.NotImplementedException; import org.apache.commons.lang3.NotImplementedException;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CacheEvictionStats; import org.apache.hadoop.hbase.CacheEvictionStats;
@ -38,6 +37,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NamespaceNotFoundException; import org.apache.hadoop.hbase.NamespaceNotFoundException;
import org.apache.hadoop.hbase.RegionMetrics; import org.apache.hadoop.hbase.RegionMetrics;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
@ -56,10 +56,10 @@ import org.apache.hadoop.hbase.quotas.QuotaFilter;
import org.apache.hadoop.hbase.quotas.QuotaRetriever; import org.apache.hadoop.hbase.quotas.QuotaRetriever;
import org.apache.hadoop.hbase.quotas.QuotaSettings; import org.apache.hadoop.hbase.quotas.QuotaSettings;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot; import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.hadoop.hbase.security.access.Permission; import org.apache.hadoop.hbase.security.access.Permission;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
import org.apache.hadoop.hbase.thrift2.ThriftUtilities; import org.apache.hadoop.hbase.thrift2.ThriftUtilities;
import org.apache.hadoop.hbase.thrift2.generated.TColumnFamilyDescriptor; import org.apache.hadoop.hbase.thrift2.generated.TColumnFamilyDescriptor;
import org.apache.hadoop.hbase.thrift2.generated.THBaseService; import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
@ -78,6 +78,7 @@ public class ThriftAdmin implements Admin {
private THBaseService.Client client; private THBaseService.Client client;
private TTransport transport; private TTransport transport;
private int operationTimeout; private int operationTimeout;
private int syncWaitTimeout;
private Configuration conf; private Configuration conf;
@ -86,6 +87,7 @@ public class ThriftAdmin implements Admin {
this.transport = tTransport; this.transport = tTransport;
this.operationTimeout = conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, this.operationTimeout = conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT); HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
this.syncWaitTimeout = conf.getInt("hbase.client.sync.wait.timeout.msec", 10 * 60000); // 10min
this.conf = conf; this.conf = conf;
} }
@ -95,8 +97,12 @@ public class ThriftAdmin implements Admin {
} }
@Override @Override
public void abort(String why, Throwable e) { public int getSyncWaitTimeout() {
return syncWaitTimeout;
}
@Override
public void abort(String why, Throwable e) {
} }
@Override @Override
@ -986,7 +992,7 @@ public class ThriftAdmin implements Admin {
} }
@Override @Override
public void snapshotAsync(SnapshotDescription snapshot) { public Future<Void> snapshotAsync(SnapshotDescription snapshot) {
throw new NotImplementedException("snapshotAsync not supported in ThriftAdmin"); throw new NotImplementedException("snapshotAsync not supported in ThriftAdmin");
} }
@ -1013,45 +1019,15 @@ public class ThriftAdmin implements Admin {
throw new NotImplementedException("restoreSnapshotAsync not supported in ThriftAdmin"); throw new NotImplementedException("restoreSnapshotAsync not supported in ThriftAdmin");
} }
@Override
public void restoreSnapshot(byte[] snapshotName, boolean takeFailSafeSnapshot) {
throw new NotImplementedException("restoreSnapshot not supported in ThriftAdmin");
}
@Override
public void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot) {
throw new NotImplementedException("restoreSnapshot not supported in ThriftAdmin");
}
@Override @Override
public void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, public void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot,
boolean restoreAcl) { boolean restoreAcl) {
throw new NotImplementedException("restoreSnapshot not supported in ThriftAdmin"); throw new NotImplementedException("restoreSnapshot not supported in ThriftAdmin");
} }
@Override @Override
public void cloneSnapshot(byte[] snapshotName, TableName tableName) { public Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName, boolean cloneAcl)
throw new NotImplementedException("cloneSnapshot not supported in ThriftAdmin"); throws IOException, TableExistsException, RestoreSnapshotException {
}
@Override
public void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl) {
throw new NotImplementedException("cloneSnapshot not supported in ThriftAdmin");
}
@Override
public void cloneSnapshot(String snapshotName, TableName tableName) {
throw new NotImplementedException("cloneSnapshot not supported in ThriftAdmin");
}
@Override
public Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName) {
throw new NotImplementedException("cloneSnapshotAsync not supported in ThriftAdmin"); throw new NotImplementedException("cloneSnapshotAsync not supported in ThriftAdmin");
} }
@ -1102,43 +1078,36 @@ public class ThriftAdmin implements Admin {
@Override @Override
public void deleteSnapshot(byte[] snapshotName) { public void deleteSnapshot(byte[] snapshotName) {
throw new NotImplementedException("deleteSnapshot not supported in ThriftAdmin"); throw new NotImplementedException("deleteSnapshot not supported in ThriftAdmin");
} }
@Override @Override
public void deleteSnapshot(String snapshotName) { public void deleteSnapshot(String snapshotName) {
throw new NotImplementedException("deleteSnapshot not supported in ThriftAdmin"); throw new NotImplementedException("deleteSnapshot not supported in ThriftAdmin");
} }
@Override @Override
public void deleteSnapshots(String regex) { public void deleteSnapshots(String regex) {
throw new NotImplementedException("deleteSnapshots not supported in ThriftAdmin"); throw new NotImplementedException("deleteSnapshots not supported in ThriftAdmin");
} }
@Override @Override
public void deleteSnapshots(Pattern pattern) { public void deleteSnapshots(Pattern pattern) {
throw new NotImplementedException("deleteSnapshots not supported in ThriftAdmin"); throw new NotImplementedException("deleteSnapshots not supported in ThriftAdmin");
} }
@Override @Override
public void deleteTableSnapshots(String tableNameRegex, String snapshotNameRegex) { public void deleteTableSnapshots(String tableNameRegex, String snapshotNameRegex) {
throw new NotImplementedException("deleteTableSnapshots not supported in ThriftAdmin"); throw new NotImplementedException("deleteTableSnapshots not supported in ThriftAdmin");
} }
@Override @Override
public void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) { public void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) {
throw new NotImplementedException("deleteTableSnapshots not supported in ThriftAdmin"); throw new NotImplementedException("deleteTableSnapshots not supported in ThriftAdmin");
} }
@Override @Override
public void setQuota(QuotaSettings quota) { public void setQuota(QuotaSettings quota) {
throw new NotImplementedException("setQuota not supported in ThriftAdmin"); throw new NotImplementedException("setQuota not supported in ThriftAdmin");
} }
@Override @Override
@ -1164,13 +1133,11 @@ public class ThriftAdmin implements Admin {
@Override @Override
public void updateConfiguration(ServerName server) { public void updateConfiguration(ServerName server) {
throw new NotImplementedException("updateConfiguration not supported in ThriftAdmin"); throw new NotImplementedException("updateConfiguration not supported in ThriftAdmin");
} }
@Override @Override
public void updateConfiguration() { public void updateConfiguration() {
throw new NotImplementedException("updateConfiguration not supported in ThriftAdmin"); throw new NotImplementedException("updateConfiguration not supported in ThriftAdmin");
} }
@Override @Override
@ -1198,46 +1165,22 @@ public class ThriftAdmin implements Admin {
throw new NotImplementedException("isMergeEnabled not supported in ThriftAdmin"); throw new NotImplementedException("isMergeEnabled not supported in ThriftAdmin");
} }
@Override
public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) {
throw new NotImplementedException("addReplicationPeer not supported in ThriftAdmin");
}
@Override @Override
public Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig, public Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig,
boolean enabled) { boolean enabled) {
throw new NotImplementedException("addReplicationPeerAsync not supported in ThriftAdmin"); throw new NotImplementedException("addReplicationPeerAsync not supported in ThriftAdmin");
} }
@Override
public void removeReplicationPeer(String peerId) {
throw new NotImplementedException("removeReplicationPeer not supported in ThriftAdmin");
}
@Override @Override
public Future<Void> removeReplicationPeerAsync(String peerId) { public Future<Void> removeReplicationPeerAsync(String peerId) {
throw new NotImplementedException("removeReplicationPeerAsync not supported in ThriftAdmin"); throw new NotImplementedException("removeReplicationPeerAsync not supported in ThriftAdmin");
} }
@Override
public void enableReplicationPeer(String peerId) {
throw new NotImplementedException("enableReplicationPeer not supported in ThriftAdmin");
}
@Override @Override
public Future<Void> enableReplicationPeerAsync(String peerId) { public Future<Void> enableReplicationPeerAsync(String peerId) {
throw new NotImplementedException("enableReplicationPeerAsync not supported in ThriftAdmin"); throw new NotImplementedException("enableReplicationPeerAsync not supported in ThriftAdmin");
} }
@Override
public void disableReplicationPeer(String peerId) {
throw new NotImplementedException("disableReplicationPeer not supported in ThriftAdmin");
}
@Override @Override
public Future<Void> disableReplicationPeerAsync(String peerId) { public Future<Void> disableReplicationPeerAsync(String peerId) {
throw new NotImplementedException("disableReplicationPeerAsync not supported in ThriftAdmin"); throw new NotImplementedException("disableReplicationPeerAsync not supported in ThriftAdmin");
@ -1248,12 +1191,6 @@ public class ThriftAdmin implements Admin {
throw new NotImplementedException("getReplicationPeerConfig not supported in ThriftAdmin"); throw new NotImplementedException("getReplicationPeerConfig not supported in ThriftAdmin");
} }
@Override
public void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig) {
throw new NotImplementedException("updateReplicationPeerConfig not supported in ThriftAdmin");
}
@Override @Override
public Future<Void> updateReplicationPeerConfigAsync(String peerId, public Future<Void> updateReplicationPeerConfigAsync(String peerId,
ReplicationPeerConfig peerConfig) { ReplicationPeerConfig peerConfig) {
@ -1261,20 +1198,6 @@ public class ThriftAdmin implements Admin {
"updateReplicationPeerConfigAsync not supported in ThriftAdmin"); "updateReplicationPeerConfigAsync not supported in ThriftAdmin");
} }
@Override
public void appendReplicationPeerTableCFs(String id, Map<TableName, List<String>> tableCfs)
throws ReplicationException, IOException {
throw new NotImplementedException("appendReplicationPeerTableCFs not supported in ThriftAdmin");
}
@Override
public void removeReplicationPeerTableCFs(String id, Map<TableName, List<String>> tableCfs)
throws ReplicationException, IOException {
throw new NotImplementedException("removeReplicationPeerTableCFs not supported in ThriftAdmin");
}
@Override @Override
public List<ReplicationPeerDescription> listReplicationPeers() { public List<ReplicationPeerDescription> listReplicationPeers() {
throw new NotImplementedException("listReplicationPeers not supported in ThriftAdmin"); throw new NotImplementedException("listReplicationPeers not supported in ThriftAdmin");