@InterfaceAudience.Public @InterfaceStability.Evolving public interface Admin extends Abortable, Closeable
Connection.getAdmin()
and
call close()
afterwards.
Admin can be used to create, drop, list, enable and disable tables, add and drop table column families and other administrative operations.
ConnectionFactory
,
Connection
,
Table
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(TableName tableName,
HColumnDescriptor column)
Add a column to an existing table.
|
void |
assign(byte[] regionName) |
boolean |
balancer()
Invoke the balancer.
|
boolean |
balancer(boolean force)
Invoke the balancer.
|
void |
cloneSnapshot(byte[] snapshotName,
TableName 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(TableName tableName)
Compact a table.
|
void |
compact(TableName tableName,
byte[] columnFamily)
Compact a column family within a table.
|
void |
compactRegion(byte[] regionName)
Compact an individual region.
|
void |
compactRegion(byte[] regionName,
byte[] columnFamily)
Compact a column family within a region.
|
void |
compactRegionServer(ServerName sn,
boolean major)
Compact all regions on the region server
|
CoprocessorRpcChannel |
coprocessorService()
Creates and returns a
com.google.protobuf.RpcChannel instance connected to the active
master. |
CoprocessorRpcChannel |
coprocessorService(ServerName sn)
Creates and returns a
com.google.protobuf.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.
|
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(TableName tableName)
Deletes a table.
|
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(TableName tableName)
Disable table and wait on completion.
|
void |
disableTableAsync(TableName tableName)
Starts the disable of a table.
|
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(TableName tableName)
Enable a table.
|
void |
enableTableAsync(TableName tableName)
Brings a table on-line (enables it).
|
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.
|
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() |
AdminProtos.GetRegionInfoResponse.CompactionState |
getCompactionState(TableName tableName)
Get the current compaction state of a table.
|
AdminProtos.GetRegionInfoResponse.CompactionState |
getCompactionStateForRegion(byte[] regionName)
Get the current compaction state of region.
|
Configuration |
getConfiguration() |
Connection |
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() |
QuotaRetriever |
getQuotaRetriever(QuotaFilter filter)
Return a QuotaRetriever to list the quotas based on the filter.
|
HTableDescriptor |
getTableDescriptor(TableName tableName)
Method for getting the tableDescriptor
|
HTableDescriptor[] |
getTableDescriptors(List<String> names)
Get tableDescriptors
|
HTableDescriptor[] |
getTableDescriptorsByTableName(List<TableName> tableNames)
Get tableDescriptors
|
List<HRegionInfo> |
getTableRegions(TableName tableName)
Get the regions of a given table.
|
boolean |
isAborted()
Check if the server or client was aborted.
|
boolean |
isBalancerEnabled()
Query the current state of the balancer
|
boolean |
isCatalogJanitorEnabled()
Query on the catalog janitor state (Enabled/Disabled?)
|
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(HBaseProtos.SnapshotDescription snapshot)
Check the current state of the passed snapshot.
|
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(TableName tableName) |
boolean |
isTableEnabled(TableName tableName) |
NamespaceDescriptor[] |
listNamespaceDescriptors()
List available namespace descriptors
|
ProcedureInfo[] |
listProcedures()
List procedures
|
List<HBaseProtos.SnapshotDescription> |
listSnapshots()
List completed snapshots.
|
List<HBaseProtos.SnapshotDescription> |
listSnapshots(Pattern pattern)
List all the completed snapshots matching the given pattern.
|
List<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(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[] encodedNameOfRegionA,
byte[] encodedNameOfRegionB,
boolean forcible)
Merge two regions.
|
void |
modifyColumn(TableName tableName,
HColumnDescriptor descriptor)
Modify an existing column family on a table.
|
void |
modifyNamespace(NamespaceDescriptor descriptor)
Modify an existing namespace
|
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.
|
void |
rollWALWriter(ServerName serverName)
Roll the log writer.
|
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,
TableName tableName)
public void snapshot(final String snapshotName, Create a timestamp consistent snapshot for the
given table.
|
void |
snapshot(HBaseProtos.SnapshotDescription snapshot)
Take a snapshot and wait for the server to complete that snapshot (blocking).
|
void |
snapshot(String snapshotName,
TableName tableName)
Take a snapshot for the given table.
|
void |
snapshot(String snapshotName,
TableName tableName,
HBaseProtos.SnapshotDescription.Type type)
Create typed snapshot of the table.
|
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(TableName tableName) |
MasterProtos.SnapshotResponse |
takeSnapshotAsync(HBaseProtos.SnapshotDescription snapshot)
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.
|
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
|
int getOperationTimeout()
void abort(String why, Throwable e)
Abortable
boolean isAborted()
Abortable
Connection getConnection()
boolean tableExists(TableName tableName) throws IOException
tableName
- Table to check.IOException
HTableDescriptor[] listTables() throws IOException
IOException
- if a remote or network exception occursHTableDescriptor[] listTables(Pattern pattern) throws IOException
pattern
- The compiled regular expression to match againstIOException
- if a remote or network exception occurslistTables()
HTableDescriptor[] listTables(String regex) throws IOException
regex
- The regular expression to match againstIOException
- if a remote or network exception occurslistTables(java.util.regex.Pattern)
HTableDescriptor[] listTables(Pattern pattern, boolean includeSysTables) throws IOException
pattern
- The compiled regular expression to match againstincludeSysTables
- False to match only against userspace tablesIOException
- if a remote or network exception occurslistTables()
HTableDescriptor[] listTables(String regex, boolean includeSysTables) throws IOException
regex
- The regular expression to match againstincludeSysTables
- False to match only against userspace tablesIOException
- if a remote or network exception occurslistTables(java.util.regex.Pattern, boolean)
TableName[] listTableNames() throws IOException
IOException
- if a remote or network exception occursTableName[] listTableNames(Pattern pattern) throws IOException
pattern
- The regular expression to match againstIOException
- if a remote or network exception occursTableName[] listTableNames(String regex) throws IOException
regex
- The regular expression to match againstIOException
- if a remote or network exception occursTableName[] listTableNames(Pattern pattern, boolean includeSysTables) throws IOException
pattern
- The regular expression to match againstincludeSysTables
- False to match only against userspace tablesIOException
- if a remote or network exception occursTableName[] listTableNames(String regex, boolean includeSysTables) throws IOException
regex
- The regular expression to match againstincludeSysTables
- False to match only against userspace tablesIOException
- if a remote or network exception occursHTableDescriptor getTableDescriptor(TableName tableName) throws TableNotFoundException, IOException
tableName
- as a TableName
TableNotFoundException
IOException
- if a remote or network exception occursvoid createTable(HTableDescriptor desc) throws IOException
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 occursvoid createTable(HTableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions) throws IOException
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
void createTable(HTableDescriptor desc, byte[][] splitKeys) throws IOException
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
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.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
void deleteTable(TableName tableName) throws IOException
tableName
- name of table to deleteIOException
- if a remote or network exception occursHTableDescriptor[] deleteTables(String regex) throws IOException
listTables(java.lang.String)
and deleteTable(org.apache.hadoop.hbase.TableName)
regex
- The regular expression to match table names againstIOException
deleteTables(java.util.regex.Pattern)
,
deleteTable(org.apache.hadoop.hbase.TableName)
HTableDescriptor[] deleteTables(Pattern pattern) throws IOException
listTables(java.util.regex.Pattern)
and
deleteTable(org.apache.hadoop.hbase.TableName)
pattern
- The pattern to match table names againstIOException
void truncateTable(TableName tableName, boolean preserveSplits) throws IOException
tableName
- name of table to truncatepreserveSplits
- True if the splits should be preservedIOException
- if a remote or network exception occursvoid enableTable(TableName tableName) throws IOException
enableTableAsync(org.apache.hadoop.hbase.TableName)
and isTableEnabled(org.apache.hadoop.hbase.TableName)
instead. The table has to be in
disabled state for it to be enabled.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(org.apache.hadoop.hbase.TableName)
,
disableTable(org.apache.hadoop.hbase.TableName)
,
enableTableAsync(org.apache.hadoop.hbase.TableName)
void enableTableAsync(TableName tableName) throws IOException
isTableEnabled(org.apache.hadoop.hbase.TableName)
to learn
when table is fully online. If table is taking too long to online, check server logs.tableName
- IOException
HTableDescriptor[] enableTables(String regex) throws IOException
listTables(java.lang.String)
and enableTable(org.apache.hadoop.hbase.TableName)
regex
- The regular expression to match table names againstIOException
enableTables(java.util.regex.Pattern)
,
enableTable(org.apache.hadoop.hbase.TableName)
HTableDescriptor[] enableTables(Pattern pattern) throws IOException
listTables(java.util.regex.Pattern)
and
enableTable(org.apache.hadoop.hbase.TableName)
pattern
- The pattern to match table names againstIOException
void disableTableAsync(TableName tableName) throws IOException
isTableDisabled(org.apache.hadoop.hbase.TableName)
to check for when disable completes. If
table is taking too long to online, check server logs.tableName
- name of tableIOException
- if a remote or network exception occursisTableDisabled(org.apache.hadoop.hbase.TableName)
,
isTableEnabled(org.apache.hadoop.hbase.TableName)
void disableTable(TableName tableName) throws IOException
disableTableAsync(org.apache.hadoop.hbase.TableName)
and
isTableDisabled(org.apache.hadoop.hbase.TableName)
instead. The table has to be in
enabled state for it to be disabled.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.HTableDescriptor[] disableTables(String regex) throws IOException
listTables(java.lang.String)
and disableTable(org.apache.hadoop.hbase.TableName)
regex
- The regular expression to match table names againstIOException
disableTables(java.util.regex.Pattern)
,
disableTable(org.apache.hadoop.hbase.TableName)
HTableDescriptor[] disableTables(Pattern pattern) throws IOException
listTables(java.util.regex.Pattern)
and
disableTable(org.apache.hadoop.hbase.TableName)
pattern
- The pattern to match table names againstIOException
boolean isTableEnabled(TableName tableName) throws IOException
tableName
- name of table to checkIOException
- if a remote or network exception occursboolean isTableDisabled(TableName tableName) throws IOException
tableName
- name of table to checkIOException
- if a remote or network exception occursboolean isTableAvailable(TableName tableName) throws IOException
tableName
- name of table to checkIOException
- if a remote or network exception occursboolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws IOException
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 occursPair<Integer,Integer> getAlterStatus(TableName tableName) throws IOException
tableName
- TableName instanceIOException
- if a remote or network exception occursPair<Integer,Integer> getAlterStatus(byte[] tableName) throws IOException
tableName
- name of the table to get the status ofIOException
- if a remote or network exception occursvoid addColumn(TableName 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 occursvoid deleteColumn(TableName tableName, byte[] columnName) throws IOException
tableName
- name of tablecolumnName
- name of column to be deletedIOException
- if a remote or network exception occursvoid modifyColumn(TableName tableName, HColumnDescriptor descriptor) throws IOException
tableName
- name of tabledescriptor
- new column descriptor to useIOException
- if a remote or network exception occursvoid closeRegion(String regionname, String serverName) throws IOException
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 occursvoid closeRegion(byte[] regionname, String serverName) throws IOException
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 occursboolean closeRegionWithEncodedRegionName(String encodedRegionName, String serverName) throws IOException
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 occursvoid closeRegion(ServerName sn, HRegionInfo hri) throws IOException
sn
- hri
- IOException
List<HRegionInfo> getOnlineRegions(ServerName sn) throws IOException
IOException
void flush(TableName tableName) throws IOException
tableName
- table to flushIOException
- if a remote or network exception occursvoid flushRegion(byte[] regionName) throws IOException
regionName
- region to flushIOException
- if a remote or network exception occursvoid compact(TableName tableName) throws IOException
tableName
- table to compactIOException
- if a remote or network exception occursvoid compactRegion(byte[] regionName) throws IOException
regionName
- region to compactIOException
- if a remote or network exception occursvoid compact(TableName tableName, byte[] columnFamily) throws IOException
tableName
- table to compactcolumnFamily
- column family within a tableIOException
- if a remote or network exception occursvoid compactRegion(byte[] regionName, byte[] columnFamily) throws IOException
regionName
- region to compactcolumnFamily
- column family within a regionIOException
- if a remote or network exception occursvoid majorCompact(TableName tableName) throws IOException
tableName
- table to major compactIOException
- if a remote or network exception occursvoid majorCompactRegion(byte[] regionName) throws IOException
regionName
- region to major compactIOException
- if a remote or network exception occursvoid majorCompact(TableName tableName, byte[] columnFamily) throws IOException
tableName
- table to major compactcolumnFamily
- column family within a tableIOException
- if a remote or network exception occursvoid majorCompactRegion(byte[] regionName, byte[] columnFamily) throws IOException
regionName
- egion to major compactcolumnFamily
- column family within a regionIOException
- if a remote or network exception occursvoid compactRegionServer(ServerName sn, boolean major) throws IOException, InterruptedException
sn
- the region server namemajor
- if it's major compactionIOException
InterruptedException
void move(byte[] encodedRegionName, byte[] destServerName) throws IOException
r
to dest
.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
void assign(byte[] regionName) throws IOException
regionName
- Region name to assign.IOException
void unassign(byte[] regionName, boolean force) throws IOException
move(byte[], byte[])
if you want to control the region movement.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).IOException
void offline(byte[] regionName) throws IOException
regionName
- Region to offline.IOException
boolean setBalancerRunning(boolean on, boolean synchronous) throws IOException
synchronous
- If true, it waits until current balance() call, if outstanding, to return.IOException
boolean balancer() throws IOException
IOException
boolean balancer(boolean force) throws IOException
force
- whether we should force balance even if there is region in transitionIOException
boolean isBalancerEnabled() throws IOException
IOException
boolean normalize() throws IOException
IOException
boolean isNormalizerEnabled() throws IOException
IOException
boolean setNormalizerRunning(boolean on) throws IOException
IOException
boolean enableCatalogJanitor(boolean enable) throws IOException
enable
- if true enables the catalog janitorIOException
int runCatalogScan() throws IOException
IOException
boolean isCatalogJanitorEnabled() throws IOException
IOException
void mergeRegions(byte[] encodedNameOfRegionA, byte[] encodedNameOfRegionB, boolean forcible) throws IOException
encodedNameOfRegionA
- encoded name of region aencodedNameOfRegionB
- encoded name of region bforcible
- true if do a compulsory merge, otherwise we will only merge two adjacent
regionsIOException
void split(TableName tableName) throws IOException
tableName
- table to splitIOException
- if a remote or network exception occursvoid splitRegion(byte[] regionName) throws IOException
regionName
- region to splitIOException
- if a remote or network exception occursvoid split(TableName tableName, byte[] splitPoint) throws IOException
tableName
- table to splitsplitPoint
- the explicit position to split onIOException
- if a remote or network exception occursvoid splitRegion(byte[] regionName, byte[] splitPoint) throws IOException
regionName
- region to splitsplitPoint
- the explicit position to split onIOException
- if a remote or network exception occursvoid modifyTable(TableName tableName, HTableDescriptor htd) throws IOException
tableName
- name of table.htd
- modified description of the tableIOException
- if a remote or network exception occursvoid shutdown() throws IOException
IOException
- if a remote or network exception occursvoid stopMaster() throws IOException
IOException
- if a remote or network exception occursshutdown()
void stopRegionServer(String hostnamePort) throws IOException
hostnamePort
- Hostname and port delimited by a :
as in
example.org:1234
IOException
- if a remote or network exception occursClusterStatus getClusterStatus() throws IOException
IOException
- if a remote or network exception occursConfiguration getConfiguration()
void createNamespace(NamespaceDescriptor descriptor) throws IOException
descriptor
- descriptor which describes the new namespaceIOException
void modifyNamespace(NamespaceDescriptor descriptor) throws IOException
descriptor
- descriptor which describes the new namespaceIOException
void deleteNamespace(String name) throws IOException
name
- namespace nameIOException
NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException
name
- name of namespace descriptorIOException
NamespaceDescriptor[] listNamespaceDescriptors() throws IOException
IOException
HTableDescriptor[] listTableDescriptorsByNamespace(String name) throws IOException
name
- namespace nameIOException
TableName[] listTableNamesByNamespace(String name) throws IOException
name
- namespace nameIOException
List<HRegionInfo> getTableRegions(TableName tableName) throws IOException
tableName
- the name of the tableHRegionInfo
.IOException
void close() throws IOException
close
in interface AutoCloseable
close
in interface Closeable
IOException
HTableDescriptor[] getTableDescriptorsByTableName(List<TableName> tableNames) throws IOException
tableNames
- List of table namesIOException
- if a remote or network exception occursHTableDescriptor[] getTableDescriptors(List<String> names) throws IOException
names
- List of table namesIOException
- if a remote or network exception occursboolean abortProcedure(long procId, boolean mayInterruptIfRunning) throws IOException
procId
- ID of the procedure to abortmayInterruptIfRunning
- if the proc completed at least one step, should it be aborted?IOException
ProcedureInfo[] listProcedures() throws IOException
IOException
Future<Boolean> abortProcedureAsync(long procId, boolean mayInterruptIfRunning) throws IOException
procId
- ID of the procedure to abortmayInterruptIfRunning
- if the proc completed at least one step, should it be aborted?IOException
void rollWALWriter(ServerName serverName) throws IOException, FailedLogCloseException
serverName
- The servername of the regionserver.IOException
- if a remote or network exception occursFailedLogCloseException
String[] getMasterCoprocessors() throws IOException
IOException
ClusterStatus.getMasterCoprocessors()
AdminProtos.GetRegionInfoResponse.CompactionState getCompactionState(TableName tableName) throws IOException
tableName
- table to examineIOException
- if a remote or network exception occursAdminProtos.GetRegionInfoResponse.CompactionState getCompactionStateForRegion(byte[] regionName) throws IOException
regionName
- region to examineIOException
- if a remote or network exception occurslong getLastMajorCompactionTimestamp(TableName tableName) throws IOException
tableName
- table to examineIOException
- if a remote or network exception occurslong getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException
regionName
- region to examineIOException
- if a remote or network exception occursvoid snapshot(String snapshotName, TableName tableName) throws IOException, SnapshotCreationException, IllegalArgumentException
SnapshotCreationException
indicating the duplicate naming.
Snapshot names follow the same naming constraints as tables in HBase. See TableName.isLegalFullyQualifiedTableName(byte[])
.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 incorrectlyvoid snapshot(byte[] snapshotName, TableName tableName) throws IOException, SnapshotCreationException, IllegalArgumentException
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 createdIOException
- if a remote or network exception occursSnapshotCreationException
- if snapshot creation failedIllegalArgumentException
- if the snapshot request is formatted incorrectlyvoid snapshot(String snapshotName, TableName tableName, HBaseProtos.SnapshotDescription.Type type) throws IOException, SnapshotCreationException, IllegalArgumentException
SnapshotCreationException
indicating the
duplicate naming. Snapshot names follow the same naming constraints as tables in HBase. See
TableName.isLegalFullyQualifiedTableName(byte[])
.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 incorrectlyvoid snapshot(HBaseProtos.SnapshotDescription snapshot) throws IOException, SnapshotCreationException, IllegalArgumentException
SnapshotCreationException
indicating the duplicate naming.
Snapshot names follow the same naming constraints as tables in HBase. See TableName.isLegalFullyQualifiedTableName(byte[])
. You should probably
use snapshot(String, org.apache.hadoop.hbase.TableName)
or
snapshot(byte[], org.apache.hadoop.hbase.TableName)
unless you are sure about the type
of snapshot that you want to take.snapshot
- snapshot to takeIOException
- or we lose contact with the master.SnapshotCreationException
- if snapshot failed to be takenIllegalArgumentException
- if the snapshot request is formatted incorrectlyMasterProtos.SnapshotResponse takeSnapshotAsync(HBaseProtos.SnapshotDescription snapshot) throws IOException, SnapshotCreationException
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 incorrectlyboolean isSnapshotFinished(HBaseProtos.SnapshotDescription snapshot) throws IOException, HBaseSnapshotException, UnknownSnapshotException
UnknownSnapshotException
.snapshot
- description of the snapshot to checkIOException
- if we have a network issueHBaseSnapshotException
- if the snapshot failedUnknownSnapshotException
- if the requested snapshot is
unknownvoid restoreSnapshot(byte[] snapshotName) throws IOException, RestoreSnapshotException
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 incorrectlyvoid restoreSnapshot(String snapshotName) throws IOException, RestoreSnapshotException
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 incorrectlyvoid restoreSnapshot(byte[] snapshotName, boolean takeFailSafeSnapshot) throws IOException, RestoreSnapshotException
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 incorrectlyvoid restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot) throws IOException, RestoreSnapshotException
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 incorrectlyvoid cloneSnapshot(byte[] snapshotName, TableName 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 namevoid cloneSnapshot(String snapshotName, TableName 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 namevoid execProcedure(String signature, String instance, Map<String,String> props) throws IOException
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
byte[] execProcedureWithRet(String signature, String instance, Map<String,String> props) throws IOException
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
boolean isProcedureFinished(String signature, String instance, Map<String,String> props) throws IOException
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 errorList<HBaseProtos.SnapshotDescription> listSnapshots() throws IOException
IOException
- if a network error occursList<HBaseProtos.SnapshotDescription> listSnapshots(String regex) throws IOException
regex
- The regular expression to match againstIOException
- if a remote or network exception occursList<HBaseProtos.SnapshotDescription> listSnapshots(Pattern pattern) throws IOException
pattern
- The compiled regular expression to match againstIOException
- if a remote or network exception occursvoid deleteSnapshot(byte[] snapshotName) throws IOException
snapshotName
- name of the snapshotIOException
- if a remote or network exception occursvoid deleteSnapshot(String snapshotName) throws IOException
snapshotName
- name of the snapshotIOException
- if a remote or network exception occursvoid deleteSnapshots(String regex) throws IOException
regex
- The regular expression to match againstIOException
- if a remote or network exception occursvoid deleteSnapshots(Pattern pattern) throws IOException
pattern
- pattern for names of the snapshot to matchIOException
- if a remote or network exception occursvoid setQuota(QuotaSettings quota) throws IOException
quota
- the quota settingsIOException
- if a remote or network exception occursQuotaRetriever getQuotaRetriever(QuotaFilter filter) throws IOException
filter
- the quota settings filterIOException
- if a remote or network exception occursCoprocessorRpcChannel coprocessorService()
com.google.protobuf.RpcChannel
instance connected to the active
master. The obtained com.google.protobuf.RpcChannel
instance can be used to access
a published coprocessor com.google.protobuf.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);
CoprocessorRpcChannel coprocessorService(ServerName sn)
com.google.protobuf.RpcChannel
instance
connected to the passed region server.
The obtained com.google.protobuf.RpcChannel
instance can be used to access a published
coprocessor com.google.protobuf.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);
sn
- the server name to which the endpoint call is madevoid updateConfiguration(ServerName server) throws IOException
server
- : The server whose config needs to be updated.IOException
void updateConfiguration() throws IOException
IOException
int getMasterInfoPort() throws IOException
IOException