@InterfaceAudience.Private @InterfaceStability.Evolving public class HBaseAdmin extends Object implements Admin
Connection.getAdmin()
to obtain an instance of Admin
instead of constructing
an HBaseAdmin directly.
Connection should be an unmanaged connection obtained via
ConnectionFactory.createConnection(Configuration)
ConnectionFactory
,
Connection
,
Admin
Modifier and Type | Class and Description |
---|---|
private static class |
HBaseAdmin.AbortProcedureFuture |
private static class |
HBaseAdmin.CreateTableFuture |
private static class |
HBaseAdmin.DeleteTableFuture |
private static class |
HBaseAdmin.DisableTableFuture |
private static class |
HBaseAdmin.EnableTableFuture |
protected static class |
HBaseAdmin.ProcedureFuture<V>
Future that waits on a procedure result.
|
private static class |
HBaseAdmin.ThrowableAbortable
Simple
Abortable , throwing RuntimeException on abort. |
Modifier and Type | Field and Description |
---|---|
private boolean |
aborted |
private boolean |
cleanupConnectionOnClose |
private boolean |
closed |
private org.apache.hadoop.conf.Configuration |
conf |
private ClusterConnection |
connection |
private static org.apache.commons.logging.Log |
LOG |
private NonceGenerator |
ng |
private int |
numRetries |
private int |
operationTimeout |
private long |
pause |
private int |
retryLongerMultiplier |
private RpcRetryingCallerFactory |
rpcCallerFactory |
private RpcControllerFactory |
rpcControllerFactory |
private int |
rpcTimeout |
private int |
syncWaitTimeout |
private static String |
ZK_IDENTIFIER_PREFIX |
Constructor and Description |
---|
HBaseAdmin(ClusterConnection connection) |
HBaseAdmin(org.apache.hadoop.conf.Configuration c)
Deprecated.
Constructing HBaseAdmin objects manually has been deprecated.
Use
Connection.getAdmin() to obtain an instance of Admin instead. |
HBaseAdmin(Connection connection)
Deprecated.
Constructing HBaseAdmin objects manually has been deprecated.
Use
Connection.getAdmin() to obtain an instance of Admin instead. |
Modifier and Type | Method and Description |
---|---|
void |
abort(String why,
Throwable e)
Abort the server or client.
|
boolean |
abortProcedure(long procId,
boolean mayInterruptIfRunning)
Abort a procedure
|
Future<Boolean> |
abortProcedureAsync(long procId,
boolean mayInterruptIfRunning)
Abort a procedure but does not block and wait for it be completely removed.
|
void |
addColumn(byte[] tableName,
HColumnDescriptor column)
Add a column to an existing table.
|
void |
addColumn(String tableName,
HColumnDescriptor column)
Add a column to an existing table.
|
void |
addColumn(TableName tableName,
HColumnDescriptor column)
Add a column to an existing table.
|
void |
assign(byte[] regionName) |
boolean |
balancer()
Invoke the balancer.
|
static void |
checkHBaseAvailable(org.apache.hadoop.conf.Configuration conf)
Check to see if HBase is running.
|
private void |
checkTableExistence(TableName tableName) |
private TableName |
checkTableExists(TableName tableName)
Check if table exists or not
|
void |
cloneSnapshot(byte[] snapshotName,
byte[] tableName)
Create a new table by cloning the snapshot content.
|
void |
cloneSnapshot(byte[] snapshotName,
TableName tableName)
Create a new table by cloning the snapshot content.
|
void |
cloneSnapshot(String snapshotName,
String tableName)
Create a new table by cloning the snapshot content.
|
void |
cloneSnapshot(String snapshotName,
TableName tableName)
Create a new table by cloning the snapshot content.
|
void |
close() |
void |
closeRegion(byte[] regionname,
String serverName)
Close a region.
|
void |
closeRegion(ServerName sn,
HRegionInfo hri)
Close a region.
|
void |
closeRegion(String regionname,
String serverName)
Close a region.
|
boolean |
closeRegionWithEncodedRegionName(String encodedRegionName,
String serverName)
For expert-admins.
|
void |
compact(byte[] tableNameOrRegionName)
Deprecated.
Use
compact(org.apache.hadoop.hbase.TableName) or (byte[]) instead. |
void |
compact(byte[] tableNameOrRegionName,
byte[] columnFamily)
Deprecated.
Use
compact(org.apache.hadoop.hbase.TableName) or (byte[], byte[]) instead. |
private void |
compact(ServerName sn,
HRegionInfo hri,
boolean major,
byte[] family) |
void |
compact(String tableNameOrRegionName)
Deprecated.
Use
compact(org.apache.hadoop.hbase.TableName) or (byte[]) instead. |
void |
compact(String tableOrRegionName,
String columnFamily)
Deprecated.
Use
compact(org.apache.hadoop.hbase.TableName) or (byte[], byte[]) instead. |
void |
compact(TableName tableName)
Compact a table.
|
void |
compact(TableName tableName,
byte[] columnFamily)
Compact a column family within a table.
|
private void |
compact(TableName tableName,
byte[] columnFamily,
boolean major)
Compact a table.
|
void |
compactRegion(byte[] regionName)
Compact an individual region.
|
void |
compactRegion(byte[] regionName,
byte[] columnFamily)
Compact a column family within a region.
|
private void |
compactRegion(byte[] regionName,
byte[] columnFamily,
boolean major)
Compact an individual region.
|
void |
compactRegionServer(ServerName sn,
boolean major)
Compact all regions on the region server
|
CoprocessorRpcChannel |
coprocessorService()
Creates and returns a
RpcChannel instance
connected to the active master. |
CoprocessorRpcChannel |
coprocessorService(ServerName sn)
Creates and returns a
RpcChannel instance
connected to the passed region server. |
void |
createNamespace(NamespaceDescriptor descriptor)
Create a new namespace
|
void |
createTable(HTableDescriptor desc)
Creates a new table.
|
void |
createTable(HTableDescriptor desc,
byte[][] splitKeys)
Creates a new table with an initial set of empty regions defined by the
specified split keys.
|
void |
createTable(HTableDescriptor desc,
byte[] startKey,
byte[] endKey,
int numRegions)
Creates a new table with the specified number of regions.
|
void |
createTableAsync(HTableDescriptor desc,
byte[][] splitKeys)
Creates a new table but does not block and wait for it to come online.
|
private Future<Void> |
createTableAsyncV2(HTableDescriptor desc,
byte[][] splitKeys)
Creates a new table but does not block and wait for it to come online.
|
void |
deleteColumn(byte[] tableName,
String columnName)
Delete a column from a table.
|
void |
deleteColumn(String tableName,
String columnName)
Delete a column from a table.
|
void |
deleteColumn(TableName tableName,
byte[] columnName)
Delete a column from a table.
|
void |
deleteNamespace(String name)
Delete an existing namespace.
|
void |
deleteSnapshot(byte[] snapshotName)
Delete an existing snapshot.
|
void |
deleteSnapshot(String snapshotName)
Delete an existing snapshot.
|
void |
deleteSnapshots(Pattern pattern)
Delete existing snapshots whose names match the pattern passed.
|
void |
deleteSnapshots(String regex)
Delete existing snapshots whose names match the pattern passed.
|
void |
deleteTable(byte[] tableName) |
void |
deleteTable(String tableName) |
void |
deleteTable(TableName tableName)
Deletes a table.
|
private Future<Void> |
deleteTableAsyncV2(TableName tableName)
Deletes the table but does not block and wait for it be completely removed.
|
HTableDescriptor[] |
deleteTables(Pattern pattern)
Delete tables matching the passed in pattern and wait on completion.
|
HTableDescriptor[] |
deleteTables(String regex)
Deletes tables matching the passed in pattern and wait on completion.
|
void |
disableTable(byte[] tableName) |
void |
disableTable(String tableName) |
void |
disableTable(TableName tableName)
Disable table and wait on completion.
|
void |
disableTableAsync(byte[] tableName) |
void |
disableTableAsync(String tableName) |
void |
disableTableAsync(TableName tableName)
Starts the disable of a table.
|
private Future<Void> |
disableTableAsyncV2(TableName tableName)
Disable the table but does not block and wait for it be completely disabled.
|
HTableDescriptor[] |
disableTables(Pattern pattern)
Disable tables matching the passed in pattern and wait on completion.
|
HTableDescriptor[] |
disableTables(String regex)
Disable tables matching the passed in pattern and wait on completion.
|
boolean |
enableCatalogJanitor(boolean enable)
Enable/Disable the catalog janitor
|
void |
enableTable(byte[] tableName) |
void |
enableTable(String tableName) |
void |
enableTable(TableName tableName)
Enable a table.
|
void |
enableTableAsync(byte[] tableName) |
void |
enableTableAsync(String tableName) |
void |
enableTableAsync(TableName tableName)
Brings a table on-line (enables it).
|
private Future<Void> |
enableTableAsyncV2(TableName tableName)
Enable the table but does not block and wait for it be completely enabled.
|
HTableDescriptor[] |
enableTables(Pattern pattern)
Enable tables matching the passed in pattern and wait on completion.
|
HTableDescriptor[] |
enableTables(String regex)
Enable tables matching the passed in pattern and wait on completion.
|
void |
execProcedure(String signature,
String instance,
Map<String,String> props)
Execute a distributed procedure on a cluster.
|
byte[] |
execProcedureWithRet(String signature,
String instance,
Map<String,String> props)
Execute a distributed procedure on a cluster synchronously with return data
|
private <V> V |
executeCallable(MasterCallable<V> callable) |
private static <V> V |
executeCallable(MasterCallable<V> callable,
RpcRetryingCallerFactory rpcCallerFactory,
int operationTimeout,
int rpcTimeout) |
void |
flush(byte[] tableNameOrRegionName)
Deprecated.
Use
flush(org.apache.hadoop.hbase.TableName) or (byte[]) instead. |
private void |
flush(ServerName sn,
HRegionInfo hri) |
void |
flush(String tableNameOrRegionName)
Deprecated.
Use
flush(org.apache.hadoop.hbase.TableName) or (byte[]) instead. |
void |
flush(TableName tableName)
Flush a table.
|
void |
flushRegion(byte[] regionName)
Flush an individual region.
|
Pair<Integer,Integer> |
getAlterStatus(byte[] tableName)
Get the status of alter command - indicates how many regions have received
the updated schema Asynchronous operation.
|
Pair<Integer,Integer> |
getAlterStatus(TableName tableName)
Get the status of alter command - indicates how many regions have received
the updated schema Asynchronous operation.
|
ClusterStatus |
getClusterStatus() |
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState |
getCompactionState(byte[] tableNameOrRegionName)
Deprecated.
|
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState |
getCompactionState(String tableNameOrRegionName)
Deprecated.
|
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState |
getCompactionState(TableName tableName)
Get the current compaction state of a table.
|
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState |
getCompactionStateForRegion(byte[] regionName)
Get the current compaction state of region.
|
org.apache.hadoop.conf.Configuration |
getConfiguration() |
HConnection |
getConnection() |
long |
getLastMajorCompactionTimestamp(TableName tableName)
Get the timestamp of the last major compaction for the passed table
The timestamp of the oldest HFile resulting from a major compaction of that table,
or 0 if no such HFile could be found.
|
long |
getLastMajorCompactionTimestampForRegion(byte[] regionName)
Get the timestamp of the last major compaction for the passed region.
|
String[] |
getMasterCoprocessors()
Helper delegage to getClusterStatus().getMasterCoprocessors().
|
int |
getMasterInfoPort()
Get the info port of the current master if one is available.
|
NamespaceDescriptor |
getNamespaceDescriptor(String name)
Get a namespace descriptor by name
|
List<HRegionInfo> |
getOnlineRegions(ServerName sn)
Get all the online regions on a region server.
|
int |
getOperationTimeout() |
private long |
getPauseTime(int tries) |
QuotaRetriever |
getQuotaRetriever(QuotaFilter filter)
Return a Quota Scanner to list the quotas based on the filter.
|
(package private) Pair<HRegionInfo,ServerName> |
getRegion(byte[] regionName) |
private byte[] |
getRegionName(byte[] regionNameOrEncodedRegionName)
If the input is a region name, it is returned as is.
|
private RpcControllerFactory |
getRpcControllerFactory() |
List<SecurityCapability> |
getSecurityCapabilities()
Return the set of supported security capabilities.
|
HTableDescriptor |
getTableDescriptor(byte[] tableName) |
HTableDescriptor |
getTableDescriptor(TableName tableName)
Method for getting the tableDescriptor
|
(package private) static HTableDescriptor |
getTableDescriptor(TableName tableName,
HConnection connection,
RpcRetryingCallerFactory rpcCallerFactory,
RpcControllerFactory rpcControllerFactory,
int operationTimeout,
int rpcTimeout) |
private HTableDescriptor |
getTableDescriptorByTableName(TableName tableName)
Get tableDescriptor
|
HTableDescriptor[] |
getTableDescriptors(List<String> names)
Get tableDescriptors
|
HTableDescriptor[] |
getTableDescriptorsByTableName(List<TableName> tableNames)
Get tableDescriptors
|
String[] |
getTableNames()
Deprecated.
Use
Admin.listTableNames() instead |
String[] |
getTableNames(Pattern pattern)
Deprecated.
Use
Admin.listTableNames(Pattern) instead. |
String[] |
getTableNames(String regex)
Deprecated.
Use
Admin.listTableNames(Pattern) instead. |
List<HRegionInfo> |
getTableRegions(byte[] tableName) |
List<HRegionInfo> |
getTableRegions(TableName tableName)
get the regions of a given table.
|
private void |
internalDeleteSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot) |
private void |
internalRestoreSnapshot(String snapshotName,
TableName tableName)
Execute Restore/Clone snapshot and wait for the server to complete (blocking).
|
private org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotResponse |
internalRestoreSnapshotAsync(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot)
Execute Restore/Clone snapshot and wait for the server to complete (asynchronous)
|
boolean |
isAborted()
Check if the server or client was aborted.
|
boolean |
isBalancerEnabled()
Query the state of the balancer from the Master.
|
boolean |
isCatalogJanitorEnabled()
Query on the catalog janitor state (Enabled/Disabled?)
|
private boolean |
isEncodedRegionName(byte[] regionName) |
boolean |
isMasterRunning()
Deprecated.
this has been deprecated without a replacement
|
private boolean |
isMetaRegion(byte[] regionName) |
boolean |
isNormalizerEnabled()
Query the current state of the region normalizer
|
boolean |
isProcedureFinished(String signature,
String instance,
Map<String,String> props)
Check the current state of the specified procedure.
|
boolean |
isSnapshotFinished(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot)
Check the current state of the passed snapshot.
|
boolean |
isTableAvailable(byte[] tableName) |
boolean |
isTableAvailable(byte[] tableName,
byte[][] splitKeys) |
boolean |
isTableAvailable(String tableName) |
boolean |
isTableAvailable(String tableName,
byte[][] splitKeys) |
boolean |
isTableAvailable(TableName tableName) |
boolean |
isTableAvailable(TableName tableName,
byte[][] splitKeys)
Use this api to check if the table has been created with the specified number of
splitkeys which was used while creating the given table.
|
boolean |
isTableDisabled(byte[] tableName) |
boolean |
isTableDisabled(String tableName) |
boolean |
isTableDisabled(TableName tableName) |
boolean |
isTableEnabled(byte[] tableName) |
boolean |
isTableEnabled(String tableName) |
boolean |
isTableEnabled(TableName tableName) |
NamespaceDescriptor[] |
listNamespaceDescriptors()
List available namespace descriptors
|
ProcedureInfo[] |
listProcedures()
List procedures
|
List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription> |
listSnapshots()
List completed snapshots.
|
List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription> |
listSnapshots(Pattern pattern)
List all the completed snapshots matching the given pattern.
|
List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription> |
listSnapshots(String regex)
List all the completed snapshots matching the given regular expression.
|
HTableDescriptor[] |
listTableDescriptorsByNamespace(String name)
Get list of table descriptors by namespace
|
TableName[] |
listTableNames()
List all of the names of userspace tables.
|
TableName[] |
listTableNames(Pattern pattern)
List all of the names of userspace tables.
|
TableName[] |
listTableNames(Pattern pattern,
boolean includeSysTables)
List all of the names of userspace tables.
|
TableName[] |
listTableNames(String regex)
List all of the names of userspace tables.
|
TableName[] |
listTableNames(String regex,
boolean includeSysTables)
List all of the names of userspace tables.
|
TableName[] |
listTableNamesByNamespace(String name)
Get list of table names by namespace
|
HTableDescriptor[] |
listTables()
List all the userspace tables.
|
HTableDescriptor[] |
listTables(Pattern pattern)
List all the userspace tables matching the given pattern.
|
HTableDescriptor[] |
listTables(Pattern pattern,
boolean includeSysTables)
List all the tables matching the given pattern.
|
HTableDescriptor[] |
listTables(String regex)
List all the userspace tables matching the given regular expression.
|
HTableDescriptor[] |
listTables(String regex,
boolean includeSysTables)
List all the tables matching the given pattern.
|
void |
majorCompact(byte[] tableNameOrRegionName)
Deprecated.
|
void |
majorCompact(byte[] tableNameOrRegionName,
byte[] columnFamily)
Deprecated.
|
void |
majorCompact(String tableNameOrRegionName)
Deprecated.
|
void |
majorCompact(String tableNameOrRegionName,
String columnFamily)
Deprecated.
|
void |
majorCompact(TableName tableName)
Major compact a table.
|
void |
majorCompact(TableName tableName,
byte[] columnFamily)
Major compact a column family within a table.
|
void |
majorCompactRegion(byte[] regionName)
Major compact a table or an individual region.
|
void |
majorCompactRegion(byte[] regionName,
byte[] columnFamily)
Major compact a column family within region.
|
void |
mergeRegions(byte[] nameOfRegionA,
byte[] nameOfRegionB,
boolean forcible)
Merge two regions.
|
void |
modifyColumn(byte[] tableName,
HColumnDescriptor descriptor)
Modify an existing column family on a table.
|
void |
modifyColumn(String tableName,
HColumnDescriptor descriptor)
Modify an existing column family on a table.
|
void |
modifyColumn(TableName tableName,
HColumnDescriptor descriptor)
Modify an existing column family on a table.
|
void |
modifyNamespace(NamespaceDescriptor descriptor)
Modify an existing namespace
|
void |
modifyTable(byte[] tableName,
HTableDescriptor htd) |
void |
modifyTable(String tableName,
HTableDescriptor htd) |
void |
modifyTable(TableName tableName,
HTableDescriptor htd)
Modify an existing table, more IRB friendly version.
|
void |
move(byte[] encodedRegionName,
byte[] destServerName)
Move the region
r to dest . |
boolean |
normalize()
Invoke region normalizer.
|
void |
offline(byte[] regionName)
Offline specified region from master's in-memory state.
|
void |
restoreSnapshot(byte[] snapshotName)
Restore the specified snapshot on the original table.
|
void |
restoreSnapshot(byte[] snapshotName,
boolean takeFailSafeSnapshot)
Restore the specified snapshot on the original table.
|
void |
restoreSnapshot(String snapshotName)
Restore the specified snapshot on the original table.
|
void |
restoreSnapshot(String snapshotName,
boolean takeFailSafeSnapshot)
Restore the specified snapshot on the original table.
|
byte[][] |
rollHLogWriter(String serverName)
Deprecated.
|
void |
rollWALWriter(ServerName serverName)
Roll the log writer.
|
private org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse |
rollWALWriterImpl(ServerName sn) |
int |
runCatalogScan()
Ask for a scan of the catalog table
|
boolean |
setBalancerRunning(boolean on,
boolean synchronous)
Turn the load balancer on or off.
|
boolean |
setNormalizerRunning(boolean on)
Turn region normalizer on or off.
|
void |
setQuota(QuotaSettings quota)
Apply the new quota settings.
|
void |
shutdown()
Shuts down the HBase cluster
|
void |
snapshot(byte[] snapshotName,
byte[] tableName) |
void |
snapshot(byte[] snapshotName,
byte[] tableName,
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type flushType)
Create snapshot for the given table of given flush type.
|
void |
snapshot(byte[] snapshotName,
TableName tableName)
public void snapshot(final String snapshotName,
Create a timestamp consistent snapshot for the given table.
|
void |
snapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot)
Take a snapshot and wait for the server to complete that snapshot (blocking).
|
void |
snapshot(String snapshotName,
byte[] tableName,
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type type) |
void |
snapshot(String snapshotName,
String tableName) |
void |
snapshot(String snapshotName,
String tableName,
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type type) |
void |
snapshot(String snapshotName,
TableName tableName)
Take a snapshot for the given table.
|
void |
snapshot(String snapshotName,
TableName tableName,
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type type)
Create typed snapshot of the table.
|
void |
split(byte[] tableNameOrRegionName)
Deprecated.
Use
split(org.apache.hadoop.hbase.TableName) or (byte[]) instead. |
void |
split(byte[] tableNameOrRegionName,
byte[] splitPoint)
Deprecated.
|
void |
split(ServerName sn,
HRegionInfo hri,
byte[] splitPoint) |
void |
split(String tableNameOrRegionName)
Deprecated.
Use
split(org.apache.hadoop.hbase.TableName) or (byte[]) instead. |
void |
split(String tableNameOrRegionName,
String splitPoint)
Deprecated.
|
void |
split(TableName tableName)
Split a table.
|
void |
split(TableName tableName,
byte[] splitPoint)
Split a table.
|
void |
splitRegion(byte[] regionName)
Split an individual region.
|
void |
splitRegion(byte[] regionName,
byte[] splitPoint)
Split an individual region.
|
void |
stopMaster()
Shuts down the current HBase master only.
|
void |
stopRegionServer(String hostnamePort)
Stop the designated regionserver
|
boolean |
tableExists(byte[] tableName) |
boolean |
tableExists(String tableName) |
boolean |
tableExists(TableName tableName) |
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse |
takeSnapshotAsync(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot)
Take a snapshot without waiting for the server to complete that snapshot (asynchronous)
|
void |
truncateTable(TableName tableName,
boolean preserveSplits)
Truncate a table.
|
void |
unassign(byte[] regionName,
boolean force)
Unassign a region from current hosting regionserver.
|
void |
updateConfiguration()
Update the configuration and trigger an online config change
on all the regionservers
|
void |
updateConfiguration(ServerName server)
Update the configuration and trigger an online config change
on the regionserver
|
private void |
waitUntilTableIsEnabled(TableName tableName)
Wait for the table to be enabled and available
If enabling the table exceeds the retry period, an exception is thrown.
|
private static final org.apache.commons.logging.Log LOG
private static final String ZK_IDENTIFIER_PREFIX
private ClusterConnection connection
private volatile org.apache.hadoop.conf.Configuration conf
private final long pause
private final int numRetries
private final int retryLongerMultiplier
private final int syncWaitTimeout
private boolean aborted
private boolean cleanupConnectionOnClose
private boolean closed
private int operationTimeout
private int rpcTimeout
private RpcRetryingCallerFactory rpcCallerFactory
private RpcControllerFactory rpcControllerFactory
private NonceGenerator ng
@Deprecated public HBaseAdmin(org.apache.hadoop.conf.Configuration c) throws MasterNotRunningException, ZooKeeperConnectionException, IOException
Connection.getAdmin()
to obtain an instance of Admin
instead.HBaseAdmin(Connection connection)
c
- Configuration object. Copied internally.MasterNotRunningException
ZooKeeperConnectionException
IOException
@Deprecated public HBaseAdmin(Connection connection) throws MasterNotRunningException, ZooKeeperConnectionException
Connection.getAdmin()
to obtain an instance of Admin
instead.connection
- The Connection instance to useMasterNotRunningException
ZooKeeperConnectionException
- are not
thrown anymore but kept into the interface for backward api compatibilityHBaseAdmin(ClusterConnection connection)
public int getOperationTimeout()
getOperationTimeout
in interface Admin
public void abort(String why, Throwable e)
Abortable
public boolean isAborted()
Abortable
public boolean abortProcedure(long procId, boolean mayInterruptIfRunning) throws IOException
abortProcedure
in interface Admin
procId
- ID of the procedure to abortmayInterruptIfRunning
- if the proc completed at least one step, should it be aborted?IOException
public Future<Boolean> abortProcedureAsync(long procId, boolean mayInterruptIfRunning) throws IOException
abortProcedureAsync
in interface Admin
procId
- ID of the procedure to abortmayInterruptIfRunning
- if the proc completed at least one step, should it be aborted?IOException
public HConnection getConnection()
getConnection
in interface Admin
@Deprecated public boolean isMasterRunning() throws MasterNotRunningException, ZooKeeperConnectionException
ZooKeeperConnectionException
MasterNotRunningException
public boolean tableExists(TableName tableName) throws IOException
tableExists
in interface Admin
tableName
- Table to check.IOException
public boolean tableExists(byte[] tableName) throws IOException
IOException
public boolean tableExists(String tableName) throws IOException
IOException
public HTableDescriptor[] listTables() throws IOException
Admin
listTables
in interface Admin
IOException
- if a remote or network exception occurspublic HTableDescriptor[] listTables(Pattern pattern) throws IOException
Admin
listTables
in interface Admin
pattern
- The compiled regular expression to match againstIOException
- if a remote or network exception occursAdmin.listTables()
public HTableDescriptor[] listTables(String regex) throws IOException
Admin
listTables
in interface Admin
regex
- The regular expression to match againstIOException
- if a remote or network exception occursAdmin.listTables(java.util.regex.Pattern)
public HTableDescriptor[] listTables(Pattern pattern, boolean includeSysTables) throws IOException
Admin
listTables
in interface Admin
pattern
- The compiled regular expression to match againstincludeSysTables
- False to match only against userspace tablesIOException
- if a remote or network exception occursAdmin.listTables()
public HTableDescriptor[] listTables(String regex, boolean includeSysTables) throws IOException
Admin
listTables
in interface Admin
regex
- The regular expression to match againstincludeSysTables
- False to match only against userspace tablesIOException
- if a remote or network exception occursAdmin.listTables(java.util.regex.Pattern, boolean)
@Deprecated public String[] getTableNames() throws IOException
Admin.listTableNames()
insteadIOException
- if a remote or network exception occurs@Deprecated public String[] getTableNames(Pattern pattern) throws IOException
Admin.listTableNames(Pattern)
instead.pattern
- The regular expression to match againstIOException
- if a remote or network exception occurs@Deprecated public String[] getTableNames(String regex) throws IOException
Admin.listTableNames(Pattern)
instead.regex
- The regular expression to match againstIOException
- if a remote or network exception occurspublic TableName[] listTableNames() throws IOException
Admin
listTableNames
in interface Admin
IOException
- if a remote or network exception occurspublic TableName[] listTableNames(Pattern pattern) throws IOException
Admin
listTableNames
in interface Admin
pattern
- The regular expression to match againstIOException
- if a remote or network exception occurspublic TableName[] listTableNames(String regex) throws IOException
Admin
listTableNames
in interface Admin
regex
- The regular expression to match againstIOException
- if a remote or network exception occurspublic TableName[] listTableNames(Pattern pattern, boolean includeSysTables) throws IOException
Admin
listTableNames
in interface Admin
pattern
- The regular expression to match againstincludeSysTables
- False to match only against userspace tablesIOException
- if a remote or network exception occurspublic TableName[] listTableNames(String regex, boolean includeSysTables) throws IOException
Admin
listTableNames
in interface Admin
regex
- The regular expression to match againstincludeSysTables
- False to match only against userspace tablesIOException
- if a remote or network exception occurspublic HTableDescriptor getTableDescriptor(TableName tableName) throws TableNotFoundException, IOException
getTableDescriptor
in interface Admin
tableName
- as a byte []TableNotFoundException
IOException
- if a remote or network exception occursstatic HTableDescriptor getTableDescriptor(TableName tableName, HConnection connection, RpcRetryingCallerFactory rpcCallerFactory, RpcControllerFactory rpcControllerFactory, int operationTimeout, int rpcTimeout) throws TableNotFoundException, IOException
TableNotFoundException
IOException
public HTableDescriptor getTableDescriptor(byte[] tableName) throws TableNotFoundException, IOException
TableNotFoundException
IOException
private long getPauseTime(int tries)
public void createTable(HTableDescriptor desc) throws IOException
createTable
in interface Admin
desc
- table descriptor for tableIllegalArgumentException
- if the table name is reservedMasterNotRunningException
- if master is not runningTableExistsException
- if table already exists (If concurrent
threads, the table may have been created between test-for-existence
and attempt-at-creation).IOException
- if a remote or network exception occurspublic void createTable(HTableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions) throws IOException
createTable
in interface Admin
desc
- table descriptor for tablestartKey
- beginning of key rangeendKey
- end of key rangenumRegions
- the total number of regions to createIllegalArgumentException
- if the table name is reservedMasterNotRunningException
- if master is not runningTableExistsException
- if table already exists (If concurrent
threads, the table may have been created between test-for-existence
and attempt-at-creation).IOException
public void createTable(HTableDescriptor desc, byte[][] splitKeys) throws IOException
createTable
in interface Admin
desc
- table descriptor for tablesplitKeys
- array of split keys for the initial regions of the tableIllegalArgumentException
- if the table name is reserved, if the split keys
are repeated and if the split key has empty byte array.MasterNotRunningException
- if master is not runningTableExistsException
- if table already exists (If concurrent
threads, the table may have been created between test-for-existence
and attempt-at-creation).IOException
public void createTableAsync(HTableDescriptor desc, byte[][] splitKeys) throws IOException
isTableAvailable(org.apache.hadoop.hbase.TableName)
-- it is not safe to create an HTable
instance to this table before it is available.
Note : Avoid passing empty split key.createTableAsync
in interface Admin
desc
- table descriptor for tableIllegalArgumentException
- Bad table name, if the split keys
are repeated and if the split key has empty byte array.MasterNotRunningException
- if master is not runningTableExistsException
- if table already exists (If concurrent
threads, the table may have been created between test-for-existence
and attempt-at-creation).IOException
private Future<Void> createTableAsyncV2(HTableDescriptor desc, byte[][] splitKeys) throws IOException
desc
- table descriptor for tablesplitKeys
- keys to check if the table has been created with all split keysIllegalArgumentException
- Bad table name, if the split keys
are repeated and if the split key has empty byte array.IOException
- if a remote or network exception occurspublic void deleteTable(String tableName) throws IOException
IOException
public void deleteTable(byte[] tableName) throws IOException
IOException
public void deleteTable(TableName tableName) throws IOException
deleteTable
in interface Admin
tableName
- name of table to deleteIOException
- if a remote or network exception occursprivate Future<Void> deleteTableAsyncV2(TableName tableName) throws IOException
desc
- table descriptor for tabletableName
- name of table to deleteIOException
- if a remote or network exception occurspublic HTableDescriptor[] deleteTables(String regex) throws IOException
listTables(java.lang.String)
and
deleteTable(byte[])
deleteTables
in interface Admin
regex
- The regular expression to match table names againstIOException
deleteTables(java.util.regex.Pattern)
,
deleteTable(java.lang.String)
public HTableDescriptor[] deleteTables(Pattern pattern) throws IOException
listTables(java.util.regex.Pattern)
and
deleteTable(byte[])
deleteTables
in interface Admin
pattern
- The pattern to match table names againstIOException
public void truncateTable(TableName tableName, boolean preserveSplits) throws IOException
truncateTable
in interface Admin
tableName
- name of table to truncatepreserveSplits
- True if the splits should be preservedIOException
- if a remote or network exception occurspublic void enableTable(TableName tableName) throws IOException
enableTableAsync(byte[])
and isTableEnabled(byte[])
instead.
The table has to be in disabled state for it to be enabled.enableTable
in interface Admin
tableName
- name of the tableIOException
- if a remote or network exception occurs
There could be couple types of IOException
TableNotFoundException means the table doesn't exist.
TableNotDisabledException means the table isn't in disabled state.isTableEnabled(byte[])
,
disableTable(byte[])
,
enableTableAsync(byte[])
public void enableTable(byte[] tableName) throws IOException
IOException
public void enableTable(String tableName) throws IOException
IOException
private void waitUntilTableIsEnabled(TableName tableName) throws IOException
tableName
- name of the tableIOException
- if a remote or network exception occurs or
table is not enabled after the retries period.public void enableTableAsync(TableName tableName) throws IOException
isTableEnabled(byte[])
to learn when table is fully online. If
table is taking too long to online, check server logs.enableTableAsync
in interface Admin
tableName
- IOException
public void enableTableAsync(byte[] tableName) throws IOException
IOException
public void enableTableAsync(String tableName) throws IOException
IOException
private Future<Void> enableTableAsyncV2(TableName tableName) throws IOException
tableName
- name of table to deleteIOException
- if a remote or network exception occurspublic HTableDescriptor[] enableTables(String regex) throws IOException
listTables(java.lang.String)
and
enableTable(byte[])
enableTables
in interface Admin
regex
- The regular expression to match table names againstIOException
enableTables(java.util.regex.Pattern)
,
enableTable(java.lang.String)
public HTableDescriptor[] enableTables(Pattern pattern) throws IOException
listTables(java.util.regex.Pattern)
and
enableTable(byte[])
enableTables
in interface Admin
pattern
- The pattern to match table names againstIOException
public void disableTableAsync(TableName tableName) throws IOException
isTableDisabled(byte[])
to check for when disable completes.
If table is taking too long to online, check server logs.disableTableAsync
in interface Admin
tableName
- name of tableIOException
- if a remote or network exception occursisTableDisabled(byte[])
,
isTableEnabled(byte[])
public void disableTableAsync(byte[] tableName) throws IOException
IOException
public void disableTableAsync(String tableName) throws IOException
IOException
public void disableTable(TableName tableName) throws IOException
disableTableAsync(byte[])
and isTableDisabled(String)
instead.
The table has to be in enabled state for it to be disabled.disableTable
in interface Admin
tableName
- IOException
- There could be couple types of IOException
TableNotFoundException means the table doesn't exist.
TableNotEnabledException means the table isn't in enabled state.public void disableTable(byte[] tableName) throws IOException
IOException
public void disableTable(String tableName) throws IOException
IOException
private Future<Void> disableTableAsyncV2(TableName tableName) throws IOException
tableName
- name of table to deleteIOException
- if a remote or network exception occurspublic HTableDescriptor[] disableTables(String regex) throws IOException
listTables(java.lang.String)
and
disableTable(byte[])
disableTables
in interface Admin
regex
- The regular expression to match table names againstIOException
disableTables(java.util.regex.Pattern)
,
disableTable(java.lang.String)
public HTableDescriptor[] disableTables(Pattern pattern) throws IOException
listTables(java.util.regex.Pattern)
and
disableTable(byte[])
disableTables
in interface Admin
pattern
- The pattern to match table names againstIOException
private void checkTableExistence(TableName tableName) throws IOException
IOException
public boolean isTableEnabled(TableName tableName) throws IOException
isTableEnabled
in interface Admin
tableName
- name of table to checkIOException
- if a remote or network exception occurspublic boolean isTableEnabled(byte[] tableName) throws IOException
IOException
public boolean isTableEnabled(String tableName) throws IOException
IOException
public boolean isTableDisabled(TableName tableName) throws IOException
isTableDisabled
in interface Admin
tableName
- name of table to checkIOException
- if a remote or network exception occurspublic boolean isTableDisabled(byte[] tableName) throws IOException
IOException
public boolean isTableDisabled(String tableName) throws IOException
IOException
public boolean isTableAvailable(TableName tableName) throws IOException
isTableAvailable
in interface Admin
tableName
- name of table to checkIOException
- if a remote or network exception occurspublic boolean isTableAvailable(byte[] tableName) throws IOException
IOException
public boolean isTableAvailable(String tableName) throws IOException
IOException
public boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws IOException
isTableAvailable
in interface Admin
tableName
- name of table to checksplitKeys
- keys to check if the table has been created with all split keysIOException
- if a remote or network excpetion occurspublic boolean isTableAvailable(byte[] tableName, byte[][] splitKeys) throws IOException
IOException
public boolean isTableAvailable(String tableName, byte[][] splitKeys) throws IOException
IOException
public Pair<Integer,Integer> getAlterStatus(TableName tableName) throws IOException
getAlterStatus
in interface Admin
tableName
- TableName instanceIOException
- if a remote or network exception occurspublic Pair<Integer,Integer> getAlterStatus(byte[] tableName) throws IOException
getAlterStatus
in interface Admin
tableName
- name of the table to get the status ofIOException
- if a remote or network exception occurspublic void addColumn(byte[] tableName, HColumnDescriptor column) throws IOException
tableName
- name of the table to add column tocolumn
- column descriptor of column to be addedIOException
- if a remote or network exception occurspublic void addColumn(String tableName, HColumnDescriptor column) throws IOException
tableName
- name of the table to add column tocolumn
- column descriptor of column to be addedIOException
- if a remote or network exception occurspublic void addColumn(TableName tableName, HColumnDescriptor column) throws IOException
addColumn
in interface Admin
tableName
- name of the table to add column tocolumn
- column descriptor of column to be addedIOException
- if a remote or network exception occurspublic void deleteColumn(byte[] tableName, String columnName) throws IOException
tableName
- name of tablecolumnName
- name of column to be deletedIOException
- if a remote or network exception occurspublic void deleteColumn(String tableName, String columnName) throws IOException
tableName
- name of tablecolumnName
- name of column to be deletedIOException
- if a remote or network exception occurspublic void deleteColumn(TableName tableName, byte[] columnName) throws IOException
deleteColumn
in interface Admin
tableName
- name of tablecolumnName
- name of column to be deletedIOException
- if a remote or network exception occurspublic void modifyColumn(String tableName, HColumnDescriptor descriptor) throws IOException
tableName
- name of tabledescriptor
- new column descriptor to useIOException
- if a remote or network exception occurspublic void modifyColumn(byte[] tableName, HColumnDescriptor descriptor) throws IOException
tableName
- name of tabledescriptor
- new column descriptor to useIOException
- if a remote or network exception occurspublic void modifyColumn(TableName tableName, HColumnDescriptor descriptor) throws IOException
modifyColumn
in interface Admin
tableName
- name of tabledescriptor
- new column descriptor to useIOException
- if a remote or network exception occurspublic void closeRegion(String regionname, String serverName) throws IOException
closeRegion
in interface Admin
regionname
- region name to closeserverName
- If supplied, we'll use this location rather than
the one currently in hbase:meta
IOException
- if a remote or network exception occurspublic void closeRegion(byte[] regionname, String serverName) throws IOException
closeRegion
in interface Admin
regionname
- region name to closeserverName
- The servername of the regionserver. If passed null we
will use servername found in the hbase:meta table. A server name
is made of host, port and startcode. Here is an example:
host187.example.com,60020,1289493121758
IOException
- if a remote or network exception occurspublic boolean closeRegionWithEncodedRegionName(String encodedRegionName, String serverName) throws IOException
closeRegionWithEncodedRegionName
in interface Admin
encodedRegionName
- The encoded region name; i.e. the hash that makes up the region
name suffix: e.g. if regionname is
TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.
, then the encoded region name is:
527db22f95c8a9e0116f0cc13c680396
.serverName
- The servername of the regionserver. A server name is made of host,
port and startcode. This is mandatory. Here is an example:
host187.example.com,60020,1289493121758
IOException
- if a remote or network exception occurspublic void closeRegion(ServerName sn, HRegionInfo hri) throws IOException
closeRegion
in interface Admin
sn
- hri
- IOException
public List<HRegionInfo> getOnlineRegions(ServerName sn) throws IOException
getOnlineRegions
in interface Admin
IOException
public void flush(TableName tableName) throws IOException
flush
in interface Admin
tableName
- table to flushIOException
- if a remote or network exception occurspublic void flushRegion(byte[] regionName) throws IOException
flushRegion
in interface Admin
regionName
- region to flushIOException
- if a remote or network exception occurs@Deprecated public void flush(String tableNameOrRegionName) throws IOException, InterruptedException
IOException
InterruptedException
@Deprecated public void flush(byte[] tableNameOrRegionName) throws IOException, InterruptedException
IOException
InterruptedException
private void flush(ServerName sn, HRegionInfo hri) throws IOException
IOException
public void compact(TableName tableName) throws IOException
compact
in interface Admin
tableName
- table to compactIOException
- if a remote or network exception occurspublic void compactRegion(byte[] regionName) throws IOException
compactRegion
in interface Admin
regionName
- region to compactIOException
- if a remote or network exception occurs@Deprecated public void compact(String tableNameOrRegionName) throws IOException
IOException
@Deprecated public void compact(byte[] tableNameOrRegionName) throws IOException
IOException
public void compact(TableName tableName, byte[] columnFamily) throws IOException
compact
in interface Admin
tableName
- table to compactcolumnFamily
- column family within a tableIOException
- if a remote or network exception occurspublic void compactRegion(byte[] regionName, byte[] columnFamily) throws IOException
compactRegion
in interface Admin
regionName
- region to compactcolumnFamily
- column family within a regionIOException
- if a remote or network exception occurs@Deprecated public void compact(String tableOrRegionName, String columnFamily) throws IOException
IOException
@Deprecated public void compact(byte[] tableNameOrRegionName, byte[] columnFamily) throws IOException
IOException
public void compactRegionServer(ServerName sn, boolean major) throws IOException, InterruptedException
compactRegionServer
in interface Admin
sn
- the region server namemajor
- if it's major compactionIOException
InterruptedException
public void majorCompact(TableName tableName) throws IOException
majorCompact
in interface Admin
tableName
- table to major compactIOException
- if a remote or network exception occurspublic void majorCompactRegion(byte[] regionName) throws IOException
majorCompactRegion
in interface Admin
regionName
- region to major compactIOException
- if a remote or network exception occurs@Deprecated public void majorCompact(String tableNameOrRegionName) throws IOException
majorCompact(org.apache.hadoop.hbase.TableName)
or majorCompactRegion(byte[])
instead.IOException
@Deprecated public void majorCompact(byte[] tableNameOrRegionName) throws IOException
majorCompact(org.apache.hadoop.hbase.TableName)
or majorCompactRegion(byte[])
instead.IOException
public void majorCompact(TableName tableName, byte[] columnFamily) throws IOException
majorCompact
in interface Admin
tableName
- table to major compactcolumnFamily
- column family within a tableIOException
- if a remote or network exception occurspublic void majorCompactRegion(byte[] regionName, byte[] columnFamily) throws IOException
majorCompactRegion
in interface Admin
regionName
- egion to major compactcolumnFamily
- column family within a regionIOException
- if a remote or network exception occurs@Deprecated public void majorCompact(String tableNameOrRegionName, String columnFamily) throws IOException
majorCompact(org.apache.hadoop.hbase.TableName,
byte[])
or majorCompactRegion(byte[], byte[])
instead.IOException
@Deprecated public void majorCompact(byte[] tableNameOrRegionName, byte[] columnFamily) throws IOException
majorCompact(org.apache.hadoop.hbase.TableName,
byte[])
or majorCompactRegion(byte[], byte[])
instead.IOException
private void compact(TableName tableName, byte[] columnFamily, boolean major) throws IOException
tableName
- table or region to compactcolumnFamily
- column family within a table or regionmajor
- True if we are to do a major compaction.IOException
- if a remote or network exception occursInterruptedException
private void compactRegion(byte[] regionName, byte[] columnFamily, boolean major) throws IOException
regionName
- region to compactcolumnFamily
- column family within a table or regionmajor
- True if we are to do a major compaction.IOException
- if a remote or network exception occursInterruptedException
private void compact(ServerName sn, HRegionInfo hri, boolean major, byte[] family) throws IOException
IOException
public void move(byte[] encodedRegionName, byte[] destServerName) throws IOException
r
to dest
.move
in interface Admin
encodedRegionName
- The encoded region name; i.e. the hash that makes
up the region name suffix: e.g. if regionname is
TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.
,
then the encoded region name is: 527db22f95c8a9e0116f0cc13c680396
.destServerName
- The servername of the destination regionserver. If
passed the empty byte array we'll assign to a random server. A server name
is made of host, port and startcode. Here is an example:
host187.example.com,60020,1289493121758
UnknownRegionException
- Thrown if we can't find a region named
encodedRegionName
IOException
- if we can't find a region named
encodedRegionName
private boolean isMetaRegion(byte[] regionName)
public void assign(byte[] regionName) throws MasterNotRunningException, ZooKeeperConnectionException, IOException
assign
in interface Admin
regionName
- Region name to assign.MasterNotRunningException
ZooKeeperConnectionException
IOException
public void unassign(byte[] regionName, boolean force) throws MasterNotRunningException, ZooKeeperConnectionException, IOException
move(byte[], byte[])
if you want
to control the region movement.unassign
in interface Admin
regionName
- Region to unassign. Will clear any existing RegionPlan
if one found.force
- If true, force unassign (Will remove region from
regions-in-transition too if present. If results in double assignment
use hbck -fix to resolve. To be used by experts).MasterNotRunningException
ZooKeeperConnectionException
IOException
public void offline(byte[] regionName) throws IOException
offline
in interface Admin
regionName
- Region to offline.IOException
public boolean setBalancerRunning(boolean on, boolean synchronous) throws IOException
setBalancerRunning
in interface Admin
on
- If true, enable balancer. If false, disable balancer.synchronous
- If true, it waits until current balance() call, if outstanding, to return.IOException
public boolean balancer() throws IOException
balancer
in interface Admin
IOException
public boolean isBalancerEnabled() throws IOException
isBalancerEnabled
in interface Admin
IOException
public boolean normalize() throws IOException
normalize
in interface Admin
IOException
public boolean isNormalizerEnabled() throws IOException
isNormalizerEnabled
in interface Admin
IOException
public boolean setNormalizerRunning(boolean on) throws IOException
setNormalizerRunning
in interface Admin
IOException
public boolean enableCatalogJanitor(boolean enable) throws IOException
enableCatalogJanitor
in interface Admin
enable
- if true enables the catalog janitorMasterNotRunningException
IOException
public int runCatalogScan() throws IOException
runCatalogScan
in interface Admin
MasterNotRunningException
IOException
public boolean isCatalogJanitorEnabled() throws IOException
isCatalogJanitorEnabled
in interface Admin
MasterNotRunningException
IOException
private boolean isEncodedRegionName(byte[] regionName) throws IOException
IOException
public void mergeRegions(byte[] nameOfRegionA, byte[] nameOfRegionB, boolean forcible) throws IOException
mergeRegions
in interface Admin
nameOfRegionA
- encoded or full name of region anameOfRegionB
- encoded or full name of region bforcible
- true if do a compulsory merge, otherwise we will only merge
two adjacent regionsIOException
public void split(TableName tableName) throws IOException
split
in interface Admin
tableName
- table to splitIOException
- if a remote or network exception occurspublic void splitRegion(byte[] regionName) throws IOException
splitRegion
in interface Admin
regionName
- region to splitIOException
- if a remote or network exception occurs@Deprecated public void split(String tableNameOrRegionName) throws IOException, InterruptedException
IOException
InterruptedException
@Deprecated public void split(byte[] tableNameOrRegionName) throws IOException, InterruptedException
IOException
InterruptedException
public void split(TableName tableName, byte[] splitPoint) throws IOException
split
in interface Admin
tableName
- table to splitsplitPoint
- the explicit position to split onIOException
- if a remote or network exception occurspublic void splitRegion(byte[] regionName, byte[] splitPoint) throws IOException
splitRegion
in interface Admin
regionName
- region to splitsplitPoint
- the explicit position to split onIOException
- if a remote or network exception occurs@Deprecated public void split(String tableNameOrRegionName, String splitPoint) throws IOException
split(org.apache.hadoop.hbase.TableName,
byte[])
or splitRegion(byte[], byte[])
instead.IOException
@Deprecated public void split(byte[] tableNameOrRegionName, byte[] splitPoint) throws IOException
split(org.apache.hadoop.hbase.TableName,
byte[])
or splitRegion(byte[], byte[])
instead.IOException
public void split(ServerName sn, HRegionInfo hri, byte[] splitPoint) throws IOException
IOException
public void modifyTable(TableName tableName, HTableDescriptor htd) throws IOException
modifyTable
in interface Admin
tableName
- name of table.htd
- modified description of the tableIOException
- if a remote or network exception occurspublic void modifyTable(byte[] tableName, HTableDescriptor htd) throws IOException
IOException
public void modifyTable(String tableName, HTableDescriptor htd) throws IOException
IOException
Pair<HRegionInfo,ServerName> getRegion(byte[] regionName) throws IOException
regionName
- Name of a region.regionName
is
a verified region name (we call MetaTableAccessor#getRegion(HConnection, byte[])
else null.
Throw IllegalArgumentException if regionName
is null.IOException
private byte[] getRegionName(byte[] regionNameOrEncodedRegionName) throws IOException
IOException
private TableName checkTableExists(TableName tableName) throws IOException
tableName
- Name of a table.IOException
- if a remote or network exception occurs.TableNotFoundException
- if table does not exist.public void shutdown() throws IOException
shutdown
in interface Admin
IOException
- if a remote or network exception occurspublic void stopMaster() throws IOException
stopMaster
in interface Admin
IOException
- if a remote or network exception occursshutdown()
public void stopRegionServer(String hostnamePort) throws IOException
stopRegionServer
in interface Admin
hostnamePort
- Hostname and port delimited by a :
as in
example.org:1234
IOException
- if a remote or network exception occurspublic ClusterStatus getClusterStatus() throws IOException
getClusterStatus
in interface Admin
IOException
- if a remote or network exception occurspublic org.apache.hadoop.conf.Configuration getConfiguration()
getConfiguration
in interface Admin
public void createNamespace(NamespaceDescriptor descriptor) throws IOException
createNamespace
in interface Admin
descriptor
- descriptor which describes the new namespaceIOException
public void modifyNamespace(NamespaceDescriptor descriptor) throws IOException
modifyNamespace
in interface Admin
descriptor
- descriptor which describes the new namespaceIOException
public void deleteNamespace(String name) throws IOException
deleteNamespace
in interface Admin
name
- namespace nameIOException
public NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException
getNamespaceDescriptor
in interface Admin
name
- name of namespace descriptorIOException
public NamespaceDescriptor[] listNamespaceDescriptors() throws IOException
listNamespaceDescriptors
in interface Admin
IOException
public ProcedureInfo[] listProcedures() throws IOException
listProcedures
in interface Admin
IOException
public HTableDescriptor[] listTableDescriptorsByNamespace(String name) throws IOException
listTableDescriptorsByNamespace
in interface Admin
name
- namespace nameIOException
public TableName[] listTableNamesByNamespace(String name) throws IOException
listTableNamesByNamespace
in interface Admin
name
- namespace nameIOException
public static void checkHBaseAvailable(org.apache.hadoop.conf.Configuration conf) throws MasterNotRunningException, ZooKeeperConnectionException, com.google.protobuf.ServiceException, IOException
conf
- system configurationMasterNotRunningException
- if the master is not runningZooKeeperConnectionException
- if unable to connect to zookeepercom.google.protobuf.ServiceException
IOException
public List<HRegionInfo> getTableRegions(TableName tableName) throws IOException
getTableRegions
in interface Admin
tableName
- the name of the tableHRegionInfo
.IOException
public List<HRegionInfo> getTableRegions(byte[] tableName) throws IOException
IOException
public void close() throws IOException
close
in interface Closeable
close
in interface AutoCloseable
close
in interface Admin
IOException
public HTableDescriptor[] getTableDescriptorsByTableName(List<TableName> tableNames) throws IOException
getTableDescriptorsByTableName
in interface Admin
tableNames
- List of table namesIOException
- if a remote or network exception occursprivate HTableDescriptor getTableDescriptorByTableName(TableName tableName) throws IOException
tableName
- one table nameIOException
- if a remote or network exception occurspublic HTableDescriptor[] getTableDescriptors(List<String> names) throws IOException
getTableDescriptors
in interface Admin
names
- List of table namesIOException
- if a remote or network exception occursprivate org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse rollWALWriterImpl(ServerName sn) throws IOException, FailedLogCloseException
IOException
FailedLogCloseException
@Deprecated public byte[][] rollHLogWriter(String serverName) throws IOException, FailedLogCloseException
rollWALWriter(ServerName)
serverName
- The servername of the regionserver. A server name is made of host,
port and startcode. This is mandatory. Here is an example:
host187.example.com,60020,1289493121758
HRegionInfo.getEncodedName()
that would allow the wal to
clean up some underlying files. null if there's nothing to flush.IOException
- if a remote or network exception occursFailedLogCloseException
public void rollWALWriter(ServerName serverName) throws IOException, FailedLogCloseException
Admin
rollWALWriter
in interface Admin
serverName
- The servername of the regionserver.IOException
- if a remote or network exception occursFailedLogCloseException
public String[] getMasterCoprocessors()
Admin
getMasterCoprocessors
in interface Admin
ClusterStatus.getMasterCoprocessors()
public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState getCompactionState(TableName tableName) throws IOException
getCompactionState
in interface Admin
tableName
- table to examineIOException
- if a remote or network exception occurspublic org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState getCompactionStateForRegion(byte[] regionName) throws IOException
getCompactionStateForRegion
in interface Admin
regionName
- region to examineIOException
- if a remote or network exception occurs@Deprecated public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState getCompactionState(String tableNameOrRegionName) throws IOException, InterruptedException
getCompactionState(org.apache.hadoop.hbase.TableName)
or getCompactionStateForRegion(byte[])
instead.IOException
InterruptedException
@Deprecated public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState getCompactionState(byte[] tableNameOrRegionName) throws IOException, InterruptedException
getCompactionState(org.apache.hadoop.hbase.TableName)
or getCompactionStateForRegion(byte[])
instead.IOException
InterruptedException
public void snapshot(String snapshotName, TableName tableName) throws IOException, SnapshotCreationException, IllegalArgumentException
Snapshots are considered unique based on the name of the snapshot. Attempts to take a
snapshot with the same name (even a different type or with different parameters) will fail with
a SnapshotCreationException
indicating the duplicate naming.
Snapshot names follow the same naming constraints as tables in HBase. See
TableName.isLegalFullyQualifiedTableName(byte[])
.
snapshot
in interface Admin
snapshotName
- name of the snapshot to be createdtableName
- name of the table for which snapshot is createdIOException
- if a remote or network exception occursSnapshotCreationException
- if snapshot creation failedIllegalArgumentException
- if the snapshot request is formatted incorrectlypublic void snapshot(String snapshotName, String tableName) throws IOException, SnapshotCreationException, IllegalArgumentException
public void snapshot(byte[] snapshotName, byte[] tableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type flushType) throws IOException, SnapshotCreationException, IllegalArgumentException
Snapshots are considered unique based on the name of the snapshot. Attempts to take a
snapshot with the same name (even a different type or with different parameters) will fail with
a SnapshotCreationException
indicating the duplicate naming.
Snapshot names follow the same naming constraints as tables in HBase.
snapshotName
- name of the snapshot to be createdtableName
- name of the table for which snapshot is createdflushType
- if the snapshot should be taken without flush memstore firstIOException
- if a remote or network exception occursSnapshotCreationException
- if snapshot creation failedIllegalArgumentException
- if the snapshot request is formatted incorrectlypublic void snapshot(byte[] snapshotName, TableName tableName) throws IOException, SnapshotCreationException, IllegalArgumentException
Snapshots are considered unique based on the name of the snapshot. Attempts to take a
snapshot with the same name (even a different type or with different parameters) will fail with
a SnapshotCreationException
indicating the duplicate naming.
Snapshot names follow the same naming constraints as tables in HBase.
snapshot
in interface Admin
snapshotName
- name of the snapshot to be createdtableName
- name of the table for which snapshot is createdIOException
- if a remote or network exception occursSnapshotCreationException
- if snapshot creation failedIllegalArgumentException
- if the snapshot request is formatted incorrectlypublic void snapshot(byte[] snapshotName, byte[] tableName) throws IOException, SnapshotCreationException, IllegalArgumentException
public void snapshot(String snapshotName, TableName tableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type type) throws IOException, SnapshotCreationException, IllegalArgumentException
Snapshots are considered unique based on the name of the snapshot. Attempts to take a
snapshot with the same name (even a different type or with different parameters) will fail with
a SnapshotCreationException
indicating the duplicate naming.
Snapshot names follow the same naming constraints as tables in HBase. See
TableName.isLegalFullyQualifiedTableName(byte[])
.
snapshot
in interface Admin
snapshotName
- name to give the snapshot on the filesystem. Must be unique from all other
snapshots stored on the clustertableName
- name of the table to snapshottype
- type of snapshot to takeIOException
- we fail to reach the masterSnapshotCreationException
- if snapshot creation failedIllegalArgumentException
- if the snapshot request is formatted incorrectlypublic void snapshot(String snapshotName, String tableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type type) throws IOException, SnapshotCreationException, IllegalArgumentException
public void snapshot(String snapshotName, byte[] tableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type type) throws IOException, SnapshotCreationException, IllegalArgumentException
public void snapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot) throws IOException, SnapshotCreationException, IllegalArgumentException
Only a single snapshot should be taken at a time for an instance of HBase, or results may be undefined (you can tell multiple HBase clusters to snapshot at the same time, but only one at a time for a single cluster).
Snapshots are considered unique based on the name of the snapshot. Attempts to take a
snapshot with the same name (even a different type or with different parameters) will fail with
a SnapshotCreationException
indicating the duplicate naming.
Snapshot names follow the same naming constraints as tables in HBase. See
TableName.isLegalFullyQualifiedTableName(byte[])
.
You should probably use snapshot(String, String)
or snapshot(byte[], byte[])
unless you are sure about the type of snapshot that you want to take.
snapshot
in interface Admin
snapshot
- snapshot to takeIOException
- or we lose contact with the master.SnapshotCreationException
- if snapshot failed to be takenIllegalArgumentException
- if the snapshot request is formatted incorrectlypublic org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse takeSnapshotAsync(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot) throws IOException, SnapshotCreationException
Only a single snapshot should be taken at a time, or results may be undefined.
takeSnapshotAsync
in interface Admin
snapshot
- snapshot to takeIOException
- if the snapshot did not succeed or we lose contact with the master.SnapshotCreationException
- if snapshot creation failedIllegalArgumentException
- if the snapshot request is formatted incorrectlypublic boolean isSnapshotFinished(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot) throws IOException, HBaseSnapshotException, UnknownSnapshotException
There are three possible states:
The cluster only knows about the most recent snapshot. Therefore, if another snapshot has been
run/started since the snapshot your are checking, you will recieve an
UnknownSnapshotException
.
isSnapshotFinished
in interface Admin
snapshot
- description of the snapshot to checkIOException
- if we have a network issueHBaseSnapshotException
- if the snapshot failedUnknownSnapshotException
- if the requested snapshot is unknownpublic void restoreSnapshot(byte[] snapshotName) throws IOException, RestoreSnapshotException
restoreSnapshot
in interface Admin
snapshotName
- name of the snapshot to restoreIOException
- if a remote or network exception occursRestoreSnapshotException
- if snapshot failed to be restoredIllegalArgumentException
- if the restore request is formatted incorrectlypublic void restoreSnapshot(String snapshotName) throws IOException, RestoreSnapshotException
restoreSnapshot
in interface Admin
snapshotName
- name of the snapshot to restoreIOException
- if a remote or network exception occursRestoreSnapshotException
- if snapshot failed to be restoredIllegalArgumentException
- if the restore request is formatted incorrectlypublic void restoreSnapshot(byte[] snapshotName, boolean takeFailSafeSnapshot) throws IOException, RestoreSnapshotException
restoreSnapshot
in interface Admin
snapshotName
- name of the snapshot to restoretakeFailSafeSnapshot
- true if the failsafe snapshot should be takenIOException
- if a remote or network exception occursRestoreSnapshotException
- if snapshot failed to be restoredIllegalArgumentException
- if the restore request is formatted incorrectlypublic void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot) throws IOException, RestoreSnapshotException
restoreSnapshot
in interface Admin
snapshotName
- name of the snapshot to restoretakeFailSafeSnapshot
- true if the failsafe snapshot should be takenIOException
- if a remote or network exception occursRestoreSnapshotException
- if snapshot failed to be restoredIllegalArgumentException
- if the restore request is formatted incorrectlypublic void cloneSnapshot(byte[] snapshotName, byte[] tableName) throws IOException, TableExistsException, RestoreSnapshotException
snapshotName
- name of the snapshot to be clonedtableName
- name of the table where the snapshot will be restoredIOException
- if a remote or network exception occursTableExistsException
- if table to be created already existsRestoreSnapshotException
- if snapshot failed to be clonedIllegalArgumentException
- if the specified table has not a valid namepublic void cloneSnapshot(byte[] snapshotName, TableName tableName) throws IOException, TableExistsException, RestoreSnapshotException
cloneSnapshot
in interface Admin
snapshotName
- name of the snapshot to be clonedtableName
- name of the table where the snapshot will be restoredIOException
- if a remote or network exception occursTableExistsException
- if table to be created already existsRestoreSnapshotException
- if snapshot failed to be clonedIllegalArgumentException
- if the specified table has not a valid namepublic void cloneSnapshot(String snapshotName, String tableName) throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException
snapshotName
- name of the snapshot to be clonedtableName
- name of the table where the snapshot will be restoredIOException
- if a remote or network exception occursTableExistsException
- if table to be created already existsRestoreSnapshotException
- if snapshot failed to be clonedIllegalArgumentException
- if the specified table has not a valid nameInterruptedException
public void cloneSnapshot(String snapshotName, TableName tableName) throws IOException, TableExistsException, RestoreSnapshotException
cloneSnapshot
in interface Admin
snapshotName
- name of the snapshot to be clonedtableName
- name of the table where the snapshot will be restoredIOException
- if a remote or network exception occursTableExistsException
- if table to be created already existsRestoreSnapshotException
- if snapshot failed to be clonedIllegalArgumentException
- if the specified table has not a valid namepublic byte[] execProcedureWithRet(String signature, String instance, Map<String,String> props) throws IOException
execProcedureWithRet
in interface Admin
signature
- A distributed procedure is uniquely identified
by its signature (default the root ZK node name of the procedure).instance
- The instance name of the procedure. For some procedures, this parameter is
optional.props
- Property/Value pairs of properties passing to the procedureIOException
public void execProcedure(String signature, String instance, Map<String,String> props) throws IOException
execProcedure
in interface Admin
signature
- A distributed procedure is uniquely identified
by its signature (default the root ZK node name of the procedure).instance
- The instance name of the procedure. For some procedures, this parameter is
optional.props
- Property/Value pairs of properties passing to the procedureIOException
public boolean isProcedureFinished(String signature, String instance, Map<String,String> props) throws IOException
There are three possible states:
isProcedureFinished
in interface Admin
signature
- The signature that uniquely identifies a procedureinstance
- The instance name of the procedureprops
- Property/Value pairs of properties passing to the procedureIOException
- if the specified procedure finished with errorprivate void internalRestoreSnapshot(String snapshotName, TableName tableName) throws IOException, RestoreSnapshotException
isTableAvailable(org.apache.hadoop.hbase.TableName)
-- it is not safe to
create an HTable instance to this table before it is available.snapshotName
- snapshot to restoretableName
- table name to restore the snapshot onIOException
- if a remote or network exception occursRestoreSnapshotException
- if snapshot failed to be restoredIllegalArgumentException
- if the restore request is formatted incorrectlyprivate org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotResponse internalRestoreSnapshotAsync(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot) throws IOException, RestoreSnapshotException
Only a single snapshot should be restored at a time, or results may be undefined.
snapshot
- snapshot to restoreIOException
- if a remote or network exception occursRestoreSnapshotException
- if snapshot failed to be restoredIllegalArgumentException
- if the restore request is formatted incorrectlypublic List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription> listSnapshots() throws IOException
listSnapshots
in interface Admin
IOException
- if a network error occurspublic List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription> listSnapshots(String regex) throws IOException
listSnapshots
in interface Admin
regex
- The regular expression to match againstIOException
- if a remote or network exception occurspublic List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription> listSnapshots(Pattern pattern) throws IOException
listSnapshots
in interface Admin
pattern
- The compiled regular expression to match againstIOException
- if a remote or network exception occurspublic void deleteSnapshot(byte[] snapshotName) throws IOException
deleteSnapshot
in interface Admin
snapshotName
- name of the snapshotIOException
- if a remote or network exception occurspublic void deleteSnapshot(String snapshotName) throws IOException
deleteSnapshot
in interface Admin
snapshotName
- name of the snapshotIOException
- if a remote or network exception occurspublic void deleteSnapshots(String regex) throws IOException
deleteSnapshots
in interface Admin
regex
- The regular expression to match againstIOException
- if a remote or network exception occurspublic void deleteSnapshots(Pattern pattern) throws IOException
deleteSnapshots
in interface Admin
pattern
- pattern for names of the snapshot to matchIOException
- if a remote or network exception occursprivate void internalDeleteSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription snapshot) throws IOException
IOException
public void setQuota(QuotaSettings quota) throws IOException
setQuota
in interface Admin
quota
- the quota settingsIOException
- if a remote or network exception occurspublic QuotaRetriever getQuotaRetriever(QuotaFilter filter) throws IOException
getQuotaRetriever
in interface Admin
filter
- the quota settings filterIOException
- if a remote or network exception occursprivate <V> V executeCallable(MasterCallable<V> callable) throws IOException
IOException
private static <V> V executeCallable(MasterCallable<V> callable, RpcRetryingCallerFactory rpcCallerFactory, int operationTimeout, int rpcTimeout) throws IOException
IOException
public CoprocessorRpcChannel coprocessorService()
RpcChannel
instance
connected to the active master.
The obtained RpcChannel
instance can be used to access a published
coprocessor Service
using standard protobuf service invocations:
CoprocessorRpcChannel channel = myAdmin.coprocessorService(); MyService.BlockingInterface service = MyService.newBlockingStub(channel); MyCallRequest request = MyCallRequest.newBuilder() ... .build(); MyCallResponse response = service.myCall(null, request);
coprocessorService
in interface Admin
public CoprocessorRpcChannel coprocessorService(ServerName sn)
RpcChannel
instance
connected to the passed region server.
The obtained RpcChannel
instance can be used to access a published
coprocessor Service
using standard protobuf service invocations:
CoprocessorRpcChannel channel = myAdmin.coprocessorService(serverName); MyService.BlockingInterface service = MyService.newBlockingStub(channel); MyCallRequest request = MyCallRequest.newBuilder() ... .build(); MyCallResponse response = service.myCall(null, request);
coprocessorService
in interface Admin
sn
- the server name to which the endpoint call is madepublic void updateConfiguration(ServerName server) throws IOException
Admin
updateConfiguration
in interface Admin
server
- : The server whose config needs to be updated.IOException
public void updateConfiguration() throws IOException
Admin
updateConfiguration
in interface Admin
IOException
public int getMasterInfoPort() throws IOException
Admin
getMasterInfoPort
in interface Admin
IOException
public long getLastMajorCompactionTimestamp(TableName tableName) throws IOException
Admin
getLastMajorCompactionTimestamp
in interface Admin
tableName
- table to examineIOException
- if a remote or network exception occurspublic long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException
Admin
getLastMajorCompactionTimestampForRegion
in interface Admin
regionName
- region to examineIOException
- if a remote or network exception occurspublic List<SecurityCapability> getSecurityCapabilities() throws IOException
Admin
getSecurityCapabilities
in interface Admin
IOException
private RpcControllerFactory getRpcControllerFactory()
Copyright © 2007–2019 The Apache Software Foundation. All rights reserved.