diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java index 290ecb5502b..b06902ae9cc 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java @@ -487,7 +487,10 @@ public interface Admin extends Abortable, Closeable { * @return Pair indicating the number of regions updated Pair.getFirst() is the regions that are * yet to be updated Pair.getSecond() is the total number of regions of the table * @throws IOException if a remote or network exception occurs + * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #getAlterStatus(TableName)} + * instead. */ + @Deprecated Pair getAlterStatus(final byte[] tableName) throws IOException; /** @@ -1181,9 +1184,8 @@ public interface Admin extends Abortable, Closeable { throws IOException, SnapshotCreationException, IllegalArgumentException; /** - * public void snapshot(final String snapshotName, Create a timestamp consistent snapshot for the - * given table. final byte[] tableName) throws IOException, Snapshots are considered unique based - * on the name of the snapshot. Attempts to take a snapshot with the same name (even a + * Create a timestamp consistent snapshot for the given table. Snapshots are considered unique + * based on the name of the snapshot. Attempts to take a snapshot with the same name (even * different type or with different parameters) will fail with a {@link SnapshotCreationException} * indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in * HBase. diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java index 66079dd9ea0..7a50458e368 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java @@ -221,7 +221,7 @@ public class HBaseAdmin implements Admin { return operationTimeout; } - HBaseAdmin(ClusterConnection connection) { + HBaseAdmin(ClusterConnection connection) throws IOException { this.conf = connection.getConfiguration(); this.connection = connection; @@ -253,13 +253,6 @@ public class HBaseAdmin implements Admin { return this.aborted; } - /** - * Abort a procedure - * @param procId ID of the procedure to abort - * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted? - * @return true if aborted, false if procedure already completed or does not exist - * @throws IOException - */ @Override public boolean abortProcedure( final long procId, @@ -280,18 +273,6 @@ public class HBaseAdmin implements Admin { } } - /** - * Abort a procedure but does not block and wait for it be completely removed. - * You can use Future.get(long, TimeUnit) to wait on the operation to complete. - * It may throw ExecutionException if there was an error while executing the operation - * or TimeoutException in case the wait timeout was not long enough to allow the - * operation to complete. - * - * @param procId ID of the procedure to abort - * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted? - * @return true if aborted, false if procedure already completed or does not exist - * @throws IOException - */ @Override public Future abortProcedureAsync( final long procId, @@ -339,23 +320,6 @@ public class HBaseAdmin implements Admin { return connection; } - /** @return - true if the master server is running. Throws an exception - * otherwise. - * @throws ZooKeeperConnectionException - * @throws MasterNotRunningException - * @deprecated this has been deprecated without a replacement - */ - @Deprecated - public boolean isMasterRunning() - throws MasterNotRunningException, ZooKeeperConnectionException { - return connection.isMasterRunning(); - } - - /** - * @param tableName Table to check. - * @return True if table exists already. - * @throws IOException - */ @Override public boolean tableExists(final TableName tableName) throws IOException { return executeCallable(new ConnectionCallable(getConnection()) { @@ -366,16 +330,6 @@ public class HBaseAdmin implements Admin { }); } - public boolean tableExists(final byte[] tableName) - throws IOException { - return tableExists(TableName.valueOf(tableName)); - } - - public boolean tableExists(final String tableName) - throws IOException { - return tableExists(TableName.valueOf(tableName)); - } - @Override public HTableDescriptor[] listTables() throws IOException { return listTables((Pattern)null, false); @@ -410,51 +364,6 @@ public class HBaseAdmin implements Admin { return listTables(Pattern.compile(regex), includeSysTables); } - /** - * List all of the names of userspace tables. - * @return String[] table names - * @throws IOException if a remote or network exception occurs - * @deprecated Use {@link Admin#listTableNames()} instead - */ - @Deprecated - public String[] getTableNames() throws IOException { - TableName[] tableNames = listTableNames(); - String[] result = new String[tableNames.length]; - for (int i = 0; i < tableNames.length; i++) { - result[i] = tableNames[i].getNameAsString(); - } - return result; - } - - /** - * List all of the names of userspace tables matching the given regular expression. - * @param pattern The regular expression to match against - * @return String[] table names - * @throws IOException if a remote or network exception occurs - * @deprecated Use {@link Admin#listTableNames(Pattern)} instead. - */ - @Deprecated - public String[] getTableNames(Pattern pattern) throws IOException { - TableName[] tableNames = listTableNames(pattern); - String[] result = new String[tableNames.length]; - for (int i = 0; i < tableNames.length; i++) { - result[i] = tableNames[i].getNameAsString(); - } - return result; - } - - /** - * List all of the names of userspace tables matching the given regular expression. - * @param regex The regular expression to match against - * @return String[] table names - * @throws IOException if a remote or network exception occurs - * @deprecated Use {@link Admin#listTableNames(Pattern)} instead. - */ - @Deprecated - public String[] getTableNames(String regex) throws IOException { - return getTableNames(Pattern.compile(regex)); - } - @Override public TableName[] listTableNames() throws IOException { return listTableNames((Pattern)null, false); @@ -490,23 +399,13 @@ public class HBaseAdmin implements Admin { return listTableNames(Pattern.compile(regex), includeSysTables); } - /** - * Method for getting the tableDescriptor - * @param tableName as a byte [] - * @return the tableDescriptor - * @throws TableNotFoundException - * @throws IOException if a remote or network exception occurs - */ @Override - public HTableDescriptor getTableDescriptor(final TableName tableName) - throws TableNotFoundException, IOException { + public HTableDescriptor getTableDescriptor(final TableName tableName) throws IOException { return getTableDescriptor(tableName, getConnection(), rpcCallerFactory, operationTimeout); } - static HTableDescriptor getTableDescriptor(final TableName tableName, - HConnection connection, RpcRetryingCallerFactory rpcCallerFactory, - int operationTimeout) throws TableNotFoundException, IOException { - + static HTableDescriptor getTableDescriptor(final TableName tableName, HConnection connection, + RpcRetryingCallerFactory rpcCallerFactory, int operationTimeout) throws IOException { if (tableName == null) return null; HTableDescriptor htd = executeCallable(new MasterCallable(connection) { @Override @@ -528,11 +427,6 @@ public class HBaseAdmin implements Admin { throw new TableNotFoundException(tableName.getNameAsString()); } - public HTableDescriptor getTableDescriptor(final byte[] tableName) - throws TableNotFoundException, IOException { - return getTableDescriptor(TableName.valueOf(tableName)); - } - private long getPauseTime(int tries) { int triesCount = tries; if (triesCount >= HConstants.RETRY_BACKOFF.length) { @@ -541,49 +435,12 @@ public class HBaseAdmin implements Admin { return this.pause * HConstants.RETRY_BACKOFF[triesCount]; } - /** - * Creates a new table. - * Synchronous operation. - * - * @param desc table descriptor for table - * - * @throws IllegalArgumentException if the table name is reserved - * @throws MasterNotRunningException if master is not running - * @throws TableExistsException if table already exists (If concurrent - * threads, the table may have been created between test-for-existence - * and attempt-at-creation). - * @throws IOException if a remote or network exception occurs - */ @Override public void createTable(HTableDescriptor desc) throws IOException { createTable(desc, null); } - /** - * Creates a new table with the specified number of regions. The start key - * specified will become the end key of the first region of the table, and - * the end key specified will become the start key of the last region of the - * table (the first region has a null start key and the last region has a - * null end key). - * - * BigInteger math will be used to divide the key range specified into - * enough segments to make the required number of total regions. - * - * Synchronous operation. - * - * @param desc table descriptor for table - * @param startKey beginning of key range - * @param endKey end of key range - * @param numRegions the total number of regions to create - * - * @throws IllegalArgumentException if the table name is reserved - * @throws MasterNotRunningException if master is not running - * @throws org.apache.hadoop.hbase.TableExistsException if table already exists (If concurrent - * threads, the table may have been created between test-for-existence - * and attempt-at-creation). - * @throws IOException - */ @Override public void createTable(HTableDescriptor desc, byte [] startKey, byte [] endKey, int numRegions) @@ -604,23 +461,6 @@ public class HBaseAdmin implements Admin { createTable(desc, splitKeys); } - /** - * Creates a new table with an initial set of empty regions defined by the - * specified split keys. The total number of regions created will be the - * number of split keys plus one. Synchronous operation. - * Note : Avoid passing empty split key. - * - * @param desc table descriptor for table - * @param splitKeys array of split keys for the initial regions of the table - * - * @throws IllegalArgumentException if the table name is reserved, if the split keys - * are repeated and if the split key has empty byte array. - * @throws MasterNotRunningException if master is not running - * @throws org.apache.hadoop.hbase.TableExistsException if table already exists (If concurrent - * threads, the table may have been created between test-for-existence - * and attempt-at-creation). - * @throws IOException - */ @Override public void createTable(final HTableDescriptor desc, byte [][] splitKeys) throws IOException { @@ -642,21 +482,6 @@ public class HBaseAdmin implements Admin { } } - /** - * Creates a new table but does not block and wait for it to come online. - * You can use Future.get(long, TimeUnit) to wait on the operation to complete. - * It may throw ExecutionException if there was an error while executing the operation - * or TimeoutException in case the wait timeout was not long enough to allow the - * operation to complete. - * - * @param desc table descriptor for table - * @param splitKeys keys to check if the table has been created with all split keys - * @throws IllegalArgumentException Bad table name, if the split keys - * are repeated and if the split key has empty byte array. - * @throws IOException if a remote or network exception occurs - * @return the result of the async creation. You can use Future.get(long, TimeUnit) - * to wait on the operation to complete. - */ @Override public Future createTableAsync(final HTableDescriptor desc, final byte[][] splitKeys) throws IOException { @@ -723,21 +548,6 @@ public class HBaseAdmin implements Admin { } } - public void deleteTable(final String tableName) throws IOException { - deleteTable(TableName.valueOf(tableName)); - } - - public void deleteTable(final byte[] tableName) throws IOException { - deleteTable(TableName.valueOf(tableName)); - } - - /** - * Deletes a table. - * Synchronous operation. - * - * @param tableName name of table to delete - * @throws IOException if a remote or network exception occurs - */ @Override public void deleteTable(final TableName tableName) throws IOException { Future future = deleteTableAsync(tableName); @@ -756,18 +566,6 @@ public class HBaseAdmin implements Admin { } } - /** - * Deletes the table but does not block and wait for it be completely removed. - * You can use Future.get(long, TimeUnit) to wait on the operation to complete. - * It may throw ExecutionException if there was an error while executing the operation - * or TimeoutException in case the wait timeout was not long enough to allow the - * operation to complete. - * - * @param tableName name of table to delete - * @throws IOException if a remote or network exception occurs - * @return the result of the async delete. You can use Future.get(long, TimeUnit) - * to wait on the operation to complete. - */ @Override public Future deleteTableAsync(final TableName tableName) throws IOException { DeleteTableResponse response = executeCallable( @@ -810,19 +608,6 @@ public class HBaseAdmin implements Admin { } } - /** - * Deletes tables matching the passed in pattern and wait on completion. - * - * Warning: Use this method carefully, there is no prompting and the effect is - * immediate. Consider using {@link #listTables(java.lang.String)} and - * {@link #deleteTable(byte[])} - * - * @param regex The regular expression to match table names against - * @return Table descriptors for tables that couldn't be deleted - * @throws IOException - * @see #deleteTables(java.util.regex.Pattern) - * @see #deleteTable(java.lang.String) - */ @Override public HTableDescriptor[] deleteTables(String regex) throws IOException { return deleteTables(Pattern.compile(regex)); @@ -833,7 +618,7 @@ public class HBaseAdmin implements Admin { * * Warning: Use this method carefully, there is no prompting and the effect is * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and - * {@link #deleteTable(byte[])} + * {@link #deleteTable(TableName)} * * @param pattern The pattern to match table names against * @return Table descriptors for tables that couldn't be deleted @@ -853,12 +638,6 @@ public class HBaseAdmin implements Admin { return failed.toArray(new HTableDescriptor[failed.size()]); } - /** - * Truncate a table. Synchronous operation. - * @param tableName name of table to truncate - * @param preserveSplits True if the splits should be preserved - * @throws IOException if a remote or network exception occurs - */ @Override public void truncateTable(final TableName tableName, final boolean preserveSplits) throws IOException { @@ -879,18 +658,6 @@ public class HBaseAdmin implements Admin { } } - /** - * Truncate the table but does not block and wait for it be completely enabled. You can use - * Future.get(long, TimeUnit) to wait on the operation to complete. It may throw - * ExecutionException if there was an error while executing the operation or TimeoutException in - * case the wait timeout was not long enough to allow the operation to complete. Asynchronous - * operation. - * @param tableName name of table to delete - * @param preserveSplits true if the splits should be preserved - * @throws IOException if a remote or network exception occurs - * @return the result of the async truncate. You can use Future.get(long, TimeUnit) to wait on the - * operation to complete. - */ @Override public Future truncateTableAsync(final TableName tableName, final boolean preserveSplits) throws IOException { @@ -947,20 +714,6 @@ public class HBaseAdmin implements Admin { return splits; } - - /** - * Enable a table. May timeout. Use {@link #enableTableAsync(byte[])} - * and {@link #isTableEnabled(byte[])} instead. - * The table has to be in disabled state for it to be enabled. - * @param tableName name of the table - * @throws IOException if a remote or network exception occurs - * There could be couple types of IOException - * TableNotFoundException means the table doesn't exist. - * TableNotDisabledException means the table isn't in disabled state. - * @see #isTableEnabled(byte[]) - * @see #disableTable(byte[]) - * @see #enableTableAsync(byte[]) - */ @Override public void enableTable(final TableName tableName) throws IOException { @@ -980,16 +733,6 @@ public class HBaseAdmin implements Admin { } } - public void enableTable(final byte[] tableName) - throws IOException { - enableTable(TableName.valueOf(tableName)); - } - - public void enableTable(final String tableName) - throws IOException { - enableTable(TableName.valueOf(tableName)); - } - /** * Wait for the table to be enabled and available * If enabling the table exceeds the retry period, an exception is thrown. @@ -1031,28 +774,6 @@ public class HBaseAdmin implements Admin { } } - public void enableTableAsync(final byte[] tableName) - throws IOException { - enableTable(TableName.valueOf(tableName)); - } - - public void enableTableAsync(final String tableName) - throws IOException { - enableTableAsync(TableName.valueOf(tableName)); - } - - /** - * Enable the table but does not block and wait for it be completely enabled. - * You can use Future.get(long, TimeUnit) to wait on the operation to complete. - * It may throw ExecutionException if there was an error while executing the operation - * or TimeoutException in case the wait timeout was not long enough to allow the - * operation to complete. - * - * @param tableName name of table to delete - * @throws IOException if a remote or network exception occurs - * @return the result of the async enable. You can use Future.get(long, TimeUnit) - * to wait on the operation to complete. - */ @Override public Future enableTableAsync(final TableName tableName) throws IOException { TableName.isLegalFullyQualifiedTableName(tableName.getName()); @@ -1088,33 +809,11 @@ public class HBaseAdmin implements Admin { } } - /** - * Enable tables matching the passed in pattern and wait on completion. - * - * Warning: Use this method carefully, there is no prompting and the effect is - * immediate. Consider using {@link #listTables(java.lang.String)} and - * {@link #enableTable(byte[])} - * - * @param regex The regular expression to match table names against - * @throws IOException - * @see #enableTables(java.util.regex.Pattern) - * @see #enableTable(java.lang.String) - */ @Override public HTableDescriptor[] enableTables(String regex) throws IOException { return enableTables(Pattern.compile(regex)); } - /** - * Enable tables matching the passed in pattern and wait on completion. - * - * Warning: Use this method carefully, there is no prompting and the effect is - * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and - * {@link #enableTable(byte[])} - * - * @param pattern The pattern to match table names against - * @throws IOException - */ @Override public HTableDescriptor[] enableTables(Pattern pattern) throws IOException { List failed = new LinkedList(); @@ -1131,25 +830,6 @@ public class HBaseAdmin implements Admin { return failed.toArray(new HTableDescriptor[failed.size()]); } - public void disableTableAsync(final byte[] tableName) throws IOException { - disableTableAsync(TableName.valueOf(tableName)); - } - - public void disableTableAsync(final String tableName) throws IOException { - disableTableAsync(TableName.valueOf(tableName)); - } - - /** - * Disable table and wait on completion. May timeout eventually. Use - * {@link #disableTableAsync(byte[])} and {@link #isTableDisabled(String)} - * instead. - * The table has to be in enabled state for it to be disabled. - * @param tableName - * @throws IOException - * There could be couple types of IOException - * TableNotFoundException means the table doesn't exist. - * TableNotEnabledException means the table isn't in enabled state. - */ @Override public void disableTable(final TableName tableName) throws IOException { @@ -1169,28 +849,6 @@ public class HBaseAdmin implements Admin { } } - public void disableTable(final byte[] tableName) - throws IOException { - disableTable(TableName.valueOf(tableName)); - } - - public void disableTable(final String tableName) - throws IOException { - disableTable(TableName.valueOf(tableName)); - } - - /** - * Disable the table but does not block and wait for it be completely disabled. - * You can use Future.get(long, TimeUnit) to wait on the operation to complete. - * It may throw ExecutionException if there was an error while executing the operation - * or TimeoutException in case the wait timeout was not long enough to allow the - * operation to complete. - * - * @param tableName name of table to delete - * @throws IOException if a remote or network exception occurs - * @return the result of the async disable. You can use Future.get(long, TimeUnit) - * to wait on the operation to complete. - */ @Override public Future disableTableAsync(final TableName tableName) throws IOException { TableName.isLegalFullyQualifiedTableName(tableName.getName()); @@ -1220,42 +878,17 @@ public class HBaseAdmin implements Admin { } @Override - protected Void waitOperationResult(final long deadlineTs) - throws IOException, TimeoutException { + protected Void waitOperationResult(long deadlineTs) throws IOException, TimeoutException { waitForTableDisabled(deadlineTs); return null; } } - /** - * Disable tables matching the passed in pattern and wait on completion. - * - * Warning: Use this method carefully, there is no prompting and the effect is - * immediate. Consider using {@link #listTables(java.lang.String)} and - * {@link #disableTable(byte[])} - * - * @param regex The regular expression to match table names against - * @return Table descriptors for tables that couldn't be disabled - * @throws IOException - * @see #disableTables(java.util.regex.Pattern) - * @see #disableTable(java.lang.String) - */ @Override public HTableDescriptor[] disableTables(String regex) throws IOException { return disableTables(Pattern.compile(regex)); } - /** - * Disable tables matching the passed in pattern and wait on completion. - * - * Warning: Use this method carefully, there is no prompting and the effect is - * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and - * {@link #disableTable(byte[])} - * - * @param pattern The pattern to match table names against - * @return Table descriptors for tables that couldn't be disabled - * @throws IOException - */ @Override public HTableDescriptor[] disableTables(Pattern pattern) throws IOException { List failed = new LinkedList(); @@ -1272,24 +905,9 @@ public class HBaseAdmin implements Admin { return failed.toArray(new HTableDescriptor[failed.size()]); } - /* - * Checks whether table exists. If not, throws TableNotFoundException - * @param tableName - */ - private void checkTableExistence(TableName tableName) throws IOException { - if (!tableExists(tableName)) { - throw new TableNotFoundException(tableName); - } - } - - /** - * @param tableName name of table to check - * @return true if table is on-line - * @throws IOException if a remote or network exception occurs - */ @Override public boolean isTableEnabled(final TableName tableName) throws IOException { - checkTableExistence(tableName); + checkTableExists(tableName); return executeCallable(new ConnectionCallable(getConnection()) { @Override public Boolean call(int callTimeout) throws ServiceException, IOException { @@ -1301,173 +919,46 @@ public class HBaseAdmin implements Admin { }); } - public boolean isTableEnabled(byte[] tableName) throws IOException { - return isTableEnabled(TableName.valueOf(tableName)); - } - - public boolean isTableEnabled(String tableName) throws IOException { - return isTableEnabled(TableName.valueOf(tableName)); - } - - - - /** - * @param tableName name of table to check - * @return true if table is off-line - * @throws IOException if a remote or network exception occurs - */ @Override public boolean isTableDisabled(TableName tableName) throws IOException { - checkTableExistence(tableName); + checkTableExists(tableName); return connection.isTableDisabled(tableName); } - public boolean isTableDisabled(byte[] tableName) throws IOException { - return isTableDisabled(TableName.valueOf(tableName)); - } - - public boolean isTableDisabled(String tableName) throws IOException { - return isTableDisabled(TableName.valueOf(tableName)); - } - - /** - * @param tableName name of table to check - * @return true if all regions of the table are available - * @throws IOException if a remote or network exception occurs - */ @Override public boolean isTableAvailable(TableName tableName) throws IOException { return connection.isTableAvailable(tableName); } - public boolean isTableAvailable(byte[] tableName) throws IOException { - return isTableAvailable(TableName.valueOf(tableName)); - } - - public boolean isTableAvailable(String tableName) throws IOException { - return isTableAvailable(TableName.valueOf(tableName)); - } - - /** - * Use this api to check if the table has been created with the specified number of - * splitkeys which was used while creating the given table. - * Note : If this api is used after a table's region gets splitted, the api may return - * false. - * @param tableName - * name of table to check - * @param splitKeys - * keys to check if the table has been created with all split keys - * @throws IOException - * if a remote or network excpetion occurs - */ @Override - public boolean isTableAvailable(TableName tableName, - byte[][] splitKeys) throws IOException { + public boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws IOException { return connection.isTableAvailable(tableName, splitKeys); } - public boolean isTableAvailable(byte[] tableName, - byte[][] splitKeys) throws IOException { - return isTableAvailable(TableName.valueOf(tableName), splitKeys); - } - - public boolean isTableAvailable(String tableName, - byte[][] splitKeys) throws IOException { - return isTableAvailable(TableName.valueOf(tableName), splitKeys); - } - - /** - * Get the status of alter command - indicates how many regions have received - * the updated schema Asynchronous operation. - * - * @param tableName TableName instance - * @return Pair indicating the number of regions updated Pair.getFirst() is the - * regions that are yet to be updated Pair.getSecond() is the total number - * of regions of the table - * @throws IOException - * if a remote or network exception occurs - */ @Override - public Pair getAlterStatus(final TableName tableName) - throws IOException { + public Pair getAlterStatus(final TableName tableName) throws IOException { return executeCallable(new MasterCallable>(getConnection()) { @Override public Pair call(int callTimeout) throws ServiceException { GetSchemaAlterStatusRequest req = RequestConverter .buildGetSchemaAlterStatusRequest(tableName); GetSchemaAlterStatusResponse ret = master.getSchemaAlterStatus(null, req); - Pair pair = new Pair(Integer.valueOf(ret - .getYetToUpdateRegions()), Integer.valueOf(ret.getTotalRegions())); + Pair pair = new Pair<>(ret.getYetToUpdateRegions(), + ret.getTotalRegions()); return pair; } }); } - /** - * Get the status of alter command - indicates how many regions have received - * the updated schema Asynchronous operation. - * - * @param tableName - * name of the table to get the status of - * @return Pair indicating the number of regions updated Pair.getFirst() is the - * regions that are yet to be updated Pair.getSecond() is the total number - * of regions of the table - * @throws IOException - * if a remote or network exception occurs - */ @Override - public Pair getAlterStatus(final byte[] tableName) - throws IOException { + public Pair getAlterStatus(final byte[] tableName) throws IOException { return getAlterStatus(TableName.valueOf(tableName)); } /** - * Add a column family to an existing table. - * Asynchronous operation. - * - * @param tableName name of the table to add column family to - * @param columnFamily column family descriptor of column family to be added - * @throws IOException if a remote or network exception occurs - * @deprecated As of release 2.0.0. - * (HBASE-1989). - * This will be removed in HBase 3.0.0. - * Use {@link #addColumnFamily(TableName, HColumnDescriptor)}. - */ - @Deprecated - public void addColumn(final byte[] tableName, HColumnDescriptor columnFamily) - throws IOException { - addColumnFamily(TableName.valueOf(tableName), columnFamily); - } - - /** - * Add a column family to an existing table. - * Asynchronous operation. - * - * @param tableName name of the table to add column family to - * @param columnFamily column family descriptor of column family to be added - * @throws IOException if a remote or network exception occurs - * @deprecated As of release 2.0.0. - * (HBASE-1989). - * This will be removed in HBase 3.0.0. - * Use {@link #addColumnFamily(TableName, HColumnDescriptor)}. - */ - @Deprecated - public void addColumn(final String tableName, HColumnDescriptor columnFamily) - throws IOException { - addColumnFamily(TableName.valueOf(tableName), columnFamily); - } - - /** - * Add a column family to an existing table. - * Asynchronous operation. - * - * @param tableName name of the table to add column family to - * @param columnFamily column family descriptor of column family to be added - * @throws IOException if a remote or network exception occurs - * @deprecated As of release 2.0.0. - * (HBASE-1989). - * This will be removed in HBase 3.0.0. - * Use {@link #addColumnFamily(TableName, HColumnDescriptor)}. + * {@inheritDoc} + * @deprecated Since 2.0. Will be removed in 3.0. Use + * {@link #addColumnFamily(TableName, HColumnDescriptor)} instead. */ @Override @Deprecated @@ -1506,52 +997,9 @@ public class HBaseAdmin implements Admin { } /** - * Delete a column family from a table. - * Asynchronous operation. - * - * @param tableName name of table - * @param columnFamily name of column family to be deleted - * @throws IOException if a remote or network exception occurs - * @deprecated As of release 2.0.0. - * (HBASE-1989). - * This will be removed in HBase 3.0.0. - * Use {@link #deleteColumnFamily(TableName, byte[])}. - */ - @Deprecated - public void deleteColumn(final byte[] tableName, final String columnFamily) - throws IOException { - deleteColumnFamily(TableName.valueOf(tableName), Bytes.toBytes(columnFamily)); - } - - /** - * Delete a column family from a table. - * Asynchronous operation. - * - * @param tableName name of table - * @param columnFamily name of column family to be deleted - * @throws IOException if a remote or network exception occurs - * @deprecated As of release 2.0.0. - * (HBASE-1989). - * This will be removed in HBase 3.0.0. - * Use {@link #deleteColumnFamily(TableName, byte[])}. - */ - @Deprecated - public void deleteColumn(final String tableName, final String columnFamily) - throws IOException { - deleteColumnFamily(TableName.valueOf(tableName), Bytes.toBytes(columnFamily)); - } - - /** - * Delete a column family from a table. - * Asynchronous operation. - * - * @param tableName name of table - * @param columnFamily name of column family to be deleted - * @throws IOException if a remote or network exception occurs - * @deprecated As of release 2.0.0. - * (HBASE-1989). - * This will be removed in HBase 3.0.0. - * Use {@link #deleteColumnFamily(TableName, byte[])}. + * {@inheritDoc} + * @deprecated Since 2.0. Will be removed in 3.0. Use + * {@link #deleteColumnFamily(TableName, byte[])} instead. */ @Override @Deprecated @@ -1591,52 +1039,9 @@ public class HBaseAdmin implements Admin { } /** - * Modify an existing column family on a table. - * Asynchronous operation. - * - * @param tableName name of table - * @param columnFamily new column family descriptor to use - * @throws IOException if a remote or network exception occurs - * @deprecated As of release 2.0.0. - * (HBASE-1989). - * This will be removed in HBase 3.0.0. - * Use {@link #modifyColumnFamily(TableName, HColumnDescriptor)}. - */ - @Deprecated - public void modifyColumn(final String tableName, HColumnDescriptor columnFamily) - throws IOException { - modifyColumnFamily(TableName.valueOf(tableName), columnFamily); - } - - /** - * Modify an existing column family on a table. - * Asynchronous operation. - * - * @param tableName name of table - * @param columnFamily new column family descriptor to use - * @throws IOException if a remote or network exception occurs - * @deprecated As of release 2.0.0. - * (HBASE-1989). - * This will be removed in HBase 3.0.0. - * Use {@link #modifyColumnFamily(TableName, HColumnDescriptor)}. - */ - @Deprecated - public void modifyColumn(final byte[] tableName, HColumnDescriptor columnFamily) - throws IOException { - modifyColumnFamily(TableName.valueOf(tableName), columnFamily); - } - - /** - * Modify an existing column family on a table. - * Asynchronous operation. - * - * @param tableName name of table - * @param columnFamily new column family descriptor to use - * @throws IOException if a remote or network exception occurs - * @deprecated As of release 2.0.0. - * (HBASE-1989). - * This will be removed in HBase 3.0.0. - * Use {@link #modifyColumnFamily(TableName, HColumnDescriptor)}. + * {@inheritDoc} + * @deprecated As of 2.0. Will be removed in 3.0. Use + * {@link #modifyColumnFamily(TableName, HColumnDescriptor)} instead. */ @Override @Deprecated @@ -1675,33 +1080,13 @@ public class HBaseAdmin implements Admin { } } - /** - * Close a region. For expert-admins. Runs close on the regionserver. The - * master will not be informed of the close. - * @param regionname region name to close - * @param serverName If supplied, we'll use this location rather than - * the one currently in hbase:meta - * @throws IOException if a remote or network exception occurs - */ @Override - public void closeRegion(final String regionname, final String serverName) - throws IOException { + public void closeRegion(final String regionname, final String serverName) throws IOException { closeRegion(Bytes.toBytes(regionname), serverName); } - /** - * Close a region. For expert-admins Runs close on the regionserver. The - * master will not be informed of the close. - * @param regionname region name to close - * @param serverName 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 - * @throws IOException if a remote or network exception occurs - */ @Override - public void closeRegion(final byte [] regionname, final String serverName) - throws IOException { + public void closeRegion(final byte [] regionname, final String serverName) throws IOException { if (serverName != null) { Pair pair = MetaTableAccessor.getRegion(connection, regionname); if (pair == null || pair.getFirst() == null) { @@ -1721,27 +1106,6 @@ public class HBaseAdmin implements Admin { } } - /** - * For expert-admins. Runs close on the regionserver. Closes a region based on - * the encoded region name. The region server name is mandatory. If the - * servername is provided then based on the online regions in the specified - * regionserver the specified region will be closed. The master will not be - * informed of the close. Note that the regionname is the encoded regionname. - * - * @param 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. - * @param 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 - * @return true if the region was closed, false if not. - * @throws IOException - * if a remote or network exception occurs - */ @Override public boolean closeRegionWithEncodedRegionName(final String encodedRegionName, final String serverName) throws IOException { @@ -1766,33 +1130,19 @@ public class HBaseAdmin implements Admin { } } - /** - * Close a region. For expert-admins Runs close on the regionserver. The - * master will not be informed of the close. - * @param sn - * @param hri - * @throws IOException - */ @Override - public void closeRegion(final ServerName sn, final HRegionInfo hri) - throws IOException { + public void closeRegion(final ServerName sn, final HRegionInfo hri) throws IOException { AdminService.BlockingInterface admin = this.connection.getAdmin(sn); // Close the region without updating zk state. ProtobufUtil.closeRegion(admin, sn, hri.getRegionName()); } - /** - * Get all the online regions on a region server. - */ @Override public List getOnlineRegions(final ServerName sn) throws IOException { AdminService.BlockingInterface admin = this.connection.getAdmin(sn); return ProtobufUtil.getOnlineRegions(admin); } - /** - * {@inheritDoc} - */ @Override public void flush(final TableName tableName) throws IOException { checkTableExists(tableName); @@ -1804,9 +1154,6 @@ public class HBaseAdmin implements Admin { new HashMap()); } - /** - * {@inheritDoc} - */ @Override public void flushRegion(final byte[] regionName) throws IOException { Pair regionServerPair = getRegion(regionName); @@ -1816,7 +1163,16 @@ public class HBaseAdmin implements Admin { if (regionServerPair.getSecond() == null) { throw new NoServerForRegionException(Bytes.toStringBinary(regionName)); } - flush(regionServerPair.getSecond(), regionServerPair.getFirst()); + HRegionInfo hRegionInfo = regionServerPair.getFirst(); + ServerName serverName = regionServerPair.getSecond(); + AdminService.BlockingInterface admin = this.connection.getAdmin(serverName); + FlushRegionRequest request = + RequestConverter.buildFlushRegionRequest(hRegionInfo.getRegionName()); + try { + admin.flushRegion(null, request); + } catch (ServiceException se) { + throw ProtobufUtil.getRemoteException(se); + } } private void flush(final ServerName sn, final HRegionInfo hri) @@ -1840,9 +1196,6 @@ public class HBaseAdmin implements Admin { compact(tableName, null, false, CompactType.NORMAL); } - /** - * {@inheritDoc} - */ @Override public void compactRegion(final byte[] regionName) throws IOException { @@ -1878,18 +1231,12 @@ public class HBaseAdmin implements Admin { } } - /** - * {@inheritDoc} - */ @Override public void majorCompact(final TableName tableName) throws IOException { compact(tableName, null, true, CompactType.NORMAL); } - /** - * {@inheritDoc} - */ @Override public void majorCompactRegion(final byte[] regionName) throws IOException { @@ -1905,9 +1252,6 @@ public class HBaseAdmin implements Admin { compact(tableName, columnFamily, true, CompactType.NORMAL); } - /** - * {@inheritDoc} - */ @Override public void majorCompactRegion(final byte[] regionName, final byte[] columnFamily) throws IOException { @@ -2001,19 +1345,6 @@ public class HBaseAdmin implements Admin { } } - /** - * Move the region r to dest. - * @param 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. - * @param destServerName The servername of the destination regionserver. If - * passed the empty byte array we'll assign to a random server. A server name - * is made of host, port and startcode. Here is an example: - * host187.example.com,60020,1289493121758 - * @throws UnknownRegionException Thrown if we can't find a region named - * encodedRegionName - */ @Override public void move(final byte [] encodedRegionName, final byte [] destServerName) throws IOException { @@ -2034,13 +1365,6 @@ public class HBaseAdmin implements Admin { }); } - /** - * @param regionName - * Region name to assign. - * @throws MasterNotRunningException - * @throws ZooKeeperConnectionException - * @throws IOException - */ @Override public void assign(final byte[] regionName) throws MasterNotRunningException, ZooKeeperConnectionException, IOException { @@ -2056,20 +1380,6 @@ public class HBaseAdmin implements Admin { }); } - /** - * Unassign a region from current hosting regionserver. Region will then be - * assigned to a regionserver chosen at random. Region could be reassigned - * back to the same server. Use {@link #move(byte[], byte[])} if you want - * to control the region movement. - * @param regionName Region to unassign. Will clear any existing RegionPlan - * if one found. - * @param force If true, force unassign (Will remove region from - * regions-in-transition too if present. If results in double assignment - * use hbck -fix to resolve. To be used by experts). - * @throws MasterNotRunningException - * @throws ZooKeeperConnectionException - * @throws IOException - */ @Override public void unassign(final byte [] regionName, final boolean force) throws MasterNotRunningException, ZooKeeperConnectionException, IOException { @@ -2085,18 +1395,6 @@ public class HBaseAdmin implements Admin { }); } - /** - * Offline specified region from master's in-memory state. It will not attempt to reassign the - * region as in unassign. This API can be used when a region not served by any region server and - * still online as per Master's in memory state. If this API is incorrectly used on active region - * then master will loose track of that region. - * - * This is a special method that should be used by experts or hbck. - * - * @param regionName - * Region to offline. - * @throws IOException - */ @Override public void offline(final byte [] regionName) throws IOException { @@ -2109,12 +1407,6 @@ public class HBaseAdmin implements Admin { }); } - /** - * Turn the load balancer on or off. - * @param on If true, enable balancer. If false, disable balancer. - * @param synchronous If true, it waits until current balance() call, if outstanding, to return. - * @return Previous balancer value - */ @Override public boolean setBalancerRunning(final boolean on, final boolean synchronous) throws IOException { @@ -2128,12 +1420,6 @@ public class HBaseAdmin implements Admin { }); } - /** - * Invoke the balancer. Will run the balancer and if regions to move, it will - * go ahead and do the reassignments. Can NOT run for various reasons. Check - * logs. - * @return True if balancer ran, false otherwise. - */ @Override public boolean balancer() throws IOException { return executeCallable(new MasterCallable(getConnection()) { @@ -2154,12 +1440,6 @@ public class HBaseAdmin implements Admin { }); } - /** - * Query the state of the balancer from the Master. It's not a guarantee that the balancer is - * actually running this very moment, but that it will run. - * - * @return True if the balancer is enabled, false otherwise. - */ @Override public boolean isBalancerEnabled() throws IOException { return executeCallable(new MasterCallable(getConnection()) { @@ -2171,11 +1451,6 @@ public class HBaseAdmin implements Admin { }); } - /** - * Invoke region normalizer. Can NOT run for various reasons. Check logs. - * - * @return True if region normalizer ran, false otherwise. - */ @Override public boolean normalize() throws IOException { return executeCallable(new MasterCallable(getConnection()) { @@ -2187,11 +1462,7 @@ public class HBaseAdmin implements Admin { }); } - /** - * Query the current state of the region normalizer - * - * @return true if region normalizer is enabled, false otherwise. - */ + @Override public boolean isNormalizerEnabled() throws IOException { return executeCallable(new MasterCallable(getConnection()) { @Override @@ -2202,11 +1473,7 @@ public class HBaseAdmin implements Admin { }); } - /** - * Turn region normalizer on or off. - * - * @return Previous normalizer value - */ + @Override public boolean setNormalizerRunning(final boolean on) throws IOException { return executeCallable(new MasterCallable(getConnection()) { @Override @@ -2218,15 +1485,8 @@ public class HBaseAdmin implements Admin { }); } - /** - * Enable/Disable the catalog janitor - * @param enable if true enables the catalog janitor - * @return the previous state - * @throws MasterNotRunningException - */ @Override - public boolean enableCatalogJanitor(final boolean enable) - throws IOException { + public boolean enableCatalogJanitor(final boolean enable) throws IOException { return executeCallable(new MasterCallable(getConnection()) { @Override public Boolean call(int callTimeout) throws ServiceException { @@ -2236,11 +1496,6 @@ public class HBaseAdmin implements Admin { }); } - /** - * Ask for a scan of the catalog table - * @return the number of entries cleaned - * @throws MasterNotRunningException - */ @Override public int runCatalogScan() throws IOException { return executeCallable(new MasterCallable(getConnection()) { @@ -2252,10 +1507,6 @@ public class HBaseAdmin implements Admin { }); } - /** - * Query on the catalog janitor state (Enabled/Disabled?) - * @throws org.apache.hadoop.hbase.MasterNotRunningException - */ @Override public boolean isCatalogJanitorEnabled() throws IOException { return executeCallable(new MasterCallable(getConnection()) { @@ -2319,21 +1570,13 @@ public class HBaseAdmin implements Admin { }); } - /** - * {@inheritDoc} - */ @Override - public void split(final TableName tableName) - throws IOException { + public void split(final TableName tableName) throws IOException { split(tableName, null); } - /** - * {@inheritDoc} - */ @Override - public void splitRegion(final byte[] regionName) - throws IOException { + public void splitRegion(final byte[] regionName) throws IOException { splitRegion(regionName, null); } @@ -2341,8 +1584,7 @@ public class HBaseAdmin implements Admin { * {@inheritDoc} */ @Override - public void split(final TableName tableName, final byte [] splitPoint) - throws IOException { + public void split(final TableName tableName, final byte [] splitPoint) throws IOException { ZooKeeperWatcher zookeeper = null; try { checkTableExists(tableName); @@ -2373,12 +1615,8 @@ public class HBaseAdmin implements Admin { } } - /** - * {@inheritDoc} - */ @Override - public void splitRegion(final byte[] regionName, final byte [] splitPoint) - throws IOException { + public void splitRegion(final byte[] regionName, final byte [] splitPoint) throws IOException { Pair regionServerPair = getRegion(regionName); if (regionServerPair == null) { throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName)); @@ -2406,20 +1644,6 @@ public class HBaseAdmin implements Admin { ProtobufUtil.split(admin, hri, splitPoint); } - /** - * Modify an existing table, more IRB friendly version. Asynchronous operation. - * This means that it may be a while before your schema change is updated across all of the - * table. You can use Future.get(long, TimeUnit) to wait on the operation to complete. - * It may throw ExecutionException if there was an error while executing the operation - * or TimeoutException in case the wait timeout was not long enough to allow the - * operation to complete. - * - * @param tableName name of table. - * @param htd modified description of the table - * @throws IOException if a remote or network exception occurs - * @return the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the - * operation to complete. - */ @Override public Future modifyTable(final TableName tableName, final HTableDescriptor htd) throws IOException { @@ -2468,21 +1692,11 @@ public class HBaseAdmin implements Admin { } } - public void modifyTable(final byte[] tableName, final HTableDescriptor htd) - throws IOException { - modifyTable(TableName.valueOf(tableName), htd); - } - - public void modifyTable(final String tableName, final HTableDescriptor htd) - throws IOException { - modifyTable(TableName.valueOf(tableName), htd); - } - /** * @param regionName Name of a region. * @return a pair of HRegionInfo and ServerName if regionName is * a verified region name (we call {@link - * MetaTableAccessor#getRegion(HConnection, byte[])} + * MetaTableAccessor#getRegionLocation(HConnection, byte[])} * else null. * Throw IllegalArgumentException if regionName is null. * @throws IOException @@ -2573,10 +1787,6 @@ public class HBaseAdmin implements Admin { }); } - /** - * Shuts down the HBase cluster - * @throws IOException if a remote or network exception occurs - */ @Override public synchronized void shutdown() throws IOException { executeCallable(new MasterCallable(getConnection()) { @@ -2588,12 +1798,6 @@ public class HBaseAdmin implements Admin { }); } - /** - * Shuts down the current HBase master only. - * Does not shutdown the cluster. - * @see #shutdown() - * @throws IOException if a remote or network exception occurs - */ @Override public synchronized void stopMaster() throws IOException { executeCallable(new MasterCallable(getConnection()) { @@ -2605,12 +1809,6 @@ public class HBaseAdmin implements Admin { }); } - /** - * Stop the designated regionserver - * @param hostnamePort Hostname and port delimited by a : as in - * example.org:1234 - * @throws IOException if a remote or network exception occurs - */ @Override public synchronized void stopRegionServer(final String hostnamePort) throws IOException { @@ -2627,11 +1825,6 @@ public class HBaseAdmin implements Admin { } } - - /** - * @return cluster status - * @throws IOException if a remote or network exception occurs - */ @Override public ClusterStatus getClusterStatus() throws IOException { return executeCallable(new MasterCallable(getConnection()) { @@ -2643,19 +1836,11 @@ public class HBaseAdmin implements Admin { }); } - /** - * @return Configuration used by the instance. - */ @Override public Configuration getConfiguration() { return this.conf; } - /** - * Create a new namespace - * @param descriptor descriptor which describes the new namespace - * @throws IOException - */ @Override public void createNamespace(final NamespaceDescriptor descriptor) throws IOException { executeCallable(new MasterCallable(getConnection()) { @@ -2671,11 +1856,6 @@ public class HBaseAdmin implements Admin { }); } - /** - * Modify an existing namespace - * @param descriptor descriptor which describes the new namespace - * @throws IOException - */ @Override public void modifyNamespace(final NamespaceDescriptor descriptor) throws IOException { executeCallable(new MasterCallable(getConnection()) { @@ -2688,11 +1868,6 @@ public class HBaseAdmin implements Admin { }); } - /** - * Delete an existing namespace. Only empty namespaces (no tables) can be removed. - * @param name namespace name - * @throws IOException - */ @Override public void deleteNamespace(final String name) throws IOException { executeCallable(new MasterCallable(getConnection()) { @@ -2705,12 +1880,6 @@ public class HBaseAdmin implements Admin { }); } - /** - * Get a namespace descriptor by name - * @param name name of namespace descriptor - * @return A descriptor - * @throws IOException - */ @Override public NamespaceDescriptor getNamespaceDescriptor(final String name) throws IOException { return @@ -2724,11 +1893,6 @@ public class HBaseAdmin implements Admin { }); } - /** - * List available namespace descriptors - * @return List of descriptors - * @throws IOException - */ @Override public NamespaceDescriptor[] listNamespaceDescriptors() throws IOException { return @@ -2747,11 +1911,6 @@ public class HBaseAdmin implements Admin { }); } - /** - * List procedures - * @return procedure list - * @throws IOException - */ @Override public ProcedureInfo[] listProcedures() throws IOException { return @@ -2769,12 +1928,6 @@ public class HBaseAdmin implements Admin { }); } - /** - * Get list of table descriptors by namespace - * @param name namespace name - * @return A descriptor - * @throws IOException - */ @Override public HTableDescriptor[] listTableDescriptorsByNamespace(final String name) throws IOException { return @@ -2794,12 +1947,6 @@ public class HBaseAdmin implements Admin { }); } - /** - * Get list of table names by namespace - * @param name namespace name - * @return The list of table names in the namespace - * @throws IOException - */ @Override public TableName[] listTableNamesByNamespace(final String name) throws IOException { return @@ -2858,13 +2005,6 @@ public class HBaseAdmin implements Admin { } } - /** - * get the regions of a given table. - * - * @param tableName the name of the table - * @return Ordered list of {@link HRegionInfo}. - * @throws IOException - */ @Override public List getTableRegions(final TableName tableName) throws IOException { @@ -2884,21 +2024,10 @@ public class HBaseAdmin implements Admin { return regions; } - public List getTableRegions(final byte[] tableName) - throws IOException { - return getTableRegions(TableName.valueOf(tableName)); - } - @Override public synchronized void close() throws IOException { } - /** - * Get tableDescriptors - * @param tableNames List of table names - * @return HTD[] the tableDescriptor - * @throws IOException if a remote or network exception occurs - */ @Override public HTableDescriptor[] getTableDescriptorsByTableName(final List tableNames) throws IOException { @@ -2933,12 +2062,6 @@ public class HBaseAdmin implements Admin { } } - /** - * Get tableDescriptors - * @param names List of table names - * @return HTD[] the tableDescriptor - * @throws IOException if a remote or network exception occurs - */ @Override public HTableDescriptor[] getTableDescriptors(List names) throws IOException { @@ -3015,18 +2138,12 @@ public class HBaseAdmin implements Admin { } } - /** - * {@inheritDoc} - */ @Override public CompactionState getCompactionState(final TableName tableName) throws IOException { return getCompactionState(tableName, CompactType.NORMAL); } - /** - * {@inheritDoc} - */ @Override public CompactionState getCompactionStateForRegion(final byte[] regionName) throws IOException { @@ -3049,22 +2166,6 @@ public class HBaseAdmin implements Admin { } } - /** - * Take a snapshot for the given table. If the table is enabled, a FLUSH-type snapshot will be - * taken. If the table is disabled, an offline snapshot is taken. - *

- * Snapshots are considered unique based on the name of the snapshot. Attempts to take a - * snapshot with the same name (even a different type or with different parameters) will fail with - * a {@link SnapshotCreationException} indicating the duplicate naming. - *

- * Snapshot names follow the same naming constraints as tables in HBase. See - * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}. - * @param snapshotName name of the snapshot to be created - * @param tableName name of the table for which snapshot is created - * @throws IOException if a remote or network exception occurs - * @throws SnapshotCreationException if snapshot creation failed - * @throws IllegalArgumentException if the snapshot request is formatted incorrectly - */ @Override public void snapshot(final String snapshotName, final TableName tableName) throws IOException, @@ -3072,86 +2173,16 @@ public class HBaseAdmin implements Admin { snapshot(snapshotName, tableName, SnapshotDescription.Type.FLUSH); } - public void snapshot(final String snapshotName, - final String tableName) throws IOException, - SnapshotCreationException, IllegalArgumentException { - snapshot(snapshotName, TableName.valueOf(tableName), - SnapshotDescription.Type.FLUSH); - } - - /** - * Create snapshot for the given table of given flush type. - *

- * 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 {@link SnapshotCreationException} indicating the duplicate naming. - *

- * Snapshot names follow the same naming constraints as tables in HBase. - * @param snapshotName name of the snapshot to be created - * @param tableName name of the table for which snapshot is created - * @param flushType if the snapshot should be taken without flush memstore first - * @throws IOException if a remote or network exception occurs - * @throws SnapshotCreationException if snapshot creation failed - * @throws IllegalArgumentException if the snapshot request is formatted incorrectly - */ - public void snapshot(final byte[] snapshotName, final byte[] tableName, - final SnapshotDescription.Type flushType) throws - IOException, SnapshotCreationException, IllegalArgumentException { - snapshot(Bytes.toString(snapshotName), Bytes.toString(tableName), flushType); - } - /** - public void snapshot(final String snapshotName, - * Create a timestamp consistent snapshot for the given table. - final byte[] tableName) throws IOException, - *

- * 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 {@link SnapshotCreationException} indicating the duplicate naming. - *

- * Snapshot names follow the same naming constraints as tables in HBase. - * @param snapshotName name of the snapshot to be created - * @param tableName name of the table for which snapshot is created - * @throws IOException if a remote or network exception occurs - * @throws SnapshotCreationException if snapshot creation failed - * @throws IllegalArgumentException if the snapshot request is formatted incorrectly - */ @Override - public void snapshot(final byte[] snapshotName, - final TableName tableName) throws IOException, - SnapshotCreationException, IllegalArgumentException { + public void snapshot(final byte[] snapshotName, final TableName tableName) + throws IOException, SnapshotCreationException, IllegalArgumentException { snapshot(Bytes.toString(snapshotName), tableName, SnapshotDescription.Type.FLUSH); } - public void snapshot(final byte[] snapshotName, - final byte[] tableName) throws IOException, - SnapshotCreationException, IllegalArgumentException { - snapshot(Bytes.toString(snapshotName), TableName.valueOf(tableName), - SnapshotDescription.Type.FLUSH); - } - - /** - * Create typed snapshot of the table. - *

- * Snapshots are considered unique based on the name of the snapshot. Attempts to take a - * snapshot with the same name (even a different type or with different parameters) will fail with - * a {@link SnapshotCreationException} indicating the duplicate naming. - *

- * Snapshot names follow the same naming constraints as tables in HBase. See - * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}. - *

- * @param snapshotName name to give the snapshot on the filesystem. Must be unique from all other - * snapshots stored on the cluster - * @param tableName name of the table to snapshot - * @param type type of snapshot to take - * @throws IOException we fail to reach the master - * @throws SnapshotCreationException if snapshot creation failed - * @throws IllegalArgumentException if the snapshot request is formatted incorrectly - */ @Override - public void snapshot(final String snapshotName, - final TableName tableName, - SnapshotDescription.Type type) throws IOException, SnapshotCreationException, - IllegalArgumentException { + public void snapshot(final String snapshotName, final TableName tableName, + SnapshotDescription.Type type) + throws IOException, SnapshotCreationException, IllegalArgumentException { SnapshotDescription.Builder builder = SnapshotDescription.newBuilder(); builder.setTable(tableName.getNameAsString()); builder.setName(snapshotName); @@ -3159,41 +2190,6 @@ public class HBaseAdmin implements Admin { snapshot(builder.build()); } - public void snapshot(final String snapshotName, - final String tableName, - SnapshotDescription.Type type) throws IOException, SnapshotCreationException, - IllegalArgumentException { - snapshot(snapshotName, TableName.valueOf(tableName), type); - } - - public void snapshot(final String snapshotName, - final byte[] tableName, - SnapshotDescription.Type type) throws IOException, SnapshotCreationException, - IllegalArgumentException { - snapshot(snapshotName, TableName.valueOf(tableName), type); - } - - /** - * Take a snapshot and wait for the server to complete that snapshot (blocking). - *

- * 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 {@link SnapshotCreationException} indicating the duplicate naming. - *

- * Snapshot names follow the same naming constraints as tables in HBase. See - * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}. - *

- * You should probably use {@link #snapshot(String, String)} or {@link #snapshot(byte[], byte[])} - * unless you are sure about the type of snapshot that you want to take. - * @param snapshot snapshot to take - * @throws IOException or we lose contact with the master. - * @throws SnapshotCreationException if snapshot failed to be taken - * @throws IllegalArgumentException if the snapshot request is formatted incorrectly - */ @Override public void snapshot(SnapshotDescription snapshot) throws IOException, SnapshotCreationException, IllegalArgumentException { @@ -3235,16 +2231,6 @@ public class HBaseAdmin implements Admin { } } - /** - * 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. - * @param snapshot snapshot to take - * @return response from the server indicating the max time to wait for the snapshot - * @throws IOException if the snapshot did not succeed or we lose contact with the master. - * @throws SnapshotCreationException if snapshot creation failed - * @throws IllegalArgumentException if the snapshot request is formatted incorrectly - */ @Override public SnapshotResponse takeSnapshotAsync(SnapshotDescription snapshot) throws IOException, SnapshotCreationException { @@ -3260,26 +2246,6 @@ public class HBaseAdmin implements Admin { }); } - /** - * Check the current state of the passed snapshot. - *

- * There are three possible states: - *

    - *
  1. running - returns false
  2. - *
  3. finished - returns true
  4. - *
  5. finished with error - throws the exception that caused the snapshot to fail
  6. - *
- *

- * 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 - * {@link UnknownSnapshotException}. - * @param snapshot description of the snapshot to check - * @return true if the snapshot is completed, false if the snapshot is still - * running - * @throws IOException if we have a network issue - * @throws HBaseSnapshotException if the snapshot failed - * @throws UnknownSnapshotException if the requested snapshot is unknown - */ @Override public boolean isSnapshotFinished(final SnapshotDescription snapshot) throws IOException, HBaseSnapshotException, UnknownSnapshotException { @@ -3293,38 +2259,12 @@ public class HBaseAdmin implements Admin { }).getDone(); } - /** - * Restore the specified snapshot on the original table. (The table must be disabled) - * If the "hbase.snapshot.restore.take.failsafe.snapshot" configuration property - * is set to true, a snapshot of the current table is taken - * before executing the restore operation. - * In case of restore failure, the failsafe snapshot will be restored. - * If the restore completes without problem the failsafe snapshot is deleted. - * - * @param snapshotName name of the snapshot to restore - * @throws IOException if a remote or network exception occurs - * @throws RestoreSnapshotException if snapshot failed to be restored - * @throws IllegalArgumentException if the restore request is formatted incorrectly - */ @Override public void restoreSnapshot(final byte[] snapshotName) throws IOException, RestoreSnapshotException { restoreSnapshot(Bytes.toString(snapshotName)); } - /** - * Restore the specified snapshot on the original table. (The table must be disabled) - * If the "hbase.snapshot.restore.take.failsafe.snapshot" configuration property - * is set to true, a snapshot of the current table is taken - * before executing the restore operation. - * In case of restore failure, the failsafe snapshot will be restored. - * If the restore completes without problem the failsafe snapshot is deleted. - * - * @param snapshotName name of the snapshot to restore - * @throws IOException if a remote or network exception occurs - * @throws RestoreSnapshotException if snapshot failed to be restored - * @throws IllegalArgumentException if the restore request is formatted incorrectly - */ @Override public void restoreSnapshot(final String snapshotName) throws IOException, RestoreSnapshotException { @@ -3333,44 +2273,12 @@ public class HBaseAdmin implements Admin { restoreSnapshot(snapshotName, takeFailSafeSnapshot); } - /** - * Restore the specified snapshot on the original table. (The table must be disabled) - * If 'takeFailSafeSnapshot' is set to true, a snapshot of the current table is taken - * before executing the restore operation. - * In case of restore failure, the failsafe snapshot will be restored. - * If the restore completes without problem the failsafe snapshot is deleted. - * - * The failsafe snapshot name is configurable by using the property - * "hbase.snapshot.restore.failsafe.name". - * - * @param snapshotName name of the snapshot to restore - * @param takeFailSafeSnapshot true if the failsafe snapshot should be taken - * @throws IOException if a remote or network exception occurs - * @throws RestoreSnapshotException if snapshot failed to be restored - * @throws IllegalArgumentException if the restore request is formatted incorrectly - */ @Override public void restoreSnapshot(final byte[] snapshotName, final boolean takeFailSafeSnapshot) throws IOException, RestoreSnapshotException { restoreSnapshot(Bytes.toString(snapshotName), takeFailSafeSnapshot); } - /** - * Restore the specified snapshot on the original table. (The table must be disabled) - * If 'takeFailSafeSnapshot' is set to true, a snapshot of the current table is taken - * before executing the restore operation. - * In case of restore failure, the failsafe snapshot will be restored. - * If the restore completes without problem the failsafe snapshot is deleted. - * - * The failsafe snapshot name is configurable by using the property - * "hbase.snapshot.restore.failsafe.name". - * - * @param snapshotName name of the snapshot to restore - * @param takeFailSafeSnapshot true if the failsafe snapshot should be taken - * @throws IOException if a remote or network exception occurs - * @throws RestoreSnapshotException if snapshot failed to be restored - * @throws IllegalArgumentException if the restore request is formatted incorrectly - */ @Override public void restoreSnapshot(final String snapshotName, boolean takeFailSafeSnapshot) throws IOException, RestoreSnapshotException { @@ -3445,64 +2353,12 @@ public class HBaseAdmin implements Admin { } } - /** - * Create a new table by cloning the snapshot content. - * - * @param snapshotName name of the snapshot to be cloned - * @param tableName name of the table where the snapshot will be restored - * @throws IOException if a remote or network exception occurs - * @throws TableExistsException if table to be created already exists - * @throws RestoreSnapshotException if snapshot failed to be cloned - * @throws IllegalArgumentException if the specified table has not a valid name - */ - public void cloneSnapshot(final byte[] snapshotName, final byte[] tableName) - throws IOException, TableExistsException, RestoreSnapshotException { - cloneSnapshot(Bytes.toString(snapshotName), TableName.valueOf(tableName)); - } - - /** - * Create a new table by cloning the snapshot content. - * - * @param snapshotName name of the snapshot to be cloned - * @param tableName name of the table where the snapshot will be restored - * @throws IOException if a remote or network exception occurs - * @throws TableExistsException if table to be created already exists - * @throws RestoreSnapshotException if snapshot failed to be cloned - * @throws IllegalArgumentException if the specified table has not a valid name - */ @Override public void cloneSnapshot(final byte[] snapshotName, final TableName tableName) throws IOException, TableExistsException, RestoreSnapshotException { cloneSnapshot(Bytes.toString(snapshotName), tableName); } - - - /** - * Create a new table by cloning the snapshot content. - * - * @param snapshotName name of the snapshot to be cloned - * @param tableName name of the table where the snapshot will be restored - * @throws IOException if a remote or network exception occurs - * @throws TableExistsException if table to be created already exists - * @throws RestoreSnapshotException if snapshot failed to be cloned - * @throws IllegalArgumentException if the specified table has not a valid name - */ - public void cloneSnapshot(final String snapshotName, final String tableName) - throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException { - cloneSnapshot(snapshotName, TableName.valueOf(tableName)); - } - - /** - * Create a new table by cloning the snapshot content. - * - * @param snapshotName name of the snapshot to be cloned - * @param tableName name of the table where the snapshot will be restored - * @throws IOException if a remote or network exception occurs - * @throws TableExistsException if table to be created already exists - * @throws RestoreSnapshotException if snapshot failed to be cloned - * @throws IllegalArgumentException if the specified table has not a valid name - */ @Override public void cloneSnapshot(final String snapshotName, final TableName tableName) throws IOException, TableExistsException, RestoreSnapshotException { @@ -3513,20 +2369,9 @@ public class HBaseAdmin implements Admin { waitUntilTableIsEnabled(tableName); } - /** - * Execute a distributed procedure on a cluster synchronously with return data - * - * @param signature A distributed procedure is uniquely identified - * by its signature (default the root ZK node name of the procedure). - * @param instance The instance name of the procedure. For some procedures, this parameter is - * optional. - * @param props Property/Value pairs of properties passing to the procedure - * @return data returned after procedure execution. null if no return data. - * @throws IOException - */ @Override - public byte[] execProcedureWithRet(String signature, String instance, - Map props) throws IOException { + public byte[] execProcedureWithRet(String signature, String instance, Map props) + throws IOException { ProcedureDescription.Builder builder = ProcedureDescription.newBuilder(); builder.setSignature(signature).setInstance(instance); for (Entry entry : props.entrySet()) { @@ -3548,19 +2393,10 @@ public class HBaseAdmin implements Admin { return response.hasReturnData() ? response.getReturnData().toByteArray() : null; } - /** - * Execute a distributed procedure on a cluster. - * - * @param signature A distributed procedure is uniquely identified - * by its signature (default the root ZK node name of the procedure). - * @param instance The instance name of the procedure. For some procedures, this parameter is - * optional. - * @param props Property/Value pairs of properties passing to the procedure - * @throws IOException - */ + @Override - public void execProcedure(String signature, String instance, - Map props) throws IOException { + public void execProcedure(String signature, String instance, Map props) + throws IOException { ProcedureDescription.Builder builder = ProcedureDescription.newBuilder(); builder.setSignature(signature).setInstance(instance); for (Entry entry : props.entrySet()) { @@ -3608,23 +2444,6 @@ public class HBaseAdmin implements Admin { } } - /** - * Check the current state of the specified procedure. - *

- * There are three possible states: - *

    - *
  1. running - returns false
  2. - *
  3. finished - returns true
  4. - *
  5. finished with error - throws the exception that caused the procedure to fail
  6. - *
- *

- * - * @param signature The signature that uniquely identifies a procedure - * @param instance The instance name of the procedure - * @param props Property/Value pairs of properties passing to the procedure - * @return true if the specified procedure is finished successfully, false if it is still running - * @throws IOException if the specified procedure finished with error - */ @Override public boolean isProcedureFinished(String signature, String instance, Map props) throws IOException { @@ -3656,8 +2475,7 @@ public class HBaseAdmin implements Admin { * @throws RestoreSnapshotException if snapshot failed to be restored * @throws IllegalArgumentException if the restore request is formatted incorrectly */ - private void internalRestoreSnapshot(final String snapshotName, final TableName - tableName) + private void internalRestoreSnapshot(final String snapshotName, final TableName tableName) throws IOException, RestoreSnapshotException { SnapshotDescription snapshot = SnapshotDescription.newBuilder() .setName(snapshotName).setTable(tableName.getNameAsString()).build(); @@ -3722,11 +2540,6 @@ public class HBaseAdmin implements Admin { }); } - /** - * List completed snapshots. - * @return a list of snapshot descriptors for completed snapshots - * @throws IOException if a network error occurs - */ @Override public List listSnapshots() throws IOException { return executeCallable(new MasterCallable>(getConnection()) { @@ -3738,25 +2551,11 @@ public class HBaseAdmin implements Admin { }); } - /** - * List all the completed snapshots matching the given regular expression. - * - * @param regex The regular expression to match against - * @return - returns a List of SnapshotDescription - * @throws IOException if a remote or network exception occurs - */ @Override public List listSnapshots(String regex) throws IOException { return listSnapshots(Pattern.compile(regex)); } - /** - * List all the completed snapshots matching the given pattern. - * - * @param pattern The compiled regular expression to match against - * @return - returns a List of SnapshotDescription - * @throws IOException if a remote or network exception occurs - */ @Override public List listSnapshots(Pattern pattern) throws IOException { List matched = new LinkedList(); @@ -3769,28 +2568,12 @@ public class HBaseAdmin implements Admin { return matched; } - /** - * List all the completed snapshots matching the given table name regular expression and snapshot - * name regular expression. - * @param tableNameRegex The table name regular expression to match against - * @param snapshotNameRegex The snapshot name regular expression to match against - * @return returns a List of completed SnapshotDescription - * @throws IOException if a remote or network exception occurs - */ @Override public List listTableSnapshots(String tableNameRegex, String snapshotNameRegex) throws IOException { return listTableSnapshots(Pattern.compile(tableNameRegex), Pattern.compile(snapshotNameRegex)); } - /** - * List all the completed snapshots matching the given table name regular expression and snapshot - * name regular expression. - * @param tableNamePattern The compiled table name regular expression to match against - * @param snapshotNamePattern The compiled snapshot name regular expression to match against - * @return returns a List of completed SnapshotDescription - * @throws IOException if a remote or network exception occurs - */ @Override public List listTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) throws IOException { @@ -3808,21 +2591,11 @@ public class HBaseAdmin implements Admin { return tableSnapshots; } - /** - * Delete an existing snapshot. - * @param snapshotName name of the snapshot - * @throws IOException if a remote or network exception occurs - */ @Override public void deleteSnapshot(final byte[] snapshotName) throws IOException { deleteSnapshot(Bytes.toString(snapshotName)); } - /** - * Delete an existing snapshot. - * @param snapshotName name of the snapshot - * @throws IOException if a remote or network exception occurs - */ @Override public void deleteSnapshot(final String snapshotName) throws IOException { // make sure the snapshot is possibly valid @@ -3840,21 +2613,11 @@ public class HBaseAdmin implements Admin { }); } - /** - * Delete existing snapshots whose names match the pattern passed. - * @param regex The regular expression to match against - * @throws IOException if a remote or network exception occurs - */ @Override public void deleteSnapshots(final String regex) throws IOException { deleteSnapshots(Pattern.compile(regex)); } - /** - * Delete existing snapshots whose names match the pattern passed. - * @param pattern pattern for names of the snapshot to match - * @throws IOException if a remote or network exception occurs - */ @Override public void deleteSnapshots(final Pattern pattern) throws IOException { List snapshots = listSnapshots(pattern); @@ -3880,26 +2643,12 @@ public class HBaseAdmin implements Admin { }); } - /** - * Delete all existing snapshots matching the given table name regular expression and snapshot - * name regular expression. - * @param tableNameRegex The table name regular expression to match against - * @param snapshotNameRegex The snapshot name regular expression to match against - * @throws IOException if a remote or network exception occurs - */ @Override public void deleteTableSnapshots(String tableNameRegex, String snapshotNameRegex) throws IOException { deleteTableSnapshots(Pattern.compile(tableNameRegex), Pattern.compile(snapshotNameRegex)); } - /** - * Delete all existing snapshots matching the given table name regular expression and snapshot - * name regular expression. - * @param tableNamePattern The compiled table name regular expression to match against - * @param snapshotNamePattern The compiled snapshot name regular expression to match against - * @throws IOException if a remote or network exception occurs - */ @Override public void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) throws IOException { @@ -3914,12 +2663,6 @@ public class HBaseAdmin implements Admin { } } - /** - * Apply the new quota settings. - * - * @param quota the quota settings - * @throws IOException if a remote or network exception occurs - */ @Override public void setQuota(final QuotaSettings quota) throws IOException { executeCallable(new MasterCallable(getConnection()) { @@ -3931,13 +2674,6 @@ public class HBaseAdmin implements Admin { }); } - /** - * Return a Quota Scanner to list the quotas based on the filter. - * - * @param filter the quota settings filter - * @return the quota scanner - * @throws IOException if a remote or network exception occurs - */ @Override public QuotaRetriever getQuotaRetriever(final QuotaFilter filter) throws IOException { return QuotaRetriever.open(conf, filter); @@ -3958,27 +2694,6 @@ public class HBaseAdmin implements Admin { } } - /** - * Creates and returns a {@link com.google.protobuf.RpcChannel} instance - * connected to the active master. - * - *

- * The obtained {@link com.google.protobuf.RpcChannel} instance can be used to access a published - * coprocessor {@link 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);
-   * 
- * - * @return A MasterCoprocessorRpcChannel instance - */ @Override public CoprocessorRpcChannel coprocessorService() { return new MasterCoprocessorRpcChannel(connection); @@ -4000,28 +2715,6 @@ public class HBaseAdmin implements Admin { } } - /** - * Creates and returns a {@link com.google.protobuf.RpcChannel} instance - * connected to the passed region server. - * - *

- * The obtained {@link com.google.protobuf.RpcChannel} instance can be used to access a published - * coprocessor {@link 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);
-   * 
- * - * @param sn the server name to which the endpoint call is made - * @return A RegionServerCoprocessorRpcChannel instance - */ @Override public CoprocessorRpcChannel coprocessorService(ServerName sn) { return new RegionServerCoprocessorRpcChannel(connection, sn); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java index dffde55a3af..d9c7caad0b0 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java @@ -718,7 +718,7 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase { } TableName tableName = selected.getTableName(); LOG.info("Deleting column family: " + cfd + " from table: " + tableName); - admin.deleteColumn(tableName, cfd.getName()); + admin.deleteColumnFamily(tableName, cfd.getName()); // assertion HTableDescriptor freshTableDesc = admin.getTableDescriptor(tableName); Assert.assertFalse("Column family: " + cfd + " was not added", diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java index 82a599ce1bd..13a5936a380 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java @@ -121,7 +121,7 @@ public class IntegrationTestIngestWithMOB extends IntegrationTestIngest { if(Arrays.equals(columnDescriptor.getName(), mobColumnFamily)) { columnDescriptor.setMobEnabled(true); columnDescriptor.setMobThreshold((long) threshold); - admin.modifyColumn(tableName, columnDescriptor); + admin.modifyColumnFamily(tableName, columnDescriptor); } } LOG.info("Enabling table " + getTablename()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index 006c3e71362..71a33441bf1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -2682,13 +2682,15 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { /** * Returns a Admin instance. - * This instance is shared between HBaseTestingUtility instance users. - * Closing it has no effect, it will be closed automatically when the - * cluster shutdowns + * This instance is shared between HBaseTestingUtility instance users. Closing it has no effect, + * it will be closed automatically when the cluster shutdowns * - * @return An Admin instance. - * @throws IOException + * @return HBaseAdmin instance which is guaranteed to support only {@link Admin} interface. + * Functions in HBaseAdmin not provided by {@link Admin} interface can be changed/deleted + * anytime. + * @deprecated Since 2.0. Will be removed in 3.0. Use {@link #getAdmin()} instead. */ + @Deprecated public synchronized HBaseAdmin getHBaseAdmin() throws IOException { if (hbaseAdmin == null){ @@ -2697,8 +2699,18 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { return hbaseAdmin; } - private HBaseAdmin hbaseAdmin = null; + /** + * Returns an Admin instance which is shared between HBaseTestingUtility instance users. + * Closing it has no effect, it will be closed automatically when the cluster shutdowns + */ + public synchronized Admin getAdmin() throws IOException { + if (hbaseAdmin == null){ + this.hbaseAdmin = (HBaseAdmin) getConnection().getAdmin(); + } + return hbaseAdmin; + } + private HBaseAdmin hbaseAdmin = null; /** * Returns a ZooKeeperWatcher instance. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java index 545014882f4..989192d3c83 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java @@ -88,7 +88,7 @@ public class TestAcidGuarantees implements Tool { // force mob enabled such that all data is mob data hcd.setMobEnabled(true); hcd.setMobThreshold(4); - util.getHBaseAdmin().modifyColumn(TABLE_NAME, hcd); + util.getHBaseAdmin().modifyColumnFamily(TABLE_NAME, hcd); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java index 0f286b66438..e510d28d556 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java @@ -465,8 +465,8 @@ public class TestAdmin2 { onlineRegions.contains(info)); } - private Admin createTable(TableName tableName) throws IOException { - Admin admin = TEST_UTIL.getHBaseAdmin(); + private HBaseAdmin createTable(TableName tableName) throws IOException { + HBaseAdmin admin = TEST_UTIL.getHBaseAdmin(); HTableDescriptor htd = new HTableDescriptor(tableName); HColumnDescriptor hcd = new HColumnDescriptor("value"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java index 7278892215e..5e302d2d5a8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java @@ -400,8 +400,8 @@ public class TestMetaWithReplicas { } } assert(moveToServer != null); - String tableName = "randomTable5678"; - TEST_UTIL.createTable(TableName.valueOf(tableName), "f"); + TableName tableName = TableName.valueOf("randomTable5678"); + TEST_UTIL.createTable(tableName, "f"); assertTrue(TEST_UTIL.getHBaseAdmin().tableExists(tableName)); TEST_UTIL.getHBaseAdmin().move(HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(), Bytes.toBytes(moveToServer.getServerName())); @@ -413,8 +413,8 @@ public class TestMetaWithReplicas { i++; } while (!moveToServer.equals(currentServer) && i < 1000); //wait for 10 seconds overall assert(i != 1000); - TEST_UTIL.getHBaseAdmin().disableTable("randomTable5678"); - assertTrue(TEST_UTIL.getHBaseAdmin().isTableDisabled("randomTable5678")); + TEST_UTIL.getHBaseAdmin().disableTable(tableName); + assertTrue(TEST_UTIL.getHBaseAdmin().isTableDisabled(tableName)); } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java index fff6f443420..267201fc06e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java @@ -100,7 +100,7 @@ public class TestExpiredMobFileCleaner { int timeToLive = expireDays * secondsOfDay(); hcd.setTimeToLive(timeToLive); - admin.modifyColumn(tableName, hcd); + admin.modifyColumnFamily(tableName, hcd); } private void putKVAndFlush(BufferedMutator table, byte[] row, byte[] value, long ts) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java index e556a5837d0..f3cfc396624 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java @@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import java.io.IOException; -import java.util.Collection; import java.util.List; import java.util.Random; @@ -41,7 +40,6 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.Store; -import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -151,7 +149,7 @@ public class TestFIFOCompactionPolicy { TEST_UTIL.startMiniCluster(1); HBaseAdmin admin = TEST_UTIL.getHBaseAdmin(); - String tableName = this.tableName.getNameAsString()+"-TTL"; + TableName tableName = TableName.valueOf(getClass().getSimpleName() + "-TTL"); if (admin.tableExists(tableName)) { admin.disableTable(tableName); admin.deleteTable(tableName); @@ -180,7 +178,7 @@ public class TestFIFOCompactionPolicy { TEST_UTIL.startMiniCluster(1); HBaseAdmin admin = TEST_UTIL.getHBaseAdmin(); - String tableName = this.tableName.getNameAsString()+"-MinVersion"; + TableName tableName = TableName.valueOf(getClass().getSimpleName() + "-MinVersion"); if (admin.tableExists(tableName)) { admin.disableTable(tableName); admin.deleteTable(tableName); @@ -211,7 +209,7 @@ public class TestFIFOCompactionPolicy { TEST_UTIL.startMiniCluster(1); HBaseAdmin admin = TEST_UTIL.getHBaseAdmin(); - String tableName = this.tableName.getNameAsString()+"-MinVersion"; + TableName tableName = TableName.valueOf(getClass().getSimpleName() + "-BlockingStoreFiles"); if (admin.tableExists(tableName)) { admin.disableTable(tableName); admin.deleteTable(tableName); diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb index 9a32b0ecb67..c61b59894ca 100644 --- a/hbase-shell/src/main/ruby/hbase/admin.rb +++ b/hbase-shell/src/main/ruby/hbase/admin.rb @@ -210,7 +210,7 @@ module Hbase def enable(table_name) tableExists(table_name) return if enabled?(table_name) - @admin.enableTable(table_name) + @admin.enableTable(TableName.valueOf(table_name)) end #---------------------------------------------------------------------------------------------- @@ -225,7 +225,7 @@ module Hbase def disable(table_name) tableExists(table_name) return if disabled?(table_name) - @admin.disableTable(table_name) + @admin.disableTable(TableName.valueOf(table_name)) end #---------------------------------------------------------------------------------------------- @@ -244,14 +244,15 @@ module Hbase #---------------------------------------------------------------------------------------------- # Is table disabled? def disabled?(table_name) - @admin.isTableDisabled(table_name) + @admin.isTableDisabled(TableName.valueOf(table_name)) end #---------------------------------------------------------------------------------------------- # Drops a table def drop(table_name) tableExists(table_name) - raise ArgumentError, "Table #{table_name} is enabled. Disable it first." if enabled?(table_name) + raise ArgumentError, "Table #{table_name} is enabled. Disable it first." if enabled?( + table_name) @admin.deleteTable(org.apache.hadoop.hbase.TableName.valueOf(table_name)) end @@ -447,15 +448,17 @@ module Hbase #---------------------------------------------------------------------------------------------- # Truncates table (deletes all records by recreating the table) - def truncate(table_name, conf = @conf) - table_description = @admin.getTableDescriptor(TableName.valueOf(table_name)) - raise ArgumentError, "Table #{table_name} is not enabled. Enable it first." unless enabled?(table_name) + def truncate(table_name_str, conf = @conf) + table_name = TableName.valueOf(table_name_str) + table_description = @admin.getTableDescriptor(table_name) + raise ArgumentError, "Table #{table_name_str} is not enabled. Enable it first." unless + enabled?(table_name_str) yield 'Disabling table...' if block_given? @admin.disableTable(table_name) begin yield 'Truncating table...' if block_given? - @admin.truncateTable(org.apache.hadoop.hbase.TableName.valueOf(table_name), false) + @admin.truncateTable(table_name, false) rescue => e # Handle the compatibility case, where the truncate method doesn't exists on the Master raise e unless e.respond_to?(:cause) && e.cause != nil @@ -463,7 +466,7 @@ module Hbase if rootCause.kind_of?(org.apache.hadoop.hbase.DoNotRetryIOException) then # Handle the compatibility case, where the truncate method doesn't exists on the Master yield 'Dropping table...' if block_given? - @admin.deleteTable(org.apache.hadoop.hbase.TableName.valueOf(table_name)) + @admin.deleteTable(table_name) yield 'Creating table...' if block_given? @admin.createTable(table_description) @@ -475,9 +478,10 @@ module Hbase #---------------------------------------------------------------------------------------------- # Truncates table while maintaing region boundaries (deletes all records by recreating the table) - def truncate_preserve(table_name, conf = @conf) - h_table = @connection.getTable(TableName.valueOf(table_name)) - locator = @connection.getRegionLocator(TableName.valueOf(table_name)) + def truncate_preserve(table_name_str, conf = @conf) + table_name = TableName.valueOf(table_name_str) + h_table = @connection.getTable(table_name) + locator = @connection.getRegionLocator(table_name) begin splits = locator.getAllRegionLocations(). map{|i| Bytes.toString(i.getRegionInfo().getStartKey)}. @@ -486,13 +490,13 @@ module Hbase locator.close() end - table_description = @admin.getTableDescriptor(TableName.valueOf(table_name)) + table_description = @admin.getTableDescriptor(table_name) yield 'Disabling table...' if block_given? - disable(table_name) + disable(table_name_str) begin yield 'Truncating table...' if block_given? - @admin.truncateTable(org.apache.hadoop.hbase.TableName.valueOf(table_name), true) + @admin.truncateTable(table_name, true) rescue => e # Handle the compatibility case, where the truncate method doesn't exists on the Master raise e unless e.respond_to?(:cause) && e.cause != nil @@ -500,7 +504,7 @@ module Hbase if rootCause.kind_of?(org.apache.hadoop.hbase.DoNotRetryIOException) then # Handle the compatibility case, where the truncate method doesn't exists on the Master yield 'Dropping table...' if block_given? - @admin.deleteTable(org.apache.hadoop.hbase.TableName.valueOf(table_name)) + @admin.deleteTable(table_name) yield 'Creating table with region boundaries...' if block_given? @admin.createTable(table_description, splits) @@ -534,18 +538,21 @@ module Hbase #---------------------------------------------------------------------------------------------- # Change table structure or table options - def alter(table_name, wait = true, *args) + def alter(table_name_str, wait = true, *args) # Table name should be a string - raise(ArgumentError, "Table name must be of type String") unless table_name.kind_of?(String) + raise(ArgumentError, "Table name must be of type String") unless + table_name_str.kind_of?(String) # Table should exist - raise(ArgumentError, "Can't find a table: #{table_name}") unless exists?(table_name) + raise(ArgumentError, "Can't find a table: #{table_name_str}") unless exists?(table_name_str) # There should be at least one argument raise(ArgumentError, "There should be at least one argument but the table name") if args.empty? + table_name = TableName.valueOf(table_name_str) + # Get table descriptor - htd = @admin.getTableDescriptor(TableName.valueOf(table_name)) + htd = @admin.getTableDescriptor(table_name) # Process all args args.each do |arg| @@ -573,11 +580,11 @@ module Hbase if wait == true puts "Updating all regions with the new schema..." - alter_status(table_name) + alter_status(table_name_str) end # We bypass descriptor when adding column families; refresh it to apply other args correctly. - htd = @admin.getTableDescriptor(TableName.valueOf(table_name)) + htd = @admin.getTableDescriptor(table_name) next end @@ -587,7 +594,7 @@ module Hbase # Delete column family if method == "delete" raise(ArgumentError, "NAME parameter missing for delete method") unless name - @admin.deleteColumn(table_name, name) + @admin.deleteColumn(table_name, name.to_java_bytes) # Unset table attributes elsif method == "table_att_unset" raise(ArgumentError, "NAME parameter missing for table_att_unset method") unless name @@ -604,7 +611,7 @@ module Hbase end htd.remove(name) end - @admin.modifyTable(table_name.to_java_bytes, htd) + @admin.modifyTable(table_name, htd) # Unknown method else raise ArgumentError, "Unknown method: #{method}" @@ -616,12 +623,12 @@ module Hbase if wait == true puts "Updating all regions with the new schema..." - alter_status(table_name) + alter_status(table_name_str) end if method == "delete" # We bypass descriptor when deleting column families; refresh it to apply other args correctly. - htd = @admin.getTableDescriptor(TableName.valueOf(table_name)) + htd = @admin.getTableDescriptor(table_name) end next end @@ -667,7 +674,7 @@ module Hbase arg.delete(key) end - @admin.modifyTable(table_name.to_java_bytes, htd) + @admin.modifyTable(table_name, htd) arg.each_key do |unknown_key| puts("Unknown argument ignored: %s" % [unknown_key]) @@ -675,7 +682,7 @@ module Hbase if wait == true puts "Updating all regions with the new schema..." - alter_status(table_name) + alter_status(table_name_str) end next end @@ -792,13 +799,13 @@ module Hbase # Does table exist? def exists?(table_name) - @admin.tableExists(table_name) + @admin.tableExists(TableName.valueOf(table_name)) end #---------------------------------------------------------------------------------------------- # Is table enabled def enabled?(table_name) - @admin.isTableEnabled(table_name) + @admin.isTableEnabled(TableName.valueOf(table_name)) end #---------------------------------------------------------------------------------------------- @@ -910,14 +917,23 @@ module Hbase #---------------------------------------------------------------------------------------------- # Take a snapshot of specified table def snapshot(table, snapshot_name, *args) + # Table name should be a string + raise(ArgumentError, "Table name must be of type String") unless table.kind_of?(String) + + # Snapshot name should be a string + raise(ArgumentError, "Snapshot name must be of type String") unless + snapshot_name.kind_of?(String) + + table_name = TableName.valueOf(table) if args.empty? - @admin.snapshot(snapshot_name.to_java_bytes, table.to_java_bytes) + @admin.snapshot(snapshot_name, table_name) else args.each do |arg| if arg[SKIP_FLUSH] == true - @admin.snapshot(snapshot_name.to_java_bytes, table.to_java_bytes, SnapshotDescription::Type::SKIPFLUSH) + @admin.snapshot(snapshot_name, table_name, + SnapshotDescription::Type::SKIPFLUSH) else - @admin.snapshot(snapshot_name.to_java_bytes, table.to_java_bytes) + @admin.snapshot(snapshot_name, table_name) end end end @@ -926,19 +942,19 @@ module Hbase #---------------------------------------------------------------------------------------------- # Restore specified snapshot def restore_snapshot(snapshot_name) - @admin.restoreSnapshot(snapshot_name.to_java_bytes) + @admin.restoreSnapshot(snapshot_name) end #---------------------------------------------------------------------------------------------- # Create a new table by cloning the snapshot content def clone_snapshot(snapshot_name, table) - @admin.cloneSnapshot(snapshot_name.to_java_bytes, table.to_java_bytes) + @admin.cloneSnapshot(snapshot_name, TableName.valueOf(table)) end #---------------------------------------------------------------------------------------------- # Delete specified snapshot def delete_snapshot(snapshot_name) - @admin.deleteSnapshot(snapshot_name.to_java_bytes) + @admin.deleteSnapshot(snapshot_name) end #---------------------------------------------------------------------------------------------- diff --git a/hbase-shell/src/main/ruby/hbase/security.rb b/hbase-shell/src/main/ruby/hbase/security.rb index c7b94e88c42..20f32983eab 100644 --- a/hbase-shell/src/main/ruby/hbase/security.rb +++ b/hbase-shell/src/main/ruby/hbase/security.rb @@ -64,7 +64,7 @@ module Hbase # Table should exist raise(ArgumentError, "Can't find a table: #{table_name}") unless exists?(table_name) - tableName = org.apache.hadoop.hbase.TableName.valueOf(table_name.to_java_bytes) + tableName = org.apache.hadoop.hbase.TableName.valueOf(table_name) htd = @admin.getTableDescriptor(tableName) if (family != nil) @@ -106,7 +106,7 @@ module Hbase # Table should exist raise(ArgumentError, "Can't find a table: #{table_name}") unless exists?(table_name) - tableName = org.apache.hadoop.hbase.TableName.valueOf(table_name.to_java_bytes) + tableName = org.apache.hadoop.hbase.TableName.valueOf(table_name) htd = @admin.getTableDescriptor(tableName) if (family != nil) @@ -165,7 +165,7 @@ module Hbase # Does table exist? def exists?(table_name) - @admin.tableExists(table_name) + @admin.tableExists(TableName.valueOf(table_name)) end def isNamespace?(table_name) diff --git a/hbase-shell/src/main/ruby/hbase/visibility_labels.rb b/hbase-shell/src/main/ruby/hbase/visibility_labels.rb index 0d50d8a3fcd..98bfb2cc323 100644 --- a/hbase-shell/src/main/ruby/hbase/visibility_labels.rb +++ b/hbase-shell/src/main/ruby/hbase/visibility_labels.rb @@ -153,7 +153,7 @@ module Hbase # Does table exist? def exists?(table_name) - @admin.tableExists(table_name) + @admin.tableExists(TableName.valueOf(table_name)) end end end diff --git a/hbase-shell/src/test/ruby/hbase/admin_test.rb b/hbase-shell/src/test/ruby/hbase/admin_test.rb index 0a1a92ea976..02700372cbe 100644 --- a/hbase-shell/src/test/ruby/hbase/admin_test.rb +++ b/hbase-shell/src/test/ruby/hbase/admin_test.rb @@ -405,21 +405,21 @@ module Hbase end #------------------------------------------------------------------------------- - define_test "Snapshot should fail with non-string snapshot name" do - assert_raise(NoMethodError) do + define_test "Snapshot should fail with non-string table name" do + assert_raise(ArgumentError) do admin.snapshot(123, 'xxx') end end - define_test "Snapshot should fail with non-string table name" do - assert_raise(NoMethodError) do - admin.snapshot(@create_test_snapshot, 123) + define_test "Snapshot should fail with non-string snapshot name" do + assert_raise(ArgumentError) do + admin.snapshot(@test_name, 123) end end - define_test "Snapshot should fail without table name" do + define_test "Snapshot should fail without snapshot name" do assert_raise(ArgumentError) do - admin.snapshot("hbase_create_test_snapshot") + admin.snapshot(@test_name) end end diff --git a/src/main/asciidoc/_chapters/cp.adoc b/src/main/asciidoc/_chapters/cp.adoc index aa2ac0fc88f..a4587ecd9b2 100644 --- a/src/main/asciidoc/_chapters/cp.adoc +++ b/src/main/asciidoc/_chapters/cp.adoc @@ -421,7 +421,7 @@ onwards. + [source,java] ---- -String tableName = "users"; +TableName tableName = TableName.valueOf("users"); String path = "hdfs://:/user//coprocessor.jar"; Configuration conf = HBaseConfiguration.create(); HBaseAdmin admin = new HBaseAdmin(conf); diff --git a/src/main/asciidoc/_chapters/external_apis.adoc b/src/main/asciidoc/_chapters/external_apis.adoc index 4aa581554bc..43a428a79cd 100644 --- a/src/main/asciidoc/_chapters/external_apis.adoc +++ b/src/main/asciidoc/_chapters/external_apis.adoc @@ -741,7 +741,7 @@ the data, and deletes the table. [source,jython] ---- import java.lang -from org.apache.hadoop.hbase import HBaseConfiguration, HTableDescriptor, HColumnDescriptor, HConstants +from org.apache.hadoop.hbase import HBaseConfiguration, HTableDescriptor, HColumnDescriptor, HConstants, TableName from org.apache.hadoop.hbase.client import HBaseAdmin, HTable, Get from org.apache.hadoop.hbase.io import Cell, RowResult @@ -753,7 +753,7 @@ conf = HBaseConfiguration() # Create a table named 'test' that has two column families, # one named 'content, and the other 'anchor'. The colons # are required for column family names. -tablename = "test" +tablename = TableName.valueOf("test") desc = HTableDescriptor(tablename) desc.addFamily(HColumnDescriptor("content:")) diff --git a/src/main/asciidoc/_chapters/ops_mgt.adoc b/src/main/asciidoc/_chapters/ops_mgt.adoc index db255aa8b10..e8d44eb691d 100644 --- a/src/main/asciidoc/_chapters/ops_mgt.adoc +++ b/src/main/asciidoc/_chapters/ops_mgt.adoc @@ -2214,7 +2214,7 @@ or in code it would be as follows: [source,java] ---- -void rename(Admin admin, String oldTableName, String newTableName) { +void rename(Admin admin, String oldTableName, TableName newTableName) { String snapshotName = randomName(); admin.disableTable(oldTableName); admin.snapshot(snapshotName, oldTableName); diff --git a/src/main/asciidoc/_chapters/schema_design.adoc b/src/main/asciidoc/_chapters/schema_design.adoc index 926df710728..e2dcbad83c0 100644 --- a/src/main/asciidoc/_chapters/schema_design.adoc +++ b/src/main/asciidoc/_chapters/schema_design.adoc @@ -53,7 +53,7 @@ Tables must be disabled when making ColumnFamily modifications, for example: Configuration config = HBaseConfiguration.create(); Admin admin = new Admin(conf); -String table = "myTable"; +TableName table = TableName.valueOf("myTable"); admin.disableTable(table);